From db33f7727cd1dc284ee7a2a9ca3435fe3dae69d6 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Mon, 15 Apr 2024 16:22:40 +0300 Subject: [PATCH 001/239] Update "dhp-stats-update" workflow to use "spark"-actions, instead of "hive" ones. Note: Currently the code is set to only test the "Step1". --- .../dhp-stats-update/installProject.sh | 18 + .../dhp-stats-update/runOozieWorkfow.sh | 20 + .../graph/stats/oozie_app/scripts/step1.sql | 4 +- .../graph/stats/oozie_app/scripts/step10.sql | 24 +- .../graph/stats/oozie_app/scripts/step11.sql | 18 +- .../graph/stats/oozie_app/scripts/step12.sql | 34 +- .../graph/stats/oozie_app/scripts/step13.sql | 34 +- .../graph/stats/oozie_app/scripts/step14.sql | 30 +- .../graph/stats/oozie_app/scripts/step15.sql | 26 +- .../stats/oozie_app/scripts/step15_5.sql | 36 +- .../scripts/step16_1-definitions.sql | 12 +- .../stats/oozie_app/scripts/step16_5.sql | 20 +- .../graph/stats/oozie_app/scripts/step2.sql | 38 +- .../scripts/step21-createObservatoryDB.sql | 38 +- .../graph/stats/oozie_app/scripts/step3.sql | 38 +- .../graph/stats/oozie_app/scripts/step4.sql | 36 +- .../graph/stats/oozie_app/scripts/step5.sql | 36 +- .../graph/stats/oozie_app/scripts/step6.sql | 30 +- .../graph/stats/oozie_app/scripts/step7.sql | 30 +- .../graph/stats/oozie_app/scripts/step8.sql | 36 +- .../graph/stats/oozie_app/scripts/step9.sql | 8 +- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 443 +++++++++++++----- 22 files changed, 627 insertions(+), 382 deletions(-) create mode 100755 dhp-workflows/dhp-stats-update/installProject.sh create mode 100755 dhp-workflows/dhp-stats-update/runOozieWorkfow.sh diff --git a/dhp-workflows/dhp-stats-update/installProject.sh b/dhp-workflows/dhp-stats-update/installProject.sh new file mode 100755 index 000000000..afd95578d --- /dev/null +++ b/dhp-workflows/dhp-stats-update/installProject.sh @@ -0,0 +1,18 @@ +# Install the whole "dnet-hadoop" project. + +# Delete this module's previous build-files in order to avoid any conflicts. +rm -rf target/ || + +# Go to the root directory of this project. +cd ../../ + +# Select the build profile. +DEFAULT_PROFILE='' # It's the empty profile. +NEWER_VERSIONS_PROFILE='-Pscala-2.12' +CHOSEN_MAVEN_PROFILE=${DEFAULT_PROFILE} + +# Install the project. +mvn clean install -U ${CHOSEN_MAVEN_PROFILE} -Dmaven.test.skip=true + +# We skip tests for all modules, since the take a big amount of time and some of them fail. +# Any test added to this module, will be executed in the "runOozieWorkflow.sh" script. diff --git a/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh b/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh new file mode 100755 index 000000000..a4825a3ae --- /dev/null +++ b/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh @@ -0,0 +1,20 @@ +# This script deploys and runs the oozie workflow on the cluster, defined in the "~/.dhp/application.properties" file. + +# Select the build profile. +DEFAULT_PROFILE='' # It's the empty profile. +NEWER_VERSIONS_PROFILE='-Pscala-2.12' +CHOSEN_MAVEN_PROFILE=${DEFAULT_PROFILE} + +# Build and deploy this module. +mvn clean package -U ${CHOSEN_MAVEN_PROFILE} -Poozie-package,deploy,run \ + -Dworkflow.source.dir=eu/dnetlib/dhp/oa/graph/stats + +# Show the Oozie-job-ID. +echo -e "\n\nShowing the contents of \"extract-and-run-on-remote-host.log\":\n" +cat ./target/extract-and-run-on-remote-host.log + +# Check oozie workflow status +# oozie job -oozie http://iis-cdh5-test-m3:11000/oozie -info + +# Get the from the previous output and check the logs: +# yarn logs -applicationId diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql index 9697a1dc8..467a98872 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql @@ -4,5 +4,5 @@ -------------------------------------------------------------- -------------------------------------------------------------- -DROP database IF EXISTS ${stats_db_name} CASCADE; -CREATE database ${stats_db_name}; +DROP database IF EXISTS ${stats_db_name} CASCADE; /*EOS*/ +CREATE database ${stats_db_name}; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index bbd7b3bbc..9088ce205 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -5,27 +5,27 @@ ------------------------------------------------------------------------------------------------ CREATE OR REPLACE VIEW ${stats_db_name}.fundref AS SELECT * -FROM ${external_stats_db_name}.fundref; +FROM ${external_stats_db_name}.fundref; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.country AS SELECT * -FROM ${external_stats_db_name}.country; +FROM ${external_stats_db_name}.country; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.countrygdp AS SELECT * -FROM ${external_stats_db_name}.countrygdp; +FROM ${external_stats_db_name}.countrygdp; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.roarmap AS SELECT * -FROM ${external_stats_db_name}.roarmap; +FROM ${external_stats_db_name}.roarmap; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * -FROM ${external_stats_db_name}.rndexpediture; +FROM ${external_stats_db_name}.rndexpediture; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.licenses_normalized AS SELECT * -FROM ${external_stats_db_name}.licenses_normalized; +FROM ${external_stats_db_name}.licenses_normalized; /*EOS*/ ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ @@ -33,23 +33,23 @@ FROM ${external_stats_db_name}.licenses_normalized; ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ create or replace view ${stats_db_name}.usage_stats as -select * from openaire_prod_usage_stats.usage_stats; +select * from openaire_prod_usage_stats.usage_stats; /*EOS*/ create or replace view ${stats_db_name}.downloads_stats as -select * from openaire_prod_usage_stats.downloads_stats; +select * from openaire_prod_usage_stats.downloads_stats; /*EOS*/ create or replace view ${stats_db_name}.pageviews_stats as -select * from openaire_prod_usage_stats.pageviews_stats; +select * from openaire_prod_usage_stats.pageviews_stats; /*EOS*/ create or replace view ${stats_db_name}.views_stats as -select * from openaire_prod_usage_stats.views_stats; +select * from openaire_prod_usage_stats.views_stats; /*EOS*/ ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -- Creation date of the database ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.creation_date purge; +DROP TABLE IF EXISTS ${stats_db_name}.creation_date purge; /*EOS*/ create table ${stats_db_name}.creation_date STORED AS PARQUET as -select date_format(current_date(), 'dd-MM-yyyy') as date; +select date_format(current_date(), 'dd-MM-yyyy') as date; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 638fb0f7a..06600db19 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -10,7 +10,7 @@ SET harvested='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${stats_db_name}.datasource_tmp d, ${stats_db_name}.result_datasources rd - WHERE d.id = rd.datasource); + WHERE d.id = rd.datasource); /*EOS*/ -- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables UPDATE ${stats_db_name}.project_tmp @@ -19,8 +19,8 @@ WHERE project_tmp.id IN (SELECT pr.id FROM ${stats_db_name}.project_results pr, ${stats_db_name}.result r WHERE pr.result = r.id - AND r.type = 'publication'); -DROP TABLE IF EXISTS ${stats_db_name}.stored purge; + AND r.type = 'publication'); /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.stored purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project stored as parquet as SELECT p.id, @@ -63,7 +63,7 @@ FROM ${stats_db_name}.project_tmp p AND r.type = 'publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 GROUP BY pp.id) AS prr2 - ON prr2.id = p.id; + ON prr2.id = p.id; /*EOS*/ UPDATE ${stats_db_name}.publication_tmp SET delayed = 'yes' @@ -73,7 +73,7 @@ WHERE publication_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); + AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ UPDATE ${stats_db_name}.dataset_tmp SET delayed = 'yes' @@ -83,7 +83,7 @@ WHERE dataset_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); + AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ UPDATE ${stats_db_name}.software_tmp SET delayed = 'yes' @@ -93,7 +93,7 @@ WHERE software_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); + AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ UPDATE ${stats_db_name}.otherresearchproduct_tmp SET delayed = 'yes' @@ -103,7 +103,7 @@ WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); + AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, @@ -116,4 +116,4 @@ FROM ${stats_db_name}.result_projects, ${stats_db_name}.project WHERE result_projects.id = result.id AND result.type = 'publication' - AND project.id = result_projects.project; \ No newline at end of file + AND project.id = result_projects.project; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql index 0a1904de7..ff95524be 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql @@ -1,42 +1,42 @@ ------------------------------------------------------------------------------------------------------ -- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.datasource purge; +DROP TABLE IF EXISTS ${stats_db_name}.datasource purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS SELECT * -FROM ${stats_db_name}.datasource_tmp; +FROM ${stats_db_name}.datasource_tmp; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication stored AS parquet AS SELECT * -FROM ${stats_db_name}.publication_tmp; +FROM ${stats_db_name}.publication_tmp; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS SELECT * -FROM ${stats_db_name}.dataset_tmp; +FROM ${stats_db_name}.dataset_tmp; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software purge; +DROP TABLE IF EXISTS ${stats_db_name}.software purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software stored AS parquet AS SELECT * -FROM ${stats_db_name}.software_tmp; +FROM ${stats_db_name}.software_tmp; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS SELECT * -FROM ${stats_db_name}.otherresearchproduct_tmp; +FROM ${stats_db_name}.otherresearchproduct_tmp; /*EOS*/ -DROP TABLE ${stats_db_name}.project_tmp; -DROP TABLE ${stats_db_name}.datasource_tmp; -DROP TABLE ${stats_db_name}.publication_tmp; -DROP TABLE ${stats_db_name}.dataset_tmp; -DROP TABLE ${stats_db_name}.software_tmp; -DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; +DROP TABLE ${stats_db_name}.project_tmp; /*EOS*/ +DROP TABLE ${stats_db_name}.datasource_tmp; /*EOS*/ +DROP TABLE ${stats_db_name}.publication_tmp; /*EOS*/ +DROP TABLE ${stats_db_name}.dataset_tmp; /*EOS*/ +DROP TABLE ${stats_db_name}.software_tmp; /*EOS*/ +DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; /*EOS*/ ---------------------------------------------- -- Re-creating views from final parquet tables @@ -54,4 +54,4 @@ SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.dataset UNION ALL SELECT *, bestlicence AS access_mode -FROM ${stats_db_name}.otherresearchproduct; +FROM ${stats_db_name}.otherresearchproduct; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index 8c1dbdc4d..68a46ded3 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -5,7 +5,7 @@ -- Sources related tables/views ------------------------------------------------------ ------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.publication_sources purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource @@ -16,9 +16,9 @@ LEFT OUTER JOIN ( SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; + WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_sources purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource @@ -29,9 +29,9 @@ LEFT OUTER JOIN ( SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; + WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_sources purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource @@ -42,9 +42,9 @@ LEFT OUTER JOIN ( SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; + WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_sources purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource @@ -55,7 +55,7 @@ LEFT OUTER JOIN ( SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; + WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; /*EOS*/ CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_sources AS SELECT * FROM ${stats_db_name}.publication_sources @@ -64,9 +64,9 @@ SELECT * FROM ${stats_db_name}.dataset_sources UNION ALL SELECT * FROM ${stats_db_name}.software_sources UNION ALL -SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; +SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_orcid purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_orcid purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_orcid STORED AS PARQUET as select distinct res.id, upper(regexp_replace(res.orcid, 'http://orcid.org/' ,'')) as orcid @@ -76,9 +76,9 @@ from ( LATERAL VIEW explode(author) a as auth LATERAL VIEW explode(auth.pid) ap as auth_pid LATERAL VIEW explode(auth.pid.qualifier.classid) apt as author_pid_type - WHERE res.datainfo.deletedbyinference = FALSE and res.datainfo.invisible = FALSE and author_pid_type = 'orcid') as res; + WHERE res.datainfo.deletedbyinference = FALSE and res.datainfo.invisible = FALSE and author_pid_type = 'orcid') as res; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_result purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_result purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_result stored as parquet as select substr(rel.source, 4) as source, substr(rel.target, 4) as target, relclass, subreltype @@ -91,9 +91,9 @@ where reltype='resultResult' and r2.datainfo.deletedbyinference=false and r2.datainfo.invisible = FALSE and r1.resulttype.classname != 'other' and r2.resulttype.classname != 'other' - and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE; + and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_citations_oc purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_citations_oc purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_citations_oc stored as parquet as select substr(target, 4) as id, count(distinct substr(source, 4)) as citations @@ -108,9 +108,9 @@ where relClass='Cites' and rel.datainfo.provenanceaction.classid = 'sysimport:cr and r1.resulttype.classname != 'other' and r2.resulttype.classname != 'other' and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE -group by substr(target, 4); +group by substr(target, 4); /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_references_oc purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_references_oc purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_references_oc stored as parquet as select substr(source, 4) as id, count(distinct substr(target, 4)) as references @@ -125,4 +125,4 @@ where relClass='Cites' and rel.datainfo.provenanceaction.classid = 'sysimport:cr and r1.resulttype.classname != 'other' and r2.resulttype.classname != 'other' and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE -group by substr(source, 4); \ No newline at end of file +group by substr(source, 4); /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index f50c13521..f61c70221 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -5,33 +5,33 @@ -- Licences related tables/views ------------------------------------------------------ ------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.publication_licenses purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.license) instances as licenses -where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_licenses purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.license) instances as licenses -where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_licenses purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.license) instances as licenses -where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_licenses purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.license) instances as licenses -where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_licenses AS SELECT * FROM ${stats_db_name}.publication_licenses @@ -40,15 +40,15 @@ SELECT * FROM ${stats_db_name}.dataset_licenses UNION ALL SELECT * FROM ${stats_db_name}.software_licenses UNION ALL -SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; +SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.organization_pids purge; +DROP TABLE IF EXISTS ${stats_db_name}.organization_pids purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_pids STORED AS PARQUET AS select substr(o.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid -from ${openaire_db_name}.organization o lateral view explode(o.pid) pids as ppid; +from ${openaire_db_name}.organization o lateral view explode(o.pid) pids as ppid; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.organization_sources purge; +DROP TABLE IF EXISTS ${stats_db_name}.organization_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_sources STORED AS PARQUET as SELECT o.id, case when d.id is null then 'other' else o.datasource end as datasource @@ -58,10 +58,10 @@ FROM ( LEFT OUTER JOIN ( SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on o.datasource = d.id; + WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on o.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result -lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute; +lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index ce6b6cc2f..7c618fd0f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -6,7 +6,7 @@ ------------------------------------------------------ ------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.publication_refereed purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_refereed purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed STORED AS PARQUET as with peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed @@ -22,9 +22,9 @@ from ( union all select non_peer_reviewed.* from non_peer_reviewed left join peer_reviewed on peer_reviewed.id=non_peer_reviewed.id - where peer_reviewed.id is null) pr; + where peer_reviewed.id is null) pr; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_refereed purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_refereed purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed STORED AS PARQUET as with peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed @@ -40,9 +40,9 @@ from ( union all select non_peer_reviewed.* from non_peer_reviewed left join peer_reviewed on peer_reviewed.id=non_peer_reviewed.id - where peer_reviewed.id is null) pr; + where peer_reviewed.id is null) pr; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_refereed purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_refereed purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed STORED AS PARQUET as with peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed @@ -58,9 +58,9 @@ from ( union all select non_peer_reviewed.* from non_peer_reviewed left join peer_reviewed on peer_reviewed.id=non_peer_reviewed.id - where peer_reviewed.id is null) pr; + where peer_reviewed.id is null) pr; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_refereed purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_refereed purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed STORED AS PARQUET as with peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed @@ -76,7 +76,7 @@ from ( union all select non_peer_reviewed.* from non_peer_reviewed left join peer_reviewed on peer_reviewed.id=non_peer_reviewed.id - where peer_reviewed.id is null) pr; + where peer_reviewed.id is null) pr; /*EOS*/ CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_refereed as select * from ${stats_db_name}.publication_refereed @@ -85,17 +85,17 @@ select * from ${stats_db_name}.dataset_refereed union all select * from ${stats_db_name}.software_refereed union all -select * from ${stats_db_name}.otherresearchproduct_refereed; +select * from ${stats_db_name}.otherresearchproduct_refereed; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.indi_impact_measures purge; +DROP TABLE IF EXISTS ${stats_db_name}.indi_impact_measures purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_impact_measures STORED AS PARQUET as select substr(id, 4) as id, measures_ids.id impactmetric, cast(measures_ids.unit.value[0] as double) score, cast(measures_ids.unit.value[0] as decimal(6,3)) score_dec, measures_ids.unit.value[1] impact_class from ${openaire_db_name}.result lateral view explode(measures) measures as measures_ids -where measures_ids.id!='views' and measures_ids.id!='downloads'; +where measures_ids.id!='views' and measures_ids.id!='downloads'; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_apc_affiliations purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_apc_affiliations purge; /*EOS*/ create table if not exists ${stats_db_name}.result_apc_affiliations STORED AS PARQUET as select distinct substr(rel.target,4) id, substr(rel.source,4) organization, o.legalname.value name, @@ -104,4 +104,4 @@ rel.properties[1].value apc_currency from ${openaire_db_name}.relation rel join ${openaire_db_name}.organization o on o.id=rel.source join ${openaire_db_name}.result r on r.id=rel.target -where rel.subreltype = 'affiliation' and rel.datainfo.deletedbyinference = false and size(rel.properties)>0; \ No newline at end of file +where rel.subreltype = 'affiliation' and rel.datainfo.deletedbyinference = false and size(rel.properties)>0; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql index 6ed686a05..54743e046 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql @@ -1,25 +1,25 @@ ------------------------------------------- --- Extra tables, mostly used by indicators -DROP TABLE IF EXISTS ${stats_db_name}.result_projectcount purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_projectcount purge; /*EOS*/ create table if not exists ${stats_db_name}.result_projectcount STORED AS PARQUET as select r.id, count(distinct p.id) as count from ${stats_db_name}.result r left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project -group by r.id; +group by r.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_fundercount purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_fundercount purge; /*EOS*/ create table if not exists ${stats_db_name}.result_fundercount STORED AS PARQUET as select r.id, count(distinct p.funder) as count from ${stats_db_name}.result r left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project -group by r.id; +group by r.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_resultcount purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_resultcount purge; /*EOS*/ create table if not exists ${stats_db_name}.project_resultcount STORED AS PARQUET as with rcount as ( @@ -33,17 +33,17 @@ select rcount.pid, sum(case when rcount.type='publication' then rcount.count els sum(case when rcount.type='software' then rcount.count else 0 end) as software, sum(case when rcount.type='other' then rcount.count else 0 end) as other from rcount -group by rcount.pid; +group by rcount.pid; /*EOS*/ -create or replace view ${stats_db_name}.rndexpenditure as select * from stats_ext.rndexpediture; -create or replace view ${stats_db_name}.rndgdpexpenditure as select * from stats_ext.rndgdpexpenditure; -create or replace view ${stats_db_name}.doctoratestudents as select * from stats_ext.doctoratestudents; -create or replace view ${stats_db_name}.totalresearchers as select * from stats_ext.totalresearchers; -create or replace view ${stats_db_name}.totalresearchersft as select * from stats_ext.totalresearchersft; -create or replace view ${stats_db_name}.hrrst as select * from stats_ext.hrrst; -create or replace view ${stats_db_name}.graduatedoctorates as select * from stats_ext.graduatedoctorates; +create or replace view ${stats_db_name}.rndexpenditure as select * from stats_ext.rndexpediture; /*EOS*/ +create or replace view ${stats_db_name}.rndgdpexpenditure as select * from stats_ext.rndgdpexpenditure; /*EOS*/ +create or replace view ${stats_db_name}.doctoratestudents as select * from stats_ext.doctoratestudents; /*EOS*/ +create or replace view ${stats_db_name}.totalresearchers as select * from stats_ext.totalresearchers; /*EOS*/ +create or replace view ${stats_db_name}.totalresearchersft as select * from stats_ext.totalresearchersft; /*EOS*/ +create or replace view ${stats_db_name}.hrrst as select * from stats_ext.hrrst; /*EOS*/ +create or replace view ${stats_db_name}.graduatedoctorates as select * from stats_ext.graduatedoctorates; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_instance purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_instance purge; /*EOS*/ create table if not exists ${stats_db_name}.result_instance stored as parquet as select distinct r.* @@ -51,9 +51,9 @@ from ( select substr(r.id, 4) as id, inst.accessright.classname as accessright, inst.accessright.openaccessroute as accessright_uw, substr(inst.collectedfrom.key, 4) as collectedfrom, substr(inst.hostedby.key, 4) as hostedby, inst.dateofacceptance.value as dateofacceptance, inst.license.value as license, p.qualifier.classname as pidtype, p.value as pid from ${openaire_db_name}.result r lateral view explode(r.instance) instances as inst lateral view outer explode(inst.pid) pids as p) r -join ${stats_db_name}.result res on res.id=r.id; +join ${stats_db_name}.result res on res.id=r.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_apc purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_apc purge; /*EOS*/ create table if not exists ${stats_db_name}.result_apc STORED AS PARQUET as select distinct r.id, r.amount, r.currency @@ -61,6 +61,6 @@ from ( select substr(r.id, 4) as id, cast(inst.processingchargeamount.value as float) as amount, inst.processingchargecurrency.value as currency from ${openaire_db_name}.result r lateral view explode(r.instance) instances as inst) r join ${stats_db_name}.result res on res.id=r.id -where r.amount is not null; +where r.amount is not null; /*EOS*/ -create or replace view ${stats_db_name}.issn_gold_oa_dataset as select * from ${external_stats_db_name}.issn_gold_oa_dataset; \ No newline at end of file +create or replace view ${stats_db_name}.issn_gold_oa_dataset as select * from ${external_stats_db_name}.issn_gold_oa_dataset; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql index b55af13d4..399381b12 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql @@ -3,26 +3,26 @@ ---------------------------------------------------- -- Peer reviewed: -drop table if exists ${stats_db_name}.result_peerreviewed purge; +drop table if exists ${stats_db_name}.result_peerreviewed purge; /*EOS*/ create table IF NOT EXISTS ${stats_db_name}.result_peerreviewed STORED AS PARQUET as select r.id as id, case when doi.doi_from_crossref=1 and grey.grey_lit=0 then true else false end as peer_reviewed from ${stats_db_name}.result r left outer join ${stats_db_name}.indi_pub_doi_from_crossref doi on doi.id=r.id -left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id; +left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id; /*EOS*/ -- Green OA: -drop table if exists ${stats_db_name}.result_greenoa purge; +drop table if exists ${stats_db_name}.result_greenoa purge; /*EOS*/ create table IF NOT EXISTS ${stats_db_name}.result_greenoa STORED AS PARQUET as select r.id, case when green.green_oa=1 then true else false end as green from ${stats_db_name}.result r -left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id; +left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id; /*EOS*/ -- GOLD OA: -drop table if exists ${stats_db_name}.result_gold purge; +drop table if exists ${stats_db_name}.result_gold purge; /*EOS*/ create table IF NOT EXISTS ${stats_db_name}.result_gold STORED AS PARQUET as select r.id, case when gold.is_gold=1 then true else false end as gold from ${stats_db_name}.result r - left outer join ${stats_db_name}.indi_pub_gold_oa gold on gold.id=r.id; \ No newline at end of file + left outer join ${stats_db_name}.indi_pub_gold_oa gold on gold.id=r.id; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql index 7faa91697..1b838ca1b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql @@ -1,6 +1,6 @@ -- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, -- peer reviewed) -drop table if exists ${stats_db_name}.result_tmp; +drop table if exists ${stats_db_name}.result_tmp; /*EOS*/ CREATE TABLE ${stats_db_name}.result_tmp ( id STRING, @@ -20,37 +20,37 @@ CREATE TABLE ${stats_db_name}.result_tmp ( peer_reviewed BOOLEAN, green BOOLEAN, gold BOOLEAN) -clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); +clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); /*EOS*/ insert into ${stats_db_name}.result_tmp select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold FROM ${stats_db_name}.publication r LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ insert into ${stats_db_name}.result_tmp select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold FROM ${stats_db_name}.dataset r LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ insert into ${stats_db_name}.result_tmp select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold FROM ${stats_db_name}.software r LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ insert into ${stats_db_name}.result_tmp select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold FROM ${stats_db_name}.otherresearchproduct r LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ -drop table if exists ${stats_db_name}.result; -drop view if exists ${stats_db_name}.result; -create table ${stats_db_name}.result stored as parquet as select * from ${stats_db_name}.result_tmp; -drop table ${stats_db_name}.result_tmp; \ No newline at end of file +drop table if exists ${stats_db_name}.result; /*EOS*/ +drop view if exists ${stats_db_name}.result; /*EOS*/ +create table ${stats_db_name}.result stored as parquet as select * from ${stats_db_name}.result_tmp; /*EOS*/ +drop table ${stats_db_name}.result_tmp; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index 8e56f98fc..4aa90b1a2 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -5,7 +5,7 @@ -------------------------------------------------------------- -- Publication temporary table -DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_tmp ( id STRING, @@ -22,7 +22,7 @@ CREATE TABLE ${stats_db_name}.publication_tmp abstract BOOLEAN, type STRING ) - clustered by (id) into 100 buckets stored as orc tblproperties ('transactional' = 'true'); + clustered by (id) into 100 buckets stored as orc tblproperties ('transactional' = 'true'); /*EOS*/ INSERT INTO ${stats_db_name}.publication_tmp SELECT substr(p.id, 4) as id, @@ -39,17 +39,17 @@ SELECT substr(p.id, 4) as id, case when size(p.description) > 0 then true else false end as abstract, 'publication' as type from ${openaire_db_name}.publication p -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_classifications purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_concepts purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case @@ -58,9 +58,9 @@ SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept from ${openaire_db_name}.publication p LATERAL VIEW explode(p.context) contexts as context -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_datasources purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_datasources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource @@ -71,44 +71,44 @@ FROM ( LEFT OUTER JOIN ( SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; + WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_languages purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_languages STORED AS PARQUET AS select substr(p.id, 4) as id, p.language.classname as language FROM ${openaire_db_name}.publication p -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_oids purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.originalid) oids AS ids -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_pids purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.pid) pids AS ppid -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_topics purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_topics STORED AS PARQUET as select substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.publication_citations purge; +DROP TABLE IF EXISTS ${stats_db_name}.publication_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_citations STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" - and p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; \ No newline at end of file + and p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql index 66620ac38..adcf23b7a 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql @@ -5,7 +5,7 @@ from ${stats_db_name}.result r select rl.id, sum(case when rl.type like 'CC%' then 1 else 0 end) as count from ${stats_db_name}.result_licenses rl group by rl.id -) rln on rln.id=r.id; +) rln on rln.id=r.id; /*EOS*/ create table ${observatory_db_name}.result_affiliated_country stored as parquet as @@ -35,7 +35,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_year stored as parquet as @@ -65,7 +65,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; /*EOS*/ create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as @@ -95,7 +95,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as @@ -127,7 +127,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as select @@ -158,7 +158,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_organization stored as parquet as select @@ -187,7 +187,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as select @@ -216,7 +216,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_funder stored as parquet as select @@ -247,7 +247,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; /*EOS*/ create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as select @@ -278,7 +278,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_country stored as parquet as select @@ -309,7 +309,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_year stored as parquet as select @@ -340,7 +340,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; /*EOS*/ create table ${observatory_db_name}.result_deposited_year_country stored as parquet as @@ -372,7 +372,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_datasource stored as parquet as select @@ -403,7 +403,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as select @@ -434,7 +434,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_organization stored as parquet as select @@ -465,7 +465,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as select @@ -496,7 +496,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_funder stored as parquet as select @@ -529,7 +529,7 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; /*EOS*/ create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as select @@ -562,4 +562,4 @@ from ${stats_db_name}.result r left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, - cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index 0384de4ec..1ff4beadb 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -5,7 +5,7 @@ ------------------------------------------------------ -- Dataset temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_tmp ( @@ -23,7 +23,7 @@ CREATE TABLE ${stats_db_name}.dataset_tmp abstract BOOLEAN, type STRING ) - clustered by (id) into 100 buckets stored AS orc tblproperties ('transactional' = 'true'); + clustered by (id) into 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ INSERT INTO ${stats_db_name}.dataset_tmp SELECT substr(d.id, 4) AS id, @@ -40,26 +40,26 @@ SELECT substr(d.id, 4) AS id, CASE WHEN SIZE(d.description) > 0 THEN TRUE ELSE FALSE end AS abstract, 'dataset' AS type FROM ${openaire_db_name}.dataset d -WHERE d.datainfo.deletedbyinference = FALSE and d.datainfo.invisible=false; +WHERE d.datainfo.deletedbyinference = FALSE and d.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_citations purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_citations STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.dataset d LATERAL VIEW explode(d.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" - and d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; + and d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_classifications purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_concepts purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case @@ -68,9 +68,9 @@ SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_datasources purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_datasources STORED AS PARQUET AS SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource @@ -82,35 +82,35 @@ FROM ( LEFT OUTER JOIN ( SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id; + WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_languages purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_languages STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.dataset p -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_oids purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids AS ids -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_pids purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.pid) pids AS ppid -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.dataset_topics purge; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_topics STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; \ No newline at end of file +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index d8f4d65e4..426d53773 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -5,7 +5,7 @@ -------------------------------------------------------- -- Software temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.software_tmp purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_tmp purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_tmp ( id STRING, @@ -22,7 +22,7 @@ CREATE TABLE ${stats_db_name}.software_tmp abstract BOOLEAN, type STRING ) - clustered by (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); + clustered by (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ INSERT INTO ${stats_db_name}.software_tmp SELECT substr(s.id, 4) as id, @@ -39,24 +39,24 @@ SELECT substr(s.id, 4) as id, CASE WHEN SIZE(s.description) > 0 THEN TRUE ELSE FALSE END AS abstract, 'software' as type from ${openaire_db_name}.software s -where s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; +where s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_citations purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_citations STORED AS PARQUET AS SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.software s LATERAL VIEW explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" - and s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; + and s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_classifications purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ CREATE TABLE ${stats_db_name}.software_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case @@ -65,9 +65,9 @@ SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.context) contexts AS context -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_datasources purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_datasources STORED AS PARQUET AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource @@ -79,35 +79,35 @@ FROM ( LEFT OUTER JOIN ( SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id; + WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_languages purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_languages STORED AS PARQUET AS select substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.software p -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_oids purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids AS ids -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_pids purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.pid) pids AS ppid -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.software_topics purge; +DROP TABLE IF EXISTS ${stats_db_name}.software_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_topics STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; \ No newline at end of file +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index fae0fbb63..6b5adff9d 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -5,7 +5,7 @@ -------------------------------------------------------------------------------- -- Otherresearchproduct temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp ( @@ -22,7 +22,7 @@ CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp source STRING, abstract BOOLEAN, type STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); +) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ INSERT INTO ${stats_db_name}.otherresearchproduct_tmp SELECT substr(o.id, 4) AS id, @@ -39,23 +39,23 @@ SELECT substr(o.id, 4) AS id, CASE WHEN SIZE(o.description) > 0 THEN TRUE ELSE FALSE END AS abstract, 'other' AS type FROM ${openaire_db_name}.otherresearchproduct o -WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible=false; +WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible=false; /*EOS*/ -- Otherresearchproduct_citations -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_citations purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_citations STORED AS PARQUET AS SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" - and o.datainfo.deletedbyinference = false and o.datainfo.invisible=false; + and o.datainfo.deletedbyinference = false and o.datainfo.invisible=false; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_concepts purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case @@ -63,9 +63,9 @@ SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+$' then concat(contexts.context.id, '::other') when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts AS context -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_datasources purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources STORED AS PARQUET AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource @@ -74,32 +74,32 @@ FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) A where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false) p LEFT OUTER JOIN(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d - WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; + WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_languages purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_languages STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.otherresearchproduct p -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_oids purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_pids purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_topics purge; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_topics STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; \ No newline at end of file +where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index 165f77946..75ec7d69c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -3,38 +3,38 @@ -- Project table/view and Project related tables/views ------------------------------------------------------ ------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.project_oids purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids -where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false; +where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_organizations purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_organizations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_organizations STORED AS PARQUET AS SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype = 'projectOrganization' and r.source like '40|%' - and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; + and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_results purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_results purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_results STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result, r.datainfo.provenanceaction.classname as provenance FROM ${openaire_db_name}.relation r WHERE r.reltype = 'resultProject' and r.target like '40|%' - and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; + and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_classification purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_classification purge; /*EOS*/ create table ${stats_db_name}.project_classification STORED AS PARQUET as select substr(p.id, 4) as id, class.h2020programme.code, class.level1, class.level2, class.level3 from ${openaire_db_name}.project p lateral view explode(p.h2020classification) classifs as class -where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false and class.h2020programme is not null; +where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false and class.h2020programme is not null; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_tmp purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_tmp purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_tmp ( @@ -61,7 +61,7 @@ CREATE TABLE ${stats_db_name}.project_tmp totalcost FLOAT, fundedamount FLOAT, currency STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); +) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, @@ -88,18 +88,18 @@ SELECT substr(p.id, 4) AS id, p.fundedamount AS fundedamount, p.currency.value AS currency FROM ${openaire_db_name}.project p -WHERE p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +WHERE p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.funder purge; +DROP TABLE IF EXISTS ${stats_db_name}.funder purge; /*EOS*/ create table ${stats_db_name}.funder STORED AS PARQUET as select distinct xpath_string(fund, '//funder/id') as id, xpath_string(fund, '//funder/name') as name, xpath_string(fund, '//funder/shortname') as shortname, xpath_string(fundingtree[0].value, '//funder/jurisdiction') as country -from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fundingtree as fund; +from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fundingtree as fund; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_organization_contribution purge; +DROP TABLE IF EXISTS ${stats_db_name}.project_organization_contribution purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_organization_contribution STORED AS PARQUET AS SELECT distinct substr(r.source, 4) AS project, substr(r.target, 4) AS organization, @@ -107,4 +107,4 @@ properties[0].value contribution, properties[1].value currency from ${openaire_db_name}.relation r LATERAL VIEW explode (r.properties) properties where properties[0].key='contribution' and r.reltype = 'projectOrganization' and r.source like '40|%' -and properties[0].value>0.0 and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; \ No newline at end of file +and properties[0].value>0.0 and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index eb16a161e..2cc7c13c4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -16,7 +16,7 @@ SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.dataset_tmp UNION ALL SELECT *, bestlicence AS access_mode -FROM ${stats_db_name}.otherresearchproduct_tmp; +FROM ${stats_db_name}.otherresearchproduct_tmp; /*EOS*/ -- Views on final tables CREATE OR REPLACE VIEW ${stats_db_name}.result_datasources AS @@ -30,7 +30,7 @@ SELECT * FROM ${stats_db_name}.dataset_datasources UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_datasources; +FROM ${stats_db_name}.otherresearchproduct_datasources; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_citations AS SELECT * @@ -43,7 +43,7 @@ SELECT * FROM ${stats_db_name}.dataset_citations UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_citations; +FROM ${stats_db_name}.otherresearchproduct_citations; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_classifications AS SELECT * @@ -56,7 +56,7 @@ SELECT * FROM ${stats_db_name}.dataset_classifications UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_classifications; +FROM ${stats_db_name}.otherresearchproduct_classifications; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_concepts AS SELECT * @@ -69,7 +69,7 @@ SELECT * FROM ${stats_db_name}.dataset_concepts UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_concepts; +FROM ${stats_db_name}.otherresearchproduct_concepts; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_languages AS SELECT * @@ -82,7 +82,7 @@ SELECT * FROM ${stats_db_name}.dataset_languages UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_languages; +FROM ${stats_db_name}.otherresearchproduct_languages; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_oids AS SELECT * @@ -95,7 +95,7 @@ SELECT * FROM ${stats_db_name}.dataset_oids UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_oids; +FROM ${stats_db_name}.otherresearchproduct_oids; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_pids AS SELECT * @@ -108,7 +108,7 @@ SELECT * FROM ${stats_db_name}.dataset_pids UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_pids; +FROM ${stats_db_name}.otherresearchproduct_pids; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.result_topics AS SELECT * @@ -121,9 +121,9 @@ SELECT * FROM ${stats_db_name}.dataset_topics UNION ALL SELECT * -FROM ${stats_db_name}.otherresearchproduct_topics; +FROM ${stats_db_name}.otherresearchproduct_topics; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_fos purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_fos purge; /*EOS*/ create table ${stats_db_name}.result_fos stored as parquet as with @@ -133,22 +133,22 @@ with select lvl1.id, lvl1.topic as lvl1, lvl2.topic as lvl2, lvl3.topic as lvl3 from lvl1 join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2) - join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4); + join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4); /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge; /*EOS*/ CREATE TABLE ${stats_db_name}.result_organization STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype = 'resultOrganization' and r.target like '50|%' - and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; + and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.result_projects purge; +DROP TABLE IF EXISTS ${stats_db_name}.result_projects purge; /*EOS*/ CREATE TABLE ${stats_db_name}.result_projects STORED AS PARQUET AS select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id = pr.result - JOIN ${stats_db_name}.project_tmp p ON p.id = pr.id; + JOIN ${stats_db_name}.project_tmp p ON p.id = pr.id; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 07204db0c..3f40dbb21 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -5,7 +5,7 @@ -- Datasource table/view and Datasource related tables/views ------------------------------------------------------------ ------------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_tmp ( @@ -22,7 +22,7 @@ CREATE TABLE ${stats_db_name}.datasource_tmp `compatibility` STRING, issn_printed STRING, issn_online STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); +) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ -- Insert statement that takes into account the piwik_id of the openAIRE graph INSERT INTO ${stats_db_name}.datasource_tmp @@ -46,14 +46,14 @@ FROM ${openaire_db_name}.datasource d1 LATERAL VIEW EXPLODE(originalid) temp AS originalidd WHERE originalidd like "piwik:%") AS d2 ON d1.id = d2.id -WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; +WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; /*EOS*/ -- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. -- Creating a temporary dual table that will be removed after the following insert -CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); +CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); /*EOS*/ -INSERT INTO ${stats_db_name}.dual VALUES ('X'); +INSERT INTO ${stats_db_name}.dual VALUES ('X'); /*EOS*/ INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`, `issn_printed`, `issn_online`) @@ -71,42 +71,42 @@ SELECT 'other', null, null FROM ${stats_db_name}.dual -WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name = 'Unknown Repository'); -DROP TABLE ${stats_db_name}.dual; +WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name = 'Unknown Repository'); /*EOS*/ +DROP TABLE ${stats_db_name}.dual; /*EOS*/ -UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository'; -UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1'; +UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository'; /*EOS*/ +UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1'; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages -where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; +where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_oids STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids -where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; +where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_organizations STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r -WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; +WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; /*EOS*/ -- datasource sources: -- where the datasource info have been collected from. -DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; /*EOS*/ create table if not exists ${stats_db_name}.datasource_sources STORED AS PARQUET AS select substr(d.id, 4) as id, substr(cf.key, 4) as datasource from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf -where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; +where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result -FROM ${stats_db_name}.result_datasources; +FROM ${stats_db_name}.result_datasources; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 19d301e27..afde8160e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -3,7 +3,7 @@ -- Organization table/view and Organization related tables/views ---------------------------------------------------------------- ---------------------------------------------------------------- -DROP TABLE IF EXISTS ${stats_db_name}.organization purge; +DROP TABLE IF EXISTS ${stats_db_name}.organization purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization STORED AS PARQUET AS SELECT substr(o.id, 4) as id, @@ -11,12 +11,12 @@ SELECT substr(o.id, 4) as id, o.legalshortname.value as legalshortname, o.country.classid as country FROM ${openaire_db_name}.organization o -WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible = FALSE; +WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible = FALSE; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource -FROM ${stats_db_name}.datasource_organizations; +FROM ${stats_db_name}.datasource_organizations; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.organization_projects AS SELECT id AS project, organization as id -FROM ${stats_db_name}.project_organizations; \ No newline at end of file +FROM ${stats_db_name}.project_organizations; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 813fffcf9..1460477ae 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -154,180 +154,354 @@ - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - + + yarn + cluster + Step1 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + + + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step2 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step3 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step4 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step5 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step6 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step7 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step8 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step9 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - external_stats_db_name=${external_stats_db_name} - + + yarn + cluster + Step10 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - external_stats_db_name=${external_stats_db_name} - + + yarn + cluster + Step11 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step12 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step13 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step14 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step15 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - external_stats_db_name=${external_stats_db_name} - + + yarn + cluster + Step15_5 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + @@ -379,23 +553,45 @@ - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step16_1-definitions + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + yarn + cluster + Step16_5 + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + @@ -461,12 +657,23 @@ - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - observatory_db_name=${observatory_db_name} - + + yarn + cluster + Step21-createObservatoryDB + eu.dnetlib.dhp.oozie.RunSQLSparkJob + dhp-stats-update-${projectVersion}.jar + + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + ${sparkClusterOpts} + ${sparkResourceOpts} + ${sparkApplicationOpts} + + --hiveMetastoreUris${hive_metastore_uris} + --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql + --stats_db_name${stats_db_name} + --openaire_db_name${openaire_db_name} + From 0b897f2f667a2bbbf9cb57af5ea4651c01a438c6 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Tue, 16 Apr 2024 18:17:54 +0300 Subject: [PATCH 002/239] Fix and add missing "DROP TABLE" statements, in "dhp-stats-update" sql-scripts. --- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql | 3 ++- .../stats/oozie_app/scripts/step16-createIndicatorsTables.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql | 4 +++- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql | 2 ++ 5 files changed, 11 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 06600db19..f4d06587b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -20,7 +20,8 @@ WHERE project_tmp.id IN (SELECT pr.id ${stats_db_name}.result r WHERE pr.result = r.id AND r.type = 'publication'); /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.stored purge; /*EOS*/ + +DROP TABLE IF EXISTS ${stats_db_name}.project purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project stored as parquet as SELECT p.id, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 70cde6481..c2231d54c 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -965,6 +965,8 @@ select allresults.ri_initiative, result_findable.no_result_findable/allresults.n from allresults join result_findable on result_findable.ri_initiative=allresults.ri_initiative; /*EOS*/ +drop table if exists ${stats_db_name}.indi_pub_publicly_funded purge; /*EOS*/ + create table if not exists ${stats_db_name}.indi_pub_publicly_funded stored as parquet as with org_names_pids as (select org.id,name, pid from ${stats_db_name}.organization org diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index 426d53773..0cffff052 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -58,6 +58,8 @@ FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.software_concepts purge; /*EOS*/ + CREATE TABLE ${stats_db_name}.software_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index 6b5adff9d..d742bcc2a 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -50,6 +50,8 @@ FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" and o.datainfo.deletedbyinference = false and o.datainfo.invisible=false; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_classifications purge; /*EOS*/ + CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype @@ -72,7 +74,7 @@ SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS dataso FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false) p - LEFT OUTER JOIN(SELECT substr(d.id, 4) id + LEFT OUTER JOIN (SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 3f40dbb21..23fa743f9 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -51,6 +51,8 @@ WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; /* -- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. -- Creating a temporary dual table that will be removed after the following insert +DROP TABLE IF EXISTS ${stats_db_name}.dual purge; /*EOS*/ + CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); /*EOS*/ INSERT INTO ${stats_db_name}.dual VALUES ('X'); /*EOS*/ From ca091c0f1e578d57467d9e1c496050420e6d8175 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 17 Apr 2024 14:03:59 +0300 Subject: [PATCH 003/239] dhp-stats-update: - Fix not passing some parameters to some Spark actions. - Allow the workflow to run up to Step7. The first 7 steps seem to work out of the box. --- dhp-workflows/dhp-stats-update/runOozieWorkfow.sh | 2 +- .../dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml | 11 +++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh b/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh index a4825a3ae..2f2fc29d5 100755 --- a/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh +++ b/dhp-workflows/dhp-stats-update/runOozieWorkfow.sh @@ -17,4 +17,4 @@ cat ./target/extract-and-run-on-remote-host.log # oozie job -oozie http://iis-cdh5-test-m3:11000/oozie -info # Get the from the previous output and check the logs: -# yarn logs -applicationId +# yarn logs -applicationId application_ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 1460477ae..c2c6f9822 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -171,8 +171,7 @@ --stats_db_name${stats_db_name} --openaire_db_name${openaire_db_name} - - + @@ -304,7 +303,8 @@ --stats_db_name${stats_db_name} --openaire_db_name${openaire_db_name} - + + @@ -369,6 +369,7 @@ --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql --stats_db_name${stats_db_name} --openaire_db_name${openaire_db_name} + --external_stats_db_name${external_stats_db_name} @@ -391,6 +392,7 @@ --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql --stats_db_name${stats_db_name} --openaire_db_name${openaire_db_name} + --external_stats_db_name${external_stats_db_name} @@ -501,6 +503,7 @@ --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql --stats_db_name${stats_db_name} --openaire_db_name${openaire_db_name} + --external_stats_db_name${external_stats_db_name} @@ -672,7 +675,7 @@ --hiveMetastoreUris${hive_metastore_uris} --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql --stats_db_name${stats_db_name} - --openaire_db_name${openaire_db_name} + --observatory_db_name${observatory_db_name} From 6f2ebb2a52fa99735e92867ffdfbd701926f3dd8 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Thu, 18 Apr 2024 15:35:03 +0300 Subject: [PATCH 004/239] Revert Step8 and Step11 to use Hive again, since their "UPDATE" statements are not supported by Spark. --- .../graph/stats/oozie_app/scripts/step11.sql | 18 +++---- .../graph/stats/oozie_app/scripts/step8.sql | 37 +++++++------- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 49 ++++++++++++++----- 3 files changed, 64 insertions(+), 40 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index f4d06587b..207c1b124 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -10,7 +10,7 @@ SET harvested='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${stats_db_name}.datasource_tmp d, ${stats_db_name}.result_datasources rd - WHERE d.id = rd.datasource); /*EOS*/ + WHERE d.id = rd.datasource); -- /*EOS*/ -- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables UPDATE ${stats_db_name}.project_tmp @@ -19,9 +19,9 @@ WHERE project_tmp.id IN (SELECT pr.id FROM ${stats_db_name}.project_results pr, ${stats_db_name}.result r WHERE pr.result = r.id - AND r.type = 'publication'); /*EOS*/ + AND r.type = 'publication'); -- /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.project purge; -- /*EOS*/ CREATE TABLE ${stats_db_name}.project stored as parquet as SELECT p.id, @@ -64,7 +64,7 @@ FROM ${stats_db_name}.project_tmp p AND r.type = 'publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 GROUP BY pp.id) AS prr2 - ON prr2.id = p.id; /*EOS*/ + ON prr2.id = p.id; -- /*EOS*/ UPDATE ${stats_db_name}.publication_tmp SET delayed = 'yes' @@ -74,7 +74,7 @@ WHERE publication_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ + AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ UPDATE ${stats_db_name}.dataset_tmp SET delayed = 'yes' @@ -84,7 +84,7 @@ WHERE dataset_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ + AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ UPDATE ${stats_db_name}.software_tmp SET delayed = 'yes' @@ -94,7 +94,7 @@ WHERE software_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ + AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ UPDATE ${stats_db_name}.otherresearchproduct_tmp SET delayed = 'yes' @@ -104,7 +104,7 @@ WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id ${stats_db_name}.project_tmp p WHERE r.id = pr.result AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); /*EOS*/ + AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, @@ -117,4 +117,4 @@ FROM ${stats_db_name}.result_projects, ${stats_db_name}.project WHERE result_projects.id = result.id AND result.type = 'publication' - AND project.id = result_projects.project; /*EOS*/ \ No newline at end of file + AND project.id = result_projects.project; -- /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 23fa743f9..07e19d68b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -5,7 +5,7 @@ -- Datasource table/view and Datasource related tables/views ------------------------------------------------------------ ------------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge; -- /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_tmp ( @@ -23,6 +23,7 @@ CREATE TABLE ${stats_db_name}.datasource_tmp issn_printed STRING, issn_online STRING ) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ +) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); -- /*EOS*/ -- Insert statement that takes into account the piwik_id of the openAIRE graph INSERT INTO ${stats_db_name}.datasource_tmp @@ -46,16 +47,16 @@ FROM ${openaire_db_name}.datasource d1 LATERAL VIEW EXPLODE(originalid) temp AS originalidd WHERE originalidd like "piwik:%") AS d2 ON d1.id = d2.id -WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; /*EOS*/ +WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; -- /*EOS*/ -- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. -- Creating a temporary dual table that will be removed after the following insert -DROP TABLE IF EXISTS ${stats_db_name}.dual purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.dual purge; -- /*EOS*/ -CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); /*EOS*/ +CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); -- /*EOS*/ -INSERT INTO ${stats_db_name}.dual VALUES ('X'); /*EOS*/ +INSERT INTO ${stats_db_name}.dual VALUES ('X'); -- /*EOS*/ INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`, `issn_printed`, `issn_online`) @@ -73,42 +74,42 @@ SELECT 'other', null, null FROM ${stats_db_name}.dual -WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name = 'Unknown Repository'); /*EOS*/ -DROP TABLE ${stats_db_name}.dual; /*EOS*/ +WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name = 'Unknown Repository'); -- /*EOS*/ +DROP TABLE ${stats_db_name}.dual; -- /*EOS*/ -UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository'; /*EOS*/ -UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1'; /*EOS*/ +UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository'; -- /*EOS*/ +UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1'; -- /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; -- /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages -where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ +where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; -- /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; -- /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_oids STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids -where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ +where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; -- /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; -- /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_organizations STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r -WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; /*EOS*/ +WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; -- /*EOS*/ -- datasource sources: -- where the datasource info have been collected from. -DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; -- /*EOS*/ create table if not exists ${stats_db_name}.datasource_sources STORED AS PARQUET AS select substr(d.id, 4) as id, substr(cf.key, 4) as datasource from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf -where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; /*EOS*/ +where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; -- /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result -FROM ${stats_db_name}.result_datasources; /*EOS*/ +FROM ${stats_db_name}.result_datasources; -- /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index c2c6f9822..5c255a488 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -308,7 +308,7 @@ - + + + + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + + + @@ -375,7 +386,7 @@ - + + + + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + external_stats_db_name=${external_stats_db_name} + + + From d46b78b65949a58447821e050ed0179a8173c404 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Thu, 18 Apr 2024 15:40:27 +0300 Subject: [PATCH 005/239] dhp-stats-update: - Set Steps 2-7 and 9 to limit the amount of files generated by Spark, from 8000, down to 100, to improve file-transfer and querying performance. - Allow the workflow to run up to Step10. The Step11 seems to have some issues even when using hive-action. --- .../oa/graph/stats/oozie_app/scripts/step2.sql | 16 ++++++++-------- .../oa/graph/stats/oozie_app/scripts/step3.sql | 16 ++++++++-------- .../oa/graph/stats/oozie_app/scripts/step4.sql | 16 ++++++++-------- .../oa/graph/stats/oozie_app/scripts/step5.sql | 16 ++++++++-------- .../oa/graph/stats/oozie_app/scripts/step6.sql | 12 ++++++------ .../oa/graph/stats/oozie_app/scripts/step7.sql | 9 ++++----- .../oa/graph/stats/oozie_app/scripts/step9.sql | 2 +- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 6 +++--- 8 files changed, 46 insertions(+), 47 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index 4aa90b1a2..8ec663573 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -44,7 +44,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.publication_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_classifications STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, instancetype.classname as type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -52,7 +52,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.publication_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_concepts STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, case +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id when contexts.context.id RLIKE '^[^::]+::[^::]+$' then concat(contexts.context.id, '::other') when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept @@ -63,7 +63,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.publication_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_datasources STORED AS PARQUET as -SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +SELECT /*+ COALESCE(100) */ p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource from ${openaire_db_name}.publication p lateral view explode(p.instance) instances as instance @@ -76,14 +76,14 @@ FROM ( DROP TABLE IF EXISTS ${stats_db_name}.publication_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_languages STORED AS PARQUET AS -select substr(p.id, 4) as id, p.language.classname as language +select /*+ COALESCE(100) */ substr(p.id, 4) as id, p.language.classname as language FROM ${openaire_db_name}.publication p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.publication_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_oids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, oids.ids AS oid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -91,7 +91,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.publication_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_pids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -99,7 +99,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.publication_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_topics STORED AS PARQUET as -select substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic +select /*+ COALESCE(100) */ substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.subject) subjects AS subject where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -107,7 +107,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.publication_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.publication_citations STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index 1ff4beadb..ebedb5dc5 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -45,7 +45,7 @@ WHERE d.datainfo.deletedbyinference = FALSE and d.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.dataset_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_citations STORED AS PARQUET AS -SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites +SELECT /*+ COALESCE(100) */ substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.dataset d LATERAL VIEW explode(d.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" @@ -54,7 +54,7 @@ WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != DROP TABLE IF EXISTS ${stats_db_name}.dataset_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_classifications STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, instancetype.classname AS type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -62,7 +62,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.dataset_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_concepts STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, case +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id when contexts.context.id RLIKE '^[^::]+::[^::]+$' then concat(contexts.context.id, '::other') when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept @@ -73,7 +73,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.dataset_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_datasources STORED AS PARQUET AS -SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource +SELECT /*+ COALESCE(100) */ p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM ( SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) AS datasource FROM ${openaire_db_name}.dataset p @@ -87,14 +87,14 @@ FROM ( DROP TABLE IF EXISTS ${stats_db_name}.dataset_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_languages STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, p.language.classname AS language +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.dataset p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.dataset_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_oids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, oids.ids AS oid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -102,7 +102,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.dataset_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_pids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -110,7 +110,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.dataset_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.dataset_topics STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.subject) subjects AS subject where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index 0cffff052..4957d8d2f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -44,7 +44,7 @@ where s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.software_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_citations STORED AS PARQUET AS -SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites +SELECT /*+ COALESCE(100) */ substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.software s LATERAL VIEW explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" @@ -53,7 +53,7 @@ where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != DROP TABLE IF EXISTS ${stats_db_name}.software_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_classifications STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, instancetype.classname AS type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -61,7 +61,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.software_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_concepts STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, case +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id when contexts.context.id RLIKE '^[^::]+::[^::]+$' then concat(contexts.context.id, '::other') when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept @@ -72,7 +72,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.software_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_datasources STORED AS PARQUET AS -SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource +SELECT /*+ COALESCE(100) */ p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource FROM ( SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource FROM ${openaire_db_name}.software p @@ -86,14 +86,14 @@ FROM ( DROP TABLE IF EXISTS ${stats_db_name}.software_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_languages STORED AS PARQUET AS -select substr(p.id, 4) AS id, p.language.classname AS language +select /*+ COALESCE(100) */ substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.software p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.software_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_oids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, oids.ids AS oid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -101,7 +101,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.software_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_pids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ @@ -109,7 +109,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.software_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.software_topics STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.subject) subjects AS subject where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index d742bcc2a..820ec4395 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -45,7 +45,7 @@ WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_citations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_citations STORED AS PARQUET AS -SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites +SELECT /*+ COALESCE(100) */ substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" and o.datainfo.deletedbyinference = false and o.datainfo.invisible=false; /*EOS*/ @@ -53,14 +53,14 @@ WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_classifications purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, instancetype.classname AS type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_concepts purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, case +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id when contexts.context.id RLIKE '^[^::]+::[^::]+$' then concat(contexts.context.id, '::other') when contexts.context.id RLIKE '^[^::]+$' then concat(contexts.context.id, '::other::other') END as concept @@ -70,7 +70,7 @@ where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_datasources purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources STORED AS PARQUET AS -SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource +SELECT /*+ COALESCE(100) */ p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false) p @@ -81,27 +81,27 @@ FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) A DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_languages STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, p.language.classname AS language +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.otherresearchproduct p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_oids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, oids.ids AS oid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_pids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_pids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_topics purge; /*EOS*/ CREATE TABLE ${stats_db_name}.otherresearchproduct_topics STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index 75ec7d69c..d2688ec07 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -6,14 +6,14 @@ DROP TABLE IF EXISTS ${stats_db_name}.project_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_oids STORED AS PARQUET AS -SELECT substr(p.id, 4) AS id, oids.ids AS oid +SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.project_organizations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_organizations STORED AS PARQUET AS -SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization +SELECT /*+ COALESCE(100) */ substr(r.source, 4) AS id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype = 'projectOrganization' and r.source like '40|%' and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; /*EOS*/ @@ -21,7 +21,7 @@ WHERE r.reltype = 'projectOrganization' and r.source like '40|%' DROP TABLE IF EXISTS ${stats_db_name}.project_results purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_results STORED AS PARQUET AS -SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result, r.datainfo.provenanceaction.classname as provenance +SELECT /*+ COALESCE(100) */ substr(r.target, 4) AS id, substr(r.source, 4) AS result, r.datainfo.provenanceaction.classname as provenance FROM ${openaire_db_name}.relation r WHERE r.reltype = 'resultProject' and r.target like '40|%' and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; /*EOS*/ @@ -29,7 +29,7 @@ WHERE r.reltype = 'resultProject' and r.target like '40|%' DROP TABLE IF EXISTS ${stats_db_name}.project_classification purge; /*EOS*/ create table ${stats_db_name}.project_classification STORED AS PARQUET as -select substr(p.id, 4) as id, class.h2020programme.code, class.level1, class.level2, class.level3 +select /*+ COALESCE(100) */ substr(p.id, 4) as id, class.h2020programme.code, class.level1, class.level2, class.level3 from ${openaire_db_name}.project p lateral view explode(p.h2020classification) classifs as class where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false and class.h2020programme is not null; /*EOS*/ @@ -93,7 +93,7 @@ WHERE p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EO DROP TABLE IF EXISTS ${stats_db_name}.funder purge; /*EOS*/ create table ${stats_db_name}.funder STORED AS PARQUET as -select distinct xpath_string(fund, '//funder/id') as id, +select /*+ COALESCE(100) */ distinct xpath_string(fund, '//funder/id') as id, xpath_string(fund, '//funder/name') as name, xpath_string(fund, '//funder/shortname') as shortname, xpath_string(fundingtree[0].value, '//funder/jurisdiction') as country @@ -102,7 +102,7 @@ from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fun DROP TABLE IF EXISTS ${stats_db_name}.project_organization_contribution purge; /*EOS*/ CREATE TABLE ${stats_db_name}.project_organization_contribution STORED AS PARQUET AS -SELECT distinct substr(r.source, 4) AS project, substr(r.target, 4) AS organization, +SELECT /*+ COALESCE(100) */ distinct substr(r.source, 4) AS project, substr(r.target, 4) AS organization, properties[0].value contribution, properties[1].value currency from ${openaire_db_name}.relation r LATERAL VIEW explode (r.properties) properties diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 2cc7c13c4..f3ab52004 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -130,7 +130,7 @@ with lvl1 as (select id, topic from ${stats_db_name}.result_topics where topic like '__ %' and type='Fields of Science and Technology classification'), lvl2 as (select id, topic from ${stats_db_name}.result_topics where topic like '____ %' and type='Fields of Science and Technology classification'), lvl3 as (select id, topic from ${stats_db_name}.result_topics where topic like '______ %' and type='Fields of Science and Technology classification') -select lvl1.id, lvl1.topic as lvl1, lvl2.topic as lvl2, lvl3.topic as lvl3 +select /*+ COALESCE(100) */ lvl1.id, lvl1.topic as lvl1, lvl2.topic as lvl2, lvl3.topic as lvl3 from lvl1 join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2) join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4); /*EOS*/ @@ -138,7 +138,7 @@ from lvl1 DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge; /*EOS*/ CREATE TABLE ${stats_db_name}.result_organization STORED AS PARQUET AS -SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization +SELECT /*+ COALESCE(100) */ substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype = 'resultOrganization' and r.target like '50|%' @@ -147,8 +147,7 @@ WHERE r.reltype = 'resultOrganization' DROP TABLE IF EXISTS ${stats_db_name}.result_projects purge; /*EOS*/ CREATE TABLE ${stats_db_name}.result_projects STORED AS PARQUET AS -select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance +select /*+ COALESCE(100) */ pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id = pr.result - JOIN ${stats_db_name}.project_tmp p ON p.id = pr.id; /*EOS*/ - + JOIN ${stats_db_name}.project_tmp p ON p.id = pr.id; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index afde8160e..1d76b89a6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS ${stats_db_name}.organization purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization STORED AS PARQUET AS -SELECT substr(o.id, 4) as id, +SELECT /*+ COALESCE(100) */ substr(o.id, 4) as id, o.legalname.value as name, o.legalshortname.value as legalshortname, o.country.classid as country diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 5c255a488..37d837e76 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -303,8 +303,7 @@ --stats_db_name${stats_db_name} --openaire_db_name${openaire_db_name} - - + @@ -382,7 +381,8 @@ --openaire_db_name${openaire_db_name} --external_stats_db_name${external_stats_db_name} - + + From 2616971e2bc23ee172ffc64e0a6730ed1356f9a8 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Thu, 18 Apr 2024 16:18:16 +0300 Subject: [PATCH 006/239] dhp-stats-update: remove leftover duplicate line --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 07e19d68b..90c3ebef6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -22,7 +22,6 @@ CREATE TABLE ${stats_db_name}.datasource_tmp `compatibility` STRING, issn_printed STRING, issn_online STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ ) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); -- /*EOS*/ -- Insert statement that takes into account the piwik_id of the openAIRE graph From ed052a3476bf5c8980412b0d1b8387491d761ab2 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 6 May 2024 16:08:33 +0200 Subject: [PATCH 007/239] job for the population of the oai database --- .../dhp/oa/oaipmh/IrishOaiExporterJob.java | 156 ++++++++++++++++++ .../dhp/oa/oaipmh/OaiRecordWrapper.java | 50 ++++++ 2 files changed, 206 insertions(+) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java new file mode 100644 index 000000000..9a608b6fa --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -0,0 +1,156 @@ +package eu.dnetlib.dhp.oa.oaipmh; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Optional; +import java.util.Properties; +import java.util.zip.GZIPOutputStream; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +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.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.dom4j.Document; +import org.dom4j.DocumentHelper; +import org.dom4j.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.provision.XmlConverterJob; +import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; +import eu.dnetlib.dhp.oa.provision.model.TupleWrapper; + +public class IrishOaiExporterJob { + + private static final Logger log = LoggerFactory.getLogger(IrishOaiExporterJob.class); + + protected static final int NUM_CONNECTIONS = 20; + + public static void main(final String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString(XmlConverterJob.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json"))); + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("inputPath"); + final String dbUrl = parser.get("dbUrl"); + final String dbTable = parser.get("dbTable"); + final String dbUser = parser.get("dbUser"); + final String dbPwd = parser.get("dbPwd"); + final int numConnections = Optional + .ofNullable(parser.get("numConnections")) + .map(Integer::valueOf) + .orElse(NUM_CONNECTIONS); + + log.info("inputPath: '{}'", inputPath); + log.info("dbUrl: '{}'", dbUrl); + log.info("dbUser: '{}'", dbUser); + log.info("table: '{}'", dbTable); + log.info("dbPwd: '{}'", "xxx"); + log.info("numPartitions: '{}'", numConnections); + + final Properties connectionProperties = new Properties(); + connectionProperties.put("user", dbUser); + connectionProperties.put("password", dbPwd); + + final SparkConf conf = new SparkConf(); + conf.registerKryoClasses(new Class[] { + SerializableSolrInputDocument.class + }); + + final Encoder encoderTuple = Encoders.bean(TupleWrapper.class); + final Encoder encoderOaiRecord = Encoders.bean(OaiRecordWrapper.class); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + + final Dataset docs = spark + .read() + .schema(encoderTuple.schema()) + .json(inputPath) + .as(encoderTuple) + .map((MapFunction) TupleWrapper::getXml, Encoders.STRING()) + .map((MapFunction) IrishOaiExporterJob::asIrishOaiResult, encoderOaiRecord) + .filter((FilterFunction) obj -> (obj != null) && StringUtils.isNotBlank(obj.getId())); + + docs.repartition(numConnections) + .write() + .mode(SaveMode.Overwrite) + .jdbc(dbUrl, dbTable, connectionProperties); + + }); + } + + private static OaiRecordWrapper asIrishOaiResult(final String xml) { + try { + final Document doc = DocumentHelper.parseText(xml); + final OaiRecordWrapper r = new OaiRecordWrapper(); + + if (isValid(doc)) { + r.setId(doc.valueOf("//*[local-name()='objIdentifier']").trim()); + r.setBody(gzip(xml)); + r.setDate(LocalDateTime.now()); + r.setSets(new ArrayList<>()); + } + return r; + } catch (final Exception e) { + log.error("Error parsing record: " + xml, e); + throw new RuntimeException("Error parsing record: " + xml, e); + } + } + + private static boolean isValid(final Document doc) { + + final Node n = doc.selectSingleNode("//*[local-name()='entity']/*[local-name()='result']"); + + if (n != null) { + for (final Object o : n.selectNodes(".//*[local-name()='datainfo']/*[local-name()='deletedbyinference']")) { + if ("true".equals(((Node) o).getText().trim())) { return false; } + } + + for (final Object o : n.selectNodes("./*[local-name()='country']")) { + if ("IE".equals(((Node) o).valueOf("@classid").trim())) { return true; } + } + + for (final Object o : n.selectNodes(".//*[local-name()='rel']")) { + final String relType = ((Node) o).valueOf("./[local-name() = 'to']/@type").trim(); + final String relCountry = ((Node) o).valueOf("./*[local-name() = 'country']/@classid").trim(); + if ("organization".equals(relType) && "IE".equals(relCountry)) { return true; } + } + } + return false; + + } + + private static byte[] gzip(final String str) { + if (StringUtils.isBlank(str)) { return null; } + + try { + final ByteArrayOutputStream obj = new ByteArrayOutputStream(); + final GZIPOutputStream gzip = new GZIPOutputStream(obj); + gzip.write(str.getBytes("UTF-8")); + gzip.flush(); + gzip.close(); + return obj.toByteArray(); + } catch (final IOException e) { + throw new RuntimeException("error in gzip", e); + } + } +} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java new file mode 100644 index 000000000..4c2766754 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java @@ -0,0 +1,50 @@ +package eu.dnetlib.dhp.oa.oaipmh; + +import java.io.Serializable; +import java.time.LocalDateTime; +import java.util.List; + +public class OaiRecordWrapper implements Serializable { + + private static final long serialVersionUID = 8997046455575004880L; + + private String id; + private byte[] body; + private LocalDateTime date; + private List sets; + + public OaiRecordWrapper() {} + + public String getId() { + return this.id; + } + + public void setId(final String id) { + this.id = id; + } + + public byte[] getBody() { + return this.body; + } + + public void setBody(final byte[] body) { + this.body = body; + } + + public LocalDateTime getDate() { + return this.date; + } + + public void setDate(final LocalDateTime date) { + this.date = date; + } + + public List getSets() { + return this.sets; + } + + public void setSets(final List sets) { + this.sets = sets; + } + +} From aa40e53c19acf6c8007b7819bea3e65ba642e057 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 7 May 2024 08:01:19 +0200 Subject: [PATCH 008/239] oai exporter parameters --- .../input_params_irish_oai_exporter.json | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json new file mode 100644 index 000000000..99a12927b --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json @@ -0,0 +1,38 @@ +[ + { + "paramName": "i", + "paramLongName": "inputPath", + "paramDescription": "The path of the input records on HDFS", + "paramRequired": true + }, + { + "paramName": "nc", + "paramLongName": "numConnections", + "paramDescription": "number of connections to the postgres db (for the write operation)", + "paramRequired": false + }, + { + "paramName": "du", + "paramLongName": "dbUrl", + "paramDescription": "the url of the database", + "paramRequired": true + }, + { + "paramName": "dusr", + "paramLongName": "dbUser", + "paramDescription": "the user of the database", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "dbTable", + "paramDescription": "the name of the table in the database", + "paramRequired": true + }, + { + "paramName": "dpwd", + "paramLongName": "dbPwd", + "paramDescription": "the password for the user of the database", + "paramRequired": true + } +] From 70bf6ac41561d487109a04ef60b0659a8785d989 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 7 May 2024 09:36:26 +0200 Subject: [PATCH 009/239] oai exporter tests --- .../dhp/oa/oaipmh/IrishOaiExporterJob.java | 11 ++- .../oa/oaipmh/IrishOaiExporterJobTest.java | 93 +++++++++++++++++++ .../eu/dnetlib/dhp/oa/oaipmh/record_IE.xml | 89 ++++++++++++++++++ .../dhp/oa/oaipmh/record_IE_deleted.xml | 89 ++++++++++++++++++ .../eu/dnetlib/dhp/oa/oaipmh/record_IT.xml | 66 +++++++++++++ 5 files changed, 344 insertions(+), 4 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE.xml create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE_deleted.xml create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IT.xml diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java index 9a608b6fa..e2ae890e5 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -98,7 +98,7 @@ public class IrishOaiExporterJob { }); } - private static OaiRecordWrapper asIrishOaiResult(final String xml) { + protected static OaiRecordWrapper asIrishOaiResult(final String xml) { try { final Document doc = DocumentHelper.parseText(xml); final OaiRecordWrapper r = new OaiRecordWrapper(); @@ -116,21 +116,24 @@ public class IrishOaiExporterJob { } } - private static boolean isValid(final Document doc) { + protected static boolean isValid(final Document doc) { final Node n = doc.selectSingleNode("//*[local-name()='entity']/*[local-name()='result']"); if (n != null) { + for (final Object o : n.selectNodes(".//*[local-name()='datainfo']/*[local-name()='deletedbyinference']")) { if ("true".equals(((Node) o).getText().trim())) { return false; } } + // verify the main country of the result for (final Object o : n.selectNodes("./*[local-name()='country']")) { if ("IE".equals(((Node) o).valueOf("@classid").trim())) { return true; } } + // verify the countries of the related organizations for (final Object o : n.selectNodes(".//*[local-name()='rel']")) { - final String relType = ((Node) o).valueOf("./[local-name() = 'to']/@type").trim(); + final String relType = ((Node) o).valueOf("./*[local-name() = 'to']/@type").trim(); final String relCountry = ((Node) o).valueOf("./*[local-name() = 'country']/@classid").trim(); if ("organization".equals(relType) && "IE".equals(relCountry)) { return true; } } @@ -139,7 +142,7 @@ public class IrishOaiExporterJob { } - private static byte[] gzip(final String str) { + protected static byte[] gzip(final String str) { if (StringUtils.isBlank(str)) { return null; } try { diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java new file mode 100644 index 000000000..6140b0907 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java @@ -0,0 +1,93 @@ +package eu.dnetlib.dhp.oa.oaipmh; + +import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.zip.GZIPInputStream; + +import org.apache.commons.io.IOUtils; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.io.SAXReader; +import org.junit.jupiter.api.Test; + +class IrishOaiExporterJobTest { + + @Test + void testAsIrishOaiResult() throws Exception { + final String xml = IOUtils.toString(getClass().getResourceAsStream("record_IE.xml")); + final OaiRecordWrapper res = IrishOaiExporterJob.asIrishOaiResult(xml); + assertNotNull(res.getId()); + assertNotNull(res.getBody()); + assertNotNull(res.getSets()); + assertNotNull(res.getDate()); + assertEquals("dedup_wf_002::532be02f990b479a1da46d71f1a4c3f0", res.getId()); + assertTrue(res.getBody().length > 0); + assertTrue(res.getSets().isEmpty()); + } + + @Test + void testIsValid_IE() throws DocumentException { + final Document doc = new SAXReader().read(getClass().getResourceAsStream("record_IE.xml")); + assertTrue(IrishOaiExporterJob.isValid(doc)); + } + + @Test + void testIsValid_invalid_country() throws DocumentException { + final Document doc = new SAXReader().read(getClass().getResourceAsStream("record_IT.xml")); + assertFalse(IrishOaiExporterJob.isValid(doc)); + } + + @Test + void testIsValid_deleted() throws DocumentException { + final Document doc = new SAXReader().read(getClass().getResourceAsStream("record_IE_deleted.xml")); + assertFalse(IrishOaiExporterJob.isValid(doc)); + } + + @Test + void testGzip_simple() { + final String message = ""; + final byte[] bytes = IrishOaiExporterJob.gzip(message); + assertNotNull(bytes); + assertTrue(bytes.length > 0); + assertEquals(message, decompress(bytes)); + } + + @Test + void testGzip_empty() { + assertNull(IrishOaiExporterJob.gzip("")); + assertNull(IrishOaiExporterJob.gzip(null)); + } + + private static String decompress(final byte[] compressed) { + final StringBuilder outStr = new StringBuilder(); + if ((compressed == null) || (compressed.length == 0)) { return null; } + try { + if (isCompressed(compressed)) { + final GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(compressed)); + final BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(gis, "UTF-8")); + + String line; + while ((line = bufferedReader.readLine()) != null) { + outStr.append(line); + } + } else { + outStr.append(compressed); + } + return outStr.toString(); + } catch (final IOException e) { + throw new RuntimeException("error in gunzip", e); + } + } + + private static boolean isCompressed(final byte[] compressed) { + return (compressed[0] == (byte) GZIPInputStream.GZIP_MAGIC) && (compressed[1] == (byte) (GZIPInputStream.GZIP_MAGIC >> 8)); + } +} diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE.xml b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE.xml new file mode 100644 index 000000000..01b7334f8 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE.xml @@ -0,0 +1,89 @@ + + +
+ dedup_wf_002::532be02f990b479a1da46d71f1a4c3f0 + 2023-03-31T18:37:45.599Z + 2023-03-31T18:45:52.701Z +
+ + + + + + 50|od______6005::55a12e2e0fee45ce8005633c6c17fe9f + oai:repository.wit.ie:3029 + 50|od_______934::e7162a5632264cd622ee7180ca66fdce + oai:generic.eprints.org:3029 + 50|od_______934::55a12e2e0fee45ce8005633c6c17fe9f + + + + + + http://repository.wit.ie/3029/1/Research%20Day%202015%20-%20Poster%20Tadhg%20Blommerde.pdf + A service innovation capability maturity model for SMEs + + Blommerde, Tadhg + Lynch, Patrick + + 2015-04-28 + There is general consensus that service innovations are prerequisite to sustained competitive advantage and are an essential mechanism for responding to changes in customer needs and the operating environment of firms (Giannopoulou et al., 2011; Stryja et al., 2013). Services have been described as ubiquitous in their role of generating economic growth and wellbeing and represent over 70% of employment and GDP in developed nations (Janssen et al., 2012; Mustak, 2014). As a consequence, service innovations must be a core ambition of all countries, regions, and firms wishing to remain competitive (van Ark et al., 2003). While acknowledging the importance of once-off innovations, more critical still is the capability to repeatedly introduce and exploit service innovations (Siguaw et al., 2006). This is generally referred to as service innovation capability (SIC) and describes the repeatable routines and behaviours that organisations have in place to transform ideas and knowledge into innovations (Basterretxea and Martínez, 2012). However, despite links between SIC and continuous, sustainable, and consistent service innovations, there is evidence that many organisations struggle with its effective management (Adams et al., 2006; den Hertog et al., 2010). This is often attributed to the lack of formal guidance available and the absence of metrics to determine an organisation’s SIC performance (Hogan et al., 2011; Szczygielski, 2011). Maturity modelling research in this discipline remains at an embryonic stage, thus far presenting only conceptual and opaque discussions that fail to address the necessity for an assessment and strategic management framework (Gryszkiewicz et al., 2013; Hipp and Grupp, 2005). Therefore, the purpose of this ongoing research project is to evaluate the maturity of an organisation’s SIC to inform its effective management and enhancement. To achieve this it dimensionalises the concept into four constituent capabilities, specifically, strategising, customer involvement, knowledge management, and networking (Blommerde and Lynch, 2014). The study then tracks the maturity of these capabilities as they progress through eight evolutionary plateaus towards a fully developed or optimal state. This is accomplished through a capability maturity model that enables organisations to rapidly diagnose key areas of strength and weakness to systematically cultivate behaviours that leverage their untapped innovative potential (Wendler, 2012; Essmann and du Preez, 2010). As a result of the immense knowledge vacuum characteristic of this discipline, it is anticipated that this ongoing research project will make a substantial contribution to both academic understanding and take strides towards filling the void in practical support (Rapaccini et al., 2013). It expands the service innovation literature by detailing key service innovation levers, bolsters the discipline through clear definitions of terminology, provides a powerful explanation of the development of SICs, and operationalises the dynamic capabilities view through a novel self-assessment reference model (Jochem et al., 2011). The next step in the project is the evaluation of the, as yet, conceptual service innovation capability maturity model. Adopting a positivistic philosophical stance, the study proposes the use of structural equation modelling on data gathered through an extensive survey to confirm the model and support theoretical assumptions. + RIKON (Research in Inovation, Knowledge & Organisational Networks) + + application/pdf + + + false + false + true + + + + true + false + 0.8 + dedup-result-decisiontree-v4 + + + + + openorgs____::54cd984fc7d3b153ec2181f985041f02 + + WIT + South East Technological University + + + + + A service innovation capability maturity model for SMEs + 2015-04-28 + + + + A service innovation capability maturity model for SMEs + 2015-04-28 + + + + A service innovation capability maturity model for SMEs + 2015-04-28 + + + + + + + 2015-04-28 + + + http://repository.wit.ie/3029/1/Research%20Day%202015%20-%20Poster%20Tadhg%20Blommerde.pdf + + http://repository.wit.ie/3029/ + + + + + + +
+
\ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE_deleted.xml b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE_deleted.xml new file mode 100644 index 000000000..00d225aa5 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IE_deleted.xml @@ -0,0 +1,89 @@ + + +
+ dedup_wf_002::532be02f990b479a1da46d71f1a4c3f0 + 2023-03-31T18:37:45.599Z + 2023-03-31T18:45:52.701Z +
+ + + + + + 50|od______6005::55a12e2e0fee45ce8005633c6c17fe9f + oai:repository.wit.ie:3029 + 50|od_______934::e7162a5632264cd622ee7180ca66fdce + oai:generic.eprints.org:3029 + 50|od_______934::55a12e2e0fee45ce8005633c6c17fe9f + + + + + + http://repository.wit.ie/3029/1/Research%20Day%202015%20-%20Poster%20Tadhg%20Blommerde.pdf + A service innovation capability maturity model for SMEs + + Blommerde, Tadhg + Lynch, Patrick + + 2015-04-28 + There is general consensus that service innovations are prerequisite to sustained competitive advantage and are an essential mechanism for responding to changes in customer needs and the operating environment of firms (Giannopoulou et al., 2011; Stryja et al., 2013). Services have been described as ubiquitous in their role of generating economic growth and wellbeing and represent over 70% of employment and GDP in developed nations (Janssen et al., 2012; Mustak, 2014). As a consequence, service innovations must be a core ambition of all countries, regions, and firms wishing to remain competitive (van Ark et al., 2003). While acknowledging the importance of once-off innovations, more critical still is the capability to repeatedly introduce and exploit service innovations (Siguaw et al., 2006). This is generally referred to as service innovation capability (SIC) and describes the repeatable routines and behaviours that organisations have in place to transform ideas and knowledge into innovations (Basterretxea and Martínez, 2012). However, despite links between SIC and continuous, sustainable, and consistent service innovations, there is evidence that many organisations struggle with its effective management (Adams et al., 2006; den Hertog et al., 2010). This is often attributed to the lack of formal guidance available and the absence of metrics to determine an organisation’s SIC performance (Hogan et al., 2011; Szczygielski, 2011). Maturity modelling research in this discipline remains at an embryonic stage, thus far presenting only conceptual and opaque discussions that fail to address the necessity for an assessment and strategic management framework (Gryszkiewicz et al., 2013; Hipp and Grupp, 2005). Therefore, the purpose of this ongoing research project is to evaluate the maturity of an organisation’s SIC to inform its effective management and enhancement. To achieve this it dimensionalises the concept into four constituent capabilities, specifically, strategising, customer involvement, knowledge management, and networking (Blommerde and Lynch, 2014). The study then tracks the maturity of these capabilities as they progress through eight evolutionary plateaus towards a fully developed or optimal state. This is accomplished through a capability maturity model that enables organisations to rapidly diagnose key areas of strength and weakness to systematically cultivate behaviours that leverage their untapped innovative potential (Wendler, 2012; Essmann and du Preez, 2010). As a result of the immense knowledge vacuum characteristic of this discipline, it is anticipated that this ongoing research project will make a substantial contribution to both academic understanding and take strides towards filling the void in practical support (Rapaccini et al., 2013). It expands the service innovation literature by detailing key service innovation levers, bolsters the discipline through clear definitions of terminology, provides a powerful explanation of the development of SICs, and operationalises the dynamic capabilities view through a novel self-assessment reference model (Jochem et al., 2011). The next step in the project is the evaluation of the, as yet, conceptual service innovation capability maturity model. Adopting a positivistic philosophical stance, the study proposes the use of structural equation modelling on data gathered through an extensive survey to confirm the model and support theoretical assumptions. + RIKON (Research in Inovation, Knowledge & Organisational Networks) + + application/pdf + + + false + false + true + + + + true + true + 0.8 + dedup-result-decisiontree-v4 + + + + + openorgs____::54cd984fc7d3b153ec2181f985041f02 + + WIT + South East Technological University + + + + + A service innovation capability maturity model for SMEs + 2015-04-28 + + + + A service innovation capability maturity model for SMEs + 2015-04-28 + + + + A service innovation capability maturity model for SMEs + 2015-04-28 + + + + + + + 2015-04-28 + + + http://repository.wit.ie/3029/1/Research%20Day%202015%20-%20Poster%20Tadhg%20Blommerde.pdf + + http://repository.wit.ie/3029/ + + + + + + +
+
\ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IT.xml b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IT.xml new file mode 100644 index 000000000..7649589d1 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/oaipmh/record_IT.xml @@ -0,0 +1,66 @@ + + +
+ od_______310::02365c51a0ed7cbb54b2bbc7c0426d1b + 2024-04-06T06:05:16+0000 + 2024-04-06T06:56:01.776Z +
+ + + + + 50|od_______310::02365c51a0ed7cbb54b2bbc7c0426d1b + oai:flore.unifi.it:2158/608965 + 2158/608965 + + + + + + Estorsione (art. 629) + + MACRI', FRANCESCO + + 2011-01-01 + + 2011-01-01 + 2011-01-01 + 2015-04-28 + UTET + + + + false + false + 0.9 + null + + + + + openorgs____::41406edad82942e9e0b29317b8a847e2 + University of Florence + + University of Florence + + + + + + + + 2011-01-01 + + 2158/608965 + http://hdl.handle.net/2158/608965 + + + https://hdl.handle.net/2158/608965 + + + + + + +
+
\ No newline at end of file From e234848af8b0a313a0c8b3988d2ceb4f425edc78 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 8 May 2024 10:00:53 +0200 Subject: [PATCH 010/239] oaf record: xpath for root --- .../main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java index e2ae890e5..fff5d015d 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -105,7 +105,7 @@ public class IrishOaiExporterJob { if (isValid(doc)) { r.setId(doc.valueOf("//*[local-name()='objIdentifier']").trim()); - r.setBody(gzip(xml)); + r.setBody(gzip(doc.selectSingleNode("//*[local-name()='entity']").asXML())); r.setDate(LocalDateTime.now()); r.setSets(new ArrayList<>()); } From c9a327bc5094f48c08f4d7d0b3274378d8d8c63f Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 8 May 2024 11:34:08 +0200 Subject: [PATCH 011/239] refactoring of gzip method --- .../dhp/oa/oaipmh/IrishOaiExporterJob.java | 14 +++++------ .../oa/oaipmh/IrishOaiExporterJobTest.java | 24 +++++-------------- 2 files changed, 13 insertions(+), 25 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java index fff5d015d..b59f0ae73 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -4,6 +4,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.charset.Charset; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Optional; @@ -145,15 +146,14 @@ public class IrishOaiExporterJob { protected static byte[] gzip(final String str) { if (StringUtils.isBlank(str)) { return null; } - try { - final ByteArrayOutputStream obj = new ByteArrayOutputStream(); - final GZIPOutputStream gzip = new GZIPOutputStream(obj); - gzip.write(str.getBytes("UTF-8")); - gzip.flush(); - gzip.close(); - return obj.toByteArray(); + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + try (final GZIPOutputStream gzip = new GZIPOutputStream(baos)) { + IOUtils.write(str.getBytes(Charset.defaultCharset()), gzip); + } + return baos.toByteArray(); } catch (final IOException e) { throw new RuntimeException("error in gzip", e); } } + } diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java index 6140b0907..e33c701c5 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java @@ -6,10 +6,9 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.InputStreamReader; +import java.nio.charset.Charset; import java.util.zip.GZIPInputStream; import org.apache.commons.io.IOUtils; @@ -57,7 +56,7 @@ class IrishOaiExporterJobTest { final byte[] bytes = IrishOaiExporterJob.gzip(message); assertNotNull(bytes); assertTrue(bytes.length > 0); - assertEquals(message, decompress(bytes)); + assertEquals(message, gunzip(bytes)); } @Test @@ -66,22 +65,11 @@ class IrishOaiExporterJobTest { assertNull(IrishOaiExporterJob.gzip(null)); } - private static String decompress(final byte[] compressed) { - final StringBuilder outStr = new StringBuilder(); + public static String gunzip(final byte[] compressed) { if ((compressed == null) || (compressed.length == 0)) { return null; } - try { - if (isCompressed(compressed)) { - final GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(compressed)); - final BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(gis, "UTF-8")); - - String line; - while ((line = bufferedReader.readLine()) != null) { - outStr.append(line); - } - } else { - outStr.append(compressed); - } - return outStr.toString(); + if (!isCompressed(compressed)) { return new String(compressed); } + try (final GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(compressed))) { + return IOUtils.toString(gis, Charset.defaultCharset()); } catch (final IOException e) { throw new RuntimeException("error in gunzip", e); } From 2b3b5fe9a172bb1fafb4815a5c52aa9fcaff6644 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 15 May 2024 14:13:16 +0200 Subject: [PATCH 012/239] oai finalization and test --- .../dhp/oa/oaipmh/IrishOaiExporterJob.java | 26 ++++- .../dhp/oa/oaipmh/OaiRecordWrapper.java | 11 ++- .../input_params_irish_oai_exporter.json | 6 -- .../eu/dnetlib/dhp/oa/oaipmh/oai-finalize.sql | 12 +++ .../dhp/oa/oaipmh/DbSerializationTest.java | 97 +++++++++++++++++++ .../oa/oaipmh/IrishOaiExporterJobTest.java | 14 ++- 6 files changed, 146 insertions(+), 20 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oai-finalize.sql create mode 100644 dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java index b59f0ae73..433baf272 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.oaipmh; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; @@ -5,6 +6,9 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.Charset; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Optional; @@ -37,6 +41,8 @@ public class IrishOaiExporterJob { protected static final int NUM_CONNECTIONS = 20; + public static final String TMP_OAI_TABLE = "temp_oai_data"; + public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -53,7 +59,6 @@ public class IrishOaiExporterJob { final String inputPath = parser.get("inputPath"); final String dbUrl = parser.get("dbUrl"); - final String dbTable = parser.get("dbTable"); final String dbUser = parser.get("dbUser"); final String dbPwd = parser.get("dbPwd"); final int numConnections = Optional @@ -64,7 +69,6 @@ public class IrishOaiExporterJob { log.info("inputPath: '{}'", inputPath); log.info("dbUrl: '{}'", dbUrl); log.info("dbUser: '{}'", dbUser); - log.info("table: '{}'", dbTable); log.info("dbPwd: '{}'", "xxx"); log.info("numPartitions: '{}'", numConnections); @@ -80,6 +84,7 @@ public class IrishOaiExporterJob { final Encoder encoderTuple = Encoders.bean(TupleWrapper.class); final Encoder encoderOaiRecord = Encoders.bean(OaiRecordWrapper.class); + log.info("Creating temporary table..."); runWithSparkSession(conf, isSparkSessionManaged, spark -> { final Dataset docs = spark @@ -91,12 +96,23 @@ public class IrishOaiExporterJob { .map((MapFunction) IrishOaiExporterJob::asIrishOaiResult, encoderOaiRecord) .filter((FilterFunction) obj -> (obj != null) && StringUtils.isNotBlank(obj.getId())); - docs.repartition(numConnections) + docs + .repartition(numConnections) .write() .mode(SaveMode.Overwrite) - .jdbc(dbUrl, dbTable, connectionProperties); + .jdbc(dbUrl, TMP_OAI_TABLE, connectionProperties); }); + log.info("Temporary table created."); + + log.info("Updating OAI records..."); + try (final Connection con = DriverManager.getConnection(dbUrl, dbUser, dbPwd)) { + try (final Statement st = con.createStatement()) { + final String query = IOUtils.toString(IrishOaiExporterJob.class.getResourceAsStream("oai-finalize.sql")); + st.execute(query); + } + } + log.info("DONE."); } protected static OaiRecordWrapper asIrishOaiResult(final String xml) { @@ -107,7 +123,7 @@ public class IrishOaiExporterJob { if (isValid(doc)) { r.setId(doc.valueOf("//*[local-name()='objIdentifier']").trim()); r.setBody(gzip(doc.selectSingleNode("//*[local-name()='entity']").asXML())); - r.setDate(LocalDateTime.now()); + r.setDate(LocalDateTime.now().toString()); r.setSets(new ArrayList<>()); } return r; diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java index 4c2766754..2fdf32c96 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/OaiRecordWrapper.java @@ -1,7 +1,7 @@ + package eu.dnetlib.dhp.oa.oaipmh; import java.io.Serializable; -import java.time.LocalDateTime; import java.util.List; public class OaiRecordWrapper implements Serializable { @@ -10,10 +10,11 @@ public class OaiRecordWrapper implements Serializable { private String id; private byte[] body; - private LocalDateTime date; + private String date; private List sets; - public OaiRecordWrapper() {} + public OaiRecordWrapper() { + } public String getId() { return this.id; @@ -31,11 +32,11 @@ public class OaiRecordWrapper implements Serializable { this.body = body; } - public LocalDateTime getDate() { + public String getDate() { return this.date; } - public void setDate(final LocalDateTime date) { + public void setDate(final String date) { this.date = date; } diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json index 99a12927b..86b2bb0d3 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json @@ -23,12 +23,6 @@ "paramDescription": "the user of the database", "paramRequired": true }, - { - "paramName": "t", - "paramLongName": "dbTable", - "paramDescription": "the name of the table in the database", - "paramRequired": true - }, { "paramName": "dpwd", "paramLongName": "dbPwd", diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oai-finalize.sql b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oai-finalize.sql new file mode 100644 index 000000000..1ec0dfee0 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oai-finalize.sql @@ -0,0 +1,12 @@ +BEGIN; + +DELETE FROM oai_data; + +INSERT INTO oai_data(id, body, date, sets) SELECT + id, + body, + date::timestamp, + sets +FROM temp_oai_data; + +COMMIT; diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java new file mode 100644 index 000000000..f33708f86 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java @@ -0,0 +1,97 @@ + +package eu.dnetlib.dhp.oa.oaipmh; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +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.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +@Disabled +public class DbSerializationTest { + + private static SparkSession spark; + + public static final String dbUrl = "jdbc:postgresql://localhost:5432/db_test"; + public static final String dbUser = null; + public static final String dbPwd = null; + + @BeforeAll + public static void beforeAll() throws IOException { + + final SparkConf conf = new SparkConf(); + conf.setAppName("TEST"); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + + spark = SparkSession + .builder() + .appName("TEST") + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + spark.stop(); + } + + @Test + public void testDatabaseSerialization() throws Exception { + final Properties connectionProperties = new Properties(); + if (dbUser != null) { + connectionProperties.put("user", dbUser); + } + if (dbPwd != null) { + connectionProperties.put("password", dbPwd); + } + + runWithSparkSession(new SparkConf(), false, spark -> { + + final List list = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + final OaiRecordWrapper r = new OaiRecordWrapper(); + r.setId("record_" + i); + r.setBody("jsahdjkahdjahdajad".getBytes()); + r.setDate(LocalDateTime.now().toString()); + r.setSets(Arrays.asList()); + list.add(r); + } + + final Dataset docs = spark.createDataset(list, Encoders.bean(OaiRecordWrapper.class)); + + docs + .write() + .mode(SaveMode.Overwrite) + .jdbc(dbUrl, IrishOaiExporterJob.TMP_OAI_TABLE, connectionProperties); + + }); + + try (final Connection con = DriverManager.getConnection(dbUrl, dbUser, dbPwd)) { + try (final Statement st = con.createStatement()) { + final String query = IOUtils.toString(getClass().getResourceAsStream("oai-finalize.sql")); + st.execute(query); + } + } + + } + +} diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java index e33c701c5..57a32e246 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.oaipmh; import static org.junit.Assert.assertNull; @@ -17,7 +18,7 @@ import org.dom4j.DocumentException; import org.dom4j.io.SAXReader; import org.junit.jupiter.api.Test; -class IrishOaiExporterJobTest { +public class IrishOaiExporterJobTest { @Test void testAsIrishOaiResult() throws Exception { @@ -66,8 +67,12 @@ class IrishOaiExporterJobTest { } public static String gunzip(final byte[] compressed) { - if ((compressed == null) || (compressed.length == 0)) { return null; } - if (!isCompressed(compressed)) { return new String(compressed); } + if ((compressed == null) || (compressed.length == 0)) { + return null; + } + if (!isCompressed(compressed)) { + return new String(compressed); + } try (final GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(compressed))) { return IOUtils.toString(gis, Charset.defaultCharset()); } catch (final IOException e) { @@ -76,6 +81,7 @@ class IrishOaiExporterJobTest { } private static boolean isCompressed(final byte[] compressed) { - return (compressed[0] == (byte) GZIPInputStream.GZIP_MAGIC) && (compressed[1] == (byte) (GZIPInputStream.GZIP_MAGIC >> 8)); + return (compressed[0] == (byte) GZIPInputStream.GZIP_MAGIC) + && (compressed[1] == (byte) (GZIPInputStream.GZIP_MAGIC >> 8)); } } From 888637773cfc4d076d27d5fc83bd1b5008f521c2 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Mon, 27 May 2024 12:34:49 +0300 Subject: [PATCH 013/239] Add missing "/*EOS*/" comments. --- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql | 2 +- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql | 2 +- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql | 2 +- .../oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql | 2 +- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql | 2 +- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql | 4 ++-- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index 9a9a507e3..8cab942e6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -1,4 +1,4 @@ -set mapred.job.queue.name=analytics; +set mapred.job.queue.name=analytics; /*EOS*/ ------------------------------------------------------ ------------------------------------------------------ -- Additional relations diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 989b92268..1f3027b7d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -1,4 +1,4 @@ -set mapred.job.queue.name=analytics; +set mapred.job.queue.name=analytics; /*EOS*/ ------------------------------------------------------ ------------------------------------------------------ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql index be5d42f96..d18cf569f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql @@ -1,4 +1,4 @@ -set mapred.job.queue.name=analytics; +set mapred.job.queue.name=analytics; /*EOS*/ ------------------------------------------- --- Extra tables, mostly used by indicators diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql index dd830a24d..0da4394c8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql @@ -1,4 +1,4 @@ -set mapred.job.queue.name=analytics; +set mapred.job.queue.name=analytics; /*EOS*/ ---------------------------------------------------- -- Shortcuts for various definitions in stats db --- diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql index e723ec8b1..416298e4c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql @@ -1,4 +1,4 @@ -set mapred.job.queue.name=analytics; +set mapred.job.queue.name=analytics; /*EOS*/ -- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, -- peer reviewed) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index 59213c4d5..f0e5ce091 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -1,4 +1,4 @@ -set mapred.job.queue.name=analytics; +set mapred.job.queue.name=analytics; /*EOS*/ -------------------------------------------------------------- -------------------------------------------------------------- @@ -113,4 +113,4 @@ SELECT /*+ COALESCE(100) */ substr(p.id, 4) AS id, xpath_string(citation.value, FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" - and p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; + and p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ From a644a6f4fef8a1d2dfdc21c9edaa463a24855d77 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 29 May 2024 12:10:11 +0300 Subject: [PATCH 014/239] Catch Spark-sql errors and show a log with the statement that failed. --- .../main/java/eu/dnetlib/dhp/oozie/RunSQLSparkJob.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oozie/RunSQLSparkJob.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oozie/RunSQLSparkJob.java index 027bf0735..01d1b9f6a 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oozie/RunSQLSparkJob.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oozie/RunSQLSparkJob.java @@ -65,7 +65,13 @@ public class RunSQLSparkJob { for (String statement : sql.split(";\\s*/\\*\\s*EOS\\s*\\*/\\s*")) { log.info("executing: {}", statement); long startTime = System.currentTimeMillis(); - spark.sql(statement).show(); + try { + spark.sql(statement).show(); + } catch (Exception e) { + log.error("Error executing statement: {}", statement, e); + System.err.println("Error executing statement: " + statement + "\n" + e); + throw e; + } log .info( "executed in {}", From 81090ad593b1bb1572c033989c86e79f795670e6 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 5 Jun 2024 10:03:33 +0200 Subject: [PATCH 015/239] [IE OAIPHM] added oozie workflow, minor changes, code formatting --- .../dhp/oa/oaipmh/IrishOaiExporterJob.java | 68 ++++++----- .../dhp/oa/oaipmh/oozie_app/workflow.xml | 106 ++++++++++++++++++ .../dhp/oa/oaipmh/DbSerializationTest.java | 14 +-- .../oa/oaipmh/IrishOaiExporterJobTest.java | 3 +- 4 files changed, 155 insertions(+), 36 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java index 433baf272..57f180fa0 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -46,15 +46,16 @@ public class IrishOaiExporterJob { public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString(XmlConverterJob.class - .getResourceAsStream("/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json"))); + IOUtils + .toString( + XmlConverterJob.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json"))); parser.parseArgument(args); final Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); final String inputPath = parser.get("inputPath"); @@ -62,9 +63,9 @@ public class IrishOaiExporterJob { final String dbUser = parser.get("dbUser"); final String dbPwd = parser.get("dbPwd"); final int numConnections = Optional - .ofNullable(parser.get("numConnections")) - .map(Integer::valueOf) - .orElse(NUM_CONNECTIONS); + .ofNullable(parser.get("numConnections")) + .map(Integer::valueOf) + .orElse(NUM_CONNECTIONS); log.info("inputPath: '{}'", inputPath); log.info("dbUrl: '{}'", dbUrl); @@ -78,29 +79,31 @@ public class IrishOaiExporterJob { final SparkConf conf = new SparkConf(); conf.registerKryoClasses(new Class[] { - SerializableSolrInputDocument.class + SerializableSolrInputDocument.class }); final Encoder encoderTuple = Encoders.bean(TupleWrapper.class); final Encoder encoderOaiRecord = Encoders.bean(OaiRecordWrapper.class); + final String date = LocalDateTime.now().toString(); + log.info("Creating temporary table..."); runWithSparkSession(conf, isSparkSessionManaged, spark -> { final Dataset docs = spark - .read() - .schema(encoderTuple.schema()) - .json(inputPath) - .as(encoderTuple) - .map((MapFunction) TupleWrapper::getXml, Encoders.STRING()) - .map((MapFunction) IrishOaiExporterJob::asIrishOaiResult, encoderOaiRecord) - .filter((FilterFunction) obj -> (obj != null) && StringUtils.isNotBlank(obj.getId())); + .read() + .schema(encoderTuple.schema()) + .json(inputPath) + .as(encoderTuple) + .map((MapFunction) TupleWrapper::getXml, Encoders.STRING()) + .map((MapFunction) r -> asIrishOaiResult(r, date), encoderOaiRecord) + .filter((FilterFunction) obj -> (obj != null) && StringUtils.isNotBlank(obj.getId())); docs - .repartition(numConnections) - .write() - .mode(SaveMode.Overwrite) - .jdbc(dbUrl, TMP_OAI_TABLE, connectionProperties); + .repartition(numConnections) + .write() + .mode(SaveMode.Overwrite) + .jdbc(dbUrl, TMP_OAI_TABLE, connectionProperties); }); log.info("Temporary table created."); @@ -108,14 +111,15 @@ public class IrishOaiExporterJob { log.info("Updating OAI records..."); try (final Connection con = DriverManager.getConnection(dbUrl, dbUser, dbPwd)) { try (final Statement st = con.createStatement()) { - final String query = IOUtils.toString(IrishOaiExporterJob.class.getResourceAsStream("oai-finalize.sql")); + final String query = IOUtils + .toString(IrishOaiExporterJob.class.getResourceAsStream("oai-finalize.sql")); st.execute(query); } } log.info("DONE."); } - protected static OaiRecordWrapper asIrishOaiResult(final String xml) { + protected static OaiRecordWrapper asIrishOaiResult(final String xml, final String date) { try { final Document doc = DocumentHelper.parseText(xml); final OaiRecordWrapper r = new OaiRecordWrapper(); @@ -123,7 +127,7 @@ public class IrishOaiExporterJob { if (isValid(doc)) { r.setId(doc.valueOf("//*[local-name()='objIdentifier']").trim()); r.setBody(gzip(doc.selectSingleNode("//*[local-name()='entity']").asXML())); - r.setDate(LocalDateTime.now().toString()); + r.setDate(date); r.setSets(new ArrayList<>()); } return r; @@ -140,19 +144,25 @@ public class IrishOaiExporterJob { if (n != null) { for (final Object o : n.selectNodes(".//*[local-name()='datainfo']/*[local-name()='deletedbyinference']")) { - if ("true".equals(((Node) o).getText().trim())) { return false; } + if ("true".equals(((Node) o).getText().trim())) { + return false; + } } // verify the main country of the result for (final Object o : n.selectNodes("./*[local-name()='country']")) { - if ("IE".equals(((Node) o).valueOf("@classid").trim())) { return true; } + if ("IE".equals(((Node) o).valueOf("@classid").trim())) { + return true; + } } // verify the countries of the related organizations for (final Object o : n.selectNodes(".//*[local-name()='rel']")) { final String relType = ((Node) o).valueOf("./*[local-name() = 'to']/@type").trim(); final String relCountry = ((Node) o).valueOf("./*[local-name() = 'country']/@classid").trim(); - if ("organization".equals(relType) && "IE".equals(relCountry)) { return true; } + if ("organization".equals(relType) && "IE".equals(relCountry)) { + return true; + } } } return false; @@ -160,7 +170,9 @@ public class IrishOaiExporterJob { } protected static byte[] gzip(final String str) { - if (StringUtils.isBlank(str)) { return null; } + if (StringUtils.isBlank(str)) { + return null; + } try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) { try (final GZIPOutputStream gzip = new GZIPOutputStream(baos)) { diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oozie_app/workflow.xml new file mode 100644 index 000000000..c4caad91e --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/oaipmh/oozie_app/workflow.xml @@ -0,0 +1,106 @@ + + + + + inputPath + The path of the input records on HDFS + + + numConnections + number of connections to the postgres db (for the write operation) + + + dbUrl + the url of the database + + + dbUser + the user of the database + + + dbPwd + the password for the user of the 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} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + Irish OAI-PHM provision + eu.dnetlib.dhp.oa.oaipmh.IrishOaiExporterJob + dhp-graph-provision-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --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=8000 + + --inputPath${inputPath} + --numConnections${numConnections} + --dbUrl${dbUrl} + --dbUser${dbUser} + --dbPwd${dbPwd} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java index f33708f86..d487fda94 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/DbSerializationTest.java @@ -42,10 +42,10 @@ public class DbSerializationTest { conf.set("spark.driver.host", "localhost"); spark = SparkSession - .builder() - .appName("TEST") - .config(conf) - .getOrCreate(); + .builder() + .appName("TEST") + .config(conf) + .getOrCreate(); } @AfterAll @@ -79,9 +79,9 @@ public class DbSerializationTest { final Dataset docs = spark.createDataset(list, Encoders.bean(OaiRecordWrapper.class)); docs - .write() - .mode(SaveMode.Overwrite) - .jdbc(dbUrl, IrishOaiExporterJob.TMP_OAI_TABLE, connectionProperties); + .write() + .mode(SaveMode.Overwrite) + .jdbc(dbUrl, IrishOaiExporterJob.TMP_OAI_TABLE, connectionProperties); }); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java index 57a32e246..c16f75e1d 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJobTest.java @@ -10,6 +10,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.ByteArrayInputStream; import java.io.IOException; import java.nio.charset.Charset; +import java.time.LocalDateTime; import java.util.zip.GZIPInputStream; import org.apache.commons.io.IOUtils; @@ -23,7 +24,7 @@ public class IrishOaiExporterJobTest { @Test void testAsIrishOaiResult() throws Exception { final String xml = IOUtils.toString(getClass().getResourceAsStream("record_IE.xml")); - final OaiRecordWrapper res = IrishOaiExporterJob.asIrishOaiResult(xml); + final OaiRecordWrapper res = IrishOaiExporterJob.asIrishOaiResult(xml, LocalDateTime.now().toString()); assertNotNull(res.getId()); assertNotNull(res.getBody()); assertNotNull(res.getSets()); From 73bd1938a5a8dfead4035f50fded1108cf45a281 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 5 Jun 2024 12:17:35 +0200 Subject: [PATCH 016/239] [graph2hive] use sparkExecutorMemory to define also the memoryOverhead --- .../dhp/oa/graph/hive/oozie_app/workflow.xml | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml index 4468382be..eec67fc5c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml @@ -116,17 +116,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=10000 --inputPath${inputPath}/publication --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Publication --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions8000 + --numPartitions10000 @@ -143,17 +145,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=4000 --inputPath${inputPath}/dataset --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Dataset --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions4000 + --numPartitions8000 @@ -170,11 +174,13 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=8000 --inputPath${inputPath}/otherresearchproduct --hiveDbName${hiveDbName} @@ -197,17 +203,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=1000 --inputPath${inputPath}/software --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Software --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions300 + --numPartitions1000 @@ -224,17 +232,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=200 --inputPath${inputPath}/datasource --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Datasource --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions100 + --numPartitions200 @@ -251,17 +261,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=1000 --inputPath${inputPath}/organization --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Organization --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions400 + --numPartitions1000 @@ -278,17 +290,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=1000 --inputPath${inputPath}/project --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Project --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions100 + --numPartitions1000 @@ -305,17 +319,19 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=15000 --inputPath${inputPath}/relation --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Relation --hiveMetastoreUris${hiveMetastoreUris} - --numPartitions10000 + --numPartitions15000 From f70dc76b61a2597ed1a62d90a5f8394cf8710f19 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 6 Jun 2024 10:43:10 +0200 Subject: [PATCH 017/239] minor --- .../oaf/utils/ResultTypeComparator.java | 87 ------------------- .../dhp/oa/oaipmh/IrishOaiExporterJob.java | 3 +- 2 files changed, 1 insertion(+), 89 deletions(-) delete mode 100644 dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ResultTypeComparator.java diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ResultTypeComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ResultTypeComparator.java deleted file mode 100644 index e10b281b8..000000000 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ResultTypeComparator.java +++ /dev/null @@ -1,87 +0,0 @@ - -package eu.dnetlib.dhp.schema.oaf.utils; - -import static eu.dnetlib.dhp.schema.common.ModelConstants.CROSSREF_ID; - -import java.util.Comparator; -import java.util.HashSet; -import java.util.Optional; -import java.util.stream.Collectors; - -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Result; - -public class ResultTypeComparator implements Comparator { - - public static final ResultTypeComparator INSTANCE = new ResultTypeComparator(); - - @Override - public int compare(Result left, Result right) { - - if (left == null && right == null) - return 0; - if (left == null) - return 1; - if (right == null) - return -1; - - HashSet lCf = getCollectedFromIds(left); - HashSet rCf = getCollectedFromIds(right); - - if (lCf.contains(CROSSREF_ID) && !rCf.contains(CROSSREF_ID)) { - return -1; - } - if (!lCf.contains(CROSSREF_ID) && rCf.contains(CROSSREF_ID)) { - return 1; - } - - if (left.getResulttype() == null || left.getResulttype().getClassid() == null) { - if (right.getResulttype() == null || right.getResulttype().getClassid() == null) { - return 0; - } - return 1; - } else if (right.getResulttype() == null || right.getResulttype().getClassid() == null) { - return -1; - } - - String lClass = left.getResulttype().getClassid(); - String rClass = right.getResulttype().getClassid(); - - if (!lClass.equals(rClass)) { - if (lClass.equals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID)) - return -1; - if (rClass.equals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID)) - return 1; - - if (lClass.equals(ModelConstants.DATASET_RESULTTYPE_CLASSID)) - return -1; - if (rClass.equals(ModelConstants.DATASET_RESULTTYPE_CLASSID)) - return 1; - - if (lClass.equals(ModelConstants.SOFTWARE_RESULTTYPE_CLASSID)) - return -1; - if (rClass.equals(ModelConstants.SOFTWARE_RESULTTYPE_CLASSID)) - return 1; - - if (lClass.equals(ModelConstants.ORP_RESULTTYPE_CLASSID)) - return -1; - if (rClass.equals(ModelConstants.ORP_RESULTTYPE_CLASSID)) - return 1; - } - - // Else (but unlikely), lexicographical ordering will do. - return lClass.compareTo(rClass); - } - - protected HashSet getCollectedFromIds(Result left) { - return Optional - .ofNullable(left.getCollectedfrom()) - .map( - cf -> cf - .stream() - .map(KeyValue::getKey) - .collect(Collectors.toCollection(HashSet::new))) - .orElse(new HashSet<>()); - } -} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java index 57f180fa0..3d69370af 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/oaipmh/IrishOaiExporterJob.java @@ -31,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.provision.XmlConverterJob; import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import eu.dnetlib.dhp.oa.provision.model.TupleWrapper; @@ -48,7 +47,7 @@ public class IrishOaiExporterJob { final ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - XmlConverterJob.class + IrishOaiExporterJob.class .getResourceAsStream("/eu/dnetlib/dhp/oa/oaipmh/input_params_irish_oai_exporter.json"))); parser.parseArgument(args); From ce2364743a445e42c434c910852dc26a1b75a7a8 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 6 Jun 2024 10:43:43 +0200 Subject: [PATCH 018/239] applying changes from PR#442: Fix for missing collectedfrom after dedup --- .../oaf/utils/MergeEntitiesComparator.java | 104 ++++++++++++++++++ .../dhp/schema/oaf/utils/MergeUtils.java | 53 +++++---- .../dhp/oa/dedup/DatasetMergerTest.java | 103 +++++++++++++++++ .../dhp/oa/dedup/EntityMergerTest.java | 10 +- .../dnetlib/dhp/dedup/json/dataset_merge.json | 2 + 5 files changed, 239 insertions(+), 33 deletions(-) create mode 100644 dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/dataset_merge.json diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java new file mode 100644 index 000000000..5792fc10f --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java @@ -0,0 +1,104 @@ + +package eu.dnetlib.dhp.schema.oaf.utils; + +import static eu.dnetlib.dhp.schema.common.ModelConstants.CROSSREF_ID; + +import java.util.*; +import java.util.stream.Collectors; + +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Result; + +public class MergeEntitiesComparator implements Comparator { + static final List PID_AUTHORITIES = Arrays + .asList( + ModelConstants.ARXIV_ID, + ModelConstants.PUBMED_CENTRAL_ID, + ModelConstants.EUROPE_PUBMED_CENTRAL_ID, + ModelConstants.DATACITE_ID, + ModelConstants.CROSSREF_ID); + + static final List RESULT_TYPES = Arrays + .asList( + ModelConstants.ORP_RESULTTYPE_CLASSID, + ModelConstants.SOFTWARE_RESULTTYPE_CLASSID, + ModelConstants.DATASET_RESULTTYPE_CLASSID, + ModelConstants.PUBLICATION_RESULTTYPE_CLASSID); + + public static final Comparator INSTANCE = new MergeEntitiesComparator(); + + @Override + public int compare(Oaf left, Oaf right) { + if (left == null && right == null) + return 0; + if (left == null) + return -1; + if (right == null) + return 1; + + int res = 0; + + // pid authority + int cfp1 = left + .getCollectedfrom() + .stream() + .map(kv -> PID_AUTHORITIES.indexOf(kv.getKey())) + .max(Integer::compare) + .orElse(-1); + int cfp2 = right + .getCollectedfrom() + .stream() + .map(kv -> PID_AUTHORITIES.indexOf(kv.getKey())) + .max(Integer::compare) + .orElse(-1); + + if (cfp1 >= 0 && cfp1 > cfp2) { + return 1; + } else if (cfp2 >= 0 && cfp2 > cfp1) { + return -1; + } + + // trust + if (left.getDataInfo() != null && right.getDataInfo() != null) { + res = left.getDataInfo().getTrust().compareTo(right.getDataInfo().getTrust()); + } + + // result type + if (res == 0) { + if (left instanceof Result && right instanceof Result) { + Result r1 = (Result) left; + Result r2 = (Result) right; + + if (r1.getResulttype() == null || r1.getResulttype().getClassid() == null) { + if (r2.getResulttype() != null && r2.getResulttype().getClassid() != null) { + return -1; + } + } else if (r2.getResulttype() == null || r2.getResulttype().getClassid() == null) { + return 1; + } + + int rt1 = RESULT_TYPES.indexOf(r1.getResulttype().getClassid()); + int rt2 = RESULT_TYPES.indexOf(r2.getResulttype().getClassid()); + + if (rt1 >= 0 && rt1 > rt2) { + return 1; + } else if (rt2 >= 0 && rt2 > rt1) { + return -1; + } + } + } + + // id + if (res == 0) { + if (left instanceof OafEntity && right instanceof OafEntity) { + res = ((OafEntity) left).getId().compareTo(((OafEntity) right).getId()); + } + } + + return res; + } + +} diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index 28db94766..f1221add3 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -40,27 +40,12 @@ public class MergeUtils { public static T mergeGroup(String s, Iterator oafEntityIterator, boolean checkDelegateAuthority) { - TreeSet sortedEntities = new TreeSet<>((o1, o2) -> { - int res = 0; - if (o1.getDataInfo() != null && o2.getDataInfo() != null) { - res = o1.getDataInfo().getTrust().compareTo(o2.getDataInfo().getTrust()); - } + ArrayList sortedEntities = new ArrayList<>(); + oafEntityIterator.forEachRemaining(sortedEntities::add); + sortedEntities.sort(MergeEntitiesComparator.INSTANCE.reversed()); - if (res == 0) { - if (o1 instanceof Result && o2 instanceof Result) { - return ResultTypeComparator.INSTANCE.compare((Result) o1, (Result) o2); - } - } - - return res; - }); - - while (oafEntityIterator.hasNext()) { - sortedEntities.add(oafEntityIterator.next()); - } - - Iterator it = sortedEntities.descendingIterator(); + Iterator it = sortedEntities.iterator(); T merged = it.next(); while (it.hasNext()) { @@ -143,7 +128,7 @@ public class MergeUtils { * https://graph.openaire.eu/docs/data-model/pids-and-identifiers#delegated-authorities and in that case it prefers * such version. *

- * Otherwise, it considers a resulttype priority order implemented in {@link ResultTypeComparator} + * Otherwise, it considers a resulttype priority order implemented in {@link MergeEntitiesComparator} * and proceeds with the canonical property merging. * * @param left @@ -161,8 +146,9 @@ public class MergeUtils { if (!leftFromDelegatedAuthority && rightFromDelegatedAuthority) { return right; } + // TODO: raise trust to have preferred fields from one or the other?? - if (new ResultTypeComparator().compare(left, right) < 0) { + if (MergeEntitiesComparator.INSTANCE.compare(left, right) > 0) { return mergeResultFields(left, right); } else { return mergeResultFields(right, left); @@ -225,9 +211,9 @@ public class MergeUtils { private static List mergeLists(final List left, final List right, int trust, Function keyExtractor, BinaryOperator merger) { - if (left == null) { - return right; - } else if (right == null) { + if (left == null || left.isEmpty()) { + return right != null ? right : new ArrayList<>(); + } else if (right == null || right.isEmpty()) { return left; } @@ -405,7 +391,7 @@ public class MergeUtils { } // should be an instance attribute, get the first non-null value - merge.setLanguage(coalesce(merge.getLanguage(), enrich.getLanguage())); + merge.setLanguage(coalesceQualifier(merge.getLanguage(), enrich.getLanguage())); // distinct countries, do not manage datainfo merge.setCountry(mergeQualifiers(merge.getCountry(), enrich.getCountry(), trust)); @@ -575,6 +561,13 @@ public class MergeUtils { return m != null ? m : e; } + private static Qualifier coalesceQualifier(Qualifier m, Qualifier e) { + if (m == null || m.getClassid() == null || StringUtils.isBlank(m.getClassid())) { + return e; + } + return m; + } + private static List mergeAuthors(List author, List author1, int trust) { List> authors = new ArrayList<>(); if (author != null) { @@ -587,6 +580,10 @@ public class MergeUtils { } private static String instanceKeyExtractor(Instance i) { + // three levels of concatenating: + // 1. :: + // 2. @@ + // 3. || return String .join( "::", @@ -594,10 +591,10 @@ public class MergeUtils { kvKeyExtractor(i.getCollectedfrom()), qualifierKeyExtractor(i.getAccessright()), qualifierKeyExtractor(i.getInstancetype()), - Optional.ofNullable(i.getUrl()).map(u -> String.join("::", u)).orElse(null), + Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), Optional .ofNullable(i.getPid()) - .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("::"))) + .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) .orElse(null)); } @@ -706,7 +703,7 @@ public class MergeUtils { private static String spKeyExtractor(StructuredProperty sp) { return Optional .ofNullable(sp) - .map(s -> Joiner.on("::").join(s, qualifierKeyExtractor(s.getQualifier()))) + .map(s -> Joiner.on("||").join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) .orElse(null); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java new file mode 100644 index 000000000..726814c43 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java @@ -0,0 +1,103 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.BufferedReader; +import java.io.FileReader; +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; + +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; +import eu.dnetlib.pace.util.MapDocumentUtil; +import scala.Tuple2; + +class DatasetMergerTest implements Serializable { + + private List> datasets; + + private String testEntityBasePath; + private DataInfo dataInfo; + private final String dedupId = "50|doi_________::3d18564ef27ebe9ef3bd8b4dec67e148"; + private Dataset dataset_top; + + @BeforeEach + public void setUp() throws Exception { + testEntityBasePath = Paths + .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/json").toURI()) + .toFile() + .getAbsolutePath(); + + datasets = readSample(testEntityBasePath + "/dataset_merge.json", Dataset.class); + + dataset_top = getTopPub(datasets); + + dataInfo = setDI(); + } + + @Test + void datasetMergerTest() throws InstantiationException, IllegalAccessException, InvocationTargetException { + Dataset pub_merged = MergeUtils.mergeGroup(dedupId, datasets.stream().map(Tuple2::_2).iterator()); + + // verify id + assertEquals(dedupId, pub_merged.getId()); + assertEquals(2, pub_merged.getInstance().size()); + } + + public DataInfo setDI() { + DataInfo dataInfo = new DataInfo(); + dataInfo.setTrust("0.9"); + dataInfo.setDeletedbyinference(false); + dataInfo.setInferenceprovenance("testing"); + dataInfo.setInferred(true); + return dataInfo; + } + + public Dataset getTopPub(List> publications) { + + Double maxTrust = 0.0; + Dataset maxPub = new Dataset(); + for (Tuple2 publication : publications) { + Double pubTrust = Double.parseDouble(publication._2().getDataInfo().getTrust()); + if (pubTrust > maxTrust) { + maxTrust = pubTrust; + maxPub = publication._2(); + } + } + return maxPub; + } + + public List> readSample(String path, Class clazz) { + List> res = new ArrayList<>(); + BufferedReader reader; + try { + reader = new BufferedReader(new FileReader(path)); + String line = reader.readLine(); + while (line != null) { + res + .add( + new Tuple2<>( + MapDocumentUtil.getJPathString("$.id", line), + new ObjectMapper().readValue(line, clazz))); + // read next line + line = reader.readLine(); + } + reader.close(); + } catch (IOException e) { + e.printStackTrace(); + } + + return res; + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java index 4a5a3bd1b..995407edb 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java @@ -93,14 +93,14 @@ class EntityMergerTest implements Serializable { assertEquals(pub_top.getJournal().getConferencedate(), pub_merged.getJournal().getConferencedate()); assertEquals(pub_top.getJournal().getConferenceplace(), pub_merged.getJournal().getConferenceplace()); assertEquals("OPEN", pub_merged.getBestaccessright().getClassid()); - assertEquals(pub_top.getResulttype(), pub_merged.getResulttype()); - assertEquals(pub_top.getLanguage(), pub_merged.getLanguage()); - assertEquals(pub_top.getPublisher(), pub_merged.getPublisher()); - assertEquals(pub_top.getEmbargoenddate(), pub_merged.getEmbargoenddate()); + assertEquals(pub_top.getResulttype().getClassid(), pub_merged.getResulttype().getClassid()); + assertEquals(pub_top.getLanguage().getClassid(), pub_merged.getLanguage().getClassid()); + assertEquals("Elsevier BV", pub_merged.getPublisher().getValue()); + assertEquals(pub_top.getEmbargoenddate().getValue(), pub_merged.getEmbargoenddate().getValue()); assertEquals(pub_top.getResourcetype().getClassid(), ""); assertEquals(pub_top.getDateoftransformation(), pub_merged.getDateoftransformation()); assertEquals(pub_top.getOaiprovenance(), pub_merged.getOaiprovenance()); - assertEquals(pub_top.getDateofcollection(), pub_merged.getDateofcollection()); + // assertEquals(pub_top.getDateofcollection(), pub_merged.getDateofcollection()); assertEquals(3, pub_merged.getInstance().size()); assertEquals(2, pub_merged.getCountry().size()); assertEquals(0, pub_merged.getSubject().size()); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/dataset_merge.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/dataset_merge.json new file mode 100644 index 000000000..86d1a8133 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/dataset_merge.json @@ -0,0 +1,2 @@ +{"publisher": {"value": "DANS Data Station Archaeology"}, "dateofcollection": "2024-04-30T12:49:55+0000", "dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": true}, "bestaccessright": {"classid": "UNKNOWN", "classname": "not available", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "language": {"classid": "und", "classname": "Undetermined", "schemeid": "dnet:languages", "schemename": "dnet:languages"}, "author": [{"affiliation": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "(Geonius)"}], "fullname": "S.A.H. Augustin", "pid": [], "rank": 1}, {"affiliation": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "(Geonius)"}], "fullname": "J.J.G. Geraeds", "pid": [], "rank": 2}], "resourcetype": {"classid": "UNKNOWN", "classname": "Unknown", "schemeid": "dnet:dataCite_resource", "schemename": "dnet:dataCite_resource"}, "originalId": ["50|datacite____::3d18564ef27ebe9ef3bd8b4dec67e148", "10.17026/ar/vt9ya1"], "description": [{"value": "In opdracht van Gemeente Beesel heeft Geonius Archeologie in maart 2023 een Inventariserend Veldonderzoek door middel van Proefsleuven (IVO-P) uitgevoerd voor het plangebied Heijackerstraat te Beesel in de gemeente Beesel. Aanleiding voor het uitvoeren van het archeologisch onderzoek vormt de aanvraag van een omgevingsvergunning voor bouw van 20 nieuwe woningen. Uit het vooronderzoek is gebleken dat het plangebied in een dalvakteterras ligt rondom opgestoven landduinen langsheen de Maas. De bodem bestaat volgens de bodemkaart uit vorstvaaggronden. Het plangebied is in het verleden voor zover kon worden vastgesteld in gebruik geweest als bouwland en is niet bebouwd geweest. Het IVO-O heeft uitgewezen dat de bodemopbouw deels intact is, al lijken in sommige boringen sporen van vergravingen of verstoringen te bevatten. Op grond van de resultaten van het vooronderzoek is een hoge verwachting opgesteld op het voorkomen van archeologische waarden uit het paleolithicum tot aan de vroege middeleeuwen. Voor de periode late middeleeuwen en nieuwe tijd is een lage verwachting opgesteld. Op grond van de resultaten van het vooronderzoek is een IVO-P uitgevoerd. Hierbij is een vindplaats aangetroffen bestaande uit drie subrecente greppels en een tweetal recente verstoringen. De vindplaats is als niet behoudenswaardig gewaardeerd. Aanbevolen is het plangebied vrij te geven voor de geplande ontwikkeling."}], "title": [{"qualifier": {"classid": "main title", "classname": "main title", "schemeid": "dnet:dataCite_title", "schemename": "dnet:dataCite_title"}, "value": "Archeologisch onderzoek IVO-P plangebied Heijackerstraat te Beesel"}], "pid": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.17026/ar/vt9ya1"}], "id": "50|doi_________::3d18564ef27ebe9ef3bd8b4dec67e148", "instance": [{"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemeid": "dnet:review_levels", "schemename": "dnet:review_levels"}, "hostedby": {"key": "10|re3data_____::84e123776089ce3c7a33db98d9cd15a8", "value": "EASY"}, "url": ["https://dx.doi.org/10.17026/ar/vt9ya1"], "pid": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.17026/ar/vt9ya1"}], "instanceTypeMapping": [{"originalType": "Dataset", "typeLabel": "dataset", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_ddb1"}], "dateofacceptance": {"value": "2024-01-01"}, "collectedfrom": {"key": "10|openaire____::9e3be59865b2c1c335d32dae2fe7b254", "value": "Datacite"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "instancetype": {"classid": "0021", "classname": "Dataset", "schemeid": "dnet:publication_resource", "schemename": "dnet:publication_resource"}}], "relevantdate": [{"qualifier": {"classid": "issued", "classname": "issued", "schemeid": "dnet:dataCite_date", "schemename": "dnet:dataCite_date"}, "value": "2024-01-01"}], "resulttype": {"classid": "dataset", "classname": "dataset", "schemeid": "dnet:result_typologies", "schemename": "dnet:result_typologies"}, "context": [], "collectedfrom": [{"key": "10|openaire____::9e3be59865b2c1c335d32dae2fe7b254", "value": "Datacite"}], "dateoftransformation": "2024-04-30T12:49:55+0000", "subject": [], "dateofacceptance": {"value": "2024-01-01"}, "metaResourceType": {"classid": "Research Data", "classname": "Research Data", "schemeid": "openaire::meta_resource_types", "schemename": "openaire::meta_resource_types"}} +{"geolocation": [{"box": "", "place": "", "point": ""}], "dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": true}, "resourcetype": {"classid": "dataset", "classname": "dataset", "schemeid": "dnet:dataCite_resource", "schemename": "dnet:dataCite_resource"}, "pid": [], "contributor": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "Geonius"}], "oaiprovenance": {"originDescription": {"metadataNamespace": "", "harvestDate": "2024-05-05T04:33:31Z", "baseURL": "https://easy.dans.knaw.nl/oai", "datestamp": "", "altered": true, "identifier": "oai:easy.dans.knaw.nl:easy-dataset:341200"}}, "bestaccessright": {"classid": "OPEN", "classname": "Open Access", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "relevantdate": [], "collectedfrom": [{"key": "10|re3data_____::730f562f9efe8a3b3742d2da510d4335", "value": "B2FIND"}], "id": "50|r3730f562f9e::ace629fb505b6b4343faca03edde1841", "subject": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies"}, "value": "Ancient Cultures"}, {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies"}, "value": "Humanities"}, {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies"}, "value": "Archaeology"}], "lastupdatetimestamp": 1716803651625, "author": [{"surname": "Augustin", "name": "S. A. H.", "pid": [], "rank": 1, "affiliation": [], "fullname": "S.A.H. Augustin"}, {"surname": "Geraeds", "name": "J. J. G.", "pid": [], "rank": 2, "affiliation": [], "fullname": "J.J.G. Geraeds"}], "instance": [{"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemeid": "dnet:review_levels", "schemename": "dnet:review_levels"}, "hostedby": {"key": "10|re3data_____::84e123776089ce3c7a33db98d9cd15a8", "value": "DANS-EASY"}, "url": ["http://dx.doi.org/https://doi.org/10.17026/AR/VT9YA1"], "pid": [], "instanceTypeMapping": [{"originalType": "Dataset", "typeLabel": "dataset", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_ddb1"}], "alternateIdentifier": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.17026/ar/vt9ya1"}, {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.17026/ar/vt9ya1"}, {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.17026/ar/vt9ya1"}], "dateofacceptance": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "2024-01-01"}, "collectedfrom": {"key": "10|re3data_____::730f562f9efe8a3b3742d2da510d4335", "value": "B2FIND"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "instancetype": {"classid": "0021", "classname": "Dataset", "schemeid": "dnet:publication_resource", "schemename": "dnet:publication_resource"}}], "dateofcollection": "2024-05-10T00:02:09+0000", "metaResourceType": {"classid": "Research Data", "classname": "Research Data", "schemeid": "openaire::meta_resource_types", "schemename": "openaire::meta_resource_types"}, "dateoftransformation": "2024-05-26T00:23:54.028Z", "description": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "In opdracht van Gemeente Beesel heeft Geonius Archeologie in maart 2023 een Inventariserend Veldonderzoek door middel van Proefsleuven (IVO-P) uitgevoerd voor het plangebied Heijackerstraat te Beesel in de gemeente Beesel. Aanleiding voor het uitvoeren van het archeologisch onderzoek vormt de aanvraag van een omgevingsvergunning voor bouw van 20 nieuwe woningen. Uit het vooronderzoek is gebleken dat het plangebied in een dalvakteterras ligt rondom opgestoven landduinen langsheen de Maas. De bodem bestaat volgens de bodemkaart uit vorstvaaggronden. Het plangebied is in het verleden voor zover kon worden vastgesteld in gebruik geweest als bouwland en is niet bebouwd geweest. Het IVO-O heeft uitgewezen dat de bodemopbouw deels intact is, al lijken in sommige boringen sporen van vergravingen of verstoringen te bevatten. Op grond van de resultaten van het vooronderzoek is een hoge verwachting opgesteld op het voorkomen van archeologische waarden uit het paleolithicum tot aan de vroege middeleeuwen. Voor de periode late middeleeuwen en nieuwe tijd is een lage verwachting opgesteld. Op grond van de resultaten van het vooronderzoek is een IVO-P uitgevoerd. Hierbij is een vindplaats aangetroffen bestaande uit drie subrecente greppels en een tweetal recente verstoringen. De vindplaats is als niet behoudenswaardig gewaardeerd. Aanbevolen is het plangebied vrij te geven voor de geplande ontwikkeling."}], "format": [], "coverage": [], "externalReference": [], "publisher": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "Data Archiving and Networked Services (DANS)"}, "context": [], "eoscifguidelines": [], "language": {"classid": "und", "classname": "Undetermined", "schemeid": "dnet:languages", "schemename": "dnet:languages"}, "resulttype": {"classid": "dataset", "classname": "dataset", "schemeid": "dnet:result_typologies", "schemename": "dnet:result_typologies"}, "country": [], "extraInfo": [], "originalId": ["oai:easy.dans.knaw.nl:easy-dataset:341200", "50|r3730f562f9e::ace629fb505b6b4343faca03edde1841"], "source": [], "dateofacceptance": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "value": "2024-01-01"}, "title": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "trust": "0.9", "inferred": false, "deletedbyinference": false}, "qualifier": {"classid": "main title", "classname": "main title", "schemeid": "dnet:dataCite_title", "schemename": "dnet:dataCite_title"}, "value": "Archeologisch onderzoek IVO-P plangebied Heijackerstraat te Beesel"}]} \ No newline at end of file From 92c3abd5a4e969cc47313b851f2f93f5c7dcfc87 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 29 May 2024 14:36:49 +0200 Subject: [PATCH 019/239] [graph cleaning] use sparkExecutorMemory to define also the memoryOverhead --- .../eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml index f6bf053cd..4188cb018 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml @@ -156,6 +156,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -190,6 +191,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -224,6 +226,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -258,6 +261,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -292,6 +296,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -326,6 +331,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -360,6 +366,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -394,6 +401,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} From 1477406ecc06eafcc704c55e2a317a74e13fcf62 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Wed, 5 Jun 2024 16:20:40 +0200 Subject: [PATCH 020/239] [bulkTag] fixed issue that made project disappear in graph_10_enriched --- .../src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 9e1acc7b2..354741690 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -172,7 +172,7 @@ public class SparkBulkTagJob { .option("compression", "gzip") .json(outputPath + "project"); - readPath(spark, outputPath + "project", Datasource.class) + readPath(spark, outputPath + "project", Project.class) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") From ec79405cc9e4cbdadeb983be3b01408c259bc751 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 7 Jun 2024 11:30:31 +0200 Subject: [PATCH 021/239] [graph raw] set organization type from openorgs --- .../dhp/oa/graph/raw/MigrateDbEntitiesApplication.java | 1 + .../dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql | 7 +++++-- pom.xml | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index da6885db3..c9a32cde6 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -398,6 +398,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i o.setEcsmevalidated(field(Boolean.toString(rs.getBoolean("ecsmevalidated")), info)); o.setEcnutscode(field(Boolean.toString(rs.getBoolean("ecnutscode")), info)); o.setCountry(prepareQualifierSplitting(rs.getString("country"))); + o.setOrganizationType(Organization.OrganizationType.valueOf(rs.getString("typology"))); o.setDataInfo(info); o.setLastupdatetimestamp(lastUpdateTimestamp); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql index 0ec303939..16ad9e265 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql @@ -28,7 +28,8 @@ SELECT (array_remove(array_cat(ARRAY[o.ec_internationalorganization], array_agg(od.ec_internationalorganization)), NULL))[1] AS ecinternationalorganization, (array_remove(array_cat(ARRAY[o.ec_enterprise], array_agg(od.ec_enterprise)), NULL))[1] AS ecenterprise, (array_remove(array_cat(ARRAY[o.ec_smevalidated], array_agg(od.ec_smevalidated)), NULL))[1] AS ecsmevalidated, - (array_remove(array_cat(ARRAY[o.ec_nutscode], array_agg(od.ec_nutscode)), NULL))[1] AS ecnutscode + (array_remove(array_cat(ARRAY[o.ec_nutscode], array_agg(od.ec_nutscode)), NULL))[1] AS ecnutscode, + org_types.name AS typology FROM organizations o LEFT OUTER JOIN acronyms a ON (a.id = o.id) LEFT OUTER JOIN urls u ON (u.id = o.id) @@ -37,6 +38,7 @@ FROM organizations o LEFT OUTER JOIN oa_duplicates d ON (o.id = d.local_id AND d.reltype != 'is_different') LEFT OUTER JOIN organizations od ON (d.oa_original_id = od.id) LEFT OUTER JOIN other_ids idup ON (od.id = idup.id) + LEFT OUTER JOIN org_types ON (org_types.val = o.type) WHERE o.status = 'approved' OR o.status = 'suggested' GROUP BY @@ -44,4 +46,5 @@ GROUP BY o.name, o.creation_date, o.modification_date, - o.country; \ No newline at end of file + o.country, + org_types.name; \ No newline at end of file diff --git a/pom.xml b/pom.xml index cc8d509f7..9e554204d 100644 --- a/pom.xml +++ b/pom.xml @@ -960,7 +960,7 @@ 1.1.3 1.7 1.0.7 - [6.1.2] + [6.1.3-SNAPSHOT] cdh5.9.2 3.5 11.0.2 From c7265724182b7747540e69eaec2732799dca2d0f Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Fri, 7 Jun 2024 12:03:26 +0200 Subject: [PATCH 022/239] changed some parameters in OSF test --- .../dhp/collection/plugin/rest/OsfPreprintCollectorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java index 0e64f8bab..a1b723e33 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java @@ -36,8 +36,8 @@ public class OsfPreprintCollectorTest { private final String resultTotalXpath = "/*/*[local-name()='links']/*[local-name()='meta']/*[local-name()='total']"; private final String resumptionParam = "page"; - private final String resumptionType = "page"; - private final String resumptionXpath = "/*/*[local-name()='links']/*[local-name()='next']"; + private final String resumptionType = "scan"; + private final String resumptionXpath = "substring-before(substring-after(/*/*[local-name()='links']/*[local-name()='next'], 'page='), '&')"; private final String resultSizeParam = "page[size]"; private final String resultSizeValue = "100"; From 3776327a8cc3e725af8af49255598681eac83d1f Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 10 Jun 2024 15:22:33 +0200 Subject: [PATCH 023/239] hostedby patching to work with the updated Crossref contents, resolved conflict --- .../SparkApplyHostedByMapToResult.scala | 39 ++++++++++++------- 1 file changed, 25 insertions(+), 14 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/hostedbymap/SparkApplyHostedByMapToResult.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/hostedbymap/SparkApplyHostedByMapToResult.scala index a900fc241..db7edf53e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/hostedbymap/SparkApplyHostedByMapToResult.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/hostedbymap/SparkApplyHostedByMapToResult.scala @@ -25,27 +25,38 @@ object SparkApplyHostedByMapToResult { val i = p.getInstance().asScala if (i.size == 1) { val inst: Instance = i.head - inst.getHostedby.setKey(ei.getHostedById) - inst.getHostedby.setValue(ei.getName) - if (ei.getOpenAccess) { - inst.setAccessright( - OafMapperUtils.accessRight( - ModelConstants.ACCESS_RIGHT_OPEN, - "Open Access", - ModelConstants.DNET_ACCESS_MODES, - ModelConstants.DNET_ACCESS_MODES - ) - ) - inst.getAccessright.setOpenAccessRoute(OpenAccessRoute.gold) - p.setBestaccessright(OafMapperUtils.createBestAccessRights(p.getInstance())); - } + patchInstance(p, ei, inst) + } else { + val cf = i.map(ii => ii.getCollectedfrom.getValue) + if (cf.contains("Crossref")) { + i.foreach(ii => { + patchInstance(p, ei, ii) + }) + } } } p })(Encoders.bean(classOf[Publication])) } + private def patchInstance(p: Publication, ei: EntityInfo, inst: Instance): Unit = { + inst.getHostedby.setKey(ei.getHostedById) + inst.getHostedby.setValue(ei.getName) + if (ei.getOpenAccess) { + inst.setAccessright( + OafMapperUtils.accessRight( + ModelConstants.ACCESS_RIGHT_OPEN, + "Open Access", + ModelConstants.DNET_ACCESS_MODES, + ModelConstants.DNET_ACCESS_MODES + ) + ) + inst.getAccessright.setOpenAccessRoute(OpenAccessRoute.gold) + p.setBestaccessright(OafMapperUtils.createBestAccessRights(p.getInstance())); + } + } + def main(args: Array[String]): Unit = { val logger: Logger = LoggerFactory.getLogger(getClass) From b0eba210c068219580cfa78c17aa23f2e1e170f8 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 10 Jun 2024 16:15:07 +0200 Subject: [PATCH 024/239] [actionset promotion] use sparkExecutorMemory to define also the memoryOverhead --- dhp-shade-package/dependency-reduced-pom.xml | 113 ++++++++++++++++++ .../wf/dataset/oozie_app/workflow.xml | 2 + .../wf/datasource/oozie_app/workflow.xml | 1 + .../wf/main/oozie_app/workflow.xml | 1 + .../wf/organization/oozie_app/workflow.xml | 1 + .../oozie_app/workflow.xml | 4 +- .../wf/project/oozie_app/workflow.xml | 1 + .../wf/publication/oozie_app/workflow.xml | 6 +- .../wf/relation/oozie_app/workflow.xml | 3 +- .../wf/software/oozie_app/workflow.xml | 4 +- 10 files changed, 131 insertions(+), 5 deletions(-) create mode 100644 dhp-shade-package/dependency-reduced-pom.xml diff --git a/dhp-shade-package/dependency-reduced-pom.xml b/dhp-shade-package/dependency-reduced-pom.xml new file mode 100644 index 000000000..04843072f --- /dev/null +++ b/dhp-shade-package/dependency-reduced-pom.xml @@ -0,0 +1,113 @@ + + + + dhp + eu.dnetlib.dhp + 1.2.5-SNAPSHOT + + 4.0.0 + dhp-shade-package + This module create a jar of all module dependencies + + + + maven-shade-plugin + + + package + + shade + + + + + eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels + + + + META-INF/cxf/bus-extensions.txt + + + + + *:* + + META-INF/maven/** + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com + repackaged.com.google.common + + com.google.common.** + + + + + + + + + + + + org.projectlombok + lombok + 1.18.28 + provided + + + org.junit.jupiter + junit-jupiter + 5.6.1 + test + + + junit-jupiter-api + org.junit.jupiter + + + junit-jupiter-params + org.junit.jupiter + + + junit-jupiter-engine + org.junit.jupiter + + + + + org.mockito + mockito-core + 3.3.3 + test + + + byte-buddy + net.bytebuddy + + + byte-buddy-agent + net.bytebuddy + + + + + org.mockito + mockito-junit-jupiter + 3.3.3 + test + + + + + DHPSite + ${dhp.site.stage.path}/dhp-common + + + 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 5401b45ca..b1bc1d6e1 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 @@ -103,6 +103,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -156,6 +157,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} 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 f9bd66ae3..9a84f4708 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 @@ -95,6 +95,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} 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 393f04e89..65ddd402b 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 @@ -125,6 +125,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} 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 ebfdeee31..0d73b498d 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 @@ -95,6 +95,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} 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 02399ed9b..ca8362c9b 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 @@ -103,6 +103,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -155,11 +156,12 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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=2560 + --conf spark.sql.shuffle.partitions=8000 --inputGraphTablePath${workingDir}/otherresearchproduct --graphTableClassNameeu.dnetlib.dhp.schema.oaf.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 57c2357b4..37310da79 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 @@ -95,6 +95,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} 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 92b114776..a4b0b237c 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 @@ -103,11 +103,12 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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=7000 + --conf spark.sql.shuffle.partitions=15000 --inputGraphTablePath${inputGraphRootPath}/publication --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Publication @@ -156,11 +157,12 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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=7000 + --conf spark.sql.shuffle.partitions=15000 --inputGraphTablePath${workingDir}/publication --graphTableClassNameeu.dnetlib.dhp.schema.oaf.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 e9e5f0b45..f72847ba8 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 @@ -95,11 +95,12 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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=10000 + --conf spark.sql.shuffle.partitions=15000 --inputGraphTablePath${inputGraphRootPath}/relation --graphTableClassNameeu.dnetlib.dhp.schema.oaf.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 1d36ddf94..dbe8a63c1 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 @@ -103,6 +103,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -155,11 +156,12 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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=2560 + --conf spark.sql.shuffle.partitions=4000 --inputGraphTablePath${workingDir}/software --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Software From 85c1eae7e0da86e25aa2da74d90ac82ecf150e27 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Mon, 10 Jun 2024 19:03:30 +0200 Subject: [PATCH 025/239] Fixes for pagination strategy looping at end of download --- .../collection/plugin/rest/RestIterator.java | 58 +++++++++++++------ 1 file changed, 40 insertions(+), 18 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java index 2518fd92f..9037a454e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java @@ -12,6 +12,8 @@ import java.util.Iterator; import java.util.Map; import java.util.Queue; import java.util.concurrent.PriorityBlockingQueue; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import javax.xml.transform.OutputKeys; import javax.xml.transform.Transformer; @@ -19,16 +21,10 @@ import javax.xml.transform.TransformerConfigurationException; import javax.xml.transform.TransformerFactory; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; -import javax.xml.xpath.XPath; -import javax.xml.xpath.XPathConstants; -import javax.xml.xpath.XPathExpression; -import javax.xml.xpath.XPathExpressionException; -import javax.xml.xpath.XPathFactory; +import javax.xml.xpath.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.http.HttpHeaders; -import org.apache.http.entity.ContentType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.w3c.dom.Node; @@ -51,7 +47,6 @@ import eu.dnetlib.dhp.common.collection.HttpClientParams; * */ public class RestIterator implements Iterator { - private static final Logger log = LoggerFactory.getLogger(RestIterator.class); public static final String UTF_8 = "UTF-8"; private static final int MAX_ATTEMPTS = 5; @@ -60,11 +55,15 @@ public class RestIterator implements Iterator { private final String AUTHBASIC = "basic"; + private static final String XML_HEADER = ""; + private static final String EMPTY_XML = XML_HEADER + "<" + JsonUtils.XML_WRAP_TAG + ">"; + private final String baseUrl; private final String resumptionType; private final String resumptionParam; private final String resultFormatValue; - private String queryParams; + private String queryParams = ""; private final int resultSizeValue; private int resumptionInt = 0; // integer resumption token (first record to harvest) private int resultTotal = -1; @@ -158,7 +157,12 @@ public class RestIterator implements Iterator { } private void initQueue() { - this.query = this.baseUrl + "?" + this.queryParams + this.querySize + this.queryFormat; + if (queryParams.equals("") && querySize.equals("") && queryFormat.equals("")) { + query = baseUrl; + } else { + query = baseUrl + "?" + queryParams + querySize + queryFormat; + } + log.info("REST calls starting with {}", this.query); } @@ -219,9 +223,8 @@ public class RestIterator implements Iterator { try { String resultJson; - String resultXml = ""; + String resultXml = XML_HEADER; String nextQuery = ""; - final String emptyXml = resultXml + "<" + JsonUtils.XML_WRAP_TAG + ">"; Node resultNode = null; NodeList nodeList = null; String qUrlArgument = ""; @@ -236,6 +239,21 @@ public class RestIterator implements Iterator { } } + // find pagination page start number in queryParam and remove before start the first query + if ((resumptionType.toLowerCase().equals("pagination") || resumptionType.toLowerCase().equals("page")) + && (query.contains("paginationStart="))) { + + final Matcher m = Pattern.compile("paginationStart=([0-9]+)").matcher(query); + m.find(); // guaranteed to be true for this regex + + String[] pageVal = m.group(0).split("="); + pagination = Integer.parseInt(pageVal[1]); + + // remove page start number from queryParams + query = query.replaceFirst("&?paginationStart=[0-9]+", ""); + + } + try { log.info("requesting URL [{}]", query); @@ -261,7 +279,7 @@ public class RestIterator implements Iterator { this.resultStream = IOUtils.toInputStream(resultXml, UTF_8); } - if (!(emptyXml).equalsIgnoreCase(resultXml)) { + if (!isEmptyXml(resultXml)) { resultNode = (Node) this.xpath .evaluate("/", new InputSource(this.resultStream), XPathConstants.NODE); nodeList = (NodeList) this.xprEntity.evaluate(resultNode, XPathConstants.NODESET); @@ -270,8 +288,7 @@ public class RestIterator implements Iterator { final StringWriter sw = new StringWriter(); this.transformer.transform(new DOMSource(nodeList.item(i)), new StreamResult(sw)); final String toEnqueue = sw.toString(); - if ((toEnqueue == null) || StringUtils.isBlank(toEnqueue) - || emptyXml.equalsIgnoreCase(toEnqueue)) { + if ((toEnqueue == null) || StringUtils.isBlank(toEnqueue) || isEmptyXml(toEnqueue)) { log .warn( "The following record resulted in empty item for the feeding queue: {}", resultXml); @@ -299,6 +316,7 @@ public class RestIterator implements Iterator { throw new CollectorException("Mode: discover, Param 'resultSizeValue' is less than 2"); } qUrlArgument = qUrl.getQuery(); + final String[] arrayQUrlArgument = qUrlArgument.split("&"); for (final String arrayUrlArgStr : arrayQUrlArgument) { if (arrayUrlArgStr.startsWith(this.resumptionParam)) { @@ -312,7 +330,7 @@ public class RestIterator implements Iterator { } } - if (((emptyXml).equalsIgnoreCase(resultXml)) + if (isEmptyXml(resultXml) || ((nodeList != null) && (nodeList.getLength() < this.resultSizeValue))) { // resumptionStr = ""; if (nodeList != null) { @@ -331,13 +349,13 @@ public class RestIterator implements Iterator { case "pagination": case "page": // pagination, iterate over page numbers - this.pagination += 1; - if (nodeList != null) { + if (nodeList != null && nodeList.getLength() > 0) { this.discoverResultSize += nodeList.getLength(); } else { this.resultTotal = this.discoverResultSize; this.pagination = this.discoverResultSize; } + this.pagination += 1; this.resumptionInt = this.pagination; this.resumptionStr = Integer.toString(this.resumptionInt); break; @@ -415,6 +433,10 @@ public class RestIterator implements Iterator { } + private boolean isEmptyXml(String s) { + return EMPTY_XML.equalsIgnoreCase(s); + } + private boolean isInteger(final String s) { boolean isValidInteger = false; try { From 9da006e98ceb53a120be36b5529f90ec765639b6 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 11 Jun 2024 10:28:32 +0200 Subject: [PATCH 026/239] [SDGFoSActionSet]remove datainfo for the result. It is not needed (qualifier.classid = UPDATE) useless since subject do not go at the level of the instance --- .../PrepareFOSSparkJob.java | 14 +------------- .../PrepareSDGSparkJob.java | 14 +------------- 2 files changed, 2 insertions(+), 26 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java index c248423d4..ec957a208 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java @@ -115,19 +115,7 @@ public class PrepareFOSSparkJob implements Serializable { .forEach( l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); r.setSubject(sbjs); - r - .setDataInfo( - OafMapperUtils - .dataInfo( - false, null, true, - false, - OafMapperUtils - .qualifier( - ModelConstants.PROVENANCE_ENRICH, - null, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - null)); + return r; } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java index bfdf14234..a88607986 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java @@ -81,19 +81,7 @@ public class PrepareSDGSparkJob implements Serializable { s -> sbjs .add(getSubject(s.getSbj(), SDG_CLASS_ID, SDG_CLASS_NAME, UPDATE_SUBJECT_SDG_CLASS_ID))); r.setSubject(sbjs); - r - .setDataInfo( - OafMapperUtils - .dataInfo( - false, null, true, - false, - OafMapperUtils - .qualifier( - ModelConstants.PROVENANCE_ENRICH, - null, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - null)); + return r; }, Encoders.bean(Result.class)) .write() From a8d68c9d294518734d9e6ac724ab40d9e4ff5d3d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 11 Jun 2024 12:40:50 +0200 Subject: [PATCH 027/239] avoid NPEs --- .../oaf/utils/MergeEntitiesComparator.java | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java index 5792fc10f..ff6c2689a 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java @@ -1,13 +1,9 @@ package eu.dnetlib.dhp.schema.oaf.utils; -import static eu.dnetlib.dhp.schema.common.ModelConstants.CROSSREF_ID; - import java.util.*; -import java.util.stream.Collectors; import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Oaf; import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.dhp.schema.oaf.Result; @@ -42,17 +38,23 @@ public class MergeEntitiesComparator implements Comparator { int res = 0; // pid authority - int cfp1 = left - .getCollectedfrom() - .stream() - .map(kv -> PID_AUTHORITIES.indexOf(kv.getKey())) - .max(Integer::compare) + int cfp1 = Optional + .ofNullable(left.getCollectedfrom()) + .map( + cf -> cf + .stream() + .map(kv -> PID_AUTHORITIES.indexOf(kv.getKey())) + .max(Integer::compare) + .orElse(-1)) .orElse(-1); - int cfp2 = right - .getCollectedfrom() - .stream() - .map(kv -> PID_AUTHORITIES.indexOf(kv.getKey())) - .max(Integer::compare) + int cfp2 = Optional + .ofNullable(right.getCollectedfrom()) + .map( + cf -> cf + .stream() + .map(kv -> PID_AUTHORITIES.indexOf(kv.getKey())) + .max(Integer::compare) + .orElse(-1)) .orElse(-1); if (cfp1 >= 0 && cfp1 > cfp2) { From 11fe3a4fe0af870ee519de5dd546d2d28483c4a4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 11 Jun 2024 14:21:01 +0200 Subject: [PATCH 028/239] [graph resolution] use sparkExecutorMemory to define also the memoryOverhead --- .../eu/dnetlib/dhp/oa/graph/resolution/oozie_app/workflow.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/workflow.xml index 74e792f07..916a9f2b1 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/workflow.xml @@ -45,6 +45,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.shuffle.partitions=15000 --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} @@ -79,6 +80,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.shuffle.partitions=10000 --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} From 4f2a61e10fc5b6983f6f637120eef0c67f38f73d Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Wed, 29 May 2024 23:05:20 +0200 Subject: [PATCH 029/239] Change the selection criteria for the pivot record of a group so that by best pid type becomes the first criteria. This will have the effect to slowly converge to records having DOI pid --- .../main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java index f73ff92ec..0ff40f6f8 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java @@ -203,8 +203,8 @@ public class SparkCreateMergeRels extends AbstractSparkAction { WindowSpec w = Window .partitionBy("groupId") .orderBy( - col("lastUsage").desc_nulls_last(), col("pidType").asc_nulls_last(), + col("lastUsage").desc_nulls_last(), col("collectedfrom").desc_nulls_last(), col("date").asc_nulls_last(), col("id").asc_nulls_last()); From d90cb099b8deababf4428626fd9e7f164fbe2ae1 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Tue, 11 Jun 2024 20:23:44 +0200 Subject: [PATCH 030/239] Fix for paginationStart parameter management --- .../eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java index 9037a454e..caef266d7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java @@ -249,7 +249,8 @@ public class RestIterator implements Iterator { String[] pageVal = m.group(0).split("="); pagination = Integer.parseInt(pageVal[1]); - // remove page start number from queryParams + // remove page start number from query and queryParams + queryParams = queryParams.replaceFirst("&?paginationStart=[0-9]+", ""); query = query.replaceFirst("&?paginationStart=[0-9]+", ""); } From 9bf2bda1c6916ea45675a71e9c115c83182371bd Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Wed, 12 Jun 2024 13:28:51 +0200 Subject: [PATCH 031/239] Fix: next returned a null value at end of stream --- .../collection/plugin/rest/RestIterator.java | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java index caef266d7..0895d5f43 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java @@ -176,19 +176,6 @@ public class RestIterator implements Iterator { */ @Override public boolean hasNext() { - if (this.recordQueue.isEmpty() && this.query.isEmpty()) { - disconnect(); - return false; - } - return true; - } - - /* - * (non-Javadoc) - * @see java.util.Iterator#next() - */ - @Override - public String next() { synchronized (this.recordQueue) { while (this.recordQueue.isEmpty() && !this.query.isEmpty()) { try { @@ -198,6 +185,23 @@ public class RestIterator implements Iterator { throw new RuntimeException(e); } } + + if (!this.recordQueue.isEmpty()) { + return true; + } + + disconnect(); + return false; + } + } + + /* + * (non-Javadoc) + * @see java.util.Iterator#next() + */ + @Override + public String next() { + synchronized (this.recordQueue) { return this.recordQueue.poll(); } } From d942a1101bfb60b536f56e39ab9a7a00925ac6f3 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Fri, 14 Jun 2024 12:14:38 +0300 Subject: [PATCH 032/239] Miscellaneous updates to the copying operation to Impala Cluster: - Show some counts and the elapsed time for various sub-tasks. - Code polishing. --- .../oozie_app/copyDataToImpalaCluster.sh | 35 +++++++++++-------- .../oozie_app/copyDataToImpalaCluster.sh | 35 +++++++++++-------- .../oozie_app/copyDataToImpalaCluster.sh | 35 +++++++++++-------- .../oozie_app/copyDataToImpalaCluster.sh | 35 +++++++++++-------- 4 files changed, 80 insertions(+), 60 deletions(-) diff --git a/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh index ca0f7a643..55deada40 100644 --- a/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh @@ -55,20 +55,20 @@ function print_elapsed_time() hours=$((elapsed_time / 3600)) minutes=$(((elapsed_time % 3600) / 60)) seconds=$((elapsed_time % 60)) - printf "\nElapsed time: %02d:%02d:%02d\n\n" $hours $minutes $seconds + printf "%02d:%02d:%02d" $hours $minutes $seconds } function copydb() { db=$1 - start_db_time=$(date +%s) echo -e "\nStart processing db: '${db}'..\n" + start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then - echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" rm -f error.log if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 2 @@ -78,6 +78,7 @@ function copydb() { fi echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" + start_file_transfer_time=$(date +%s) # Using max-bandwidth of: 70 * 150 Mb/s = 10.5 Gb/s # Using max memory of: 70 * 6144 = 430 Gb # Using 1MB as a buffer-size. @@ -93,7 +94,7 @@ function copydb() { ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} if [ $? -eq 0 ]; then # Check the exit status of the "hadoop distcp" command. - echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster.\n" + echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster, after: $(print_elapsed_time start_file_transfer_time)\n" else echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log @@ -118,6 +119,7 @@ function copydb() { fi echo -e "\nCreating schema for db: '${db}'\n" + start_create_schema_time=$(date +%s) # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" @@ -128,7 +130,8 @@ function copydb() { all_create_view_statements=() num_tables=0 - entities_on_ocean=`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'` # Get the tables and views without any potential the "WARN" logs. + entities_on_ocean=(`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'`) # Get the tables and views without any potential the "WARN" logs. + echo -e "\nGoing to create ${#entities_on_ocean[@]} entities for db '${db}'..\n" for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elements are single-words. # Check if this is a view by showing the create-statement where it should print "create view" for a view, not the "create table". Unfortunately, there is no "show views" command. create_entity_statement=`hive --database ${db} -e "show create table ${i};"` # We need to use the "--database", instead of including it inside the query, in order to return the statements with the '`' chars being in the right place to be used by impala-shell. However, we need to add the db-name in the "CREATE VIEW view_name" statement. @@ -152,8 +155,9 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log exit 6 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. fi fi fi @@ -208,8 +212,11 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done + entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" + + start_compute_stats_time=$(date +%s) echo -e "\nComputing stats for tables..\n" - entities_on_impala=`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"` for i in ${entities_on_impala[@]}; do # Use un-quoted values, as the elemetns are single-words. # Taking the create table statement from the Ocean cluster, just to check if its a view, as the output is easier than using impala-shell from Impala cluster. create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. @@ -221,20 +228,18 @@ function copydb() { fi done + echo -e "\nFinished computing stats for tables, after: $(print_elapsed_time start_compute_stats_time)\n" + rm -f error.log # Cleanup the temp log-file. + # Check if the entities in both clusters are the same, down to the exact names, not just the counts. (they are sorted in the same way both in hive and impala) - if [ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]; then - echo -e "\nAll entities have been copied to Impala cluster.\n" + if [[ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]]; then + echo -e "\nAll entities have been copied to Impala cluster.\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" else - echo -e "\n\nERROR: 1 OR MORE ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n" - rm -f error.log + echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 8 fi fi - - rm -f error.log - echo -e "\n\nFinished processing db: ${db}\n" - print_elapsed_time start_db_time } diff --git a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh index ca0f7a643..55deada40 100644 --- a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh @@ -55,20 +55,20 @@ function print_elapsed_time() hours=$((elapsed_time / 3600)) minutes=$(((elapsed_time % 3600) / 60)) seconds=$((elapsed_time % 60)) - printf "\nElapsed time: %02d:%02d:%02d\n\n" $hours $minutes $seconds + printf "%02d:%02d:%02d" $hours $minutes $seconds } function copydb() { db=$1 - start_db_time=$(date +%s) echo -e "\nStart processing db: '${db}'..\n" + start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then - echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" rm -f error.log if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 2 @@ -78,6 +78,7 @@ function copydb() { fi echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" + start_file_transfer_time=$(date +%s) # Using max-bandwidth of: 70 * 150 Mb/s = 10.5 Gb/s # Using max memory of: 70 * 6144 = 430 Gb # Using 1MB as a buffer-size. @@ -93,7 +94,7 @@ function copydb() { ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} if [ $? -eq 0 ]; then # Check the exit status of the "hadoop distcp" command. - echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster.\n" + echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster, after: $(print_elapsed_time start_file_transfer_time)\n" else echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log @@ -118,6 +119,7 @@ function copydb() { fi echo -e "\nCreating schema for db: '${db}'\n" + start_create_schema_time=$(date +%s) # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" @@ -128,7 +130,8 @@ function copydb() { all_create_view_statements=() num_tables=0 - entities_on_ocean=`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'` # Get the tables and views without any potential the "WARN" logs. + entities_on_ocean=(`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'`) # Get the tables and views without any potential the "WARN" logs. + echo -e "\nGoing to create ${#entities_on_ocean[@]} entities for db '${db}'..\n" for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elements are single-words. # Check if this is a view by showing the create-statement where it should print "create view" for a view, not the "create table". Unfortunately, there is no "show views" command. create_entity_statement=`hive --database ${db} -e "show create table ${i};"` # We need to use the "--database", instead of including it inside the query, in order to return the statements with the '`' chars being in the right place to be used by impala-shell. However, we need to add the db-name in the "CREATE VIEW view_name" statement. @@ -152,8 +155,9 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log exit 6 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. fi fi fi @@ -208,8 +212,11 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done + entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" + + start_compute_stats_time=$(date +%s) echo -e "\nComputing stats for tables..\n" - entities_on_impala=`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"` for i in ${entities_on_impala[@]}; do # Use un-quoted values, as the elemetns are single-words. # Taking the create table statement from the Ocean cluster, just to check if its a view, as the output is easier than using impala-shell from Impala cluster. create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. @@ -221,20 +228,18 @@ function copydb() { fi done + echo -e "\nFinished computing stats for tables, after: $(print_elapsed_time start_compute_stats_time)\n" + rm -f error.log # Cleanup the temp log-file. + # Check if the entities in both clusters are the same, down to the exact names, not just the counts. (they are sorted in the same way both in hive and impala) - if [ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]; then - echo -e "\nAll entities have been copied to Impala cluster.\n" + if [[ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]]; then + echo -e "\nAll entities have been copied to Impala cluster.\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" else - echo -e "\n\nERROR: 1 OR MORE ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n" - rm -f error.log + echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 8 fi fi - - rm -f error.log - echo -e "\n\nFinished processing db: ${db}\n" - print_elapsed_time start_db_time } diff --git a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh index dd2203eef..43498abd2 100644 --- a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh @@ -55,20 +55,20 @@ function print_elapsed_time() hours=$((elapsed_time / 3600)) minutes=$(((elapsed_time % 3600) / 60)) seconds=$((elapsed_time % 60)) - printf "\nElapsed time: %02d:%02d:%02d\n\n" $hours $minutes $seconds + printf "%02d:%02d:%02d" $hours $minutes $seconds } function copydb() { db=$1 - start_db_time=$(date +%s) echo -e "\nStart processing db: '${db}'..\n" + start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then - echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" rm -f error.log if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 2 @@ -78,6 +78,7 @@ function copydb() { fi echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" + start_file_transfer_time=$(date +%s) # Using max-bandwidth of: 70 * 150 Mb/s = 10.5 Gb/s # Using max memory of: 70 * 6144 = 430 Gb # Using 1MB as a buffer-size. @@ -93,7 +94,7 @@ function copydb() { ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} if [ $? -eq 0 ]; then # Check the exit status of the "hadoop distcp" command. - echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster.\n" + echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster, after: $(print_elapsed_time start_file_transfer_time)\n" else echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log @@ -118,6 +119,7 @@ function copydb() { fi echo -e "\nCreating schema for db: '${db}'\n" + start_create_schema_time=$(date +%s) # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" @@ -128,7 +130,8 @@ function copydb() { all_create_view_statements=() num_tables=0 - entities_on_ocean=`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'` # Get the tables and views without any potential the "WARN" logs. + entities_on_ocean=(`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'`) # Get the tables and views without any potential the "WARN" logs. + echo -e "\nGoing to create ${#entities_on_ocean[@]} entities for db '${db}'..\n" for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elements are single-words. # Check if this is a view by showing the create-statement where it should print "create view" for a view, not the "create table". Unfortunately, there is no "show views" command. create_entity_statement=`hive --database ${db} -e "show create table ${i};"` # We need to use the "--database", instead of including it inside the query, in order to return the statements with the '`' chars being in the right place to be used by impala-shell. However, we need to add the db-name in the "CREATE VIEW view_name" statement. @@ -152,8 +155,9 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log exit 6 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. fi fi fi @@ -208,8 +212,11 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done + entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" + + start_compute_stats_time=$(date +%s) echo -e "\nComputing stats for tables..\n" - entities_on_impala=`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"` for i in ${entities_on_impala[@]}; do # Use un-quoted values, as the elemetns are single-words. # Taking the create table statement from the Ocean cluster, just to check if its a view, as the output is easier than using impala-shell from Impala cluster. create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. @@ -221,20 +228,18 @@ function copydb() { fi done + echo -e "\nFinished computing stats for tables, after: $(print_elapsed_time start_compute_stats_time)\n" + rm -f error.log # Cleanup the temp log-file. + # Check if the entities in both clusters are the same, down to the exact names, not just the counts. (they are sorted in the same way both in hive and impala) - if [ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]; then - echo -e "\nAll entities have been copied to Impala cluster.\n" + if [[ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]]; then + echo -e "\nAll entities have been copied to Impala cluster.\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" else - echo -e "\n\nERROR: 1 OR MORE ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n" - rm -f error.log + echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 8 fi fi - - rm -f error.log - echo -e "\n\nFinished processing db: ${db}\n" - print_elapsed_time start_db_time } diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh index 918775f49..1d5842d06 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh @@ -57,20 +57,20 @@ function print_elapsed_time() hours=$((elapsed_time / 3600)) minutes=$(((elapsed_time % 3600) / 60)) seconds=$((elapsed_time % 60)) - printf "\nElapsed time: %02d:%02d:%02d\n\n" $hours $minutes $seconds + printf "%02d:%02d:%02d" $hours $minutes $seconds } function copydb() { db=$1 - start_db_time=$(date +%s) echo -e "\nStart processing db: '${db}'..\n" + start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then - echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" rm -f error.log if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 2 @@ -80,6 +80,7 @@ function copydb() { fi echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" + start_file_transfer_time=$(date +%s) # Using max-bandwidth of: 70 * 150 Mb/s = 10.5 Gb/s # Using max memory of: 70 * 6144 = 430 Gb # Using 1MB as a buffer-size. @@ -95,7 +96,7 @@ function copydb() { ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} if [ $? -eq 0 ]; then # Check the exit status of the "hadoop distcp" command. - echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster.\n" + echo -e "\nSuccessfully copied the files of '${db}' from Ocean to Impala cluster, after: $(print_elapsed_time start_file_transfer_time)\n" else echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log @@ -120,6 +121,7 @@ function copydb() { fi echo -e "\nCreating schema for db: '${db}'\n" + start_create_schema_time=$(date +%s) # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" @@ -130,7 +132,8 @@ function copydb() { all_create_view_statements=() num_tables=0 - entities_on_ocean=`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'` # Get the tables and views without any potential the "WARN" logs. + entities_on_ocean=(`hive -e "show tables in ${db};" | sed 's/WARN:.*//g'`) # Get the tables and views without any potential the "WARN" logs. + echo -e "\nGoing to create ${#entities_on_ocean[@]} entities for db '${db}'..\n" for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elements are single-words. # Check if this is a view by showing the create-statement where it should print "create view" for a view, not the "create table". Unfortunately, there is no "show views" command. create_entity_statement=`hive --database ${db} -e "show create table ${i};"` # We need to use the "--database", instead of including it inside the query, in order to return the statements with the '`' chars being in the right place to be used by impala-shell. However, we need to add the db-name in the "CREATE VIEW view_name" statement. @@ -154,8 +157,9 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log exit 6 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. fi fi fi @@ -210,8 +214,11 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done + entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" + + start_compute_stats_time=$(date +%s) echo -e "\nComputing stats for tables..\n" - entities_on_impala=`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"` for i in ${entities_on_impala[@]}; do # Use un-quoted values, as the elemetns are single-words. # Taking the create table statement from the Ocean cluster, just to check if its a view, as the output is easier than using impala-shell from Impala cluster. create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. @@ -223,20 +230,18 @@ function copydb() { fi done + echo -e "\nFinished computing stats for tables, after: $(print_elapsed_time start_compute_stats_time)\n" + rm -f error.log # Cleanup the temp log-file. + # Check if the entities in both clusters are the same, down to the exact names, not just the counts. (they are sorted in the same way both in hive and impala) - if [ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]; then - echo -e "\nAll entities have been copied to Impala cluster.\n" + if [[ "${entities_on_impala[@]}" == "${entities_on_ocean[@]}" ]]; then + echo -e "\nAll entities have been copied to Impala cluster.\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" else - echo -e "\n\nERROR: 1 OR MORE ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n" - rm -f error.log + echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 8 fi fi - - rm -f error.log - echo -e "\n\nFinished processing db: ${db}\n" - print_elapsed_time start_db_time } STATS_DB=$1 From 38636942c7af3f400f01618a6667f07fa29268d0 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Fri, 14 Jun 2024 15:11:19 +0300 Subject: [PATCH 033/239] filtering out deletedbyinference and invinsible results from accessroute --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index 7bad34e86..dafec9a6f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -65,4 +65,5 @@ DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge; CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result -lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute; + lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute +WHERE datainfo.deletedbyinference=false and datainfo.invisible = FALSE; From 3095047e5e6cf1cba744264296f5e01f96bfb4b4 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Tue, 18 Jun 2024 14:40:41 +0300 Subject: [PATCH 034/239] Miscellaneous updates to the copying operation to Impala Cluster: - Fix not breaking out of the VIEWS-infinite-loop when the "SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR" is set to "false". - Exit the script when no HDFS-active-node was found, independently of the "SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR". - Fix view_name-recognition in a log-message, by using the more advanced "Perl-Compatible Regular Expressions" in "grep". - Add error-handling for "compute stats" errors. --- .../oozie_app/copyDataToImpalaCluster.sh | 24 ++++++++++++------- .../oozie_app/copyDataToImpalaCluster.sh | 24 ++++++++++++------- .../oozie_app/copyDataToImpalaCluster.sh | 24 ++++++++++++------- .../oozie_app/copyDataToImpalaCluster.sh | 24 ++++++++++++------- 4 files changed, 64 insertions(+), 32 deletions(-) diff --git a/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh index 55deada40..978cf4a9a 100644 --- a/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh @@ -32,9 +32,7 @@ while [ $COUNTER -lt 3 ]; do done if [ -z "$IMPALA_HDFS_NODE" ]; then echo -e "\n\nERROR: PROBLEM WHEN SETTING THE HDFS-NODE FOR IMPALA CLUSTER! | AFTER ${COUNTER} RETRIES.\n\n" - if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 1 - fi + exit 1 # This is fatal and we have to exit independently of the "SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR" config, as none of the DBs will be able to get transferred. fi echo -e "Active IMPALA HDFS Node: ${IMPALA_HDFS_NODE} , after ${COUNTER} retries.\n\n" @@ -148,7 +146,7 @@ function copydb() { echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 5 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` @@ -188,7 +186,7 @@ function copydb() { specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" - echo -e "\nView '$(cat error.log | grep -Eo "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" + echo -e "\nView '$(cat error.log | grep -Po "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" ((new_num_of_views_to_retry++)) # Increment it here, instead of acquiring the array's size in the end, as that doesn't work for some reason. else all_create_view_statements=("${all_create_view_statements[@]/$create_view_statement}") # Remove the current successful statement from the list. @@ -200,9 +198,11 @@ function copydb() { # Although the above command reduces the "active" elements to just the few to-be-retried, it does not manage to make the array return the its true size through the "${#all_create_view_statements[@]}" statement. So we use counters. if [[ $new_num_of_views_to_retry -eq $previous_num_of_views_to_retry ]]; then - echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! EXITING..\n\n" + echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! BREAKING-OUT..\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 7 + else + break # Break form the inf-loop of views and continue by computing stats for the tables. fi elif [[ $new_num_of_views_to_retry -gt 0 ]]; then echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" @@ -224,7 +224,15 @@ function copydb() { # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; + impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` + if [ -n "$log_errors" ]; then + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" + if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log + exit 8 + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. + fi fi done @@ -237,7 +245,7 @@ function copydb() { else echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 8 + exit 9 fi fi } diff --git a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh index 55deada40..978cf4a9a 100644 --- a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh @@ -32,9 +32,7 @@ while [ $COUNTER -lt 3 ]; do done if [ -z "$IMPALA_HDFS_NODE" ]; then echo -e "\n\nERROR: PROBLEM WHEN SETTING THE HDFS-NODE FOR IMPALA CLUSTER! | AFTER ${COUNTER} RETRIES.\n\n" - if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 1 - fi + exit 1 # This is fatal and we have to exit independently of the "SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR" config, as none of the DBs will be able to get transferred. fi echo -e "Active IMPALA HDFS Node: ${IMPALA_HDFS_NODE} , after ${COUNTER} retries.\n\n" @@ -148,7 +146,7 @@ function copydb() { echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 5 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` @@ -188,7 +186,7 @@ function copydb() { specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" - echo -e "\nView '$(cat error.log | grep -Eo "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" + echo -e "\nView '$(cat error.log | grep -Po "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" ((new_num_of_views_to_retry++)) # Increment it here, instead of acquiring the array's size in the end, as that doesn't work for some reason. else all_create_view_statements=("${all_create_view_statements[@]/$create_view_statement}") # Remove the current successful statement from the list. @@ -200,9 +198,11 @@ function copydb() { # Although the above command reduces the "active" elements to just the few to-be-retried, it does not manage to make the array return the its true size through the "${#all_create_view_statements[@]}" statement. So we use counters. if [[ $new_num_of_views_to_retry -eq $previous_num_of_views_to_retry ]]; then - echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! EXITING..\n\n" + echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! BREAKING-OUT..\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 7 + else + break # Break form the inf-loop of views and continue by computing stats for the tables. fi elif [[ $new_num_of_views_to_retry -gt 0 ]]; then echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" @@ -224,7 +224,15 @@ function copydb() { # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; + impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` + if [ -n "$log_errors" ]; then + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" + if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log + exit 8 + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. + fi fi done @@ -237,7 +245,7 @@ function copydb() { else echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 8 + exit 9 fi fi } diff --git a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh index 43498abd2..55ae3114e 100644 --- a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh @@ -32,9 +32,7 @@ while [ $COUNTER -lt 3 ]; do done if [ -z "$IMPALA_HDFS_NODE" ]; then echo -e "\n\nERROR: PROBLEM WHEN SETTING THE HDFS-NODE FOR IMPALA CLUSTER! | AFTER ${COUNTER} RETRIES.\n\n" - if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 1 - fi + exit 1 # This is fatal and we have to exit independently of the "SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR" config, as none of the DBs will be able to get transferred. fi echo -e "Active IMPALA HDFS Node: ${IMPALA_HDFS_NODE} , after ${COUNTER} retries.\n\n" @@ -148,7 +146,7 @@ function copydb() { echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 5 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` @@ -188,7 +186,7 @@ function copydb() { specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" - echo -e "\nView '$(cat error.log | grep -Eo "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" + echo -e "\nView '$(cat error.log | grep -Po "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" ((new_num_of_views_to_retry++)) # Increment it here, instead of acquiring the array's size in the end, as that doesn't work for some reason. else all_create_view_statements=("${all_create_view_statements[@]/$create_view_statement}") # Remove the current successful statement from the list. @@ -200,9 +198,11 @@ function copydb() { # Although the above command reduces the "active" elements to just the few to-be-retried, it does not manage to make the array return the its true size through the "${#all_create_view_statements[@]}" statement. So we use counters. if [[ $new_num_of_views_to_retry -eq $previous_num_of_views_to_retry ]]; then - echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! EXITING..\n\n" + echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! BREAKING-OUT..\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 7 + else + break # Break form the inf-loop of views and continue by computing stats for the tables. fi elif [[ $new_num_of_views_to_retry -gt 0 ]]; then echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" @@ -224,7 +224,15 @@ function copydb() { # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; + impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` + if [ -n "$log_errors" ]; then + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" + if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log + exit 8 + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. + fi fi done @@ -237,7 +245,7 @@ function copydb() { else echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 8 + exit 9 fi fi } diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh index 1d5842d06..07a8a4534 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh @@ -30,9 +30,7 @@ while [ $COUNTER -lt 3 ]; do done if [ -z "$IMPALA_HDFS_NODE" ]; then echo -e "\n\nERROR: PROBLEM WHEN SETTING THE HDFS-NODE FOR IMPALA CLUSTER! | AFTER ${COUNTER} RETRIES.\n\n" - if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 1 - fi + exit 1 # This is fatal and we have to exit independently of the "SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR" config, as none of the DBs will be able to get transferred. fi echo -e "Active IMPALA HDFS Node: ${IMPALA_HDFS_NODE} , after ${COUNTER} retries.\n\n" @@ -150,7 +148,7 @@ function copydb() { echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 5 - fi + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` @@ -190,7 +188,7 @@ function copydb() { specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" - echo -e "\nView '$(cat error.log | grep -Eo "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" + echo -e "\nView '$(cat error.log | grep -Po "Query: CREATE VIEW ([^\s]+)" | sed 's/Query: CREATE VIEW //g')' failed to be created, possibly because it depends on another view.\n" ((new_num_of_views_to_retry++)) # Increment it here, instead of acquiring the array's size in the end, as that doesn't work for some reason. else all_create_view_statements=("${all_create_view_statements[@]/$create_view_statement}") # Remove the current successful statement from the list. @@ -202,9 +200,11 @@ function copydb() { # Although the above command reduces the "active" elements to just the few to-be-retried, it does not manage to make the array return the its true size through the "${#all_create_view_statements[@]}" statement. So we use counters. if [[ $new_num_of_views_to_retry -eq $previous_num_of_views_to_retry ]]; then - echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! EXITING..\n\n" + echo -e "\n\nERROR: THE NUMBER OF VIEWS TO RETRY HAS NOT BEEN REDUCED! THE SCRIPT IS LIKELY GOING TO AN INFINITE-LOOP! BREAKING-OUT..\n\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then exit 7 + else + break # Break form the inf-loop of views and continue by computing stats for the tables. fi elif [[ $new_num_of_views_to_retry -gt 0 ]]; then echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" @@ -226,7 +226,15 @@ function copydb() { # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; + impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` + if [ -n "$log_errors" ]; then + echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" + if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then + rm -f error.log + exit 8 + fi # This error is not FATAL, do we do not return from this function, in normal circumstances. + fi fi done @@ -239,7 +247,7 @@ function copydb() { else echo -e "\n\nERROR: $((${#entities_on_ocean[@]} - ${#entities_on_impala[@]})) ENTITIES OF DB '${db}' FAILED TO BE COPIED TO IMPALA CLUSTER!\n\n\nFinished processing db: '${db}', after: $(print_elapsed_time start_db_time)\n" if [[ SHOULD_EXIT_WHOLE_SCRIPT_UPON_ERROR -eq 1 ]]; then - exit 8 + exit 9 fi fi } From ac270f795b998e6774da8ff4a6d4d729816f2c10 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Wed, 19 Jun 2024 11:11:52 +0200 Subject: [PATCH 035/239] [IrishFunderList]make changed according to 9635 comment 14, 15 and 16 --- .../dhp/collection/crossref/irish_funder.json | 25 +++---------------- 1 file changed, 4 insertions(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json index e4f491e5c..9482904c5 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json @@ -58,7 +58,7 @@ "uri": "http://dx.doi.org/10.13039/100010414", "name": "Health Research Board", "synonym": [ - "501100001590" + "501100001590", "501100023273" ] }, { @@ -85,12 +85,6 @@ "name": "Irish College of General Practitioners", "synonym": [] }, - { - "id": "100012734", - "uri": "http://dx.doi.org/10.13039/100012734", - "name": "Department for Culture, Heritage and the Gaeltacht, Ireland", - "synonym": [] - }, { "id": "100012754", "uri": "http://dx.doi.org/10.13039/100012754", @@ -233,7 +227,7 @@ "id": "100018064", "uri": "http://dx.doi.org/10.13039/100018064", "name": "Department of Tourism, Culture, Arts, Gaeltacht, Sport and Media", - "synonym": [] + "synonym": ["100012734"] }, { "id": "100018172", @@ -319,12 +313,7 @@ "name": "Centre for Ageing Research and Development in Ireland", "synonym": [] }, - { - "id": "501100001583", - "uri": "http://dx.doi.org/10.13039/501100001583", - "name": "Cystinosis Foundation Ireland", - "synonym": [] - }, + { "id": "501100001584", "uri": "http://dx.doi.org/10.13039/501100001584", @@ -605,7 +594,7 @@ "id": "501100009315", "uri": "http://dx.doi.org/10.13039/501100009315", "name": "Cystinosis Ireland", - "synonym": [] + "synonym": ["501100001583"] }, { "id": "501100010808", @@ -763,12 +752,6 @@ "name": "Institute of Technology, Tralee", "synonym": [] }, - { - "id": "501100023273", - "uri": "http://dx.doi.org/10.13039/501100023273", - "name": "HRB Clinical Research Facility Galway", - "synonym": [] - }, { "id": "501100023378", "uri": "http://dx.doi.org/10.13039/501100023378", From d35edac212181c32037f20dc2a3e59e7458f9eb0 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 20 Jun 2024 12:28:28 +0200 Subject: [PATCH 036/239] [IrishFunderList]make changed according to 9635 comment 20, 21, 22 and 23 --- .../dhp/collection/crossref/irish_funder.json | 30 ++----------------- 1 file changed, 3 insertions(+), 27 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json index 9482904c5..e50dc2dee 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json @@ -85,18 +85,6 @@ "name": "Irish College of General Practitioners", "synonym": [] }, - { - "id": "100012754", - "uri": "http://dx.doi.org/10.13039/100012754", - "name": "Horizon Pharma", - "synonym": [] - }, - { - "id": "100012891", - "uri": "http://dx.doi.org/10.13039/100012891", - "name": "Medical Research Charities Group", - "synonym": [] - }, { "id": "100012919", "uri": "http://dx.doi.org/10.13039/100012919", @@ -275,13 +263,13 @@ "id": "100019637", "uri": "http://dx.doi.org/10.13039/100019637", "name": "Horizon Therapeutics", - "synonym": [] + "synonym": ["100012754"] }, { "id": "100020174", "uri": "http://dx.doi.org/10.13039/100020174", "name": "Health Research Charities Ireland", - "synonym": [] + "synonym": ["100012891"] }, { "id": "100020202", @@ -510,7 +498,7 @@ "id": "501100003037", "uri": "http://dx.doi.org/10.13039/501100003037", "name": "Elan", - "synonym": [] + "synonym": ["501100021694"] }, { "id": "501100003496", @@ -584,12 +572,6 @@ "name": "Technological University Dublin", "synonym": [] }, - { - "id": "501100009269", - "uri": "http://dx.doi.org/10.13039/501100009269", - "name": "Programme of Competitive Forestry Research for Development", - "synonym": [] - }, { "id": "501100009315", "uri": "http://dx.doi.org/10.13039/501100009315", @@ -716,12 +698,6 @@ "name": "Insight SFI Research Centre for Data Analytics", "synonym": [] }, - { - "id": "501100021694", - "uri": "http://dx.doi.org/10.13039/501100021694", - "name": "Elan Pharma International", - "synonym": [] - }, { "id": "501100021838", "uri": "http://dx.doi.org/10.13039/501100021838", From 66cd28f70a94536e38ff6368b65c88b9b2d3bbfc Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Thu, 20 Jun 2024 14:33:46 +0300 Subject: [PATCH 037/239] - Fix not using the "export HADOOP_USER_NAME" statement in "createPDFsAggregated.sh", which caused permission-issues when creating tables with Impala. - Remove unused "--user" parameter in "impala-shell" calls. - Code polishing. --- .../oozie_app/copyDataToImpalaCluster.sh | 14 +++++----- .../oozie_app/copyDataToImpalaCluster.sh | 14 +++++----- .../oozie_app/copyDataToImpalaCluster.sh | 14 +++++----- .../oozie_app/copyDataToImpalaCluster.sh | 19 ++++++------- .../stats/oozie_app/createPDFsAggregated.sh | 28 +++++++++++-------- 5 files changed, 46 insertions(+), 43 deletions(-) diff --git a/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh index 978cf4a9a..09ea1b393 100644 --- a/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-hist-snaps/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-hist-snaps/oozie_app/copyDataToImpalaCluster.sh @@ -63,7 +63,7 @@ function copydb() { start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" @@ -120,7 +120,7 @@ function copydb() { start_create_schema_time=$(date +%s) # create the new database (with the same name) - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" + impala-shell -i ${IMPALA_HOSTNAME} -q "create database ${db}" # Because "Hive" and "Impala" do not have compatible schemas, we cannot use the "show create table " output from hive to create the exact same table in impala. # So, we have to find at least one parquet file (check if it's there) from the table in the ocean cluster for impala to use it to extract the table-schema itself from that file. @@ -148,7 +148,7 @@ function copydb() { exit 5 fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" @@ -182,7 +182,7 @@ function copydb() { new_num_of_views_to_retry=0 for create_view_statement in "${all_create_view_statements[@]}"; do # Here we use double quotes, as the elements are phrases, instead of single-words. - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" @@ -212,7 +212,7 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done - entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + entities_on_impala=(`impala-shell -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" start_compute_stats_time=$(date +%s) @@ -222,9 +222,9 @@ function copydb() { create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. if [ -z "$create_view_statement" ]; then # If it's a table, then go load the data to it. # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" + impala-shell -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" diff --git a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh index 978cf4a9a..09ea1b393 100644 --- a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/copyDataToImpalaCluster.sh @@ -63,7 +63,7 @@ function copydb() { start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" @@ -120,7 +120,7 @@ function copydb() { start_create_schema_time=$(date +%s) # create the new database (with the same name) - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" + impala-shell -i ${IMPALA_HOSTNAME} -q "create database ${db}" # Because "Hive" and "Impala" do not have compatible schemas, we cannot use the "show create table " output from hive to create the exact same table in impala. # So, we have to find at least one parquet file (check if it's there) from the table in the ocean cluster for impala to use it to extract the table-schema itself from that file. @@ -148,7 +148,7 @@ function copydb() { exit 5 fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" @@ -182,7 +182,7 @@ function copydb() { new_num_of_views_to_retry=0 for create_view_statement in "${all_create_view_statements[@]}"; do # Here we use double quotes, as the elements are phrases, instead of single-words. - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" @@ -212,7 +212,7 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done - entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + entities_on_impala=(`impala-shell -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" start_compute_stats_time=$(date +%s) @@ -222,9 +222,9 @@ function copydb() { create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. if [ -z "$create_view_statement" ]; then # If it's a table, then go load the data to it. # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" + impala-shell -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" diff --git a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh index 55ae3114e..d75412df8 100644 --- a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/copyDataToImpalaCluster.sh @@ -63,7 +63,7 @@ function copydb() { start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" @@ -120,7 +120,7 @@ function copydb() { start_create_schema_time=$(date +%s) # create the new database (with the same name) - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" + impala-shell -i ${IMPALA_HOSTNAME} -q "create database ${db}" # Because "Hive" and "Impala" do not have compatible schemas, we cannot use the "show create table " output from hive to create the exact same table in impala. # So, we have to find at least one parquet file (check if it's there) from the table in the ocean cluster for impala to use it to extract the table-schema itself from that file. @@ -148,7 +148,7 @@ function copydb() { exit 5 fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" @@ -182,7 +182,7 @@ function copydb() { new_num_of_views_to_retry=0 for create_view_statement in "${all_create_view_statements[@]}"; do # Here we use double quotes, as the elements are phrases, instead of single-words. - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" @@ -212,7 +212,7 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done - entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + entities_on_impala=(`impala-shell -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" start_compute_stats_time=$(date +%s) @@ -222,9 +222,9 @@ function copydb() { create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. if [ -z "$create_view_statement" ]; then # If it's a table, then go load the data to it. # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" + impala-shell -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh index 07a8a4534..96c61d91a 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh @@ -65,7 +65,7 @@ function copydb() { start_db_time=$(date +%s) # Delete the old DB from Impala cluster (if exists). - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "drop database if exists ${db} cascade" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE FROM IMPALA CLUSTER! EXITING...\n\n" @@ -122,7 +122,7 @@ function copydb() { start_create_schema_time=$(date +%s) # create the new database (with the same name) - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" + impala-shell -i ${IMPALA_HOSTNAME} -q "create database ${db}" # Because "Hive" and "Impala" do not have compatible schemas, we cannot use the "show create table " output from hive to create the exact same table in impala. # So, we have to find at least one parquet file (check if it's there) from the table in the ocean cluster for impala to use it to extract the table-schema itself from that file. @@ -150,7 +150,7 @@ function copydb() { exit 5 fi # This error is not FATAL, do we do not return from this function, in normal circumstances. else - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "create table ${db}.${i} like parquet '${CURRENT_PRQ_FILE}' stored as parquet;" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN CREATING TABLE '${i}'!\n\n" @@ -184,7 +184,7 @@ function copydb() { new_num_of_views_to_retry=0 for create_view_statement in "${all_create_view_statements[@]}"; do # Here we use double quotes, as the elements are phrases, instead of single-words. - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later + impala-shell -i ${IMPALA_HOSTNAME} -q "${create_view_statement}" |& tee error.log # impala-shell prints all logs in stderr, so wee need to capture them and put them in a file, in order to perform "grep" on them later specific_errors=`cat error.log | grep -E "FAILED: ParseException line 1:13 missing TABLE at 'view'|ERROR: AnalysisException: Could not resolve table reference:"` if [ -n "$specific_errors" ]; then echo -e "\nspecific_errors: ${specific_errors}\n" @@ -214,7 +214,7 @@ function copydb() { previous_num_of_views_to_retry=$new_num_of_views_to_retry done - entities_on_impala=(`impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) + entities_on_impala=(`impala-shell -i ${IMPALA_HOSTNAME} --delimited -q "show tables in ${db}"`) echo -e "\nThe schema of db '${db}', along with ${#entities_on_impala[@]} entities have been created, on Impala cluster, after: $(print_elapsed_time start_create_schema_time)\n" start_compute_stats_time=$(date +%s) @@ -224,9 +224,9 @@ function copydb() { create_view_statement=`hive -e "show create table ${db}.${i};" | grep "CREATE VIEW"` # This grep works here, as we do not want to match multiple-lines. if [ -z "$create_view_statement" ]; then # If it's a table, then go load the data to it. # Invalidate metadata of this DB's tables, in order for Impala to be aware of all parquet files put inside the tables' directories, previously, by "hadoop distcp". - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" + impala-shell -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA ${db}.${i}" sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log + impala-shell -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}" |& tee error.log log_errors=`cat error.log | grep -E "WARN|ERROR|FAILED"` if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN COMPUTING STATS FOR TABLE '${i}'!\n\n" @@ -271,8 +271,7 @@ copydb $MONITOR_DB'_institutions' copydb $MONITOR_DB'_ris_tail' contexts="knowmad::other dh-ch::other enermaps::other gotriple::other neanias-atmospheric::other rural-digital-europe::other covid-19::other aurora::other neanias-space::other north-america-studies::other north-american-studies::other eutopia::other" -for i in ${contexts} -do - tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'` +for i in ${contexts}; do + tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'` copydb ${MONITOR_DB}'_'${tmp} done \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh index 46631a0c2..9eec0bb20 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh @@ -6,21 +6,26 @@ then ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} fi +export HADOOP_USER_NAME=$3 + +IMPALA_HOSTNAME='impala-cluster-dn1.openaire.eu' + function createPDFsAggregated() { db=$1 -impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} -q "drop table if exists indi_is_result_accessible"; + impala-shell -i ${IMPALA_HOSTNAME} -d ${db} -q "drop table if exists indi_is_result_accessible"; -impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} -q "create table indi_is_result_accessible stored as parquet as + impala-shell -i ${IMPALA_HOSTNAME} -d ${db} -q "create table indi_is_result_accessible stored as parquet as select distinct p.id, coalesce(is_result_accessible, 0) as is_result_accessible from result p left outer join - (select id, 1 as is_result_accessible from (select pl.* from result r - join pdfaggregation_i.publication p on r.id=p.id - join pdfaggregation_i.payload pl on pl.id=p.id - union all - select pl.* from result r - join pdfaggregation_i.publication p on r.id=p.dedupid - join pdfaggregation_i.payload pl on pl.id=p.id) foo) tmp on p.id=tmp.id"; + (select id, 1 as is_result_accessible from (select pl.* from result r + join pdfaggregation_i.publication p on r.id=p.id + join pdfaggregation_i.payload pl on pl.id=p.id + union all + select pl.* from result r + join pdfaggregation_i.publication p on r.id=p.dedupid + join pdfaggregation_i.payload pl on pl.id=p.id) foo) + tmp on p.id=tmp.id"; } STATS_DB=$1 @@ -35,8 +40,7 @@ createPDFsAggregated $MONITOR_DB'_institutions' createPDFsAggregated $MONITOR_DB'_ris_tail' contexts="knowmad::other dh-ch::other enermaps::other gotriple::other neanias-atmospheric::other rural-digital-europe::other covid-19::other aurora::other neanias-space::other north-america-studies::other north-american-studies::other eutopia::other" -for i in ${contexts} -do - tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'` +for i in ${contexts}; do + tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'` createPDFsAggregated ${MONITOR_DB}'_'${tmp} done \ No newline at end of file From 9f6e16a03cb5f55f58902e97b05a041ff66abd4b Mon Sep 17 00:00:00 2001 From: Serafeim Chatzopoulos Date: Thu, 20 Jun 2024 16:03:15 +0300 Subject: [PATCH 038/239] Add support to cretate/update solr collection aliases --- .../dhp/oa/provision/ProvisionConstants.java | 3 + .../oa/provision/SolrAdminApplication.java | 55 +++++++++++++++++-- .../provision/input_solradmin_parameters.json | 13 +++++ .../dhp/oa/provision/oozie_app/workflow.xml | 29 ++++++++++ .../provision/SolrAdminApplicationTest.java | 25 ++++++++- 5 files changed, 118 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java index 01d161b6b..81398016a 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java @@ -14,4 +14,7 @@ public class ProvisionConstants { return format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION; } + public static final String PUBLIC_ALIAS_NAME = "public"; + public static final String SHADOW_ALIAS_NAME = "shadow"; + } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java index 0033978bf..459ca0b50 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java @@ -9,6 +9,7 @@ import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.response.UpdateResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -23,7 +24,7 @@ public class SolrAdminApplication implements Closeable { private static final Logger log = LoggerFactory.getLogger(SolrAdminApplication.class); enum Action { - DELETE_BY_QUERY, COMMIT + DELETE_BY_QUERY, COMMIT, UPDATE_ALIASES } private final CloudSolrClient solrClient; @@ -62,8 +63,21 @@ public class SolrAdminApplication implements Closeable { final String collection = ProvisionConstants.getCollectionName(format); log.info("collection: {}", collection); + final String publicFormat = parser.get("publicFormat"); + log.info("publicFormat: {}", publicFormat); + + final String shadowFormat = parser.get("shadowFormat"); + log.info("shadowFormat: {}", shadowFormat); + + // get collection names from metadata format profiles names + final String publicCollection = ProvisionConstants.getCollectionName(publicFormat); + log.info("publicCollection: {}", publicCollection); + + final String shadowCollection = ProvisionConstants.getCollectionName(shadowFormat); + log.info("shadowCollection: {}", shadowCollection); + try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) { - app.execute(action, collection, query, commit); + app.execute(action, collection, query, commit, publicCollection, shadowCollection); } } @@ -73,21 +87,28 @@ public class SolrAdminApplication implements Closeable { } public SolrResponse commit(String collection) throws IOException, SolrServerException { - return execute(Action.COMMIT, collection, null, true); + return execute(Action.COMMIT, collection, null, true, null, null); } - public SolrResponse execute(Action action, String collection, String query, boolean commit) + public SolrResponse execute(Action action, String collection, String query, boolean commit, + String publicCollection, String shadowCollection) throws IOException, SolrServerException { switch (action) { case DELETE_BY_QUERY: UpdateResponse rsp = solrClient.deleteByQuery(collection, query); if (commit) { - solrClient.commit(collection); + return solrClient.commit(collection); } return rsp; + case COMMIT: return solrClient.commit(collection); + + case UPDATE_ALIASES: + this.updateAliases(publicCollection, shadowCollection); + return null; + default: throw new IllegalArgumentException("action not managed: " + action); } @@ -98,4 +119,28 @@ public class SolrAdminApplication implements Closeable { solrClient.close(); } + private void updateAliases(String publicCollection, String shadowCollection) + throws SolrServerException, IOException { + + // delete current aliases + this.deleteAlias(ProvisionConstants.PUBLIC_ALIAS_NAME); + this.deleteAlias(ProvisionConstants.SHADOW_ALIAS_NAME); + + // create aliases + this.createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, publicCollection); + this.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, shadowCollection); + + } + + public SolrResponse deleteAlias(String aliasName) throws SolrServerException, IOException { + CollectionAdminRequest.DeleteAlias deleteAliasRequest = CollectionAdminRequest.deleteAlias(aliasName); + return deleteAliasRequest.process(solrClient); + } + + public SolrResponse createAlias(String aliasName, String collection) throws IOException, SolrServerException { + CollectionAdminRequest.CreateAlias createAliasRequest = CollectionAdminRequest + .createAlias(aliasName, collection); + return createAliasRequest.process(solrClient); + } + } diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json index 23eca2f7b..6e3f21ef2 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json @@ -28,5 +28,18 @@ "paramLongName": "commit", "paramDescription": "should the action be followed by a commit?", "paramRequired": false + }, + { + "paramName": "pf", + "paramLongName": "publicFormat", + "paramDescription": "the name of the public metadata format profile - used to create an alias", + "paramRequired": false + }, + { + "paramName": "sf", + "paramLongName": "shadowFormat", + "paramDescription": "the name of the shadow metadata format profile - used to create an alias", + "paramRequired": false } + ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index a754c7a5d..02195c0aa 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -110,6 +110,14 @@ sparkNetworkTimeout configures spark.network.timeout + + publicFormat + the public metadata format - used to create the public collection alias + + + shadowFormat + the shadow metadata format - used to create the shadow collection alias + @@ -133,6 +141,7 @@ ${wf:conf('resumeFrom') eq 'create_payloads'} ${wf:conf('resumeFrom') eq 'drop_solr_collection'} ${wf:conf('resumeFrom') eq 'to_solr_index'} + ${wf:conf('resumeFrom') eq 'update_solr_aliases'} @@ -721,5 +730,25 @@ + + + + + + oozie.launcher.mapreduce.user.classpath.first + true + + + eu.dnetlib.dhp.oa.provision.SolrAdminApplication + --isLookupUrl${isLookupUrl} + --format${format} + --actionUPDATE_ALIASES + --publicFormat${publicFormat} + --shadowFormat${shadowFormat} + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java index 9d5bff3cf..43e4c9ed7 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java @@ -4,9 +4,9 @@ package eu.dnetlib.dhp.oa.provision; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.client.solrj.response.SolrPingResponse; import org.apache.solr.client.solrj.response.UpdateResponse; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; class SolrAdminApplicationTest extends SolrTest { @@ -24,7 +24,7 @@ class SolrAdminApplicationTest extends SolrTest { SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost()); UpdateResponse rsp = (UpdateResponse) admin - .execute(SolrAdminApplication.Action.DELETE_BY_QUERY, DEFAULT_COLLECTION, "*:*", false); + .execute(SolrAdminApplication.Action.DELETE_BY_QUERY, DEFAULT_COLLECTION, "*:*", false, null, null); assertEquals(0, rsp.getStatus()); } @@ -39,4 +39,25 @@ class SolrAdminApplicationTest extends SolrTest { assertEquals(0, rsp.getStatus()); } + @Test + void testAdminApplication_CREATE_ALIAS() throws Exception { + + SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost()); + + CollectionAdminResponse rsp = (CollectionAdminResponse) admin + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, DEFAULT_COLLECTION); + assertEquals(0, rsp.getStatus()); + + } + + @Test + void testAdminApplication_DELETE_ALIAS() throws Exception { + + SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost()); + + CollectionAdminResponse rsp = (CollectionAdminResponse) admin.deleteAlias(ProvisionConstants.PUBLIC_ALIAS_NAME); + assertEquals(0, rsp.getStatus()); + + } + } From 6055212f7706b26d2e95c52ebf90c638918ea110 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 25 Jun 2024 12:39:02 +0200 Subject: [PATCH 039/239] merged from the json_payload branch --- .../model/ProvisionModelSupport.java | 83 +++++++++++++++--- .../dhp/oa/provision/xml/part-00000.json.gz | Bin 66906 -> 0 bytes 2 files changed, 72 insertions(+), 11 deletions(-) delete mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/xml/part-00000.json.gz diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index f46aebdcf..a8c9ceeb3 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -32,6 +32,7 @@ import eu.dnetlib.dhp.schema.solr.Datasource; import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines; import eu.dnetlib.dhp.schema.solr.Instance; import eu.dnetlib.dhp.schema.solr.Journal; +import eu.dnetlib.dhp.schema.solr.Measure; import eu.dnetlib.dhp.schema.solr.OpenAccessColor; import eu.dnetlib.dhp.schema.solr.OpenAccessRoute; import eu.dnetlib.dhp.schema.solr.Organization; @@ -76,6 +77,7 @@ public class ProvisionModelSupport { r.setCollectedfrom(asProvenance(e.getCollectedfrom())); r.setContext(asContext(e.getContext(), contextMapper)); r.setPid(asPid(e.getPid())); + r.setMeasures(mapMeasures(e.getMeasures())); if (e instanceof eu.dnetlib.dhp.schema.oaf.Result) { r.setResult(mapResult((eu.dnetlib.dhp.schema.oaf.Result) e)); @@ -106,6 +108,14 @@ public class ProvisionModelSupport { final RelatedEntity re = rew.getTarget(); final RecordType relatedRecordType = RecordType.valueOf(re.getType()); final Relation relation = rew.getRelation(); + final String relationProvenance = Optional + .ofNullable(relation.getDataInfo()) + .map( + d -> Optional + .ofNullable(d.getProvenanceaction()) + .map(Qualifier::getClassid) + .orElse(null)) + .orElse(null); rr .setHeader( RelatedRecordHeader @@ -113,7 +123,9 @@ public class ProvisionModelSupport { relation.getRelType(), relation.getRelClass(), StringUtils.substringAfter(relation.getTarget(), IdentifierFactory.ID_PREFIX_SEPARATOR), - relatedRecordType)); + relatedRecordType, + relationProvenance, + Optional.ofNullable(relation.getDataInfo()).map(DataInfo::getTrust).orElse(null))); rr.setAcronym(re.getAcronym()); rr.setCode(re.getCode()); @@ -131,11 +143,20 @@ public class ProvisionModelSupport { rr.setOfficialname(re.getOfficialname()); rr.setOpenairecompatibility(mapCodeLabel(re.getOpenairecompatibility())); rr.setPid(asPid(re.getPid())); - rr.setProjectTitle(rr.getProjectTitle()); + rr.setWebsiteurl(re.getWebsiteurl()); + rr.setProjectTitle(re.getProjectTitle()); rr.setPublisher(re.getPublisher()); rr.setResulttype(mapQualifier(re.getResulttype())); rr.setTitle(Optional.ofNullable(re.getTitle()).map(StructuredProperty::getValue).orElse(null)); + if (relation.getValidated() == null) { + relation.setValidated(false); + } + if (ModelConstants.OUTCOME.equals(relation.getSubRelType()) + && StringUtils.isNotBlank(relation.getValidationDate())) { + rr.setValidationDate(relation.getValidationDate()); + } + return rr; } @@ -266,6 +287,7 @@ public class ProvisionModelSupport { ds.setOfficialname(mapField(d.getOfficialname())); ds.setDescription(mapField(d.getDescription())); ds.setJournal(mapJournal(d.getJournal())); + ds.setWebsiteurl(mapField(d.getWebsiteurl())); ds.setLogourl(mapField(d.getLogourl())); ds.setAccessinfopackage(mapFieldList(d.getAccessinfopackage())); ds.setCertificates(mapField(d.getCertificates())); @@ -311,6 +333,7 @@ public class ProvisionModelSupport { ds.setSubjects(asSubjectSP(d.getSubjects())); ds.setSubmissionpolicyurl(d.getSubmissionpolicyurl()); ds.setThematic(d.getThematic()); + ds.setContentpolicies(mapCodeLabel(d.getContentpolicies())); ds.setVersioncontrol(d.getVersioncontrol()); ds.setVersioning(mapField(d.getVersioning())); @@ -326,6 +349,7 @@ public class ProvisionModelSupport { rs.setOtherTitles(getOtherTitles(r.getTitle())); rs.setDescription(mapFieldList(r.getDescription())); rs.setSubject(asSubject(r.getSubject())); + rs.setLanguage(asLanguage(r.getLanguage())); rs.setPublicationdate(mapField(r.getDateofacceptance())); rs.setPublisher(mapField(r.getPublisher())); rs.setEmbargoenddate(mapField(r.getEmbargoenddate())); @@ -341,14 +365,14 @@ public class ProvisionModelSupport { rs.setCountry(asCountry(r.getCountry())); rs.setEoscifguidelines(asEOSCIF(r.getEoscifguidelines())); - rs.setGreen(r.getIsGreen()); + rs.setIsGreen(r.getIsGreen()); rs .setOpenAccessColor( Optional .ofNullable(r.getOpenAccessColor()) .map(color -> OpenAccessColor.valueOf(color.toString())) .orElse(null)); - rs.setInDiamondJournal(r.getIsInDiamondJournal()); + rs.setIsInDiamondJournal(r.getIsInDiamondJournal()); rs.setPubliclyFunded(r.getPubliclyFunded()); rs.setTransformativeAgreement(r.getTransformativeAgreement()); @@ -375,6 +399,13 @@ public class ProvisionModelSupport { return rs; } + private static Language asLanguage(Qualifier lang) { + return Optional + .ofNullable(lang) + .map(q -> Language.newInstance(q.getClassid(), q.getClassname())) + .orElse(null); + } + @Nullable private static List getOtherTitles(List titleList) { return Optional @@ -422,7 +453,7 @@ public class ProvisionModelSupport { Instance i = new Instance(); i.setCollectedfrom(asProvenance(instance.getCollectedfrom())); i.setHostedby(asProvenance(instance.getHostedby())); - i.setFulltext(i.getFulltext()); + i.setFulltext(instance.getFulltext()); i.setPid(asPid(instance.getPid())); i.setAlternateIdentifier(asPid(instance.getAlternateIdentifier())); i.setAccessright(mapAccessRight(instance.getAccessright())); @@ -453,7 +484,8 @@ public class ProvisionModelSupport { private static AccessRight mapAccessRight(eu.dnetlib.dhp.schema.oaf.AccessRight accessright) { return AccessRight .newInstance( - mapQualifier(accessright), + accessright.getClassid(), + accessright.getClassname(), Optional .ofNullable(accessright.getOpenAccessRoute()) .map(route -> OpenAccessRoute.valueOf(route.toString())) @@ -508,7 +540,25 @@ public class ProvisionModelSupport { } private static Provenance asProvenance(KeyValue keyValue) { - return Optional.ofNullable(keyValue).map(cf -> Provenance.newInstance(cf.getKey(), cf.getValue())).orElse(null); + return Optional + .ofNullable(keyValue) + .map( + kv -> Provenance + .newInstance( + StringUtils.substringAfter(kv.getKey(), IdentifierFactory.ID_PREFIX_SEPARATOR), + kv.getValue())) + .orElse(null); + } + + private static List mapMeasures(List measures) { + return Optional + .ofNullable(measures) + .map( + ml -> ml + .stream() + .map(m -> Measure.newInstance(m.getId(), mapCodeLabelKV(m.getUnit()))) + .collect(Collectors.toList())) + .orElse(null); } private static List asContext(List ctxList, @@ -581,7 +631,14 @@ public class ProvisionModelSupport { .map( pids -> pids .stream() - .map(p -> Pid.newInstance(p.getQualifier().getClassname(), p.getValue())) + .filter(p -> Objects.nonNull(p.getQualifier())) + .filter(p -> Objects.nonNull(p.getQualifier().getClassid())) + .map( + p -> Pid + .newInstance( + p.getValue(), + p.getQualifier().getClassid(), + p.getQualifier().getClassname())) .collect(Collectors.toList())) .orElse(null); } @@ -607,7 +664,9 @@ public class ProvisionModelSupport { .stream() .filter(s -> Objects.nonNull(s.getQualifier())) .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) - .map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassname())) + .map( + s -> Subject + .newInstance(s.getValue(), s.getQualifier().getClassid(), s.getQualifier().getClassname())) .collect(Collectors.toList())) .orElse(null); } @@ -620,7 +679,9 @@ public class ProvisionModelSupport { .stream() .filter(s -> Objects.nonNull(s.getQualifier())) .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) - .map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassname())) + .map( + s -> Subject + .newInstance(s.getValue(), s.getQualifier().getClassid(), s.getQualifier().getClassname())) .collect(Collectors.toList())) .orElse(null); } @@ -689,7 +750,7 @@ public class ProvisionModelSupport { private static CodeLabel mapCodeLabel(KeyValue kv) { return Optional .ofNullable(kv) - .map(q -> CodeLabel.newInstance(kv.getKey(), kv.getValue())) + .map(k -> CodeLabel.newInstance(k.getKey(), k.getValue())) .orElse(null); } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/xml/part-00000.json.gz b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/xml/part-00000.json.gz deleted file mode 100644 index 8dfcea4fa86f1b6dff330b0f9d0bba873f5b0768..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 66906 zcmaI7Ra9I})GgYLI|O%kcXxNU;1+_r2WcFFYmnd)EVyeU!QBb2!Gb%4oaW1a@5>P$ z7<*Ljf?B<5?={z)v&do)0RR0!pLVS#tjTx(xi-mkKAt1akgkDITD z$DbdyaKmX9Agt{vMb}>a7%W#_bUbbqL6i|; zSyn3N|L1q;#wtA7(L(HxUwO|-t=jmHqxVHC?D=yx+}e{=VXS;WavE+ludaTAKJc9z{eDi9-8q0Kv7 zi0qfHwe%uTDcxOtV7z3I^2?FmTk?2GQYqaTeC&??8*kju*kv|c8u{Zx!p@j#@$Qu@ zF3ICQvRJyQlVHa-rq$eGmg#4c&5IOn!zN}GZ*4tymCI9!(k*&~W$Jd~jo6jFi$u;< z>(U>@p>P#PFsuzy6Q~S`jyV~@c$@kb2{SHF!&~Av3g7f)gF9D zef=#aCUo!9of^+fK97CZbaGc9VM$AEU6gq|`aTV6s4Fyy=2#X>pTykSh?uJq1 zyg?XSm?)6VFvK4ly+XPiK`8Dm&S zS1&pn#k#QqU$lCwdQm=R&ISE2nH=AKTuSDBu83dg;1^@a71%$0lyP!M1y@ooE*hHdNrMo zhWACGt`+mf>Q0?3knbbu_hEq$4HucnFI;{E&av0X?nJV|7o-onB{N1|thWJY{fK|# zIxxr{2v!g+t5F5tMQ2<#`Xe8a@}i8c)=+I=1jRFKj*@v9Tp5SSMA5~Ew6+Wj67as_ z(>yVqV?=&>p&a>LGE>*+?B&i=*!yy~LwuH{I+SF3babUlJVwwGTxm0`7~zfu`w~4nN$VcA;*)iAQO}^nChO4a%cv|* z-!tL~t>84@-{8s}`+-U`K_-iR{gWUzUzgdh7@G%*x^K)lK3Xpm=!qM!wnIZn)IeP6 z4_y0{3cG3S6NQe@3EDsw~{}Nv`=mJ1YCr8Pg#z32&7xbrkF5q{YAA7 zY$^&6Ba%zW|GBID7>8h(Vj5YTm;T+?+#oT}H?3VdhO=yzg61-cd2@TM=^$Jy$mkTs zC4uSp^)5-v{=M$7;|oV%gT2>BO_lJpo0V%R!~I%&-jcoyJd(+CcgjJ2-tOGkz7=WS zl|(Ym4`QmJGv5mEfuodcdAflC(zX^qC?pj%GVaY*Sbk`ro&%FbkH$hbLdj}sNQp&# zZoFT56n|2Q9{h>8$bNOY@OqHn(;Wi-RnmCaOj%jb-i}10i|pn1GaBu=K8V*8!_Zt{y{zUeX{#6v@(G|p5;hfiE34+ie%UM zH(wJtnV5c(&anTE;Y~YS-CjLfEw^E|1-0mNC))ouPdCGB--}G0%$AP4>`*X`O5BFg2jn)>^EF{FTDZ(_tAxI^hKO7FC*QGQVo#U$pW&Y@Gd<*FfO zV~{3-@3Qhsr=vF{WW5Ob2;T{zq~de(-PeRK^P{Pq*Z#~4BdJwA{+*r2CDr#aCOw27M3Zt|;`_WM3mag`^Dqkv!?|j!?b7MLXT!a^cZ>!{qks zPHlzNABz=&M0?qG9zXY(X|4w!73yO8T6F&W8k!)srtz95`{mTmDL?C-fX0c4K*dN$ zjA^UG$C&W>&SNp(}%h=JHGPtpkVSCyWkNoo@l^ z4<`v|$9GH#oX)}>k1s9ww@By@&CN_>#J^z!yt)nRi~Kg*9e9xAM`STC1yM;qz@@DF zygtwPV}BvhYWSt$z!JKw`!F{oM!FE@3BA2qs#kvMtIeaWpc{vxo1TlkKG7A(b7?Sf zqU${$e1X62@G1uf8&L`n9zC)Ssos^Dqqyy2w-?XG==9*7* zpoQnl3q_R5kuoF^V69z^3sAoas}{-tx)II>b=jNzEQ>2Q3sP!%VqoDCX~0~Cbzh(Oit zs8)*3r2Y0qB9n2D^5eU6B)7*>Bf0JQUq+F8V|>Z@E@)fCU;&VOBUkco#~AkqK|~&v z24O88)KY=@=CUa@Bo;;J%`2iU;tIvDqIC`(9Fqk|OJw%Az+ZsLnfG6J!{VGCY-nRY zL@c;?DL9GGdNUraOA=btDi10iA|}dy3P+ZQP8Zw25|Yr{wm86$&&)!TzqDg_ zl@YgP)ep~tjPseP^}6&$iY}%r`+(}ed?;~P&}XlR!Y;?)&(kHTW%1h$;A#KT4xl^B zrYlMu;rQ$+oOt*FW{S{m1=@diBLQ$+v3f}2YM^`<>b^yPDesgYL!o~<#i$3}+!J?V zgxYbIN~!_aeli(>@BnOQ4W9uffX>7G_bo7;HV1=~Eb$uD)PC3Ovh`YX07wW1Kuf?N zAa(|2P(SSgHWr9)8Tx~xo&t{@&I|>N?@&AIZj5BZ&EL^Gi*JimhddpzH;XqdyFJ49 z4qqG%L<|rBxHWRc{&qxii`|(Y+%7os$yGZA&x}vkvJ|f9+GwaXNE*94Oc+49S*c*G zay)m6@eqX)7uO_pXR&RgB|?+S-O2*pP6dW9J$Mh$RU$`|^gqNZ9VosCWzk=qgaroi z5yOY|Y~n}aYo@%z@q-$iG1-RN1q{08>A=K73dhHW-=jb z<=fU}O?BlQhHc^sy^T#gfLRE<0UQFSp+8~C-8PrN+u*lai$}>aGzbsPN=bPWvt0zN z7qqvXb`Z9u6jgYcws_+GyFVX(ht`B+Bl;RHbvH9gJ+deolRaFnjx<0FGe~rQT^HkW zqQ>W3V5i*-D;zcL+uAyT@!uEcayQN2uFFG1*OhWm)w(;W99)v{&2rx-$@MT5#F zl4~S?47EX*L&Q!=$PP6DcJ@Jbs1NJ~(%TNbKm@DAAnNR}+#_li_Q1PVj@ z=mMvRQ!=wAI%HS5jZ-=`n4oCmBz;wpL;E4%BZ&d zz$sX?+os5$(cldJs`8Q4J6deK`VjuC11@Y?SQg3C*pp<4OS}4roV?olsFcvhoW)(& zDa0mXu4UK&8Kzt{@6c=#{54qM&ercd???ihL=zHEvNOYUKZ@{&n-P&r4_7})TWO^Woc zNlwk;JOhM8QgRV-zxK$R;*cDK^W*1t=U#M4Q|ncrJoY_rVU*NeR}T`@a2_Dp3e5zx zN+ZhNI|j=1cs{HoDnUPAQV}uG;VSR~;av8CJ6Z`k2e4%esnZvYSX&GXRAlylhu4j_ z*%*EWQV;M5jPttO9~%q5ijFr&xn>U0>-kr`4pd7DTBH9Ov}Jzpna7N&j-SsiZq9!4 zC!%jLLG~AJ9phD8mjJtR193~uN){}86<0bkJBrzI4#+%jw>L#`F(G6r_ZGgZ6qc>} za4Gr|k!M4cp9s->@CbH8o$6lc zF}q=jlIKs$W&e)}m5s~St+Ssuat-Ed3EI!O4R|~O$UVf)juJ!3+)d0<6eO24zEskE zJDw}j;qOPbSA4iTyIA9#=QwVe25}}MfcG5VEYltLL~vw@VtK!lIQmRcFyDpTFRv~8 zEIEZ&f5ggR=ebedL?Fo~A}@*9p7Chg%8BJAT9bp9%^tn?@{9=!+&ZdoTuCQr?XDms zMD;M{QppiW)BBT}H_l>TD6D|R`klKffv_=+t5UKp!^sn0nk@rzzLc_3ZzHy{l6h#;)v?mY;;NO|(GhxNmv1;J z>Q+ebF{e%)!)L4*VRy_chS*_O3}TujHpZcY4rJqZZP_cLBj@dS9Zw;P2-|g0`h35E zdxjbDn6ARr);P0t-!BpOhMy>vETX^R^1ikb`l2r?5x8gLE8OF4T9UG%H7hV&ZHPL{ zbvn2X(`Uv<;V6|R{fKt=qL2ac+>xK-)Tj%@cg4uj6KYYbd>*k3{bZqaLKAW9aLjZf zwy3M}ROpb3^P5#PFca3pg4dh|{=gy(JgLt-*Kcrr^tT9Qf2N^Yp4FNh4Sgp`EywTYt%K); zY?#$|Ivq9TnxFBwkT{AkxR5%yi!M${_MIE1tk-7t+(gauXhAvN{LUCx8@H$6Dt#}y zxy6lyNIi=!?Az z2icrYU|vVE2O%IUu~FVGj%_$K)_P>+z?LsrXx5aK&9hK;wF_$%+~QK-NSp*OLVq8E z<6%A4q^-Go;Plm#+1vBqGO5}5`p!7K#e@yb#c->+R*|dqvlko$Y_=iISqIA)Z6I?l zm=BZCvWgWd+MPGw63a>a;HT8CYs)jOxZAEf{fX&UzrNn!AsRHw}>)6QAI(=Oi zZc~j?*1_3#zpv@u+ z1t;&7-LJ&}mU_IdEc6h=V&{zZq;r^-y7NI~C+Ib});R06!HQG%R#X?F6L+G=#DAlx zy!uN?#z>x3o6B}PRAYQz!4HYVfk#>!ZFxPxFQ6#EJ0K;Lba4)cv$LV}fE)Bwvq~xF z(7|?X0jCNVRJI>b%Xoo9_U0ZV{FJ*KM+Lqe;z$8;$$ z7{q4!vq;vPv}?)C#K3{vmSVak0elu>vyH6F3D2W}dsHmT=x?7b=Vvapr#{&H9I%_D zQlj<>zE1Ri8pMPe0VYFJ!^_8e$<}r`DxleAQ&7NAGcg-@=_@2a@Jhu4}1tr^ko9N4 zv!g>;{MvC_F!(Vz9k{`~QtYEdF%iz>e^9g^)8h*+yBrM|p z;_4$WvVhixHPv*)H8g9SB9Lo$9E@Yrt<*Ztc|b{S&{ zp>}`;Kt(XB;>MX+g2^!{YWc4vQ7P6+7F2qQ`)x|4l zHFwX0?12)|CLKFJRWPyKLL+FU{UdM1{;I(y@4{w5dV+zd-DBDQkje|z@pI=?ROE@C zt)kRp&EE?#CQiZVtbR26muYc2yT8rpvEQe$?EXS>WB0f0&*n#*g1dyDL2scn{XM95 z;G!j0)4u`3c=W6H!hjHMP_b$m%aLdS%3xY$aY2V*L;x0m7KBRZ8?N*|ID&=S93h$8 zSPnmAP0Wdrq88Q+8I0?}RCVn5tN_2@$uoA1YRxgHek<-SqPPa0g<7zzK4CLQzM|DG zyft2R>{bPc9s!yG4iYUZP`2AN@1PP2$zSys8@5y9q|W>x=qL4NjI?f6()bgB$%i%Ea zzXRq408nO58_+B3!4IdL3d&xz=5APRsKAi^xP6zx=#w9uETGAW9=rgcePT2Wda zEl^cKKJX8rTmLr-kU!uNa5Xcz+}y%)W$p_%BVv3jwS<;=P??x~}2ZM#3 z7GzX+h~fDpQ23Alpz!}E0Kgjw1DWtf!o2;9w*Ub6Zvg=Q3jl~RcOaw#Pj-A59}Wcu zfPjD^0#F@e=6`eG0J6M$|3*Vm?Yb690D~c1gaCY7zgjGS>~=dIZ$WSYGV?PD!Te zSHu_TKOjW1%IbqbPWJ3H^tKNGk2VqhRf=TJVgAKO=Hz7eIFG}TM*sFmZ7;6qZ%uZq zN-w=#s`rxh$PZE2aC;9?dw-ni6vb&l%79D&r|n1ewN+fsOO!K?hXTt-(gb8;D(VQi zOqq3d?EuMjxWwqvY&djeBa%vg04LB2YM9z-Cl>Kz+(dO6*-n`9>xli$ZAXC|`Nf}p z$${z2*ss%AwTX~NNFSj#5%LI`QGg}i9-&@IT)%3a4XIJ?n*(3;h3TgVd-LdoGkkg< z^b680k*rJ7O3pU@9q7)Xr%pN1y^wprIdCHs_4w;T3Ax-K#7e|CX1--uNh!BAWXYn7 zCA?vTL)dBLd^~Zsp9Fb#yfSREu9G_#)6c7DnCE7r+mA19d2)04`FGwse|t9b{{{3KJ6V7xz_&yy0X;ps&%KLgXnVDI`!I zZc!hu*1KVs=e&|s6X$~@C!a{150V^^6JY(n>)^?(3mV7CGDuu&_vAhC`1F0J(yeh9`2^>1U-8<+ZOJ$s$o0b*+g`vEYGU@`-<=n15O= zBIQ4iXdmW=(cnzb9GxZ#as%uGMw9kUeDmb7?C$({ zODrSYW}Mnkhd*tSqgv4J(%!(x7eQ8A43yu6tTqvjJPxwjPvmh1LrnxpN#98Q$T{21 z)5$r-tw6&zvQ|~28x{m;7zEiE1V?2^l+bS=6#rXUds*d0%;Ro$Ev|xSEmf7wS=Q1R zv>7ZI34{Y+`biLd*9^-9HKFA=bt5&z!NSH(GPa1izMm^V6(Rf8+AMtCo5yg7kO060 z&;?YMBA}+}*DVeN+Bae`XK~CY|@(bqm*u9P)( z2etxSfLpT~;Qg^KH+6{O=!5^&!H*cW%iF)~yWgXqCOWZwxNH@WU-G$dm=HCw)of#H@B9d<6-mIo|;uAB7#?jGb41v(i}cljE!r!7s*R z_|OR0P&xQnfotK6uS4BTq|9Tk3RzPPS~NfK3@fu76Fn<5oKqT7$C-Z7{isIT)V(dr z4w0$rf93aHN|Q()_~enOuw~fmN7FG2QJYNnY_hJdh_Etgoq#XPZJut9CC78qU=7e47Qz_ zF044JG+^1&we9nW?N%C)gC~-7QZ=OsV5z}_c7xv2?a~l;`h?b25|Tq-q`v(ce?@Z6 zdZg|<0(BN|!r&Zn`6<3T=%YRn_B2wW%HE()Z2!65-Ua!EmzjNE=(|9ron~=5@LP9l zX8#HF{YQUAhC;Pf5JS*4G*LZA*OMXG;eN6n&;YcC9*1k#aL)Mditzr&QgNzTjWv*K z?no%>vBRH^zd3+U-^F2T`7F%xnK&U+yFPHf_^wLqkmtLCt4xn-CuZA|t6+C=|KYcJ zm}I`gG!Lt_Q04lZ&N$0Um&0JfvZ0~!n3;!U zqHGmZwt1CxWaWvgsLIHS3L@H)_yXJX@{fbGlw-+B*fR$yv(LSGr?Rn&N3}09-LJno z!&+7Xl({bBDt7h<-DuXO845BN&6COB-_Y^ReGZ=#P6|q2*C>Del>l|eSX0+k?%~)_ zX}iK;d@d!~E+Z*+;jz%{Fot4es)HH_*Iu1jI{9_9GH_JwAe~1ew>bB_?_RW@aX-Jn z=$F2>dDS{8-c`L9u6A%eYpU_VlFzeQK?4pMukVoQ)56db#N=}ZQ$aFRIhPf`y>OVd z3THgcLhZ<+#3V20uWH99Lg`Xc2#QR>x>bWaEJe#j}HXp&hM5X$9g-*xei8bhpR$IWLj) z-&dPJ>2Bcan&`IMqV_R=m@lRgyOuVPVuBe)Nqlq?KiKnL|MWhf$1dzbe;_Tp8+^s0 zEEmmBs%QE4M!`P9QQ);}SW{)EJUmXUdT=Q8R?^W4i|Xig=hJvmyTl)%_or$J6H*-? z=&<3V_1Ms^sHHGxj7QY3hMgvri!;6t3h#BnhqDxucUTcWk2mci|9H-ks|qZ5*pW>w z^r_mdXdm-wbr6cE;h6I>UgaWQclkZN^-har)x;?zW8lazeV_YPvW~Hvx}aQh1I0mT z*{k-qvBj$CEJXm@PwiL3{SF(>SlXlZCxD+9LZj5NX{5ze#XIlEA7*p}QU!FNMMzg8 ze&Q9f2(ksvL6wF>C}!o3wSXK$K)t;1&9NxZXazCEYx6rc3=4um6-?U;7FY$VgD3!s zUu6v{#*PEGDMbx1wf__>3ygyz`~jCdq*&wp+EEXo1ELLFt&7yI1xG4<_&nzLj00*f z7WZpztygssk;l<8NSH+t_6q9GgQJSg*ofb4iFC>Hu@6C+A1 z4K_X32TfPr>Lhfc><~qqG6dkfBSYHaxUzXL{kt8MO|MoMO;1Jdfltr&n9ZyRXF)}l z*Gd=3Ig9oV?XWln6p0T?xrC4eTW`(i1z_u~83jXw{v!!ML%fr}*P!fV6pJFygS?hJ zBmzKBfZLhaC;c=AxLe?{Wxz2NneD&PUO1xmIwjx4xK$&S&sJ?itcjG#OFeDF;201d zkPXVDI7b4))tC&>#Q%zjw>|2%Jrwn<0&{3%5>|h=ug2rulCgM*Hw@dzz?+dy73aR7 z+>!yuc%_40H?wQ|GkMgf^4H6ZMdVBZ?6qHH)PDTrcyUT-p7i!PhEU9KD0eJ(} zlAB3DfEqXi1bYNgKEq*cPe&J7Uy_8>7sACHRu8epc&>BA#qZWFf#srm&-X<;%f_!EkD!XywwpH{Vc>)(o!DQmkkP!7!w?0Dg#bGuQDhM8c z4j=^x|0I~bYlah+jJyPl7kBqE%c`7R6G^-4^v|l$Q3qmyrMxG)zYD-1hX5eoED%iP zv~zpv$E{ks5_SJ2_Wo#hMtvSbUF9dm0FGpqyR4&udcXOx_+#g29e}RY3C$#;ShO?2l0rFfT8dw>}AX8Lk%9jL4yO zS<86Tj#=nCFh7YFQ3z;*-Tkat4gv#$fl$yevO-038_h6qmb4tGbSTVy4x#1oYRqa} zWq5+QC(B;YP*R>MT~mKP;lRKlpkUyWv+vMOu4J5N(${Qq&|vWf_2QiUrC$-2cK9San=))al6_|__+Id zpWCYiA640jpPkyw_`xoqZJR-tDn+EwZ9spWXg@?p|2JwD+SIvRqCb?y@|{+sp4$>n z9(`nC>dQ`|@QY_SJUNUw0w@s)!(v&~J(Gx#0u?1@Mewp;y$NOFgJ?;v9#B&CaYRiu z!32n43&f!0CWt-cpDA|M(BW!*B@Sd?Dv|SC;&D92(E7kw4lg;Xa!BZ!T~D4hqqp8| z-TWDrq3==PwoBEUq2_A0M+9H9?aLL3vsY10pG0%Y)KY}_##do6fj_?$Ccfv!mX5My zGb-Ho4I&FAIfFmdu4IUjk!Z@eKPpZ9-OvBrC{+P{wZYS&f!UHuHA<6*Y+_}xjIpzs zOW2C6Y>jO+@R!;N!2?RU6oPqap_LCHm^TB3JrgbKmT$m{5$9fDZCj^ZNSY>yW%w4MO-$$Iaq45cgEjmukMIC64aT2&{GjZ{6U!=tryFfvNnIxijqt|G> zzs0+Um;QdG8{z9nu%+$R(CJ*$uP)A}|3GrAg-Y2^UNS6W>dy6D|7;wqjdGv_@Z1ds^V$!4r5>JqPV$?-jxX{l4S(%x!8Q z$IJGtIpypbT!mNT;u-KJJkRdj6#<7pzsxw`dqXc&@9xPMYks-xbsW!G&WAgXF?}rV z`mg|f6vY;;6C;1%xiTJxI6QgrPqNW^lWZa|CM`Ph;@>x4j(t(ZbUC@z&c<+gUycl8 z6oc5>Cs$6jr7I+i1Lf4s-p=r%>jS3oD`+Ntyj5tOUD$s%{M&3W+~auXVH)W-RvBJG zR9T4~7y$lWD%*r@St-(uQi;<&n=heoFca+YB78)vhjf?hQAss^0QDk6o<(du1++}V z?}WpvBy}ZkaHN!HD{ur4dqpJC^rR`rced%|bCIt_Pg?69(S;-%`%wpV&(6yd`yn>;~$gF+C3Y;*Cy387^RV3yw zd84L^{qs^Oi&vrhvz%Ab<5h3hmy>^w-ej_ zDc^>EIZviZ(GW?dIiL;GYgfO=&^BTc|KA2x)8d& z8cdIrC+i~+o&wQ9H5|UV1k*QTX)rVF%8Y`4#AM*PtD~oJk-i|4J>5}kGS?O%CKS^~ zu7oWXzg|ILZ=MiA!S2V0{@BeVdbtmoH_XS`@5bD${ zLPXN$t2v_i#q`TL-X%r+jOr=>?6$1lzl+*C*%btE6iPjJS2zzxviEfiVyLR^_i_A3 zMfSZ}ur~(2T(K4{E`F6?UE5S}gXfqg0SaA_)VF>Ci?Y<726ONV>lVp^m)e-0{fxhS zlhj}XOEJ>l)?+0fq9P}p5%|P~%X#;G;8ROu{?W8#TZ;I@_`7aRe#<`PvGL66`(T+_ zCzC~bhL&Z61(Q%WnK-2EuR4(rPV#4y1-+pimEssv#Cs+J80Siwk70qgYt!7B zPLa5l^PK*nHY%XN35tMKukV8SVk^J+0Z+gi=tKBvGQP`Lp%NpAjCx@4s&>pGbjj0aF5HVjEd3b~6X{hdT?3eOn`k9(CPCEc1tL+F<~9Tic)! zh<5^ZOI3ZFOoA>0d0KVbf)z|U4$Du5SfePQCiE&LN(23}TAltz3;l1V!)cz) z2K@43q4;o?s~N(T30!4d3@IwTojCf$%fm@;;-Ti>>gXhf?ya;U6;A~?nhQL{hBZ0N z>P{igAM8w*l$(yi0FqCHN0sTR6a8TTWdv{-h*br_=ZC25>0H<+woxVA3-(`%z66)? zb1^$@BW6in2+vY?rdWGZ$28thUwX*UYX(4BvEQb#wyq!yql>T5w%ar~KUe~~A7aCS zbJ`i0OzM_lEaX#rq7`DnD|Y_q(3E?VMrzOJ!KVJRN#mz#pVrB~CSUNkdLxFda@>9` znE)za*&<82HlPCb$n!#ds{kxj*NZgd!f?`fwZ=K5{UL7ebj>(PelsB&+6)Pd-59}u|D*11yTEBRwifgZPXA_Dx(!=#QdS_NVn?1g`m_xR*eQbs7p6bN zEFV8z3q_@&g-8R%)(aQAKcEQz0P`syUz`JEVHQe==gH8f{^`ZefMN{(jOTk|0b;Vf z2^%`|6tETwLWh0!k}d4|-us#T29XOYbeX!b&HF670Czd~`w=Z7N)1q7!ejjqtA7>< zp}_$9z*&?s!wpi6P?(!it&%9YBg=>)!0I=uOAk=}wtORC zYvJ4STP^BNe!|owVq%DF;jFVOZ{dKMKziH*OuBn6!FqrieX^QiIh}u_wG!V`+Occ( zU1URHDQ2t8=rE8+L`bh#y|KJF z;!JNWuP)&Ge=P6!vRi{nvbJHqQEizd*Tdk7!w8j~Ra`KxK`2Yr=;!|~dBM04f88?F zn^NVV?g{vUiF^&u2f~>ZHsmu4?>qVa4z61z_*k^!dHr*Y->U9C0f&ll3&C~hv~|SU zVY3=b+UhWz9euVK>J=}Jt5oY?&e5qKUJ(bIoc1KacxFD7d13f z{@3U5iUF*+wz*xkR*Hvr*@diKN(cbKBU&af{Fc>AcrF+T(muR#oRIb*7(#PSzqJoc zwQ=sNHU$xs6Eo17)2MSlBvJLtfCExoJLWp2_{cmUpJk8_gy+RTCnpsbuI-I*I@N@n zwcsVXz_LB7BYKhE-f?E-VulBr!GMR!u<-66CMeARvk8iW5j>nSSF{PhwFNR`#vVRb zt?~gXnP-;%erPYx9{2ugpkKTOQ|HiiE?BXMk2~{Rv9Y8KrUc0V zVmz>M1^!&#C8KJX|D<>ZzB>o((<|t!4jIAo%}Bk@ewYFGBJA`~MQyQ`5p1U*s8PF0 z=VPgJK|1!bOfO9?nh#%}lQ1R=+{W2R#ik0}2A=<{+P=u>img_>Xp{@xthjJP$fRMm z_}ILCbgsaUg~^<38l-kXJMaABmiI)iC6&mwDQ%gmo37;?Fyp!TuaQq(kw3vi!lEG) z<3dV-05ik1L!+ir62>X8@;HJfuqe#{r0EGPc_Ky!Y1jK;XWdnyAM&tpzmZo)C~d{A zghP^ICA5j15;etCyQEghzxN!8VO9x(dW}eEh;ZL!j~3r$2{N zl-KyeMWt4)6tBJ1-ljs4WvaQH2qx>u^~7B}uOiUefwGmEK;fN>C_@+7%ZSs!bas55 z>E21uQ3KJg2i-&%)&rpu@vWc-U^8CFssr({3iv`ava?uAHdjqHAhcQ~-(| zj}12s72=*+XoI+?N^+`@Km6yO@_F1l&2+nw^3S`>9T9rXT{^pi7^Yq-^g1GUO_w@f zf8`Y(sX2^WX2_6;&u-K^fIJ4i47ka-ZF#PwhCS`8+XxeGn0y&5ILdg!p+fjr=cqQs zygDqk%X-yxaw|)8*<^c3?C%)!_;UbWVfm+LP9~CJ2dD{Ph}?|pz(Pb_9$ixA;3c2w z&tg<45z2TDIUDid>!8ajx`D%+jYCWQd+!4%+3oXpog+T^^QE%x~vzLN@NidWTIM* zp6NIHJW``}U3pN{@*1s;(xhf$m`|v34N#;KDhrOyy2`$+4#>Cn%mZU~sTC$D#Tm=% zu!$!aG?6~pVbh=E@a7G#SU|WjYS%#d+VP7z)t(?K&s0&mltW&)G^!oLr5je3eRe>9tr0T=f+BG$KmBF# zZnm+zv$)kz+U~%QuYiaZ3QjcN^L0(o|9Fzr-63jgSzC@e7lt-@nYg<7;&WU+tD8}! za0SGWv=Cfwuvw==WM=pA6zk* zEUmE}po=GsDBTXra`X$_;sdenu+A?!isOM;0mguKd079agN2b9(Zx*Dt^7ke~N^lT9)ao!w zjY{1dm!lYZi+}JK(w6uD8U>`y#E$Bx{en#cGFk>OQjxj+8|{S~TVE^z4OVDn`)hHG z5PJnLBz}G5tW0{iU@C!ifd0p_{EjvGlx;z@@~cM2qM~4w>?c*cWp;46PytgQe7yOU zu~Kc^3-V7|j_2wMAuuoKKf)&I=5bA=V3h03(na9;r9^d;+~c;xoQM4U4>b1!I5ReT z8Ve8yi2wu#h=W7`AOvu~MF8Z+6{!BeSOjKJWS33yI|L3MM!bKsjzDbt0SdaK|6f6aI7Me4uKw7aVP z(E@#dsdJt2`!L=1A>P`idj+Pd7V*fQR)Z)M=f!1KkL67<1L%5SSNRZ*cNbxmT7{5? zBmE_LMB3cO+nkr3>LHqh+S&Yc8iM}sm`Dc%00bKYI^F<)7x45y0B~AZKxSx6M72q; zg)fs^2#-`mq-&sN{^r=ff`Zr%mF8OSqU}4z z5ui#rj2E(4X7V!wR2H&QVDt-t)Ij+VrxX}A>;AE5$;jZV^^BRL4t~qX>cEeO0LX!_ zDgYUOiYt~M6w@?>%YsDi0^zbC;79z;*HQ(h9BNv@MD;x`falLAvW})9td)jd4!1t- zt2kvzw*hq2j$oofWJpNZP|L9YWugxWl8N_)6eI9np{n#o*M7cc(QyC1(nc;!hzmiN zKdjB^kn_vRJVMkI@e~TGk~h!Pz0?6OkjY``=%nIRh<(*7Y_M?tfy3lM0Y(u6l_j{E}k0DA$!F(&w$&8$_I&aew`# zw2w$+D*Msn1t=_z>;pO)->rL%T*3LXMM8RIsmyZYujcMB#4X9>QtkxDJg+^p9kg@> z*r$#9z7nvs(8S{dSUAEW5eE1`_*3J3;Afh=f#>Y*l~Lbh+1T>2FCOx2yUa~jj4Fxw zE_F&~0LI5VX$b|Y_XboLq_e2^iPCrb+aBWqzo13MNjei~mpo8V9ZV;?kpnGn0^KOS z<_s(atAd>&aOq8LdxJ}xfSxzD6}hL*ms$f>xcF4VIsOK)HA67=i+t!8~+Acy)hzG(si4Je@LH*e0DP z(1SdOQ|-s_$aq(&g&RF>0u^sM|Kcm)uRGf#89Tq}mtI8Kn}9n-cW<$I$4J5IeP2=z z;wks&qsrzH2N91b+XYU32xY>t&|Qkz!Vp?Sz!9IjR0rMy^_FkY<9`+T=#qP>k*B2J zIcYSWtmknTf3>vsk!vFe0mn7+@O1sRhewJ-!gr-p0Q413EwN2GkAZ>eaq%M=ZdXqm zf!Iy>2~2PcaQt0&;spp39zhV8F$a+Y#%+NaTFISwafQrEduo1C8+U1#u^+#DE|Uzf z+Fdo_-3g3(*#9B1rqCH(>VK+ozb5IudaPPntoZc!Pn~rQ4a#s``0TdyVi{o8k|(>H zf3UGHa_yaKat8rd2NVy+JkeA{5F$mkE?fhYwu-J>UHmQrU3epS<;z|+5)KR{!hrMU z`^GTw&eh@(;#B?Oxlh+)YVZ1g8&+6?R8M1dLRDdM5CmP{<+Pf#xnT)%Fv*3eI>6`9 z_kxg-CIE0i6exVFP$75eFPvfCK4~TT+eXp$url;|CTE{1Q#G4Hg+?ZjbidnZ`lfy$ zq-hv)52LAMYSx{okT|OuY|i>xVr3#I#^7cA&|G1Q1{1hLQlr)QV~P&cBQ&9 zhR|a6t%3_uONb)MF7jg=Qi zGh=);xs1pnbxVapcnd_Y*S4oVz`|t)Yoq{=GfOxBEtNszL&R}QNI^fSgB_>BrCK!v zBgG^EVeJaux4^>woUY4>47}o&twDQwCxi+=t@iO0@6AmqE;c3>y~M53Ig@NdDv&H7 z4nS!8@nUtAl+aaqsyS*Z1QZ6;%YTPIU+86akq-((&3`vB4^cv}u%;36-@Q%Qp;ave zG13*;GP2!VbeKz!9|TAU53py(CQoBwBtQya6g)-(qyWBkBmb3fVa8Xxw6#Px#4e;Z zjSGAL@7)F=&;hUvsGY%H*3T`3tp%Dv_83Hs`ZwAO7yhw5wLcNvFiy`(p@($~C)wE~ z843Xr1CyLuT)vU(=c%KTyib-R9Jf_Psequna<(QWe)G{|M8SJ6D#^aO9~DtMG3jsj zf)W7~z1cp0;u*fZl&4v4xOHl;Xq_FR671#-mQeO+9lm%&tBFG!Z!tb>Wxcr}TR%er zfVikXLjrKY* zXB?)tQ2kVm!2W`Yup6C)zHb_D!pi@m<1K*V3c7CX!9BPHcL*BXgKK~Q!GgQH27(O1 zo!}DO-3jh4!6LZ32Oso5lf3U=_xtMJdaBZ=8cx;pOop?1_u328r%G_fs2<~GIe@?7v4L;~(kZFX)E69c^rozr@I_GD%`8k2)OMqYlo|sy38ltd z=`S!XQqi$`yHZF@>n$JSD%D?Pf`zY_U0}RG=b)`=^mFYbz-I=~|gq(zb{~(K2 z>*LhRR*}pv%fIhj03lOJ{l0(Th6B!e{@#HDQ-8y?O?%(GlmYlm?3V)lMQ{WfZMWOz zrMC3wYY(pyKhfkL4r>_{Dkq*tA+0ZEe&xg~_CSdgduu;E;~cy&4obvhsjnud%>aJePdy!wC}gjHW%)(@Kj%ZmWqo7fzl!V(r!rBbm#r45UH zii9sQZsr!0B^C@ks&XX6f}xKpR2cjj=*}LW8d7FnAeqNouq{#WRRU8^mX8Hn4k_*h zCE3`Qm?G;Blw@Lp=`>`nY2MH)uv?AzMiSdLN1bGPH=cZ+W3@4A@+@8pTvjBA5ncs3 z2TP1<;iU=S-KHv$h2?XDOotqECH(2~DdUuFRLbM@oaZEK2`toAUY+5_`0ou0u;js` zH(K?KYtQ7qYJL-Y+=K57&n@4_e@u^79fAmE+Xqlzla;SY+H=lGyz6FydPVTNnV`iM z4N&>tVoSO9V!l1RA>M~5^0i2ZIp#IAZbC47dJkbaP7jzJWM~I6G|OlDH>+T%{b9e2 z_fh?AU~Wy!sX68hN(}_>;O5=KoEjY;cu-{?z^idAItwJy{)_#q4N6 zDzfX}sh(By$J|Y3hyo050KuevZ?{tT+v;|DHihmk%cVHG@j~<>{J<;A`cTg#q?{z3 z;s-S)piwONef8@nijc#0t~ENU|8s7<&j@=S^@&ukcmK1Q>;wCWkIMG&E_R!?7t2T| zQyTe;W;#U&2*v83NIk<^XHFYM>L2+YVWDF3_NrK4%Ip3rQIBBROv~v!Um! zLsw72SFU2xgI+b<0}nv^z6&`;JXiK|cWA`N4WQ+q0oe+BDi2jhD%A#(E)(X6l$*Pl z{1VY`&!#M2upikPfz{jF+ppp)!z17kmA2QYj9hI}Pzdl_T?D#3myx&e%O4_2c*bJ^d z;G=ZS$`TgQiU(19MNVm*pw|RcbTKi|)Ulbwg&D^2^IiP8GaiP0w&6yTLPbPv7BaB# z-Ix8Hg0vCc{L#gaYQgLFX$OaRUXupjsI(win=IkU%!9_^p)0nd#0oXdk@*6jKlO~Y zqPte_9nuR#(u$wG2rquYl2*_{exIhEmm6y64H-TvV7n`S=~UlS!Nyj2mv(y@G3a$> zoYUWDm%WO(ip3~l@8eHvR`$B8@sAZ_301Ic$HP>ZY$uCA*g56bNXNXWkWn``njHn@ z0kZym0&Ju4+b@<6ba;QxtI3%rixhM`vKd#b?v&FMWlg=)R-}AVx&Ua%9Y15d$ttq0Nw^No* zDkNQ)J*LZzg?1C^HJ%MUFwy8S3nDtREXusR745fF1hcIwj{XK+D@3Y6q~BCw`;Y#v zuzy^azNpFyDXZe{yz_kYc2Iqm<`d{!EwLai|SU}ZqRR@=7c>`;WCpXrsi zMk9n{jo-RuonBT{`arR9Fp_bGgw2Br@?071%Ek!P+Lv`1En+z><=A4>u4M`UDs9VM z7dv&HZ^&w{FEXEJm|xwvkRo7W@@{ehs%(o&xr%YTv(FQ}^!UQZM5#rZUmtw8QFiHUjXGqNPyi~B zw7aMd9B-{_#vX_rDTFhlWqtn?>aOHR@5fC8C19x=0>ZwSS(BMRLW1pztf_vT0d@V z&)^e(j0gx4{uL3DGRrZ>>mqcxfj4t?4o|}RoF(x)RbpY|RYBx=uzP~6Rxvemg6wak zyNvJ56V8`2iC;|)wX$Ws8Ti~hFJvZd9gF7{%s4H+H>E6;Y%X*?wU41%)VkemE!DoY z6t9xfX0E4tPuw)$;(lA6dYj7qMR99P?7O%~3uG_HZR){IF-^F=d_NVZjC+qMgPWyw zlnqB=X7d=Kn05zDAoKg9OijP<8R4E9shSY_2VVQa6&7JU zpN`UBBYaJ44*hi>LqiEvC#D+o0@Fk6RZNIFmB})(BKNZ7ag#KPby6~%G=3kqRh9E( zy;Hg)UZ3tzryIHStFrTd%?=z%pDQ z;pi~(V+)lU$-z^PnG1u}kyQTk+ zXs&o^ijNhK;h}r~!+mY7q1b1I453?9j0rUV*u*iZQM z#bOTKTSpy@Y})>J$AKGMeb#IrEq%WC1{q#gt_!sMAm{9AxcZkcwM0lMDHZ~MR&Nks zj%&uqoCA7k!@=3mj-?ZdU0Kzj`8p55o!>~RJIn)&-RRX*v=eiv_SGuWBAM-?p$=B% z!%itG%}Drun?zDwK~lkt@Ups^b!}tW5TbPfu{p0mxx=eO3@!dNT?z9zv+aLD3QNg& z^p)n$2tPD7!I-G4LPY^P*X+_gW{%fQO_rlycM$ZQI@?+qr@vgv-hc??D z6Ayz?=St?3A>vnVl12aL#38NeIH4>p1^Q#R+{yD@f-DS%-h?drNMk?3=()J?SXt># zm&7XSaGaGF*t}L{8tDti3VbFt%7Zq0C~F$P2X$7KxrQG{0&VI7Z12%aOC4jng)RjJ za*hs#q$ZJi4V!U33};_`zuBz`bi0Cgxpk*4HD@qK3sIM2QkYzheJPcu-s1Z;o? zdLrJ~ITah6s0HoDR5y4_=dwi42jd49sE8ZCtW=Xi_GHxqv4V6!4InFPOUC6z0xJ&g z6oImrulcxP74+V|m&ZD2#l4fsX>owd2`;Hx`DXXt5xt>=`g-cTfe%aoV*N>iFjaRd zHa0!G+a)66NEsjGUnK$inMa?0*RjW}z@5}XD43T%o% z%Al`MwY(P)_1<2L|W8`(5iHKS)l{v=k%Xm%pu-u+IkHWV2lZ#yJU-25HIL&eF7-?xA&x`iM)*$ zL?uJqaQrf(Y7n!5K1zowD+JG%6Tk){^>I=f)Z9Y>LA&?Ba3E;+9_R(=Z}%REbV7d~ za(b~f623w@oZHem&5+!HOF7i3%knRVCq00)JiZGW$72U_F_UEbH#?W}<)#&H&uUHr zu7*a5!2|Y)QSYAgC{FmUNCFOUqQU_V)>w~_h7NleOusmL5*s;*=u*9e&H)j>pL`hl z?xZ5<#31r?8=&ctixlc~g_dB$G8ozY=qsZ?qmy0=TwEv-WS862d%zJt{RjSnYFtAy4JWQ z-)I&F1^}IH+7$n1FB|cjBegTrN)OFViVvS_(QZSxb_j1}*O#jq1jQ)9VxRyJthHs_ z;vxZTZn7-d7*DD#eANMQzcVWhC7Z@86!d7G05QGAuc=x4Szt(Ugiu)(~p z11xjLouwZHLH}uB>{sfy%>aUfo*o=!P}?*+DIhfJ0UmOOVoUcS24TirFL`nqatMK) zBD84^*eQy!gAM++O&hSB>5eT)!zd;?MN*$m!8uH0l*2o~EWtJpxGd#a9}qMnI7RL> z59sky09R;h1a_}abJ@&U4_-K)K@zxMHvi6%R@T6a@v7tptAk2GYA^+PnZ{O*x&^&_ z8Voh|q5DfT49VIZ%3SCs2gx?xtwns|Om>-)ld^B&H- z9?mKJAVY>vtA_49f4bHwu_#q*sgW7k9VgXnX&{{vFz zAuv(>bjQ94cN^p17rS2kbbk7LH+f6HvhsZS+kb5l0`t6`@?3$H3`2t(i>KT9s1kx!gI+}QykhUjSRZm!Cuyb?G>Yyj{WTgEv~QGZ<4)6 zWHes0Lw{ByUY@yKc&jva+75$~XkptX4vW98u*d_)h@yg%Ln|hKJdS(W6dd2%eZe|Z z9806GHSkP*-+Ne)>*bY*FO@wEB_4_19DT&LwuS1-Un2EuT zmtelEpjf85i;D?4Z`GD16tj553P@`k15#iDwn1Ypz%g7h97{3Ohr?1FfVmhJ`N{=X z#EE0MM#YI^&7^yPd(%X-(6g2Ry*U-5In_=bfe?-jQ4gJj^_W>PL+np}@b%NM7J;5#waFhryJY0)S*5DP3uWuUf;({x@tTCpoY#*CVfy*TAANNebM)pCjw0tarx2Ol zjQDZe&_8VD&zj41K%QHJqz4?+Eh4(CJe)Nm0|`6DogKFif0o&_OgBD`z7qM&$V99^ zArkA1Hfp6E8vYW3G}nJ~qNbraQC17`)Bigs$~+#(iHdu`jotbmIZ@sMKu*+`|K^WU z)uQq0}p}bE@(&|=k zOf9uhVXh3U)YLL#uKD>{jm(I-TN7Lf#779V;kDWQDF_jeKc_??vpvV6heCZUhcCsoXWw9Yqs zlnLG6@)p&al_H9WGbo|T0K8Q5q$1>s)S>j7qO}?O1AWy9xO}4S<0T#-pW~)hv+Z#jfoCF@$*eMJ~^< z_oOC~%eYy4P*%50BVb9&Ul@@Ih=9R|V?wm(SXEcdoYfgHEchYe=x0wgJ9XDudH_*& zdVTMZv1ZiW0}lMq*`zZ2gY_4|0V7gP>5eO6_wk+j0?hQkY|3+o%Za&Xt$^7g%iAPhtqXN1n&`ICx)xP!FOwv5!E<_dkb|LJRsgeT?S_)YkYst zDENMPTJIXgP+L!D=<3ldmp-zsqLi7gvJ$EzmAp_Pn<$CVkT+H#%Ms+w5d`uy{>|Fe zBonsb{n4&3L~NH$w>W}7Rc*e5^peJhc2%8$fI^ zAowG*cE)>L(z{|ghir|pTlI$%96IkE;=a!KdKX+UCjcW9_l-8 zfsUSHMZj!87=|+P!GSgtHe4H)@1IUIa{LUK4JgIM!fft%Z$2qK`CGK)THj?kg!X%~ zPH2$?!1M06(L=~Ilm97B@tNH&64#qSEoG4vuJ5;p&tG0y)|Ven$HgF&3WQ&*a1M`j zU~S)C9)+JjTJeq_&=Zk*3=)e;R!&V-Im2hrnsQyjljZ`lDh5Ql4#+BgSf?PUta1u! zb9LVh4;F?@XV4Jrc*5$>c}5PR;%n!^nn`ojnP|AYWL5`P%mr1*$!f&mQ!^aL2QcG@A9IJ_7Tx(P=mgKvu7N-dXMaUK-R} z5%1E2nRfp9UQym11Vqzl<1yx&&Y@8Ji>C3I0c4gCjE4&#vyR~B7U;bxs8Csg}XErQfIITo?B3#g!u&A?eq44B6EatnC6`elB;5 z3F*2bFO`yXc{So=LZS&h7*;{5{6(AlR-^a6+bE}WMf)T{p#c$YXP7{z5H=9umUpTo6!$m6?d9eFif|hV z!#Ti)q&WXt7aPrt^xLt}sa;58h?lMF`?9nBcI9Xl*Q7k8jLsDoX#AGI)*Ek+e&)TS z?3cD>X=x}QjLSEY{;n>#F?O=*MIAWomT2wKd`I9Ta_ zg{d*%ZW%!bM``q;g9#=C3jBy8T>_0GTQaezQ1a+z zdrj^SAY~3YIkAkA7*(t@c%Kn_Xw#-w)ORM`z);!S(2R#3XnCrk&`Q_{ygEPpTbCTT z(e}7NS{m4`_VLp7R;lx^6r;9hvZuo!p^e1(BQat!qiy_Mr+_hMPyyO=EMIUTM@uqa zZB18l?7O^AzLepcUFrPkqm@vRErNJAsYPh|%Y_$oRkEaj^>3oW%zpa7E!G8>cp9CY z$l763fI>z%K_}rrbJg!V5<%I~6gMV|I%L8!z-P!%a)kzC0Ci2YzX9bfXVBh>Cods9 z=d&i4gp}H0e0bKM>)hwrfD|U!c*mh2to0gXKDs|@Ik<|*fRN{)DnP)MP>Ao!)k=(R zXFOT?gi9a=d;89h?*5@XOqOlmkGxhfe~hEUE9H2HM)I5lbxAYe94|9I{!Mnf*Po3s zCW#HWwnuJaB%T|#)3mOOzbOeEQS5437}TAJmVO8}d3`bg;W~!96N?^sn({R0k?X7f z@#4&`zJ5Gj^dw;YF#7R>L_w=bZQ|g`M5T6x0MouuP%1>ZLc9HxH^R|mGh2lJYC#Xf)Jz?VFF2R`O$!QmZ98w>&T zte7jFhC(Ac;C*Up5NB*_c)!w^P*_%|!em&c6OlvEg?7utKc!t!8D0JP8<+Le;&u5N zF;DoeKvSu1BKF(uE<2@0pDMM?7H<_a-Q)}vlvA1T6J7(Oj1u3UC$?TN@PGqg>>wJp zX~}i&e5u$Znf;LnI8_)NYMeWk3APWdpp)v1HM}M0v*h%acn1b8GV6(fS;3%MrpuLH z8Et%GZACQTv8Fwav)ZUIAO{IqdGKs!o!D=)Xx$(!No7lAEFn_i$T3OxweZn+3p&ryr3e4M&qsV+NqpYu@%}AWY*m!tbYh`Vkg8ccOr_vF!14QKbt31 zKp{c|0y&AgUoX5YylxtJ?Y3-@>wwT%DcI>4{Da%w(0xg){-Im-@OOLJ$EO(UYo#$2 z5>q1DkmbdzOjqcAVB;mSkRG8|W$k9-Td{Ym(k48Mo_)i`j7Z0Eec8PnsCPS649 zhcUUkb`sYzvb&sXo*3B)^Fds!wHOaezYi11=v?U(VoK)G=!3)*jlIq~z@LTLt< zZy#$@Q5RqgIpAnOGC62}-aG2bbYCu0s=E()*y~@}{d9*j@^F)m1=NmIC)yRh^Pdhn(h@6#$@~MIt-(i>9y$PE^XRPFf4s{ERt_# zKAVHaK*<3Z$%5r?!$a^bt9FR%?2HMU5k~?I>4+*8K|n(&Q(Vey&g#D6bbkRcx}aP!M~th+e#Q6Xs!&ePJWuG(9M1{wbnFT z>$JWtLMx0L+@1`Vo2dpW(H`PdG)8Chm*@sDK4_{@94IO&%a$u9Uz`i0bNmZEejCVL zuJchx4R#%-7cK}1Io3m_zQY~`Q}^3|ag$@q4|g(#hOgk4Ap~=+>$;k_y^>t4m8W}@ z!eSr6XQ1^j*N}HI6jfGE%swn9=Zp-0B3a>2E}9<%SNy&|#^gGEyIozaWgcy_!%Ov$ z`y@uEPb=DgmJ!uCg*y!Oer+KQL%m;d6qKUSZ3&fV)0zG5n|swQE^pWDM@o8*yZ)(0 z9|os5Uq0hCN@y-4cb~ZCc4R;x2x)$;oEE6Pk;3yMJ7Mw8jc4jdgxD>Z1v&!F&BJU) z)bSVD{%2%JB#Ah&vwkWdxLDB7F?|O~a5JisV zP3Lur^M=flwoyEb0_fI%HS7(JJa{WXeku?H$%2oY}Lm)6ZYPfB6CXbyqjmA7{PvI$BQXU25EN%eWw0b*=D~xd)0k2|7{4 zFbpq=;j}WC`VoUI2RUb$gAW@zAh)@{%Ngb!v3nbmNQX*x9~kwm95x8pjs=wj#3>~i zg0aAHWMXn0Gyf)k@LumJ=xXpG?Y{|yg$&z<=TlOcYgUs}e|+(m7(Zo`s}7M+!U2MB zc7u6gF0rcbnmc!eI`PNwR!y0*0y$=DM6$bG|6%sqN8nPA)hsA`H0X$_C7+x(mA_zF ztUYv>d#gJwI>U;AKl%xgP#I2nGc^Ml7)*nfTN#u$i~1Wobcf2ubNbApzUlUwbptS_ zM`8TbO+`ziJNRp9 zK_zXmWq@Lp$vb)r;jMs?3GjP?+T3K4`|~eT36<>xdOzM7bMCx+!6TyKB(B#-i}(k` z2Bj?f{&DQop!05Xj3@hM`$4UZAVtTc6g!fHozbvIvmLj%ejih~<>>e@N6M zZ5P}O3nSWGRWcyNUXRzelBqIo=}9nVGJ`eZW9Qw}-j_n?4@ub%%IBob)40|@>B3m$ zd~SY-<3?S8?<_p+33l8sb}Z!?oMc}W^?wrZeWs}1nvZXc^1#~~zf1TUR}CrqX}Ms? z?Z1dQ-!hYA2dP!3Cqk9=lc8A1#}pTUlB86={xA3+bVbHqFSud; zBJL4kIkMwdf4R%jr`IyHkW=uP?BZH9wGzyMeYnnepJWWU;V%o#!| zX!#NQROIKiLUQ9vtI@C=6kx2NOix;ucCt4Zo#UT-?|D1mh14Ev*P3Po!4njV<3WmG z8(xd7aR4;qsEoqoo1iNQ)=8L-r3m}1^Z^yI)s}It* zi)GWKVVyz%kghY7$|^p;h+O4y`H4ZI^9!Hcu0GTr~{z`jHd! zPV-%_0eXjjuR%8e<3c2cx!xSwTqK#pvnqN^4IvbhI(n^RT0&dHYNySZ_SgJ~)~fQ0 zVARa^tTe z`DnZ~pd8x4s4PzQn^SN0D)AUW-o;D7h#iWI4LiBg?k=_PHdmUq@yITC2{wQ{rbaGg z)UH5ENL(9uzt*2CD~!T4aC^jmp9#7e63tXyL*H zMtrkgj`=;qu~L?Rapo?#0v6+(hj?qsljzTp-4^f_Fa*}IMz{w33P1`Gz@XmZDX-~5igeGI za8Cw@NDd2dt_2w&;4Cy44lEROsv&O4m}b}?L<1r}Sn*G(F0NZ8C-QF0O0ckuUl+}*4Ax2(>krnr+@_|*NJ ziuAk4SkzE{PU^Qu?Tm3=P3q~%^FSJ|Gm2vyZ*2%h(D%PRI#b0KT_iL##Oqn7EI5+; zUg%b^lX>uA^xx-{r46hE2zw+xf`cQUe9xv7qFfNo$IuuD)p6{ z>kMUNjioJxs$_Zoq2h13__G0DS?g%ay{1+vAK&ub|AQkOyw57w@wM@l9!bxeC=}Dn z7-^A7D&#*sW~u%|S1gfgXdG|AnB~Db%HK+j1rZT07es$$^y{;pi+1@uKR|Xb_5Ir8 zj}ut3nz{}#H|}PAuqz=FQ0>6*S37J7v#XVgGm=7HikQb{!Fr+qggb}`d(MmLVY$gN z?9BvC^CG|cW!cK2e>fmpe$b?=wJ7})Khi?QT}0D;TgS8Lph=(Kv7|L4MF-{(boc^v zXmc3b_-?ND`c}h3#HPyIe&M0y@lG&=`L_@Ka-{Ihsir~t+&8xYle$?PX_M``J}yj6 zyhR4vvcsFxUk1}VpRUFvkydr_`5nc4$~W{f&w0?466)#_&x`L879R81)bI~BQ zkZeQTD?3!E%Uv~Zk#R@Z{J9Y{StPoxPkE2fQQFSg@tGx(VOxNl!H~_@rqw6Y(BQmH zfaL=JbEa0cS(XQN9dG`QA-hzB?VE8z28wvxkG*_>Q@dX_hp=nIB>5XZbZ?2tc4%0> z-AQUxR{PGA{lkyqJVT$>)alKfUsdPxWVw0s@|z)Z>`pJP_PYA78gHa3i_&7p=J5nt zvh?(Ly{%wvlZ$`S*4Kp)ey90k(Wrdg>1^F5*j7=TzPOAr;$lTA6JXvNb67KssWeM3 zT*uOzO4mO4P9#9B&XTo}!RyVZ&C^bmFF#Agd|@_=GkLlcfWXo1G%|H!GpYo6PUc*BQS}{G92Z z2GkN*Q_o-7ldm=uN?#-Dy9bRjzp>O8dBxmbe$!FVrOu!l2|2oJ=XKI&`#879)AF%R zFKk0xpp=}(V&&w#ah4z#vLX1|HI~CQxxp;XYw@TUCe5*iNM&4Q97$!IIY;H*?a-+= zrFbm*PMSg2p~D0BgF!v&(%PsJfu8=?_DT`aSi+~3!O`*GrhNm~rI3@-Bqp6UhGdfb z&lrIi)#u2N8+Xx~9ZYj2EOJ_|ZhimopRrB?lf8d(1l~?|?We0Y2)=(*!3&Oe%tRN& zOy`;(@klV4XV^)aOexmg=PsQYkIl%}uy?q$WpmA5s5w{IeObpjgpm-ck8QW|8qz0S=SQo{NnmWZAdWYj z7X3C@x^F7t`1R{Fx^aFT<_VHwC)P2eAnj&j&H{k~$eS4-DL%!jCC{~WiSN+{XD>zN zGo?LlV4hIUcfizS>lAkraS6seBpTDols^DHgX}Jf!gI?ErS+-?Qf@-Nh*z7G9<+j0 zOZPE2jLe69U@&^Bw)x8WrZQIJZAbIfA36P5aR=X}vZp@Y9^Um|Lnd_srpMeWOregQ zmCmZdq?h*5t|{+Fwv+M#KM}#*8X1a(Mf-i%x$+%A`DxJQTPE^!y?(L# z&d~2d{nOQ_yUU_pjc zgCX!uD7;MDNMu5`QQtoGT6Y7K2t+Gs=<{RnM;~Vn+7txOf&s7+V$>TKt=!1I&0FHR z_j24uDO|6R5%8n`CCX1(<;s^#=16S_wS&@yh0jEssGp@Ku$zY5Rq}*U06o+zWLwoP z4<;1K7sHEfQ?x35mS+AW!XYmgNhuZ>GAtv?HMm4Vm%VWrX<$R*1>@xmY)F8C z1G*vcayH#rpe8gg6n8xSGVOp#${44&&e;dXqdS6FxFon^Dv^e+d&Y07iuZ)DoZ8X$%o^YboF>j*%n!N*t$%XM?g zSG9+)G&~N5QDXzIDYu(Q%i;ZZ;{qbUf4OuWEkAuLO}!d!ClN^bLP zTKmY;eo2+yhn$CHwr7wmhZ@TKt0zd;WAXEG_Va}3qtd| zHpLq=I3G&*dj4;6cr&k&Q@4Zr(XH2NQn_LmFpBgOV0R~y1lhp0VD*l`%!nFt*Ikws*7bs8(qEykTrfhT3U zCY6tTCj4tYe!sA1>_8E&t_&1{Q=pgv%y?{eTUkQrBL;#UK&G<+=6l>SX^x zZ>a;1Vj6Vc2VRVOE&n3HjP1k5XWoFIeMcuM^BD##&j9VsgL7ET5cEw%nfl<4!Dj>wzB_7dQ`au==mWfLBe~# zWKXYO?>Tk9hXT}m2H$5Cr%o(VRFYSWmex6yNGeC?2TCZ5D=$ZUy%Qj#@hZGAb+$43 z4X5w!tiXHO#)4ML7HF1GoPEv~_xO25-5||CJb57DoHRCpJ>?CLe&=^~b}v@nGtD3`6O8)b`-R!3vE8%Oq2HOIyCBPX!c@ zvfzIePrmCLbpOAKr?LHiE1vIZm}~QT|4}@98gW1J%l9qKsa}7T-I%bcJ@(z4@o@L} z@J`~}B3t+WP(1Td|G(mSOcK_}b1M^_Rt~?Rq{JRukK6k9O%5?7j@K zG4ajS9rR|!M`cv%27d$&$K~bM^*cQ2MDhd!bjB@z?>6S{yY%$QbcI#VocOh14FtMK zh{zbXEhE?v8tYc{NX}hr&skW@7ukNBa3pTMMy!L;M+0D{&qRnm`LyLu@ximvT$x8z zho#g1n~PM&w&w5Gj*;60ha^eA^jLX+5ZC)O%yf}2TAeq)zLJ!F zo^r#vHsV2;r59pZTk6scIxG*|sY`AueJjBw$+a3|@%^duM0Wncxe}!}&`i1Iu=?8_ zHrwt8esb>JM+l!jmyOnLCq)G3`}4|#H%~bNd|7sU6Un|N1`r=F?-b>JririP-8K_* zUws=F>Nu-)WziX?T}@qB4tZ`{7np*uk_ufB|;8uPFd<3EBm zafR#M&|LfRt(|eLm)2nK{p1a7uO62Z+#-YU!4lZX9d&~ru{JCa137mQiY-j?E06R0 z3IBPZ&YAT+p4_bxC<4~KjJh^!MX#O-7Cm4huAsERUT`g7vX4(zkLvrCHEb1+W?QRnwkNU%suOsiCpUL$;Zj z^s4P-=lPNV%aozgWKR02KmUz8t86O0yU6TypIAV8YUd_NKRrPS%3tK`SMDXGW>ud4 z*f(RDge*UF648mTA<0Pp_fZBt0VzQ)fu4Xs2jI{CbII@>@F)`p9%UWU;$g>}K${nY zmT`DWh=~yyAdDDcUVR+_0R~|{>vKhjo(dK~T>6|9U1ix1(}{qLSF);Hq;T6MwCBvD zsw1m>{iy12f633j(DZdI!}hM-*5yFcvyhDv46v@9LMLhzVG=GA<0+92TgP66`w9As z)i2#H>gpJ2=w*=vU;>oUYUdiYHSHN{TTuUarmPY2%ueMi5 zg7I3MLt+|CT)~p?xXB2ADa?Lvq5UY0gXq;QlW4ZSD0z?d-18~H%xGdzqXD;F z9?(mUu#PT(UIGSAs9ti+OCN5!rhYYs^a=R-Er#mN3K|7s-@^*H+WLfuDUrtCN8 zSls+54znDR8!0k`azBLhP5m_PF?{Rv>e^=qumh;?!Cy;~>L=FT z9B7PJoI{`BCD`X{()}9Bx$TqZ`2ytyZrXd+T}!R~64r|{Px-=R_NvBE%k>90KSWw5 zxs(82B7`4uh;w#6Kta?7P#9ohFfpJopaWE41Q0LM*1NZ;#z!*El_avdfuo(>G{6`D z(GE4aIuL|8(3}bqN#y|O?<59?s-mtCLg={w-bq>fZyY2gLDVi)`SqWs@KZ241RsH<BjaIE0P*j`;3#S?pSs{PYLMF%JGjNm27dhRt~n)blh zzi}QTJh2=*`+@e!Wq0Fp!?1`8Jn9r4GUplo$g3zHKS3J}h@DsapPY;pyp0rWzUpskUnW+C7wx_a)Wbjz${Umwb*!+t@a*Q*Oat`TJ#HFDQIH?Z>|;Jp;T3h-r)r zM2!pv2A6BR(M6E}80eSh%^AuT-K_K%lE$-n_BH#Nq+%6(lXS!?NJHcW;0gb;16YE1+OYuIO_q)yv} ziZ|ZR4%jg6dLK$R70tR74b6BVM9+RgdCJyl4y7dpX(htPTrjw> zfLZAY`^>6x{>8Hc)NKDP`Vk3--w z%(7WeudL{k_cBCQOI~ux4UZ~HL2s3l_=@76Pt)6c-QVheB$nySszr{KLNx*(NIguI z8JlRy*bQw_oCuuI76lBnMKN-|&8_AOlq*iiZ{c^XW3$d^Hih$uDR}|s0fx&wnBd|u zLUq7v72*#A2d#$?e*(#`pIFJ7VT&ld%r%SX>`_yzMBOsuOpi~kjbIr6axHi_ za}kQ`hldj`1p7D?cOcPpiYY=*jE}r$BZB4dn{iw2T3Z~V02mGgBfThbmlmmRt&k@Jg9VoBLLSzMw{D5^w1)Fx1ph@5vDMtsg)m= zsf&D+=#2u%{&JTyuy-RFN43s?u|DB0O@v6t$2J=CktqcksSm#zZj&m`hPaQPme@}22fXpCA=lnRRPR>SA}H}i>IA> ztm3eQ!%^zmOKFRU|Szf->*Al$qyv^@ODnLg+LetWjlCmN8xQOu5UfE{FUqU zlNkTkxW1HhwYk1(l5Qf1D_;NW(XPBGiYK@sD=-XQ|pKrFgtUdw?z9qpQjk*C$q%ofr+0l{E z2iPxIz!WDDIs}(?dZpV@Lftt`YI}xRQxDhoXvK8{`zW>T{r{0qG4#To3P4EQ5%iO; z37x)}n}G5k>DU_@5($B@_hzbHETo zTQT7RZHrxDqN~`!`kOoh;nugt}i2ec+si|{@f@IblmneT8u_3Gr`EzCm7(G-X zAEJIt$~CE?iVPO~4D3t~Sn!a67W_=lVx{IVZVEm!d9D$eFHSB@Za#mm(Q{2aaAbu1 zOch=CCjx}YloE_^vG`3tDiT#qLu@ZhiuO^Kk&!E9#f@;yk5_{8;SU!$3IF$ElBw&T z;fiqo?h(I-TIWTAe_qD96)Zx}s{ipNU!!zV0-cq7 zT4=sPpc(vp= z^f|RUdL+J>+h0}SHGFj*Vt@PY4q65t$xYO3H^mP)Air*7vSXXAHiexK*g z^xdr=*<;AtnP}56P`4Cj418p8Q<55|3+jt|g=8c8Vgs*a7=5+Hck7ML390%=mmWau z>vw5K+h{$nW|yvJk4`T%wkc0ftx83A@@hrg#9_k1h$b;PoJ&8+>NMhxse^c^Vdhx_ zLKPe!$X?X8s@_$0Y5A^Ly>4P|Su1QFsleijPJVPkm|Y$X@brS6w;X@%ja%6;OD7eZ zC<7bvLUBoxJj@rkFG2d&eP-Eo&C@-$?lhra?M_HG^4(h=@wGg{y&F+>(7L)ytAhs- zFU0PS$cxqVpm~ju9Pee944Yv$yj-buDk{20wxtUZd{mRR&Na^&xPLwkBD}$?b=GnQ z?Ha-panH8fh6#fRd5UoFQL>bPd{|j=vCWCB*h)*riFmLL{eu4mZ9~A^Z$rPN1e5I& zhnfBEI|qx`6fU+ z>lNkCR+4^F8ExfL*57q&h<3PebtNJgoN;%O=O^#W`m>w*42i_((uOlUj7n{$6JApv7iXNJiJ|?3+V*J_7yKd2J>PeHv{uvIIPU|k9^akm}58Td^8B&*L7tKXs=;k*Y&Ot z_sc^^u&=2Ztkf6Fm``3X9rz`_xEknYU?%p&h9P=zPlTTe$C)oihy!vTUmF&7k;L}| z&J&UG`bb-?^xAnXE-Zp$4B4?GV=AM2rk;!Fsk9wug)1yHz7L;Ad$Pf3)n?$G%&0_? zCj*>f+uu94bMe?~MvicwxOMEP!`E!_WQ8SkLHs#EFafo-Kn99GCkec_TMn19E3HYr zyo1rBae7~ub$Q`vnSTFDqy-O<`ebcePTp(XZGGUJm+UhXIvW50nfvF9D&OkHtu%i` zCTY^j6-bUKh__|RX5Jj(EBwT{C&}+~#MuYWNE_Kea&4_X^61(j9_6KCg2$owgKmBJ z9+A6OGmgHaupECG-s@->o$kMVcg69?Dw4iT$_u;ClL>~7n5n@D=MVvA3jF>7Fagl- z4-EACBL?u`d@1=}nu7Z)?K>5F1NyYHeni~lucr~15Hgc>1$tws0{x zZL)E^S8k!9x_g%%`fi77O^7F;hmRU##7^i9Fc|kT&MgWh;Ch2EziV z8E)RiypiAFQ>n=pIN4jx#~+NcP02i~3aQ~CCl`Xpfx|I*6Eeiz`;fA}haW3QJ4cre z)rtsV?8MBPV5-}g>lH;4Db70N5d;K%4-A4b5U`~`zBYsqMsmIm&TtvT3t3FtxegL$ zZG>C8m}8E(exx*dhB4}7tBEKf(n}X6lTQUPP)_w^7_Y<&zPaVpG3)gk8`c~$b$+S? zLs_<&LW8O>>2)(FSGI;z;S)lyi}B~jyFUioN8_LN`Ts#YUBaSKI7xcTsyzS_4_Hu@ zMf&=YG^n!tlOkCCGsxoEY<5;ajwl%ao9^AJ4N&RJJSY6LdJf7d0vv~>W4w4ELGYwe zjxq=rLb8RT6?ZR}tJ!|xxz^1uBm`#9hqvS3ycwG#dI-3_Dexb^JtRP#qARxHqVp9;47t5C4r-6ynzT zi5tdNB0vTS;ZmbVJucxWp9p|D1N&sd z`j{W*JM-5#uQzlKO2j~{rPnes+F;%he;xEj1yJxJIV$S-n_|*>C=bt74wl zKl^W5UK$JMGe`;hu$gP&89N*@SPzS5$2M3pmc9d*yH}RChq8hQ0WNjLd?}=P*NAxw zmAsukwxvC&D**6%bxgSlwPX6z@(DA?<vP+-ol>)lrBD>$BtjJB;p+H0>|lrSIMedGp)>_E@nkBPn;db6FBpH%MExB6 zo}JexK6B)AFi8kW3H%N$fJ=Snm?wwy*;c3zS023$66oH&59|85r(o5PgZxVKDYYEw zq%8S@3g1u(0GIdy=7Ts>dm6}KyY1&bi2Hod)MTfi?K+{mlvYUrKS$)XsyEKdY`D)U z4G$U;O=836RoT7{OKr}odPJC>VO12M-*+lXD(OU0iFIA-=c2p8Mf`y0j}9-y6PE2hIFe+kP8-d4|P#QB`j*O2I&3%`-tfhR@c9F_#lV@z5ODu&~tZdV^fA|ss= zJ5KB8wlJ`uofU>i6%ABc4f2k)wTxyq@>=OM60DBk2)JCj-MI{NL z30O6xbrG@}c&~5$y`fTBs3E?!eUg|(3ryg*uY)oJtm3RtW&oJ`%m6FjW(}K@+82snX{}T- zZDG%RAFiA5e8pRb14ydr*_Iwk>WS+eP(fe+?@1+l743G1xk;ytpuwSv(4wD2<~Us^ zM5)Uk9;<&N{>xoX-QLYf%tV~e=c0^|>R#d&$sfXe^ocfJ%a4mx17Bh38t9kpFaG;+ z?KfxPJfTy5Riat!ubzJo2rfiN@K(j6CFX9x2BnWEsPthHU;-SSKwuz8C)A{JR3{sz zNL?R9E3f)qW32wF-Rk!cVuV2ggpqX5YkWZLhUuQCFakw3C^rVOW7_oKMd9o2p7o2$ z9A7AEiS_YZd${tUJWX4}5VK(ywd^;7zue{I?X^2cd+IcRE)o<=Gr>Bas{QvfCOIKv z$293-eW)T9?vByxcqK@khjbJ?N&_qL~?DQ4kzON>rNj}2#0dkW1A73WyK z^D^kq3FXeKU0fTL!T7Cgb> zU>dAbi~CXz!-Nb7&miNjIFP0a_caZ*6FVtf&%ctdJCw^TOJDL!LAeez)z?YE)P(DS z1#o5T=6AMydtkYFUGfyZCj?ihzzR{#^6_I#8nKeNpl|sEt{CZpt%E3_iw7A&&MoTQ z@H_pX+YKde=7yR~tP^V|n$yh6_raPuQY$A2b(Hl*@<=nbA~M1~YT&{g@f`8Mg+T_o zFef}_1>0mv4Ml9PU`Y+W^?(2(FT#EU(s>(raVcy|+tQ!wPncjxmf&M&<W$X=Ji9Al`biA2aZVgxv$^HvmT9I=I4yIsP>m+StnSMEgso2cwfNgW zB%;n4`LHjxW-`JOP%SQjBmt_$K?b^jM3RwATbrFI5JCro2DrQ+n*czmQ!L0mJ7PG- zlAlk5=ouUV?xW{VAa#y+-U8*G14vVS47i?5kpAT^r+x2c4`Xocq?k4C<3zNN2{w%_ zKULgTc8qqxyL|rAea_7H!qVCfn->xxD**tavX#>1*FWr>C-tmXKM@9vkKpakEp9aI zvW7O)JZrn%4L^;p(9W1$kLf%ok0q`X{RN|(;v&FaHwWrGvulVaBUk(hwHkXtvb z;S;&*5w?@@lm0IYj|}wK-WgA6RyWPvN^aY}Ocu%~mS_y$7T?UzjLifSickdu69E&d zU;uMp!N4^0U0kduHxHC>n2&qAYy%3%*ISrW;uKOxZ^S7eqJ~VRU zlVeGol2z@wjCJEs8}oBtwa&OZk`joc0ityRcIwV8g}U(8c3GcRUa~4uWPUEJnB~Qt z`xAV$5GYFoD%VoBd>?^OC-iWuMX!Y(ZjgZz>+Py?QhQs0NVwghQ7{@xZf?*e6O%9WmutcJ*$Yi_ zYTA-<&&|Oxl%OHPv9|>|`-?FPYTtC2%X^K-enuR!RQwIzN-+U(iRE!E({FPthF5<` z8srna5F4UX@MiSOQ1+i#QQ;_gwdk<&#QXTiuDgSO^z2*skC=_U9=73RmJLn>hX_kx z3;O8R?FP1>9$=s?sL#zdd!pGxPP=+yDci}T#KwcK--2vow`Kp0+?EabfE$cDjCvz= ze~H(=N&-#IJ2-VRwzo_JJmRPjTnjV`2aR~TZ{z}>Z{l~TIYx2Ev%3#5P465^zF>~V zQvIT9JqJHhe60tyJe zc~*W%o>Mv${1c)X&(Y9^ur7*aLxud=c3E1uhXkzfTBKSeu)-k&t?)V|e)?**=c;MM zztSF2b;@Am%~L4#SY9)p$91n@Q$4iaG=5G@=q(o@asci$YQjxeKtc1NRANQBJ+4}( zK_KU5Xpb6J7`L4vc9>#w=M(99ENFbJFp#|?LPm>Bn&D}feYlGlCOJ;R1O_XKpaHyo zN7^B+uq|EVoC(+6){!$RkD)d8+{!owtjRAJ6TP<>uX)je5Yvf$@Pz8@kWZ$*%T*Kh z(?sRW*C!!-c42)_U;=W|Wn0A68_zW(z(DNVu(rh^GAwwkqALklE+kAYgFh!1kDZeT?|KM$Mg?n{cs4w+xx%Mr5AOtOg%+%tr{Rp9} zZnm||Wpkyn8e2Jnr{+#sf2&U;v2^J5r!4t$vSNIcS`nsCsGFvrQZ#4X-?kkGYg$w) z*E`SOwaXxM@`*4rKH6G&v}KyA0%wO{_c~?q!TQcjz0yX(mtTQJmVk`qg)EicZv#j?cEa{;P^35fg+p|1OCs(E zH+ns_zo3Uzt!?TjzQm`+iUsg(F)cOVtuKJmueSEjj|us+NuM}ho$zPt#x_noo_IXJ zr>LZH`bEX1$S>(rOEJqxq-iUgK7Gw9jjSs9U+!`S_ioBDvjG;C zY8BnO^uGs)|K(b6c=keuik0sJ6Ba8c;@NG+Rc?goxU1c^>SJfk=5wk(XY{30Wpf0{ zByHG!yJ6_B2BO4s4cIo*w;xwU{1W*>@p^&fEtmx#J`f00Tx7W|bYxj}n@lQ`O zv_!MhG-U)eagr3z&b3y&3q#oZy?03E_~)I?q4BSs%~$XH-cP`4BEgB0g(ae=F33l^ z!SzC+I>@#{p*k@4p*p7yvCXu;1~aPAk5ulJ-w*6z0R;nWeE8ON;Yh9h=gTI%Q$e}` zfCvR%{=;ARXN2}h_%gc===p_7dHm}zY_RuCjtjo)Y)Sh`Cj3A|xY5%VDNyR`#A{I; zZi*>ROd5Q)HKFFUmz`CW^raG|#zGS~5VW&6@dUSwSJ&?)e0#>n+YX-0>#rsF!4*(8 zmL~LC@nwND2{Uj!B2Xiufa3uf$nl6k4JB(fXB`m=Sy@@4@tiVCjpYbn3tQ2gGmQqd zVf5MnivWw4MZ|n~>h4qMo&|VWYnHrHTqP2Z#Q_lt?gsO_qcvMDC^Q_7tP24pM_3%^g$dsE)OH^~a{gld!D0!Q%su{7 zKY1OAcme@p_t8q&^yd0dovjm)L!o*o^?**wj876=0Roihfqf7Nt^hLkiJm~6=EX*s zROX&vSiThF8zPIwK!fKLN zg`aNoOK)NWDK&6@B~hk$deXDc2f&)a0Sfb6p7|PbQ-|zaibRYBgv7k;92?ZO#U6h} z{jPabV+et(Y)v)qn;SkAF2wF5icjN7Xi4V>q#JijbBjxczne2q4~KcKxBOGA&-C47jiAR)A8m?O&=GhDE@%Ewrd+FEJT*!oKG){Ae_|0M`{DQbjZPa zHbORn)-zbi)XUdbJ65$caFwC!9ahPadVF8m>!WuR`6XX-*d+&>yL}2bu zQZw+=*Fb3q8{B!7qq^P&JbT=v7sSYH;#$i^O8f4S1TYU^hScD&x??!d9v6c8{Ow!w z#ZB-6CO0FrrM18Z8Or7+_DcmFIi40oHKD|3&#}f^(DMXGID_YRzg|1i;&h{T$JBj{ zA$&1vReDFbxG&=@`YD@e?y7el4dEugqmW>~Bg;8ABxk%ZT4Ynf9NBo)U*ZoR!qcQY zI&Fotlze5Xm(V?!M)4p`XS0FjCR0mqA>Hekv5M)#_u+$Ak0^CSgPIpxs>*#zzfk-; z&Kn|TFkGTU5VB~vkL+}3)fw@{=NwEtJj?Am3_s9&gqM(~XwLr5ar4#dC{ZCqtOZUi z?J5~C@A32Z*G_^}kxI>|W@^=!6Og$t(JalavND7+_s>D*eitg#FO^4C^F;d9 zDi(mueM|=^)Cas5s>D-fs9&Xnw)@2}wedY$-G*sguK=ZG`xLJ8a4tvwi zi78{3vNGoK@H4?bi8c;09!>HV$~A=3VlJ7m!@Gg!?CtiZN&*=P(!IZApRZh7Xb!99 z#P(S-s&;|6ai#`&=3%xb#usAa2Ihzi3h2p~@!|1uTJ9qjGA|q4idzMX>Rf~(VlO$Q zB3OW0yUaofDmPyp$&;e1lNKgjjJBVG5GM7{)R(M0&xMb-tE!?vGkr@lbE&)__RB*z z&EY>6G4!SXbjOSB^j~+*^zrG>McU<|{mPqv-pPkG%!om6REd>yhKUa za_^X26&H}4sGckbx;sy|*Y+J^U+MU^Pnx}<*;*Rq`Q%0Od500r&q)ZQiE-{kNdW^> z>SNDaltuH#2|D741}CCJ1WTyknFW&t6+D5tFF0o%WPjs(qfp#Y(BhDmCZa%eL-E;+ z=g#B9W2kmEj{}wM_%`P?^1h-tZ=Ssnk24J=17acx@Bk^Lb9oYB@>%Bzk^P32xCRS}Y z#ydplod6yi*j=Hx5}5nAlCzYq?y!BhQW0C|2gBnP_9#2fS`4ILAz8We8f*Q0L=(8e zw!vyKjT5*KCiT?%hj=)o@tX5vG&RI=5)vBqm!ArHR`C9&fbMcC_inDy(~&;Ltxpt} z?0aLb)nUbLc{aaF6v&tEd8Hct9_h5>XzlU0d#quBkiz~IW1alh@6(wtbM!|4fcP6L zx=}uA^iNL?(xMTb0WGmn!!hVtsDXlFn{FHE%L8-&& z+o)Nb?HR+L!`$+oCGpXUt;+K(k{JfHre99{yWk6hX z^Ji9UHCReeAIS{-4Ae&g4Ae)$F(W0KuJ6K_Soox%rad|0&2s-@;4dQO^w3`c0M%rQ z?7c2wh4_w6v|XphDd2cgi&6plq}hBA!ziNOslNA(d~+ut_f2GEv(xJ)iRRslIFbzH zd*sAkHC=a((Hikx=qGcMa6R?tiz}i+&A4^3Md~Cj<$So^7Piq!asCVD6athd;{7k; z|J;fFnfr6qbv5|sF8=OaUKZ}MzS=oG2V|vZbT|v8*B0waR~;1 zl}w5|t5P=Ulmhp8h$W<`o7nt+*Bu$s`^v!3LOaTDy5~@v2SYrtd^$U1ydpWCJ>4X~-aAI&`6kDlQCI2uZ zIT2z~>aEKR)7m8~fdv>h#Y@v-Xe@XYx<*WC;j-QwW4wHaKzl+4+tTc}~E?tyy!(gzUdD5ACL z{4$&?#mO9ZDeEZav&4!QJR;pbX8HRENAvnuUQxQ>8h@g+`3-qO+NXx;jpc0TH6s{8 z1UG||^mX&`{l8aO&mEW}P3kD4@7h_!;Ey_q{}Kse|Pw~avyF(TL@h?FEjf^FzX$e_j`BXHSPYp<*R15HyJDR&{; zTu+L$IVd$#4}I{C!*!fH|2W@*(!`2=naGhn^N?+TR`F_)_YTCQ(eJ~g?6!D)*GU+7 zzgsKA+g!h#cJNcJ+ z_b?5qu5UPKSHrU9r;_O&4LjpBqAW{;t~370XZ({SCQ^d2DlI?KIBr2(A9vYSixl;{ z(Y#Go?@t#{TVL~)w?Am>V^1C#FX1kR+WLg1D9BMlkRU+jC8UH`+4ee9k@%T|1j)KU zu7vNAw|Lg%Yo+B<1b*f={GC%5A-)sEqd}pm?rP-sdqPRJ-`%y()%5E;ngrGwcAt3? z4*5wqFxM#6P>rrDc*YG5r$5chVmI7LPwzPRF!ln$&7dkNb18CYF(wNPOkQREJ207; zkodN!e};f2PH*Y8(qO#)Dk||qc!Sr?W*=Yf({0h6C>&l;cP+?pGw<{s?d;wB>Ckao zI&{$^Sj*<`Wf30L-jbS$O1oNA;JZ++W+Y6HrSQV1YudD4)Yfi5+e!3j&>{3*ZEYM@ za1^~-{*f1?iy7O?RI4=SM9#4EiO23E?I_K#s6UX1`0v1^BQ!91P$?u>TBE$&$P5ik z9#?^ZNf!S93`|~&J{v(0nW8BWO>0ZLg?XxMVQ@mXhHp63wYzw;z@v;pdsUw9pk8vv!eG!pJ2tcY&^~V#3Vr(A~9cUc%h)`p2Wu(=n@#Yr~(vXy%u8 zth<~?E9~JP?dc2by3H;um^^8JQWNW+jz+;JWayLnyM&#v&_ng}m%#{?mUh%1K`PO_ zp83-hMboSo!`-{h3L5Y^E8+q$DzJe5w>oUQGbLw)U&^d9)0Fu*C|0|4Jw3C_wB~pm zqRR;O$y390*wkkh%fM767HafE8yzWVsJ5vCKoxp|+bd-|$@;labCh3)FD(LK4=oL6fe6O|7( zS6_cp)qR8S&xEQv+@f-Q8N6NTS^pJ5?>y3T>$Etp?#ZL<<$(|-gM@v?Ppezat5jm{ zPtGb{&zOCFu^*9oWl%w@W#MciWOTOBZtgHT>0Nw-vCwVm&~7}(k?p;Ne}T^6WZ6!~ zZMRoHj~64$^HbYsAa|!v`t{eZ)dHc9R);x7N7YxB*t?$dl8FFr<;i_?&_>ydpW0E_ za8G_Qc;FeE^#;%1*MoiYb;x*Q%Zq5-E93=|(Jj4PN6n+bR!1V?_<^g%SeN;)r%BX; zq=}&4_!|ga@I1uu1-^4Y0Lh8nqbjpDHg-KN;a^CE?AV+sKfm2@rU1Z8vsN%vn<6@< z?E^d$i8w0;IrpjBVVl3;2Q)zRnXURXCam4?!w%D1%5v>m12F~WA zO0+7&UCzV(8)jSrCq}nY{EcY?@;7GWYG2<1Ps*n9f{l~ z=xRq3uUIF6d_v?fh;Ra44}793MUidY3ROdvXyZYB?LKo&&0v!}qtfQoi;ddiKUNWB z`i&?x=B^~5`{iGLxd@dX*|4_Ch3>RBZy^=nsu zfV&6CPxWS^*SutCw|V2lroz8-z2$IH^OWstI>%V9H-DESd5~f5?~|6zi$5Eh z6s9q#$nTLEvMlRO#})O82+gh)Az_ zpP?wq4%e@0^!whCf^8AL*+eMABt8J}OWM zC^!06{%vzK=|9eTOWSsz^#H6I?7Eo8^91zDh1MvyphqAjW{hft3Q}T5 zz(6T66I2b{1))bwYT|aIk@Xq{!FVx6*(ReXwt}$7`3w|%n%?d>4?TGa*VkV5_gZU& zdISKla6?m?a7DlnspC>&ho1oIk*dPi+4bNJbYgov9Z5uc0rtm4#jEi4F!#KD=W#h0?vQJ&_C0|gS0V&CfMfGpMr^Dl<2R_a z@)GV6YOMt3zP0kQP)Re@Lq=P5qK9Q*61#_`8!+S1Uy{Y@f#}hX*-=QO`S)ZWow8!P z!&u@6$w2u{%}yt*NbWm!8W;pv!0{Zb_U%t<(s+34jQ%!y7w-Szas*Q4c6ePP!$eTP zL_l?jbc{^RHilv@YCkd^wIq4Lx}Ua7ErJ)N#Md^k(sy_>jmp6u)hBnH5jI#RSfns; z0F$>cg+Yqu?~6op^mEy_wO19~X1#>dQNoq-x29`;)A(Ove{saLq@N%%Ox2uOEYwk2 zvyxiQ{IiAm@JR2@@uKLqhx+`cW_uOMP$4OYu4$N?v=QY}5x5|oQ_nipZDJhyZq0f) zRBDbikd3Zd_-XVae$8B1kH4in8)Uf}7$%VuHgdN9T}D*7uU7VCI!@s?ikR7s;#}jF zcZHH6)Nc5rS6ERWO7x4T)dm4544q1ZU(wNSEbt|9P52DE`E!ukt%B2J06TU26(Pq_a|H_UeZPw9qjNV;M3ztRmU z6&1u*=DUBT8$5sfBi%6mwB=^!CQWLC6P0z^RWDq|&oYZ|Ezul%fOU=6Z?9&v=dW7T z38Yq);5F+hD|W2w;1N4oNngOmw`3Bpc{A zAjt+DUq)~Jl*Uhs`?78$)l*Vi_<0SOc^Wh;v))EiHsz<){XTBK2Be+rgd(=T-dr-@ zDd@3FALr+U$0sh>yJIhfE)cNX7&VJpG;8lAVK3RaV-NeP+l`%Lfge9j(fK6)Z(9GJesT$%-yD1YFit`cD>ZRYLnCdF{%bQ?!)kHYKbW z*ePvPYBk8Mg_6B%vj(k_tI|V(W^B;eY}sq=xTOMbNlhuu)wpsQOt!w_H*{}gL8$8qhYBoq_sCBLT1K~XQ8yG7k%f>gStzKGiF$3#258M_%f zAKSr7qiT~^v@)7pc^PKCm2pM>Lp4MIJ%f$D0eep( z;^jh7xIa+t>4&O~Y;Lu)kUxg)-x z-;j!Rvt?!pb_;Mj5Elj|{J$ra?5SH0L#SJv{SdX12$|)26i2Kih80H^x(`F}m=NzN z&Pz7;HB7;gp*S^Ny$I&a4^p*XjO!2WWm_`id$HIQeA&E~ky{j_K4#_y#L#W?@$yr) zkr5S%1r$A*rr%|1wdt<8{8j*xoZig1@mE(odM3R40ow>`lp>E3|7n9$1U^CnP6Fg3 zKnC&=5^z8p9KGxaP(mZ14Os{~#De4E&ia80^o-%S3K?YiW_{doXg!q)O*Sq2<+(N` zW5fXw!`m}WxFMxv)!#%t1pAs1%R&zdDW zO}R<=TrXO)7=8+qr@u5KoXoyq64YO}fh!jTCe48>7YGdG%7u6yBuqCC$!WJt%-Q0Y z+=OjuxM9Iw5qZraUcmrVX(_Y!K2lLqQ3@x-gXzorlUT`zQ}Zgn>kiBCnB)zt$Pf`9 zCEtyFp{yYD-~S#M8gS1qm*$G25y@>U#LX4a@#IzhxHkF>TXhOAB?@u<7ZfsV96yC$ z%7Z}$oN1~Vni1BJk#y4Sy;Y82@Yg(IOfQGJwTDQlIo9g~{qt^1p|Q(!lrJJFb58^D z-@)TxfzHuosed|0FQ8JOXtyX-3Iyi96etQhN0k@rv|gv2Js>YMCHvs=;d?1axiVoU zpd>^lDV2Uva;L%dRK_z~pC6L0h{5EB1%T?uyP!8Bh5MX{sEIGukd&#leiV#_2=}U! zgkx605W^5B48v5;`!C?%KyJhQ&Wh-&@iJy4y^HcH2I4Up&g5pbUo~;K)6iV?{M<7M z6c_YCF8x`2ZWooV2Ic{JwPlwBsw-HB;SRoCe0!)(6l= zUcAilck=bja*I+$`cwXZ1g`jZ~d}-GR?7B>D8HIm}9X2?!}G|--ncqI*O4tr|V}SlR7s|TB+}5bT)U= zkMhL+j+MG^EP(m*|>{y+7UgelGh^(q-@NyZ5pZ?_vApVRyxzd%3V zyJ(qb-{%Gv%c;eM0&{ONOhGXEKw?~oz8`Rm>{cD|~Ycfrd0;mQ?1 zYe~PjN(_k)VTuoFMZal!)?#09I%-o_eNZ zyHU~nb-n2$o@3oDg~l5hOIZd5U>kiq1*A~B65@H24K*p# zEEXPW2{!W$GpAo15n!t_`lX_dN}rc+(r>{ljDE?jeBzp~Yf;7UjMNWq9v0>EG1D7b zH*pg2K=IX!ftPeVv}?&3nQ>d!&9^LQCh5^bnBoI-@W#gGt4{L%rlIr7-gMH@(jPl=cwy&6L*PpBl<+4@XyEqQ8#dzk!q2V zu)OIo(kPAN>BKXkT8en49qwXDc3BF`>(F%5KT))cI9vzd@p<59~=JWds&f4 zKm`Wpy@eu&3Ce6PL~EM`$G7$KqTa9PyZeq9%h9*5vMV`K{N*lZV(&)i;nT}%JMQSk zI;vb^?&(vzAHy-LcnpMTL8nxGbr{Jy=>JR@(6S8vmSI%^S}KnZGshmARVGt5qYE^< zUc)uRgFe)qfBR5fK+*Ll*-tW1bcGB^HToWtp(@x;JAyt`A@o6yb|!RWKzP;~C?|xj z1+2}lI9AH)5Me@aEhJRCwZ|$C+BcKPd~!tBDoW~iz{I$3KK*a|X#t%{+|J8Hg;3M? zA~enT_1!IBsZM;JgJQz;yRcV66#stQE6V+y(q7x>^lqrFOQxfq zfob6P0BQI&%J)C0PUCnl6x`CMiL-~xlJN-?1-*clr+jBgM2SD32#!A*P!KV zSci?&hY{3`98;P%Od&xPs-z}(NF z=9;!NHa9(#)9#)i?Jzx~JALu&qbK@@i;te&fKtJDWkeH!hXV2Be49*Rdf*7awIGyy zXhtcVqij4K*Rd&pgs84?#jEDeI{eEWg)0BSJ@R7jK3sRI(rxFyx{4Vz5D#?-<0%n| z3E^ORXdCW=&twAL>w(e$Bl8z9BI$FZ>P_b#byCjL>opCF@c>SgYK1!dUg0~?9o#qH z$iI<;`vx*lih7M4D16iNk3J19vami?wz#lB z$|9y{T+%`u@Ol(|wvf{d$t>*sm}M0%bV}SJVz~Lr<^n^b`AO!-LyZ&ejr>r#IvZK~ zvQ_Gre@9M@sP1yU?cH!=jq<(W;jPpEQATHt4o^V~oNAc1f(!K@}b0iaD zhR5Ic@-Tw~0|LR$OElC?EKgtM1*ydAt4#P+eMlW1V0CcPJAYX81nb6__Mu?D#m1vW zlp&~U15#N~)draRs?CY2f=PC}T_r_jR$RSELGSCBqUdUV^3)nvqCx1z43o0B*@~sY z1(6-ns60-MmW4#k0Spj*n_)f)d4{*uLny*;LZ*&Ud1yF3#|F!$wAJbOC}K>mkr+KN zdhmrfh}H831oo8ofxS#&LYPE1D+is0JbZY6j^~U$Q4nJd#cx=@d}KeWtkZZz(f97- zgdYj*<;j1HXt|h`Zy?V&(~xkkbCXn`!Jj?P$h9J#Igc!-q^dSnr#+(n2@kPx3eT|< zz4dVMxhF5y&UE9bQO|mS8O+*~TAWgucAfkT{m<1u#c@nyW~k7($vPlnJHfX#+#VSJSE+qtTn8 z3fS8s{_JNR}${QGiQTN1}vaAZagG50~!=|X{v4~n3J(WX~l}d4<~ye zc^DWEph=D4VV|MS_erJr_FcQ)%AD(FDeasontR>+zhsouKrVL(Eg7!<3gf2ip6vOZ=FxI@v@d($VueV?66r3 zT$|0FK=D1<>A#f*zMbHoe=7~u|FhD7nsb%)l#f^CZ>1qi$tnHa>}Es#m%n1vx&INP z&R@E7aeeC7|Nn{axd}o*Ju)`$n#HsBK>O*TLhjDWWjjZyu4|O4;dHANuhZeiZlS(Y zE~%DJ)YDP{?GJK^)^~pFHJJ5PO&*<=JNmoWqtCByR`+Th{dk{pV0;=K9(`fMPIwZN zcX@^}7j>YapBf5DPrng)!~>+K-~Z|Q;Ii*TI(%~bf_l$*5<^~Pg`-UcmVHUoriG)e z)0V#$kuJ!L^Le!rSJl(D20cIRv5tee`S1lGY)grBQ#*b}zr1Qe>w%%3f|sPur*QX* zGq|2fR(*$TpzY3Mh)qVR0tk!#C|R%HevUJ9K(9mB(&M9e`cOOTs~n$ed#nv1AFo4K zu(BOjc2*e6qxAQ7%g;B|UrdmEQ|~DbdRknx#XYEl-n6@~xz`v|E~|SUNtyd#hkc{M zh6-XtD0oyxnhZFoamU{@J5fXtqxFZaQgO zDBJzh?NL{oWnYsHA1LSse~OTMrU*Z*>egfOUa8qs$e^Vn`fS&uU${6D8?o8=+Twds zfb{=U*jGnI-FW=u8H*yUgXYb0D&}}yyqUSPBN1<9qSI5Re8KhrSJ)^$0~`$-piQN zAKMQUngPRs5#Ue|u;qnPzxQ@yM_D1}okW(FwWS{-1XF}pPQnFEsq=|77@(p^d+7&Q zx79FV1n*15I}E*|!WO53iu=9@{;0YQ)et>pA-Nmwqu6CLDb#p~z3>(6iRr&}8tY)m zCVmUUKTlt;W3zmLyO|`7k~XU<9M-Wh4=<$(lLE%@ZBWCc5_Du>Qqah&>1w@@$YX-F zMID^NN-k|vnUv7d?X*S-Sd9BxRE`eLrWGUt3mgX~HX|8Y)=XbR9U2*{fpR4f3^DqX z8l%IApV7*zmv>H?W+qp#08=5po8fwe1|%Y z1MGu&EwrZ?nA0W#MX|VWGGQ4#)1hO9^KzzMb6a%X|SNgYtu%ns|aC$1j`)6x6q`=BWb< zWWl?eox3Cn1X%;un*EsMtFx>PNb4gR(~vb{!AdI=CBzMk=am{@F_EAv$vY|bp8^_r zrNL#cDpIZ_M(cE!y%cNfhKVbQzFd2Jntr+c8%lWZ3IItr3musG+Z-v+-M|ute}Vo-`s35BQHd+EaLNu zx^-)}Q%+%Vd$S_hO2i><`}V%nFdwf?b1!k_qM8Tn71RN=N@e)_{(Vs3^&gH}(wg5| z!ta0W>npU73L3N(`X`d$FQqB%rwvlOj};}h_F-nv5D^6RvUO50HP9h+?;tsS}o215(j==AgU7tapo~^xzH+w zi<-r^yL9(X+zJuF+`}SyYJw0|_Eg;o3=;|ZOO#_hr%bC`X>TnK51cooo|9=mv>7Mh zS$7={^)1WB8WmbMFnk653OYBASS{0@GALD5KeV*)-u-F4)wn3rZpXiHYlIoIvL*+l z-3(JO2I)>8T^&U~h_Yw$dMBFe#EQ*zsGjrMt9|e`*HO<}W}p3%@{SL$N-3MxWsm4P zyx(_tSSFMUY!;&9MP{a(ia3t*MrP9$Bw0b3ewT|vr&M+T&wKsaB}%%^n|DC485!N2 z%v5RSbQUZD7oef#`3 zJ9mQBu2gum@7ssp0bkFeNhH2pHdJ=4-1^MQOf2clgp9SQ`RY8x=LdR^#qDO45GPS) z8`P6^-Fahk(2q_U{-84KvOY9d-_s~#tw1I7H*_^CJNjIOh4SMz^>4zS+6y`0maPW3W2YOh@k+BxDDd6=^iw<_4PDq5&LhYEFDWkekR}3K+)#_HMb3r7%0nv0 zd$XEM)MbLsFB#->o-$e!2**sr?$a3@grXR%c-r(62qndn^_#lO3VTJ5)SFzG0dT^_IyVHWLQq}mB7o2JOFYG7Fq~EVG&ZE?S;q<%m z5V0sYtl?=wm-EeceWfN0?iOf3uembY|OIlsjsXz*Icv zBK{rZyN}@ogE$;o6*4#@pAfLO!aE}mSJOBnZ(#ZYV@EeJn=!19Kp|>1#H{cr*>)HP zH)0@cU$d4;P+!pFK_rpAUmkAr||__@(m0GjC5?QN9u?ndryRP}e@EA6|ezTJ>S z5G7bNB%wLVWU1$+OOD4YJiKQXF%Sl@hyiJmH7AC4++1rEdN~X7tgG)C31(v`|8fIe zwBR$MBIN7}naTsTw=M3bkpsW=E0MELWGfL6ipVPY=U0s2|6v~}R7vL1a+@5ZO{NyY z>W(8}`t$*ayJB@gN&j`oq~{^4BgjdKX76y4u&5|S zc#qPlD1ij#USR_XBt1~SAOi^tI(-5O1H$Vp30@dD2ml76Wr{OKVK(57P709lzO~lC z8%M_{GmJPSxn39A>QL-s{JT;Ufn4r^FT{*nJ+E#uMbZD& z_9=uB&$|yRd#EGX5-gjSMs+atq?46MV+4IH(Y_@8eGxCks|77nqki9D7NSI?( zs?Zqj5Xe|gY=XcT8HNdk9Q9(ZFmpaf7skUHW}tsx{SZx-Bl&)16q2))EH2#8Zd!(u zXsRSfGUmj<0HpwlB;a0wpyv+YI8J)X3%x&A^YQ)*&b&V~9>p;NL^K~qiU#JQkHSmm zghh+n5$@kEs_|gx5yGY?c0Pp;hV}wCco@%f;DcnB!063S%p6fuglZ}un-p#}Bh?m0L80vtLw2Uz zK*RFzhNcejPou&zfP8K!qQNp^cIK3G#hw}q8hgV6HM9`~qMd#r<-4j1g_d^vVi`BzVl=0vk?Ra+cb5ib2u?s$sS4W+|pbhuxeCh4vyTwQt>SYJ^pNGT$2@&M_T+A{b{9`ygz#4G#MJQ1|CU zQ-j)*Y(MT~Lav`p8Z#ihNj;F5K*rE3uy`r$$3}Py0Pi|IbIQEgnMBwmN zw$g_TUh^_lYg}iGeyd9_1hb1p>NY*bpMESq>&#|a6;(vIuMh;AfUOkr4Nef>Y(`MA z!i1GnKA$k#Ls358FWNPuJip-^7wCGuzqDP`1!dh$`ik`lex+>{uJ}{9H%rsO+JT>|PrEghSST~=P;zpX- zB!&BaGBs-UUc0ZpcPP@WtW{r1Do}EtbB`#sBb`l)=FW)#cxNPt0|{e|1y{47;#Rn)4qq+SemJ&j#%U`{ z|NhC~`&lA8fft-;#)L3dcX}+c<8fB)0W8T6hXcH!Nrw>>*7NT|VNuov_nwz}`<062 z5uY)Xm4-l#AWQIsHIjt z83m_?T!1MKIZ?7izL$j5SX9wobcxSB!b;S4!fG~GbmiU0oAjB%9{cHzn2@DQjTsT$%+GQ~03hvhi=Bw-4X}paDVBY}X zE&;%S4NMs183$mHRfI-Gc!n|DL1Ui{Y2RtKGIh||e)}Zc8E69xlfpb()`UX>BC|*z z;Z+9K? z^$HOU4|jm%1ZP98d_4MA@^m9)4K)hkxc6@iKQEgW{69}48z;I?8P%Qma=*`0BFY#O z>MN?aLw_uhN;rnXgd$y>m$H0S_ECxD{KC=V;E9L1s!)iLITI*={~gYR56O19E7;5! zp`?xKDa7J@f=(IYd{A4^LXs38FbXyph?Xf904P(#C?FN*+b?I5H4yVJJ`bD3mNmpZ zh7Y42IAYlWx5K$3?!TBOIGK<^?wBQT(MT5%;d3Xo2YL}5Zx8r4fdBB)>}xk(335i>uAwW=@LZ@5*(}@cX0aE3>0~)M}Bw0 zu1T|k0jX$n0GuHGWYkmmT1uJBmOc~A32%(#JBE((veuRO^7>@qAx8g4+Ns1)chK2y zNAcUpG9=~(B3oukIQoZHY<~^o;QimHT0f9K_L>n>w3P&-xx)z*13Tr%85M!zqBvk< z?VS$Gd)4|a5v^I+iI6ome0M2dTX+3zb@1hQk8npqto!E@j`5ne^vKBqmuPXeSjBLS z@RrCEm=o=*6A7A_Oo%6G*S+9^+`wf<%Ti;@u`W)Td^~;ygd;huO0)Z3@n+;f{6}^S z=>f;#qXW9ExS&ZTs;|DeLtaFWuh#=t-w#AzJ>@$3Ya${ojZ# z@;~rJzWo03*}+AmcBpVwq||ZQ7eB1KP_Wso(TKdfvLbn!lXslET(4uns4+FmVls_E zr>^6odkTl2OSSxl@0|xenV2f6*PS3syXYO^8;;nX^Es@b=oJ;=T&AFIL;oT`Vq*m5 z4dkb}v7htGsGg4RRId=M!Z(ZZMG>W%$Cf*>Od0aolTBsLBopY#w$HXRhn~fARD8ZH z>hGvLcL~LLTWurU)=}YIbC-s|*BDJWy2hD5`ykbf+?xnXbRS4m!;O%}fXDroT#IOr zxH@mW!zsmsj(huKEK%M7`9Xf^VdSkPYY*+GJxeR~*WI%(JAG&Bz@0^`d^v`x3d>1C zUuHS3E|SAfZi)>kbVg9wgd3)}^6Ei!Aapp4;T~5#`j%oX#$T?q$4aMK zy{w=xF(F!2CfSgPM*Dh6C)tImnBe3wlA%q$``(SpB1figkMjjyV$=7jmu$~|an~PU z{y5+lGyU>S{f}SVcn2sw0V}TJu8~-|fBj;s&f=4py8kFWdb5|&m5^2cs-I^aEThW^ zdYgkCF>L1H9dtt28nsA|v2u58Z_PjG>0A58hv?pJs}vXWCMhovpu&&xTqChCelob9 zeZ#Ve8Zh{VdGsN$xuN9DQ218_SKAspUac=`eOZNci^p1Xg)QD1zxKpV=J8We5<_zDH8ko4cy3Z!lHo#Ge~2abp9ATsQ>2@z>2u0RU94 z8C=3}2SH98ziKPC6v&06Ll3%LojBGt?*><6{bI%-S0{tvWr*y^`k1Zc5QT`(b``G= z_+F{rPufN2=H28-7)ok#aM08l8h3Ga6AQ};r7rh&*NQB9aHzc;FRZb^9lwUd&8(6A zoTR~VW$%VDrOs+{Ff{FXXBS83gBdUPXc6>67}-1FPk6{rKs%gAG$?;$5-1rv(j$C{ zc-&pevkSqXj|L-}M_|r^budSIF3FL-sw<4b^I$Gb?^`znU@qj1F)$bC_I{yGOW1Z2 zhItiwB-AEqPIFvi5QGls3w(5BVoXkQS10Rr&6Bn0&5_DBV}XSDO>J8cbQ9DMjtEVl zm0bMt5HWv;>ty@&Ff!MYKL%CAVxOIX5=sU%8#{u&TU+f3E)#usc2MMFlbe(Lae-2( zF3$TBC6nZC;1d2l(syT8ukxI9W(-GSsRhzQq~{$R#P9pUQ8?)DzP^vXCi90&8*;U= zNH;LC0Bs$(&5LM$BlG}Q?1cz;-n!NqYzvwPxC?k5R6YZ67u524Cg=y>H3yv(V$k${ zq|YmO(q~*r1H-`~3~+<%E9~3xz{wwEB*e+tz{wqK*w|E5;^NaY+1k#J_j!xC3=+cT z;jMw#SQf0EhU|0)brCusN{}eXU)&0}thd`AN_m^`t1{vZ+uZwTt3T-#LJ1W}?>3Hl z+lf3-n(9QZ(1&5Mf-qz#7Kjq?h|OgQ{)U_%_?N~pj2#_3#3)dcBOlK-5r(J?f?tD2 zdhR8HiT@LF*n;6%oV*&*j;$+uoh)pvd)}fDS-lgA}o(678&QG62CEWItqL4}g z)ToMSj;nEb=v!i`_P!J#oBQGhW!A&^N`m~H{XG0|Y5x$fV{o*IZYlVpyjh{dZpEUB z80HYRRR|^AN03$mXc0|muC$U2iIW)x5|dLXSH{e-rq3%>B69A6M&vOxZqyHhaTiY) z+XodFDM}+2Aqv8b-6LPSnM6gkTE$d@riKwa1lL7$(wP+}#p>g_CET}?EqX1)wrLDl z4n8&JGo^ORf4oC{)fTB7(g?$OV;uOwtM}#kJPZw60&D z$4b`)UR4P;Q=*&SYdyQ2BT~n4fZLfedi(!zJ3lO0>HIYLSwkHAjoqe_$KdfKeb9D7 z)nH&w(b9gPfp0EW-?QihTiFYR%is`C%-uoG4b5!Z$7)c1O#ez@_pfo!ak}5m#C0If z)8C&(T`j8~6K>~&9_QR-hjLLxZ`QqI2}GUzF<1SbcGZuP~sHxclG|; zVsmTWd-$mn%NNAW6o`dR?aV5)bA&yH_7JzVwkctW=COet6rGcoG?d~q+|r*fht?z386 zyj@ovQs5kkGoO&;^5jGt`+Iggb9X;{PMP{6vs$pj)Ni1R12Pb+&@edP@=4B#v^n>| zDVbnZrTts`Y#4yJ2akO$1gDiq6Y(k&=_hN;(ZRY>mw#I!yBoR<2137q0X*^IMl=vbfMqO1GJgfpzu^)2af z7Dy$kWVGT#y5KSWanpDy7&ADlt{Q*r@=kWD7-(7re z$%l}v3;zbw2ub!f^9afGW#hoDnbg@uZhIrnZuYki#68O=W7Yu=TZ3$;6}fqW$)AMY z6;u{d5+4Ui^*82DYeKfNsOc(?($pfhGgctf6zt) z`?-t9Q5K0(^#`3w5wvwEbvN+ zhF|V|fT<+PO%y<=wK>EosAY{(Fc~S-gozVSs*7@0_Kl6IY0$|@iAk^FZ=M_oFWR$Z zmD3ehYE&J@KfsvX7kDCSY)7!<5#E8T;k0FX>SOd_E+w4-cba!|0v%@ctCZ-q(2g75gaxeo<*(luDC$)U%4~f~(=n;Tnh(crtW86aM@u54ViVVG z;&}-8WY-xjj>O+mFzr!oH&-jlG|bu;owm}#uOwQ^XI&{F0=_` z1D@SVOXRgoR{usTneh|qjkxn2QX2AlY+e;G7?__81_EQ|XM=$mrumP-pg9k}$ShA? zhr2~~IELE~qLLAtz@b7Ee*wiml6bMg%=R2zPz`EqQvCbs!wIU4b$fbVxxXr0*2rOC zt}K1b(s){-w-zRd4)g|;24bA~8s}EW!MKS_igVM`ZIRPWslkA*Kgp*pU$X*6^(Rpa z$?Bcvf#{K@{VV#QxX&xE@Y4i-Ivr!CwY0pqpVC^NHX6FZ^A$j*rKdW>EW(@cD9yw3 zg%g}H+=R;o3AzBqPC>AH(Q~oW-PB@7mr_>UpOG8i;@SihQAFvvGNFbdcEtG|;rXW? zPwFduaiEeU`3{U&;@p%Z`KuYk1kwPNh|4KUZ0PD1-XWmcQ&+WJ!|O2?{~&;4x(nx{ zg=_!2|A*h>anAdAi^jv}e<(eqb|<}ml%B_9erMnnIpm4Z{h>e8f5e1Zv@7lw$zuT; z&Z=9!Qdsq^rj#tNMkd%{dMF{t1J$6Vy3md!-G11}Eh78XypsFF%(>ifPc7sL;2@+8 z-j+kd%JpZGJ`#I9anTon+$uCAgoO?8iR@+D(6;g&1>)9_elU1?*lQ1w`4D?t|H`$! zHD~Ss-cvFEOyF^+fAr)!MEvi4OXX?k|!hhbW29@g>F zW;5@JzkT1Az0-BLQn)YuHQL0MIXZSlh1|Xx4fH&xjRgM`)qGg87igdb#;q;?6GcroH9%QUEThB?$e?cc9@gDLVP+A{<_xZ zC_6)?{D09X;&ag077G zMFN%?2kYsY37t22{`ILTt=R$ZR5Q%8W@$NuSQ@x3n%e3w0WkXN6&XHm`_0iZTF1Je z=rl4$kSK)8$ zkCv}+(3Ud7Llyey%d@O(3HbJOzNKJ>KU7J0ZK*r{T9|N2vxv7Zt_{o!sNWPiFp=mc z4bA%lpm%(UVdUr1Xp$v0 zS7&f==QX7EC88ZnO2I2^eVaFXw!rxZu`SLj=8Sufa@3xj5x`oCIxJi3oKA zhTt{TXAqq~3mrMJnlbyU>$+(?P&i;%Lg7H+fME$j1ig4REI}{yWEvHfBy~O!vnHZ! zBYR8^-aq{@Ya+&pz%emq*jbKN*4oHjj;NN{uq&39qD$rE`_`1hoobs8r%;2bA`-vuQ{|U84FiB!HP6b!Jp>^Br?0^pgfPH zVXormC>13=+KI!`Ot3$2dgKDU^OC@s`G?gKKiclpJY{6qT7xw+B;^!3AcW$^`7`Se z;8$`!`<0pSoX>tGC$jOgU#VxD@S*Ssw@sap3~78PV%J;h?E)l;tgBuviHwS0+)+M- zj4Y<@IXV`1++io$K^nnGB6FW7{#-N6n?03e%-wKd%#j(#?aq-E$8BmvM+@Bk4pf^! zFl(zAKuW?;cx?{~kFp8*;8kCm?BoQW+6^?4rkLS2^Z)VXpmaY#l8~dRNBRy zz(|dJ_>BubI}tuIBc>X;dy_JD2jQ<}$HK`8IYYB!_)BU8pyr@H^jmKXlMzJ?yMU_a zlf==W3@OoVnl2J`1R;_j8Xj9e1~1=qz-mm??x7_HqR+*j88>4pHkf!HfMPB2b8qM9 zYaqngR=`Q}+k_);)*7DWcO+-GUu_1&D;2Gqt1wQaM^cERUTfZ|pra!Yqi|xMxBuw* zb6K?%1^uiSx1%uv2Aw zO7ap|nAbLG1>!w8xbjgF**mP0>!hQOe-f+5@7#=xl}iuu^u!@L&{eqnXi%cmO$~IK zXwOp+BG$byCV(+#dJOk6x8b{@mNBygg<5;R1)#oKcU!@FqHY8KLfF^;0b%%Djf9HXnE$Q5 zSiYKqn44}wp4HdQ%KuSc{nMwORX?%wGf6b{NUHvX$LBLRwE5=}h~)Y&m1GS5xIKvc zZJ(+T(sc*1QXT`@l?&&jG%0=j$kk!m^6;(<@!T}!80i|avl8UEOdT_q&L8Ps=&_5i zNoJ%`wYL!NrNy&dO1-Bs=k4nC_FVmv3Dp(}myMn}7USpJ?4S|XBT?>_)e}u2t`;3X z$b>^|)~fs58=qANuO3pTj*uac6WaO>sM8^sQM0B7RrMCr`ecZE$hY9#))lj?o~&ju zM!*IDSzZ};di5ecMh_0_?bvDWn@E_)Sj0ZtDHR1w7re@9OcU-mWuWWjxv$Q{8!gIJ zxBy1x8IF}#@IOP=@Ne1F@gkjQ`iw8!Of0Oy<^_Y|v9a=@j~xfyl|LvkBn7q)Hfs0_ zyiqR&2!?e;ET#b9$|smxjO~;JfXhqK>bl9Z$sl6lKPrlc%t5|(IZ)T1Pw2D!6zvpS z&y>>$tvcRBtd%vvr9xvP-8jxg4N@#lVqV%_^2}aHmph5*FnOQjdxZCs)SB>H(yTo* z7($!|{T?FYm%{H`AcsIL{}tT+7driMw=l)EGob9OYS>5gI^(!Zxd~)p!h}POu?%+{mh_Bux`YC$ny6?iJrpu)t&+cl$(w z5-^h0U&Mic1cxsDDA_CS{xSI0MKG@srvB0OW64)0vx-jQ%ZsBp8O}xoK0-wkXB6aZ z`hpL64TRrO%S4HN;q0C%=@*dQGn2OiFaE>i7fpN4n!j41gva+@#MSQhg)=JLq#nv2@NYK&XpSaay39vF->lx+-1SWKq=5?#O?c+=^&^t9&JZc_H^aFx#2CWm4 zW#G)n@>SYLb(aKE1W1xT0I1yI_QRe<;eOaq`Ii?E6gNug#e!-9bQtMXKRp*doD?-w zvRaPXJhoY=5L-u`ka3{C5L?m1zW`et_|CMth^T?00BevAoc_odG#82J#hXI(F|x4W zdK?%dR2@`oZ3PG5R51!xs=ZOIN8by1i-j%%9)E@> z3()v8JdK0W|AD9U`uFZE;ZA8E62NEify?Y?7|#{6owIo5LDT2R6UOga`@$do7U>K_ z@Ec!(L9EYlcR2!P$_L8(;LtRQzre7`S8(<_jP*Dh!KkypBne0sr2e}VPV{cXMo-@$YB^Ak}SzvL-*G+Ky8(U1}Ozk5GePig<@ z{a7pWCVi#f;cb35)+pweTti^-CvEgM?SEQ7iag+@tE|J|uzQdjYe)F;n9oEv0)7NF zTGOq!Xrfj0Tq%^BVWrE9PA+~g+ykzE>|W1gk{3<#r@DKW?7w=A>P3BbAxb6R?V02c zr_@r^kM>+>hIR;&G03LtQF$!PRZi=#*phxeViOU@>28ht7R{D;FL@G1day4VQ$)VU z{9cn{k505#IA%+KP;gB3=rDt^oo^WmFntQ;#?%z-(13S|K^7+Uw9}$2efAb8i{fD zOu(5oK>YwAGa1e7=zE-65;3bAPpJvI!8jXMNX|j~myjGVKfI&mBLeG62nW5fM}E$o zCth8;F9_rcOiddpERIh;(q)(1I-cbDe1U6PW~FP<6oEaPfgndRI*x))g)EK3?o zl7v%^sRk(!6!&Q2Ndycqj{yY*pwE1Y&ZBxRUY!oHv FzW}!ckwpLi From 1c30eacac2ead489dc4ee1cba26ff79ca9a08b58 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 25 Jun 2024 15:27:38 +0200 Subject: [PATCH 040/239] updated index feeding procedure to exploit the collection aliases --- .../oa/provision/SolrAdminApplication.java | 20 +- .../dhp/oa/provision/SolrRecordDumpJob.java | 19 +- .../dhp/oa/provision/XmlIndexingJob.java | 23 +- .../input_params_solr_record_dump.json | 2 +- .../provision/input_params_update_index.json | 4 +- .../provision/input_solradmin_parameters.json | 6 - .../dhp/oa/provision/oozie_app/workflow.xml | 23 +- .../provision/SolrAdminApplicationTest.java | 12 +- .../oa/provision/SolrConfigExploreTest.java | 142 +++++++- .../dhp/oa/provision/SolrConfigTest.java | 20 +- .../dhp/oa/provision/SolrExploreTest.java | 52 --- .../eu/dnetlib/dhp/oa/provision/SolrTest.java | 31 +- .../dhp/oa/provision/XmlIndexingJobTest.java | 70 +++- .../eu/dnetlib/dhp/oa/provision/fields.xml | 7 +- .../conf/exploreTestConfig/managed-schema | 339 ++++++++---------- .../oa/provision/xml/xml_json_sample.json.gz | Bin 0 -> 354497 bytes pom.xml | 2 +- 17 files changed, 402 insertions(+), 370 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/xml/xml_json_sample.json.gz diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java index 459ca0b50..44426e8c5 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java @@ -40,9 +40,6 @@ public class SolrAdminApplication implements Closeable { final String isLookupUrl = parser.get("isLookupUrl"); log.info("isLookupUrl: {}", isLookupUrl); - final String format = parser.get("format"); - log.info("format: {}", format); - final Action action = Action.valueOf(parser.get("action")); log.info("action: {}", action); @@ -60,9 +57,6 @@ public class SolrAdminApplication implements Closeable { final String zkHost = isLookup.getZkHost(); log.info("zkHost: {}", zkHost); - final String collection = ProvisionConstants.getCollectionName(format); - log.info("collection: {}", collection); - final String publicFormat = parser.get("publicFormat"); log.info("publicFormat: {}", publicFormat); @@ -77,7 +71,7 @@ public class SolrAdminApplication implements Closeable { log.info("shadowCollection: {}", shadowCollection); try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) { - app.execute(action, collection, query, commit, publicCollection, shadowCollection); + app.execute(action, query, commit, publicCollection, shadowCollection); } } @@ -86,24 +80,24 @@ public class SolrAdminApplication implements Closeable { this.solrClient = new CloudSolrClient.Builder(zk.getHosts(), zk.getChroot()).build(); } - public SolrResponse commit(String collection) throws IOException, SolrServerException { - return execute(Action.COMMIT, collection, null, true, null, null); + public SolrResponse commit(String shadowCollection) throws IOException, SolrServerException { + return execute(Action.COMMIT, null, true, null, shadowCollection); } - public SolrResponse execute(Action action, String collection, String query, boolean commit, + public SolrResponse execute(Action action, String query, boolean commit, String publicCollection, String shadowCollection) throws IOException, SolrServerException { switch (action) { case DELETE_BY_QUERY: - UpdateResponse rsp = solrClient.deleteByQuery(collection, query); + UpdateResponse rsp = solrClient.deleteByQuery(shadowCollection, query); if (commit) { - return solrClient.commit(collection); + return solrClient.commit(shadowCollection); } return rsp; case COMMIT: - return solrClient.commit(collection); + return solrClient.commit(shadowCollection); case UPDATE_ALIASES: this.updateAliases(publicCollection, shadowCollection); diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrRecordDumpJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrRecordDumpJob.java index faa18851b..06a35eda5 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrRecordDumpJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrRecordDumpJob.java @@ -36,7 +36,7 @@ public class SolrRecordDumpJob extends AbstractSolrRecordTransformJob { private final String inputPath; - private final String format; + private final String shadowFormat; private final String outputPath; @@ -61,8 +61,8 @@ public class SolrRecordDumpJob extends AbstractSolrRecordTransformJob { final String inputPath = parser.get("inputPath"); log.info("inputPath: {}", inputPath); - final String format = parser.get("format"); - log.info("format: {}", format); + final String shadowFormat = parser.get("shadowFormat"); + log.info("shadowFormat: {}", shadowFormat); final String outputPath = Optional .ofNullable(parser.get("outputPath")) @@ -95,27 +95,24 @@ public class SolrRecordDumpJob extends AbstractSolrRecordTransformJob { final String isLookupUrl = parser.get("isLookupUrl"); log.info("isLookupUrl: {}", isLookupUrl); final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl)); - new SolrRecordDumpJob(spark, inputPath, format, outputPath).run(isLookup); + new SolrRecordDumpJob(spark, inputPath, shadowFormat, outputPath).run(isLookup); }); } - public SolrRecordDumpJob(SparkSession spark, String inputPath, String format, String outputPath) { + public SolrRecordDumpJob(SparkSession spark, String inputPath, String shadowFormat, String outputPath) { this.spark = spark; this.inputPath = inputPath; - this.format = format; + this.shadowFormat = shadowFormat; this.outputPath = outputPath; } public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException { - final String fields = isLookup.getLayoutSource(format); + final String fields = isLookup.getLayoutSource(shadowFormat); log.info("fields: {}", fields); final String xslt = isLookup.getLayoutTransformer(); - final String dsId = isLookup.getDsId(format); - log.info("dsId: {}", dsId); - - final String indexRecordXslt = getLayoutTransformer(format, fields, xslt); + final String indexRecordXslt = getLayoutTransformer(shadowFormat, fields, xslt); log.info("indexRecordTransformer {}", indexRecordXslt); final Encoder encoder = Encoders.bean(TupleWrapper.class); diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java index 78154e0ab..386445057 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java @@ -40,6 +40,8 @@ public class XmlIndexingJob extends AbstractSolrRecordTransformJob { private final String format; + private final String shadowCollection; + private final int batchSize; private final SparkSession spark; @@ -63,8 +65,11 @@ public class XmlIndexingJob extends AbstractSolrRecordTransformJob { final String inputPath = parser.get("inputPath"); log.info("inputPath: {}", inputPath); - final String format = parser.get("format"); - log.info("format: {}", format); + final String shadowFormat = parser.get("shadowFormat"); + log.info("shadowFormat: {}", shadowFormat); + + final String shadowCollection = ProvisionConstants.getCollectionName(shadowFormat); + log.info("shadowCollection: {}", shadowCollection); final Integer batchSize = Optional .ofNullable(parser.get("batchSize")) @@ -85,15 +90,17 @@ public class XmlIndexingJob extends AbstractSolrRecordTransformJob { final String isLookupUrl = parser.get("isLookupUrl"); log.info("isLookupUrl: {}", isLookupUrl); final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl)); - new XmlIndexingJob(spark, inputPath, format, batchSize) + new XmlIndexingJob(spark, inputPath, shadowFormat, shadowCollection, batchSize) .run(isLookup); }); } - public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize) { + public XmlIndexingJob(SparkSession spark, String inputPath, String format, String shadowCollection, + Integer batchSize) { this.spark = spark; this.inputPath = inputPath; this.format = format; + this.shadowCollection = shadowCollection; this.batchSize = batchSize; } @@ -103,12 +110,6 @@ public class XmlIndexingJob extends AbstractSolrRecordTransformJob { final String xslt = isLookup.getLayoutTransformer(); - final String dsId = isLookup.getDsId(format); - log.info("dsId: {}", dsId); - - final String collection = ProvisionConstants.getCollectionName(format); - log.info("collection: {}", collection); - final String zkHost = isLookup.getZkHost(); log.info("zkHost: {}", zkHost); @@ -130,7 +131,7 @@ public class XmlIndexingJob extends AbstractSolrRecordTransformJob { .javaRDD() .map( t -> new StreamingInputDocumentFactory().parseDocument(t.getXml(), t.getJson())); - DHPSolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd()); + DHPSolrSupport.indexDocs(zkHost, shadowCollection, batchSize, docs.rdd()); } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_solr_record_dump.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_solr_record_dump.json index 7e5734222..3c2c1e05d 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_solr_record_dump.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_solr_record_dump.json @@ -13,7 +13,7 @@ }, { "paramName": "f", - "paramLongName": "format", + "paramLongName": "shadowFormat", "paramDescription": "MDFormat name found in the IS profile", "paramRequired": true }, diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_update_index.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_update_index.json index 3396020e0..c8364bb28 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_update_index.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_update_index.json @@ -13,8 +13,8 @@ }, { "paramName": "f", - "paramLongName": "format", - "paramDescription": "MDFormat name found in the IS profile", + "paramLongName": "shadowFormat", + "paramDescription": "MDFormat name found in the IS profile bound to the shadow index collection to feed", "paramRequired": true }, { diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json index 6e3f21ef2..23a378857 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json @@ -5,12 +5,6 @@ "paramDescription": "the URL to the ISLookUp Service", "paramRequired": true }, - { - "paramName": "f", - "paramLongName": "format", - "paramDescription": "metadata format profile name", - "paramRequired": true - }, { "paramName": "a", "paramLongName": "action", diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 02195c0aa..6c58d2466 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -35,9 +35,13 @@ maximum number of relations allowed for a each entity grouping by target - format + shadowFormat metadata format name (DMF|TMF) + + publicFormat + the public metadata format - used to create the public collection alias + batchSize number of records to be included in each indexing request @@ -110,14 +114,6 @@ sparkNetworkTimeout configures spark.network.timeout - - publicFormat - the public metadata format - used to create the public collection alias - - - shadowFormat - the shadow metadata format - used to create the shadow collection alias - @@ -650,8 +646,8 @@ eu.dnetlib.dhp.oa.provision.SolrAdminApplication --isLookupUrl${isLookupUrl} - --format${format} --actionDELETE_BY_QUERY + --shadowFormat${shadowFormat} --query${solrDeletionQuery} --committrue @@ -681,7 +677,7 @@ --inputPath${workingDir}/xml_json --isLookupUrl${isLookupUrl} - --format${format} + --shadowFormat${shadowFormat} --batchSize${batchSize} @@ -698,7 +694,7 @@ eu.dnetlib.dhp.oa.provision.SolrAdminApplication --isLookupUrl${isLookupUrl} - --format${format} + --shadowFormat${shadowFormat} --actionCOMMIT @@ -723,7 +719,7 @@ --inputPath${workingDir}/xml_json --isLookupUrl${isLookupUrl} - --format${format} + --shadowFormat${shadowFormat} --outputPath${workingDir}/solr_documents @@ -741,7 +737,6 @@ eu.dnetlib.dhp.oa.provision.SolrAdminApplication --isLookupUrl${isLookupUrl} - --format${format} --actionUPDATE_ALIASES --publicFormat${publicFormat} --shadowFormat${shadowFormat} diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java index 43e4c9ed7..3834f530e 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplicationTest.java @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.provision; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import org.apache.solr.client.solrj.request.SolrPing; import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.client.solrj.response.SolrPingResponse; import org.apache.solr.client.solrj.response.UpdateResponse; @@ -13,7 +14,10 @@ class SolrAdminApplicationTest extends SolrTest { @Test void testPing() throws Exception { - SolrPingResponse pingResponse = miniCluster.getSolrClient().ping(); + final SolrPing ping = new SolrPing(); + ping.getParams().set("collection", ProvisionConstants.SHADOW_ALIAS_NAME); + SolrPingResponse pingResponse = ping.process(miniCluster.getSolrClient()); + log.info("pingResponse: '{}'", pingResponse.getStatus()); assertEquals(0, pingResponse.getStatus()); } @@ -24,7 +28,7 @@ class SolrAdminApplicationTest extends SolrTest { SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost()); UpdateResponse rsp = (UpdateResponse) admin - .execute(SolrAdminApplication.Action.DELETE_BY_QUERY, DEFAULT_COLLECTION, "*:*", false, null, null); + .execute(SolrAdminApplication.Action.DELETE_BY_QUERY, "*:*", false, null, SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); } @@ -34,7 +38,7 @@ class SolrAdminApplicationTest extends SolrTest { SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost()); - UpdateResponse rsp = (UpdateResponse) admin.commit(DEFAULT_COLLECTION); + UpdateResponse rsp = (UpdateResponse) admin.commit(SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); } @@ -45,7 +49,7 @@ class SolrAdminApplicationTest extends SolrTest { SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost()); CollectionAdminResponse rsp = (CollectionAdminResponse) admin - .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, DEFAULT_COLLECTION); + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); } diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java index 424262eef..41eac2a30 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java @@ -1,29 +1,59 @@ package eu.dnetlib.dhp.oa.provision; +import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.file.Path; +import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; +import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.embedded.JettyConfig; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.ConfigSetAdminRequest; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.cloud.MiniSolrCloudCluster; import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.params.CollectionParams; import org.apache.solr.common.params.CommonParams; +import org.apache.solr.common.params.CoreAdminParams; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.util.NamedList; import org.apache.spark.SparkConf; import org.apache.spark.sql.SparkSession; 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 eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.jupiter.api.Assertions.assertEquals; @ExtendWith(MockitoExtension.class) -public class SolrConfigExploreTest extends SolrExploreTest { +public class SolrConfigExploreTest { + + protected static final Logger log = LoggerFactory.getLogger(SolrConfigExploreTest.class); + + protected static final String SHADOW_FORMAT = "c1"; + protected static final String SHADOW_COLLECTION = SHADOW_FORMAT + "-index-openaire"; + protected static final String PUBLIC_FORMAT = "c2"; + protected static final String PUBLIC_COLLECTION = PUBLIC_FORMAT + "-index-openaire"; + + protected static final String CONFIG_NAME = "testConfig"; + + protected static SolrAdminApplication admin; protected static SparkSession spark; @@ -35,15 +65,17 @@ public class SolrConfigExploreTest extends SolrExploreTest { @Mock private ISLookupClient isLookupClient; + @TempDir + public static Path workingDir; + + protected static MiniSolrCloudCluster miniCluster; + @BeforeEach public void prepareMocks() throws ISLookUpException, IOException { isLookupClient.setIsLookup(isLookUpService); int solrPort = URI.create("http://" + miniCluster.getZkClient().getZkServerAddress()).getPort(); - Mockito - .when(isLookupClient.getDsId(Mockito.anyString())) - .thenReturn("313f0381-23b6-466f-a0b8-c72a9679ac4b_SW5kZXhEU1Jlc291cmNlcy9JbmRleERTUmVzb3VyY2VUeXBl"); Mockito.when(isLookupClient.getZkHost()).thenReturn(String.format("127.0.0.1:%s/solr", solrPort)); Mockito .when(isLookupClient.getLayoutSource(Mockito.anyString())) @@ -54,12 +86,12 @@ public class SolrConfigExploreTest extends SolrExploreTest { } @BeforeAll - public static void before() { + public static void setup() throws Exception { SparkConf conf = new SparkConf(); conf.setAppName(XmlIndexingJobTest.class.getSimpleName()); conf.registerKryoClasses(new Class[] { - SerializableSolrInputDocument.class + SerializableSolrInputDocument.class }); conf.setMaster("local[1]"); @@ -69,16 +101,76 @@ public class SolrConfigExploreTest extends SolrExploreTest { conf.set("spark.sql.warehouse.dir", workingDir.resolve("spark").toString()); spark = SparkSession - .builder() - .appName(XmlIndexingJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); + .builder() + .appName(SolrConfigExploreTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + + // random unassigned HTTP port + final int jettyPort = 0; + final JettyConfig jettyConfig = JettyConfig.builder().setPort(jettyPort).build(); + + log.info(String.format("working directory: %s", workingDir.toString())); + System.setProperty("solr.log.dir", workingDir.resolve("logs").toString()); + + // create a MiniSolrCloudCluster instance + miniCluster = new MiniSolrCloudCluster(2, workingDir.resolve("solr"), jettyConfig); + + // Upload Solr configuration directory to ZooKeeper + String solrZKConfigDir = "src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/exploreTestConfig"; + File configDir = new File(solrZKConfigDir); + + miniCluster.uploadConfigSet(configDir.toPath(), CONFIG_NAME); + + // override settings in the solrconfig include + System.setProperty("solr.tests.maxBufferedDocs", "100000"); + System.setProperty("solr.tests.maxIndexingThreads", "-1"); + System.setProperty("solr.tests.ramBufferSizeMB", "100"); + + // use non-test classes so RandomizedRunner isn't necessary + System.setProperty("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler"); + System.setProperty("solr.directoryFactory", "solr.RAMDirectoryFactory"); + System.setProperty("solr.lock.type", "single"); + + log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString()); + log + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); + + NamedList res = createCollection( + miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); + res.forEach(o -> log.info(o.toString())); + + // miniCluster.getSolrClient().setDefaultCollection(SHADOW_COLLECTION); + + res = createCollection( + miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); + res.forEach(o -> log.info(o.toString())); + + admin = new SolrAdminApplication(miniCluster.getZkClient().getZkServerAddress()); + CollectionAdminResponse rsp = (CollectionAdminResponse) admin + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); + assertEquals(0, rsp.getStatus()); + rsp = (CollectionAdminResponse) admin.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, SHADOW_COLLECTION); + assertEquals(0, rsp.getStatus()); + + log + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); } @AfterAll - public static void tearDown() { + public static void tearDown() throws Exception { spark.stop(); + miniCluster.shutdown(); + FileUtils.deleteDirectory(workingDir.toFile()); } @Test @@ -86,8 +178,9 @@ public class SolrConfigExploreTest extends SolrExploreTest { String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml"; - new XmlIndexingJob(spark, inputPath, FORMAT, batchSize).run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus()); + new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) + .run(isLookupClient); + Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", @@ -101,14 +194,13 @@ public class SolrConfigExploreTest extends SolrExploreTest { query.add(CommonParams.Q, q); query.set("debugQuery", "on"); - log.info("Submit query to Solr with params: {}", query.toString()); - QueryResponse rsp = miniCluster.getSolrClient().query(query); + log.info("Submit query to Solr with params: {}", query); + QueryResponse rsp = miniCluster.getSolrClient().query(ProvisionConstants.SHADOW_ALIAS_NAME, query); // System.out.println(rsp.getHighlighting()); // System.out.println(rsp.getExplainMap()); for (SolrDocument doc : rsp.getResults()) { - System.out - .println( + log.info( doc.get("score") + "\t" + doc.get("__indexrecordidentifier") + "\t" + doc.get("resultidentifier") + "\t" + @@ -122,4 +214,18 @@ public class SolrConfigExploreTest extends SolrExploreTest { } } } + + protected static NamedList createCollection(CloudSolrClient client, String name, int numShards, + int replicationFactor, int maxShardsPerNode, String configName) throws Exception { + ModifiableSolrParams modParams = new ModifiableSolrParams(); + modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name()); + modParams.set("name", name); + modParams.set("numShards", numShards); + modParams.set("replicationFactor", replicationFactor); + modParams.set("collection.configName", configName); + modParams.set("maxShardsPerNode", maxShardsPerNode); + QueryRequest request = new QueryRequest(modParams); + request.setPath("/admin/collections"); + return client.request(request); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java index 625b6d131..2c62389c6 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java @@ -2,24 +2,15 @@ package eu.dnetlib.dhp.oa.provision; import java.io.IOException; -import java.io.StringReader; import java.net.URI; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.SolrDocument; -import org.apache.solr.common.SolrInputField; import org.apache.solr.common.params.CommonParams; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; -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.dom4j.io.SAXReader; import org.junit.jupiter.api.*; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -50,9 +41,6 @@ public class SolrConfigTest extends SolrTest { int solrPort = URI.create("http://" + miniCluster.getZkClient().getZkServerAddress()).getPort(); - Mockito - .when(isLookupClient.getDsId(Mockito.anyString())) - .thenReturn("313f0381-23b6-466f-a0b8-c72a9679ac4b_SW5kZXhEU1Jlc291cmNlcy9JbmRleERTUmVzb3VyY2VUeXBl"); Mockito.when(isLookupClient.getZkHost()).thenReturn(String.format("127.0.0.1:%s/solr", solrPort)); Mockito .when(isLookupClient.getLayoutSource(Mockito.anyString())) @@ -95,9 +83,9 @@ public class SolrConfigTest extends SolrTest { String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml"; - new XmlIndexingJob(spark, inputPath, FORMAT, batchSize) + new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus()); + Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", @@ -109,8 +97,8 @@ public class SolrConfigTest extends SolrTest { SolrQuery query = new SolrQuery(); query.add(CommonParams.Q, q); - log.info("Submit query to Solr with params: {}", query.toString()); - QueryResponse rsp = miniCluster.getSolrClient().query(query); + log.info("Submit query to Solr with params: {}", query); + QueryResponse rsp = miniCluster.getSolrClient().query(ProvisionConstants.SHADOW_ALIAS_NAME, query); for (SolrDocument doc : rsp.getResults()) { System.out diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrExploreTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrExploreTest.java index 34a9465a7..5b5e42fbd 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrExploreTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrExploreTest.java @@ -34,58 +34,6 @@ public abstract class SolrExploreTest { @TempDir public static Path workingDir; - @BeforeAll - public static void setup() throws Exception { - - // random unassigned HTTP port - final int jettyPort = 0; - final JettyConfig jettyConfig = JettyConfig.builder().setPort(jettyPort).build(); - - log.info(String.format("working directory: %s", workingDir.toString())); - System.setProperty("solr.log.dir", workingDir.resolve("logs").toString()); - - // create a MiniSolrCloudCluster instance - miniCluster = new MiniSolrCloudCluster(2, workingDir.resolve("solr"), jettyConfig); - - // Upload Solr configuration directory to ZooKeeper - String solrZKConfigDir = "src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/exploreTestConfig"; - File configDir = new File(solrZKConfigDir); - - miniCluster.uploadConfigSet(configDir.toPath(), CONFIG_NAME); - - // override settings in the solrconfig include - System.setProperty("solr.tests.maxBufferedDocs", "100000"); - System.setProperty("solr.tests.maxIndexingThreads", "-1"); - System.setProperty("solr.tests.ramBufferSizeMB", "100"); - - // use non-test classes so RandomizedRunner isn't necessary - System.setProperty("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler"); - System.setProperty("solr.directoryFactory", "solr.RAMDirectoryFactory"); - System.setProperty("solr.lock.type", "single"); - - log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString()); - log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); - - NamedList res = createCollection( - miniCluster.getSolrClient(), DEFAULT_COLLECTION, 4, 2, 20, CONFIG_NAME); - res.forEach(o -> log.info(o.toString())); - - miniCluster.getSolrClient().setDefaultCollection(DEFAULT_COLLECTION); - - log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); - - } - @AfterAll public static void shutDown() throws Exception { miniCluster.shutdown(); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrTest.java index 79527b891..2caf09799 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrTest.java @@ -1,6 +1,8 @@ package eu.dnetlib.dhp.oa.provision; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.File; import java.nio.file.Path; @@ -10,6 +12,7 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.ConfigSetAdminRequest; import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.cloud.MiniSolrCloudCluster; import org.apache.solr.common.params.CollectionParams; import org.apache.solr.common.params.CoreAdminParams; @@ -21,14 +24,21 @@ import org.junit.jupiter.api.io.TempDir; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import sun.security.provider.SHA; + public abstract class SolrTest { protected static final Logger log = LoggerFactory.getLogger(SolrTest.class); - protected static final String FORMAT = "test"; - protected static final String DEFAULT_COLLECTION = FORMAT + "-index-openaire"; + protected static final String SHADOW_FORMAT = "c1"; + protected static final String SHADOW_COLLECTION = SHADOW_FORMAT + "-index-openaire"; + protected static final String PUBLIC_FORMAT = "c2"; + protected static final String PUBLIC_COLLECTION = PUBLIC_FORMAT + "-index-openaire"; + protected static final String CONFIG_NAME = "testConfig"; + protected static SolrAdminApplication admin; + protected static MiniSolrCloudCluster miniCluster; @TempDir @@ -72,10 +82,21 @@ public abstract class SolrTest { .toString()); NamedList res = createCollection( - miniCluster.getSolrClient(), DEFAULT_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); - miniCluster.getSolrClient().setDefaultCollection(DEFAULT_COLLECTION); + // miniCluster.getSolrClient().setDefaultCollection(SHADOW_COLLECTION); + + res = createCollection( + miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); + res.forEach(o -> log.info(o.toString())); + + admin = new SolrAdminApplication(miniCluster.getZkClient().getZkServerAddress()); + CollectionAdminResponse rsp = (CollectionAdminResponse) admin + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); + assertEquals(0, rsp.getStatus()); + rsp = (CollectionAdminResponse) admin.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, SHADOW_COLLECTION); + assertEquals(0, rsp.getStatus()); log .info( @@ -83,12 +104,12 @@ public abstract class SolrTest { .getClusterStatus() .process(miniCluster.getSolrClient()) .toString()); - } @AfterAll public static void shutDown() throws Exception { miniCluster.shutdown(); + admin.close(); FileUtils.deleteDirectory(workingDir.toFile()); } diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJobTest.java index 522c34ef1..8149c2526 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJobTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJobTest.java @@ -10,6 +10,7 @@ import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.params.CommonParams; import org.apache.spark.SparkConf; @@ -50,9 +51,6 @@ public class XmlIndexingJobTest extends SolrTest { int solrPort = URI.create("http://" + miniCluster.getZkClient().getZkServerAddress()).getPort(); - Mockito - .when(isLookupClient.getDsId(Mockito.anyString())) - .thenReturn("313f0381-23b6-466f-a0b8-c72a9679ac4b_SW5kZXhEU1Jlc291cmNlcy9JbmRleERTUmVzb3VyY2VUeXBl"); Mockito.when(isLookupClient.getZkHost()).thenReturn(String.format("127.0.0.1:%s/solr", solrPort)); Mockito .when(isLookupClient.getLayoutSource(Mockito.anyString())) @@ -103,46 +101,72 @@ public class XmlIndexingJobTest extends SolrTest { long nRecord = records.count(); - new XmlIndexingJob(spark, inputPath, FORMAT, batchSize).run(isLookupClient); + new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) + .run(isLookupClient); - assertEquals(0, miniCluster.getSolrClient().commit().getStatus()); + assertEquals(0, miniCluster.getSolrClient().commit(SHADOW_COLLECTION).getStatus()); - QueryResponse rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "*:*")); + QueryResponse rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.SHADOW_ALIAS_NAME, + new SolrQuery().add(CommonParams.Q, "*:*")); assertEquals( nRecord, rsp.getResults().getNumFound(), "the number of indexed records should be equal to the number of input records"); - rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "isgreen:true")); + rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.SHADOW_ALIAS_NAME, + new SolrQuery().add(CommonParams.Q, "isgreen:true")); assertEquals( - 0, rsp.getResults().getNumFound(), + 4, rsp.getResults().getNumFound(), "the number of indexed records having isgreen = true"); - rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "openaccesscolor:bronze")); + rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.SHADOW_ALIAS_NAME, + new SolrQuery().add(CommonParams.Q, "openaccesscolor:bronze")); assertEquals( - 0, rsp.getResults().getNumFound(), + 2, rsp.getResults().getNumFound(), "the number of indexed records having openaccesscolor = bronze"); - rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "isindiamondjournal:true")); + rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.SHADOW_ALIAS_NAME, + new SolrQuery().add(CommonParams.Q, "isindiamondjournal:true")); assertEquals( 0, rsp.getResults().getNumFound(), "the number of indexed records having isindiamondjournal = true"); - rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "publiclyfunded:true")); + rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.SHADOW_ALIAS_NAME, + new SolrQuery().add(CommonParams.Q, "publiclyfunded:true")); assertEquals( 0, rsp.getResults().getNumFound(), "the number of indexed records having publiclyfunded = true"); - rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "peerreviewed:true")); + rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.SHADOW_ALIAS_NAME, + new SolrQuery().add(CommonParams.Q, "peerreviewed:true")); assertEquals( - 0, rsp.getResults().getNumFound(), + 35, rsp.getResults().getNumFound(), "the number of indexed records having peerreviewed = true"); rsp = miniCluster .getSolrClient() .query( + ProvisionConstants.SHADOW_ALIAS_NAME, new SolrQuery() - .add(CommonParams.Q, "objidentifier:\"iddesignpres::ae77e56e84ad058d9e7f19fa2f7325db\"") + .add(CommonParams.Q, "objidentifier:\"57a035e5b1ae::236d6d8c1e03368b5ae72acfeeb11bbc\"") .add(CommonParams.FL, "__json")); assertEquals( 1, rsp.getResults().getNumFound(), @@ -158,6 +182,22 @@ public class XmlIndexingJobTest extends SolrTest { log.info((String) json.get()); + admin + .execute( + SolrAdminApplication.Action.UPDATE_ALIASES, null, false, + SHADOW_COLLECTION, PUBLIC_COLLECTION); + + rsp = miniCluster + .getSolrClient() + .query( + ProvisionConstants.PUBLIC_ALIAS_NAME, + new SolrQuery() + .add(CommonParams.Q, "objidentifier:\"57a035e5b1ae::236d6d8c1e03368b5ae72acfeeb11bbc\"") + .add(CommonParams.FL, "__json")); + + assertEquals( + 1, rsp.getResults().getNumFound(), + "the number of indexed records having the given identifier, found in the public collection"); } } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/fields.xml b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/fields.xml index ba3fa83df..8567acf85 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/fields.xml +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/fields.xml @@ -68,15 +68,12 @@ - - - - + @@ -93,6 +90,7 @@ + @@ -122,6 +120,7 @@ + diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/exploreTestConfig/managed-schema b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/exploreTestConfig/managed-schema index 9720d3f37..d4e1cbede 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/exploreTestConfig/managed-schema +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/exploreTestConfig/managed-schema @@ -194,228 +194,173 @@ - - - - - - + - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + - - - - - - + + + + + + - - - - - + + + + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - - - - - - - - - - + - - + + + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + - - + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/xml/xml_json_sample.json.gz b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/xml/xml_json_sample.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..81e4a1c3afb50979b571bcf857a3a809b147a03d GIT binary patch literal 354497 zcmV)1K+V4&iwFRpJ9K6M1MI!qZX`#NDEPi#;j4u-RXCZ+xaNgb%0U&2C6!bcqR5i^ zXw<0ni15q|W@JQmT#`l3G_VUS&^SQrp|L+O0(u|LLoar5fb+1xnTJu;TYtrT$(p&x zJtJ>q@{+7jw?s0+!`$86+}v(%_Rn9O2A=!H{ulrJ^A{sVE$09H#XkIQTk!MeFJz_L z(R!VMLfiWOzDx&-G-zvVpsA`(Ex9u=1|8b92C~}Knep=%ji0~pe0ylS)Ompr{^{p0 zbm?C%1BKfAJ~Pk5)`0()1^d(wZPQ_UTD#YgfBxe4&L$Z%|vhPLN& zAQp32$SmXBb_dL7u8GYC)Ct%h;jQU84m08X!1pFNR+zRGyoe_*{fjqcF13C39{$|ffgJ)iJaQOJ>2Lq9=MamL z0~^L+Q*usDnCW_sH~j5y|Nb}n^A~><6AB_H6iXm}4;M|2M&Zcw<8con$8nRczip9& z7J1$xub#XNo;3+ZoRok5 z;sA!@dx-kOAdD>A3&>I22mE?Y4m>x2xkV;Gqz(%NTD-{T)bdZbk~2F9=>iCk$m`$! zze$U9dgOuJ>!}a-33Hk6H7C@+5JQG3_nJ~$R{39&q;;E}c25%|Jn;Mp4Mk6MI!)%b zH?;<$fx)12(_}#a8*3EPhUr;i>hE5E|N703uQ_Cg8qDEO0kwqAs0~kW&5+uWnxAkm z2S9WJHHPoG7c7TX=lpr(TFgqe3B0fYJ5=l(B(%Aa%%~HwMV%%er$qcj>|x&J$P4)9 z6mV{H=fU@p?}#-Rh2b>V-`guPpFN;v_MtT$P4|Et4g*;Ky@VglY<8AsN@c!s`;)_s z;tg==rpa2)Z5p2;i)2>3loxnc)&O#(%-z0Ce9oPG4BSYwUpyuQogK9`DUVC;-P5d^0!@lm)oyE0UrC zPie}^iKOjo>b}%&=~71knrmGF>jW5lAec`EPi-pti6rf7^1jk;DRSqhJiyW?F)sEB z$ap+qA%*5N>(iryeVkQzep$5t?EW1793n6Q03c=n(xq)BI|~;aniReZ?EQd)ed!4X zy9NM~@B$oT{+)od2h@qpn((vrA+R`kV26Chf=K(>5FQu!0>NItzQleew)M5(=9@5) zy*{i%Dwy>*mdPIP<-RU}S|zBJzML=$A_o(*0I>`X55B~N1hByoBTB2+lNGtA9yWXU z+;nIV!1Dv(Q)I6UV(LvJ2gHN$JP$)_^>kV8N$o@6aI1l&<9JU|_{!j{@ohGMxsO}x zbUlI31Gc(TE`Eu}{6iGUNGy|y$1z0}UT{if`~%BnA#P-PV2A8|u|fQ4brZs+XP+Su z&amOx)c0oa6`+io@ooyv1DK5Ghx?}Q1;H70#`_r)hHpYY3PO$-T-W|>Ns*8D_{0FE zCCMs>+*GP6V74US%pq66dOrc#ET}|nsR1sl=`#w0BPed;fd`M{FETNm#6tSlc@e!o zC*?ohi-7^y@`>NiPJ7_!<(K5KL)~FShj{`6W>lL0@szqCS3e>L6eMN`R-*z538ZQa z&Om9!u4#ZS^#sn|XL>M8<$4w^U_`z0a%@m~Bqky}s} z&Y`)W))mlQd{f4=0QMfB;o)E}z;{)vJuwtM?9yPW={|e9kewv(mUEO`D;$I!aEY?d znabn6Nt%|P^4$YW zbl~+LC#)}pn*9BE-nNjmsd)34^S(ZCKHxt5*T?D0jFt9X(Bk7hN^c&Q8E$@~u426T zwfvU)GlqIs#cF1KJmGy*lT=l!y*}qE0%;_Xd4S5CAeDd~?-lm*5%GPg)21+#*`;j8x!n*r4<;BCbgdl}^s0#f)H zGZKc=&%XeA0&Ty46-EE?9zrPewwLcNHxS&atc{;HvlAsQV3_*&x`yqCUtZ4fyFsXza@yc5~12X5GS<^S0d}GEMK^!0%c8(U}%^Qdd$wxV1dxKRQSTN{}9=hw_f8PfSwSxO*a9)URyZ&_Y4Z+7w8L9 zAj;NypP3$tpw7m9{Py3llZEr1H=zIgbAJBhARCMgzSKqI3*bVVf%p>e-l3KM+uu+WAK0$#1JsF^%*hK{D4$dE+kc&UT(cvf?>I17KnR(E z!2T;}B@8x4JQXp>Z1^ViL3m=&L68m`c1#e$yg-FCZ6-hg3Pyk(R0==?jf@!s&IYq+ zkcnuGm<2Hi#01n(Coo$+8_++qxwpuZ;J3dKIt6ZqVkj;LO?$!5QQU+*>V}LA=*;s` zULm0BCYXd@aTYir512lTpaIzB6LZ**0#2y~{`Oz}APy8wc;S+ld^3^!7BABVb*(&gaGD5UAq_7INjPGJ+ZoJJj8hv?8W3;5--XN^xzGpW zPtpki$ROV(EQk;2APABhEhg|mIHDlrd68ogg9$=_Ca^++1GLD=$cA<0rwv+O2&j_{ z{QPWWna zC;+hr8qqNf2baU-v%sbtRU?>Mir6Wxfeqx?5(j9tk4)}1Wd;6U;Ujy3J;xRpP_)Cy zV7_a6bFQ#M*YrUHKgB)WB8lF7;HS2h9eC+g5fGE)uL}U~gb}M8qTr9WxD4cne3WXfyO2>q_j& z^5|*2@q_(+xuP|nCZ>3s4bJxdYfbg z18UT#SABY2{`C6&{f`P+WevM*REXS^yC+h2U+L~kz5gV^KU)k7ky{;Aufsy_Wmw26 zqweA;kc}<9B~bS+y5XxoL|*u)LQEAeRtTzDj6#i=1&CQ3ys=HmqJZM90>0khk;D~{ z3mqxB0s55=g%vR*$q`=Z`OBH5`?W>7wn(pKk;ZU|jBJWTR2!=Auc5lU?y(m#$Hr3! zT7d^=PsfiQ;p&gx#$G`K4yE zyrZ%hBrlh+oO&?lxlnevEN9*e8t&b8p#FLh_$&OMA_~6{^uHwRfBnG!wYn|!x+we~ zUu72nY8z>7BNaB%vi)mQ*8b+`uq@~m!fCFBt6nM^CO#cE48vKhjN#glE`BRvAXU^mEj(TFyW*mz8fcSuXm^Y=*km&N;| zjXCJdaXfeG*<%oUk3a&CzsNO6pA}$d-{pHyDlSj{_J92LS2z6auMIMeW|WdsVBi%nhRoXZ`p}{; zuM|MNeZsE_n9gw!V7YsjU?Lj_PG8H!Dg&iU#JL04d?eH8bO!LNYP1KIEO(hq6&>c) zW}Sh_w98%d*>In)hs!Hn*0~8DzrppHdIR5&4SX+eI&3DNUG}Df0te`ctnSObeMR5S zO$Vua0z6I9_jR=;N$vWk!!_S@Sh>%t^w8Ctx#DKSmGq9KS06Sd_WOPHVf}-9ooN2N zT{`kA*X!^WTl#8g36{7x}N|m>9c-GDFIIzoh z(dV{{!9iD!2#PHl1$Bx}rjHh>RA134rdr%UuZq7|p$u zyO@2l!hQLQI+F%DVZK*{nm#F(OLHbgXXQM+-~Kvg4$6xsbZVD@rjH6x<4^LC)p^L@ zQ|C0|I5@Ew9iA;=b@g^$%HvJj+Tub zT1Ia2C=WRKBoDbg5Ba(02XMgs_TORM-Z^E^>61LSC_x1C8q;Anaao2wamP5eo|ef8}44LO1nU_!|=p-tdNc&q$1bv@8H zeC)*?E-7dyw{tFpVmi8$Cr3vw$dR96!W{%?}AE(>- z2zDG|$?&`}@qD~~KMk3mJlZjr)OvjV4!fjE^t1y$33{|F*(UBBI z?kR%-W%P1B@a^2Q}b)sVu?Y@A((ph@l!7<$YlM7tF2Qag%^@}#SCGhkNo-nHA;UxyOe!&#C1X??f zH*QMHfy>Nj{gM%G3B*nYQCQG&0Q1(fe%boA>T+Rwv+yK_8g+rR58a`&jqoI&;T?B3 z#Bt<0S+bv^7af@r>M0w2SXI98Ck~171Ffyg1W{fpsydlQ8o?+tJ7~}fgUy?D#?5J9M-I;xM zlYw{jwyU?@wzl08yT=;Zang91A!}3ey+ayzsvoVgT`e`2RY`tZJeIc*AANI%($6S7 zdL^0Pqwwg2Y{%1pH~JbKPav>udYUaNts zcTG*znjOjj@yftz8cb4}JUHD(8r(ZrinyY4aB|aK+h_Vl1+p2hZsF|BNix;su5NUtqFAlg%pQlgqFtPE9Fzp}MXiTTf zrwzPJJT8S(XO9Xo<;gcI0=0jn?yy0AXH(Cb8*kxh9yoqf4AlNTZ^7$4#MsVaXOKdM z=VToZ~ecNgR z7?-nOBF;+U!lXs-6FUqz+d^K4<}0Le0f{#{vqMo$4TTJGrs7O#VXhCJKYJ)9EQ*%d z!LnIrTM0{~vG~Ja>fliDUpQ)*J;Tx3Lpc5j7xp;1Z6Ek8P9E06Lrjn$1VMQckO8LA zF2m=$#LgWE7Sg|sJOfzol<{h6&UuTxf{kQuV1&uR z0lO9-RD=azE_lC~);qBrVyjW&9Et}CGxx}Zey}I@1ruH18u)OcH29xlzi_1RnSk(z zK4hFwJWZsHXu|0QZJ$Yp6qfze1N=BZlJFKgu;OBISjL4fUjQH-psn^LbLIRC}r;US1@fWa^d`J@g}Q1m(F z;lRb89z5j0FiyWkS9pc=p4uzauy*_+`6% zoV0K`&f;3<{KYIQ+Kt6$zso-o!JYzoYM)Jg(@rZnc@M8l1AkW$ZvQLhW zp8R+q2H%3=cXYMgl_mJ?Qiz1_u5GZ5{8$_05ZJB}`NI+X{tm=pK;-uT$pX&aAnKmf z+2d16lts(4c`&0awPac9?QvKwgw=}YyiQ`z+qs^_i)PtI#5n@89haR2Q=~$)k)AO& zhTS!xLkdR)+p)On_4UQukE^DRAZS*&3xx}uF4Z$outOm2H=@*CzF{N3>8P8hDrwF0kR_B@UTNgYay14gnY*Cb2KF_!HXc)vq4y=i~jy+z~#7va5~ z?*gypy`68_5qobZn)4ms+c_que|~~Ddet)v=Dd^xq8JSt2Xd|L&EYBUYmd%-@#xgm z1Qyddw<5sOO3SgtpEyr{wuRhg!Mfz%UX$>!*s1w=uVUi&E#`bZAI*J=Ik$TJAGHIE z%;bUEp}N@}7|b-7rgZ4$DY<)wW*G(@G?jtTYib?ZX;MQqn_Wv+6|H9tbd6P~QNKaH zrS5qVYWgG(leb!`V2P!^Tk%;@)968Zs!eFeg@amac3QEtO z6+kCn6%%qViylAnoO6@84s;FrN=SW+y5V}?gU~(&x%s8o-av~oTIX_(0&b~;GtY|nX_vFp$qH%WR!qJE zr6qGOpqg})ezo3UpP2r+16mbmS^&+6fD41X%t#QP1CLp_i1RVxg@u4|9PB-LdQ4!5 zt`!K^Pyi^0na2(Q-i8ku+kqN#K0%F+w`7^F$Nmu69AV^)1~F01#g3nh>_E6VK%+Hr zAVI(qmr#mwR4PXxW;Q{m<}3`Y?!|FtuA%%OANb9qgNLyLB7Te?!E^ZWurzo+&Ugwi z0qzQ_n$LqDL2G>o-w2T=AaJD3Y=?V{LYiy}5EM)MA|MZ*zYQL?$h+y#r|2fbiA|U} zqAu5#(aXeOO9KXAEa*=_!$1O!T9%!-yk<7KUie&anBot^uG1 zW?=nM7s z244g;`M@e5dE>Qwn>)y0N29;;IeMcTB$z;xW&qPb(}3am7-fL&GoWuyzQS6-d^^D? zI*cPTSyHhPN<$DUnr@B)Co?GOs<7%48#I(Ae(6OnpDX(SXta?F1c(A0RCxsH*ReOT znGKY?pb=f%&FlF?Se*;enBt(3$qFNw0n{=gV$R{2?Q$Sz+<6!wZn70WK71v{39AKj z22sO7L?XGvcrLQtjKkr~5|+@14GjD4oMVA+jfgu&MP3=lpO~ub|t6$*0Eo70`481IjT(fX+Z46{F(pc@hDYWoHpwp?tt>PHF z!WlHYDV$xxbDulyIit)S89+XaA5J{>h7(PO#-%qz6qQ#8m*;MFaCHVt+`_-H3Ie}j zIa;nfz|U{tUm^2zC&5=FmD0JB0Aa%~|3Gd=8iS4+^RO**%T|f4Z9memwiVNal$vVkkY`>b2z#?XkAh*3GWm z#iX8QyUQe{cgX@jH>uFBu@1$XgojRmYv7ZoKZv*726~!rzwKCUx21<%)BYm$Z`9!;x z(H^t`D$@X#!n%f{${Ov_p2|8_uMGJ3d^n*MP!Em|-Y5S6RK@6-tgETI+LH{^7?_ei zP)*9zp4RPXojE4`;_b<^Jf@%q=QY47m+R zQk34nRHb%XHMJM zZR?WU8(3Y|?r1XY=pZwI4q=v8OuH#6$v?P67=7+1Xg)1IEZ@BH&fj#0=;f>&L zZuJPdCNjBp+4c_FueMZ6^j?U=HCbIVS#2gaZ&#k#vJ$yMDz1_Ra@PI$MQf0PZdq(^ zL|w~a$;&-Wg~PPH_5rv332v9Fu3J31ZSDm4Kc%8&a5wa>t-6zLsSNg=<813XeC`5C z-i#@Lo^qOuX@8HIy4G!XOw;Npn$*(_&=Pu9R~eYHrc<^aL)(r$xh9`GO+G+9rxAv? z0aFtG#yv^xb$dNcSJW%G(bu4_EqOuiAc57z;WxNYa@ov+N>k~b$m+h_+gJ2$6o-d} z>z+u;zNGK#ot9i@7Q7#s1+RQhYJ=MFt7-jy*M`5DIFxlV=ZR3d0ul@7^|e=rcZ)gm z;@e;S+?}!Ya^)qrzH+eA8(;m>eX;ej=7}3$Ie77HuYT^1*m}A0;(J~>P{|#ye&Kf5 zdNtvbyIpy3*}bm*+|95RrN|OVZ)H-v`bJKDBd5NRbJZI;Upe6;!{5VT_}s7c?VQi! zc24!`*Ee-Olbbr#%YV0T?O=|oUB9`r{S=uog!ckMa`voB(X`H2*Q9b3?(uIoo!rjh zVsV0k&m1wu%nP48GE(mHnIj_2mGPP5;f^E$`OH^gMRt>0OC%a6)Y|;MFHZtmdFS^l z3TSwAQx|{Z-mdfZZ!STmuwwU1w|^h+#n0;un&k|dm&|2}*~G4p%Chf9juUrLdh?m& zxQsoCo^hIG68w7e>#D^Ze3gElmEm*3&$DTaUd{r)&~$ynZl~I;{CS&|%lddiiOc*n zRV12SiKwE*)fX&~7Atbt#ykHOJ!a*MvQ;+{s;0d9rbB+prSnXZFdU;Wvo!c-o-t?V z?eg<%LBqY<&c^yUzN6C5-%okGmpskVux=h|Et0Ml^1Uv}_*%&MdfCoOeJ#rIim$RP zuN%7qt2ZH!U=!51X)N5rY|T}1z8S%95;szf-$qG*sUpx`*8FGvIE!ye&43&|k7G7IO1i49IVU z&21Tc8?2qJeZ3I6OK+{q4S(szTEFA%%=>b?{0`ZPDkYgLOGK5xzB=c`&gdxDf57i3 zCo7L(dY#$~&j?ec#$1xW-4R~2aQokmXy0FNT0%UpWDE0G$l}^a8(vIX$aj_`8WBBh zn#bgJYn9)4%=C`vjG28MQ5K+~(Hx0YuBeu?Hp#|zw}s)!o1RG3zLL_l=^ZegZB6yO z6DQNH$k;0~>0I^PSquElD3rHH^xHL`)kMEV#d0CV7e3e^>%#T~wM*@m^?TZQPM6!I z$sx7i3Q?^~?XoU)JDYdIv;kM%k(cW4lDnl!cZk!7Rf$QfGm?Fd28+d;Mk`ILB1PuC z-#+)w=bARs&Ont6n6yTvj;VoeTUktAZg-V9ho3I} z%ccM?TM?d<`{pI#RZTa0mfVpU$TYguk=mBjl{DGXrM9Nbr#(OLJs1cD7Q>TS;I%ID z)Q0gTSM{6Z;O(2^r!Tfy9R3w`PRHKN{`jW``N^T-*eg~t&ppdmGEY7;|Ng(Ps1^SW z{q*Y;b15{)iy1A|Nl%~TVSht|zyFzkvZ7f00SiOV@yILN9~TPCSCoPO!Ka`8jn5Kf zN4qr1>`{Ik@X5vB|6HD8MKSnqsP*aJSc5#HGtVzV&K?zKXaQtatO7p=_K41K5n+0m z*u~AaZqK*=i5Yx)pZ*8Kr75i<^p-x&j|9IeO|UZ$`zdog!>6abgSXVOXWnVCgZ#rh z{PfFGC*7q*@4vMzA5+~hp0k8Jf`i6BWe&Y?8JT*gbjIMB9Zd1Y1$9s9vXbg1me0VG zhxd`zu>AIIse)y^`zi~uA5WQSpHjTu0^_G-a*Dj+l!CB8fXnjaJcS`gf_q^!>j^>$ z1p9jJxtQU?wWx2A_@KiqCZ1!F(K!HUrVs?Tnk&m*9!U-fr6Eu<_D*dQ0LC2ZLThqC zCxAtaLjWWO%y2ytM9>NjB^Te4Xy$$V3(N~M+RW?`0iN$!x!Fb}nE9iKcyNY%{7c*` zZd5!Rd_;v0i_;)u>Jb|O=0_J1;9{=*@fX*gG>FR*JiwpCXRtRX_Jk524w)8thd(16 zyG2iVHkcvC6Xe0cF5VZ$_I#O!b2y5~w}<2h z+x0BGVtM2<;FAd9lqW#}DGEO)CQ={bG9(sYBy<5e0dp@9f(>Jw(T~5R^G0;JHjoD1 zB_7)#@T4gjJ-acuG)qH#2Z-#m*bJpfEJjc4GX4q`6!W~cX6wU4YD0{5w*x03)T{r$vL%W_9;$1%DxL` zbWFz{iC`^Z)?)wwa}Hpag2+SyXyDqw9C4L7Mm$*jDI93*)311G*mlPl2Xa9X#qffR zqcB2T$lAaEKQOzb0TKj3x&?+Y%tlK2G+^w|?`gyv1%L)lR8vd__377;_YK0O7{bI0 zy-BhPr)-j+Dwk*B1*|750Z<=80-EFnHD`cU+!m*}{U8jWc!OJ(thtJAKfa75fs(k$$F-GV7dE{C=KQ5OA8=KTOu7G;w!d*8f zXjbL%#$2n0ZM1CPXpQ{6L@3|5y3RF^dqL)7X_L&y&}4{d$O&c4TQxBVH z_YxK=#DpFjJPRPE-V}4|0rBKvsI7Ld*Oj{6!=@~)29l2BJw-D?24{_LvjNO~+*+sW z349)~)t%nE)Y6-)ido2W85x}O@Dl4M6&RU+dxz1of#$nT+U!JpD~5Iy1+8mNot+rQ za%rFoMzQ|9Wo^Ao+QKH51Dm&p^-Jco<*Bs$1q<1F$+BZxSg!xP9jsrnqpg=RJGO!q z<`i4N`UT_J5^!zIN?x>xEot4!EmUgz%JrYxz4|4)*%FxD3}$NX%BMRon7aB^HnURrW7T%7-L_-Zd5!yeH&~w)kcVB7yuF3?bwt1F|0NAc$CbfzpQy zMQUNOTqaS+Ev^ag`Li}Zv%7U8e?jI(Gc$Ki>68bi7bn3{sO`SXP(zzoyi)*_%rSjHHV&y2e8|idY zJz?jo0u3PFX2-aE&unyLm z57zn~R1a<>K+_MM52d1W@i+X7)j?W6d9$&HDYEUb1U7qA2phuoE26W$cj%P54f2we zf{~I3g>mx6#Tc$-5kdhKS|tU&Y2=pbj}`9dRH3j>RIje z^`~eauxyW8Jpg1hn$eGcLL4=n8^iZDiQsjVuHzVy%C$T(Mi?ctJggoBABh1|ZW7{(a~|z2ayXgVxj_TIhQeS& z3_9bMCM;!yW=_;tK(&S`fD#BeG;QaUMB&WGT)zY~RG0+ll@XsKsHHfJl+a>g1dUQq zBMjj!6MNlG)K>2_!A!a)@F>Fu!o#7tASVFsp&?EipxL0bfBXd=fO;I;zCxB*Zn(My z)Wg>tq^op*usF#o+G&TZdL1&1kq0M~b10%cTpGF|>p9Q*PQ_m=$UC_^F&N2NyCX3e z1xl%7Fs?HOV}+o#tr!e>TQL~gzOJ=^Lh2ZddlrKsSkxvl7%NHl7Gf~umfTfz_;)QB zbqL07gkY>Ifqxc}7i%jNI}&#h>$y3{d#2;|iwg33@fNX$n-2~U+HF6t6|YyboU69^ zp~Y%#Kc@^Ag8**RTc1c_E4}r#QdujN*Qr$2e$@Wu4O+EFbz2_Qj34ftM>S+0!qO<> zN44j)_MFz9(|hANT`IHfy5}@85!@rssob^<*_4a{Yg;N~?Jn&~13I9c0fom(b()Jg zQ^;CSw2m&R?b>r%lw@sOQ{R8XL*k#|6+yP^ewf#Prot|xc!W}82MJt z|ByY*L&quY`Pvp=AD z?Hp@mz6(Tk%oqF!<^@XO>2Wfb6Pyd$JCYf{N@om@@GLhmR+^;!Re4umxJgk|V6s?C z)0@Ej8z_F3u!x}G2lG)C0~FbSC1(PM2)y8RK0yz}I^$}$g)|rk&N0;U45Xfi&} zhj|b>^!QO?MOe)q!;D9sg==Cu5wFF1boln9K@N^y;RjKGj{;D~sTGVE3*%y^XUuV$ zqNV^2<_vm=z9Wz|8XOO12)KlZ<`rhThNDeb} z8(5JaqbNCkpi}CApa-iCD?85p3HxCZxiLQ?1`E$XG)`Ay%CiaEQ(lFZg=UL9!AGJX z2DN5Luoi){752zD8*t#^RTnG=`{3~CRf8NIy?V&!Y5T+h@dz4BN1&(2)DkcyfiKlu@&JVf!DYw z3Fu_%@Etc5HEUr7Qcj<73ZD&ONpN9tsiw)k6Q7kSHbnv_1u-JeK;f2QHDNpokb1;U zf$0%GGthpySR&Rjk&YI15wCyJRf`;>k%QyY0kXn-4BQX^ZF8KBa0VI^WI7}yq8asV z+}HsDF{=rv#E#3oAeMs(bWmUsG!Sb+mwyXO!M|+=FhrJ2Dp>BBmzE_}co}{AH|jT! z;Vd3Fm|pV-7R0ec-$9S!su}c-(JJ^n$@WgD9 z^ESsOi+;h_v7Qq=wNvE4m24FKDr?7jZgF9#$bpWH6#blOW6|CCYTA}Zu1z2Km~16DsR z!OlI)1IJHR2ZDV1*O_OKPyZV7^pDRX!>6;k!RMdmyEy*)pYdz>5tb^eEDsQQisv2X8hye<%NGizL3mL&0 zxr9v&|D3Pk*oMR9oILpU@cV~E>8Zp7bu?CT(Y`{sNe2Z22j|5Ba~!b&LJVRZJH}Iv z1RlVO(joLA@NZxSL|BpwHpLq$MY;ewC@Ii3R{OhcRi&`g71 zP_l=?%u&k#c7;`1{lldH2U?em9^!X^q#TZ7?7}!GU{uiy}YVXZZ4q^F4A?aMk%`T~q_GqLAz z<5dnj0FZMW6t?&RfFS1i$w!ogXI>LWDyX$ZUV8;oj%5$xgEvSGbvVwU??uzptP?OF zO=4)twRye*J{2z%ZV(^I9>57kU?ua1?vokKO$hT+#`aQE*xD!{*c@{^Pu|iLZ|dC8 zY@fV1(52YzLO;Y7sMJiyZDBz)gwN2JOvCh`$o9_l6KynY#@r4@&-c$Vs%a1dDAC5; zsN7x#eNLy0`g{pFeF|0`^FZpr$I+DAj&qm^Si(5~HOp8_i#$QrBn~K$W%(?S7(n1` zq?~*SCf7LQaJ=z`tKcom>#*!>GLN)6?e2v3BqIKfg!V{v`i`3l?OCDU-L=pj4M6Ix zwqA$!+_%sk!J;-u-?5r>Zy~ftYbi=wQMGm*+jD1Qdu~;S-;wB_*hZBPsma@?`eh5( zdQm`$bt(rowM_MkX07!Ce==)S4rJL9)vvZ|-JLuc(zZwgVwW+5z_0*=G zJ7(&s!?F6?^<>rYSKF#%vb$M#(s)#wm>dfus(4+;U)Awfb^O)6iN9K^8SZ-gRbuhG zPw`hwHhQ`)OQuR?)}ft2r)Le6UYoYH_MpS6j13z0IW~{>V5jv z{`e>IJ;>BnTu}BcJuMeC$sUzLnebv7+NA`5i?!miUeS+#L9IoeAlyL#V1kHgfgOA8-cmlFJU>Ae_Q7H{GOkHC{RK73N46R;yB^gj{xE9Lr7KB4a zfKqrCh$hSu8jrA{@Jz;Z2)=C_wB&?S^K&&4A?AfAg?bUe_?l1~vO zjF-RU*LMH_Y-M+dS_tYNV#CBXiZH&}b9ypJS_eVgw;%|?qc(|xSV_CL5Cx&MB(;q*t&5o4)gXu~i|soS$PjCI z#qfXy+FgIK+PGePL#*TFI?L;K{j!3%C?!cjOuS>MLwf6mD%AR0t-sw({jGLY_BS&j z)vm~GrGn_~tSd5~>w)jtpBLwXNZ!}Z$l4iMJ0tInGjgfyxAV@(MDxB!-bh7lTWzar z%1lz#cGu7;>nPM5FlNZIVVLXH23ZS=tVr!%uTBS1>5Y6v!_YQ?GCa`e0v>i+@2Vo^82hv@N=2+AUeqYq!ijbIWWj6D@Y&#LVuN+%g@xBk8(cyJhaGTPBlh zK8xb<;-p;1l_bQi!b}UI(REjaPX+XXb0-td`&B~z-MM0#$Sf=UHYLIOD+-JEXaniG zHm>)#q<^ijy2Cdm6utv^qa$D^wm$#}TKfk9&^&j9E z6A5EpGpl4yPoLyX>EeUc*Lc2n*+uiM%`69V(ST5E&gEF<5o7Z`WM8g-nddoP_QvoZ z9z27EPzLFfSSJo1k#9YR7il3==5t#T`q|^xqVC5roO*CR@FGi3sUwnQ9Ha7VQr~(+ zo|UQ`!7(Ga_+le}(Oi$ALGMJ3mg1OS1JLZrt9Z;9ca!${O7wXo`a}?ixk4~UxjETb zT!*W$Ha~_BdC{r!>N`RwJaWm0qnL@bF?NuZ@JYg!K1>}CD-x06>OkVK*F#I>P# zi6@FFa$@5L=4CN3ya)$gE)~VYta3BKL8N2Hkg}v?RtwalI7-c6E*sb)rOaT-;>U%4 zFcXC+gM)H~Jw(O6k*}R2GH8H=0)k7DM5a8>c(KbxjsjxPKvXo!W>LV9jhL{afNPhz z8g>E_wUcx`b7<#MjELbX6wO;ezZ{cEh6=<52Djm2QYS6im5Y-r6&qWKc3gG99TvM` zK*=@;R=A943i2#^{tp2eI-Ze!agHdNAhlo?6&^&`B3vO3o%6=Ru_MVCn79Ejf)Q+3 zI?(1JRX8girbprYf`e6L*x+O0=`>heMTwG8vcc&TPzTLL$X(^4Ab#-kxMC9j0$Pcp zP}Di+#hkKoV`<4UUZ~8ChmK@nf~0L@0WR1%95DzPt9)^kF6bR5$0Np)oGwTufJek_ z9ErG)r12DPkpOl;iN9AOOfw!ItZQr7YGnKiwqkf*!NMA%gqL+1q1eXJCK8)~z{E3< zAW4F_1kD?9W*&P&{qvO^BDb?rF5&S^EoT_LX)#oW(k&%+W4dn)+}6vtF<{Y+UxDf- z{y;%K_6Rii8+#CMZQHE?Ed$kK~T#Iv(Kf1T2u@|QPdMj+SheBvi_3<|7t5VyJ+_x2`*IbZ9&g|MR+HBX&8J<{Mb3pT!)_#S- zcG1n+#oB54n(bB6bk}yt+Aet&yQB!Zsg0AhadKyklL-l5n}xEDGwa{7>#)8RUMHx& z{afLej3pC+ZgnWxr9;VL%Zq26+Nq)9>&=bl#h9Q(XkRJO-~IsABWy4EtPJ`J(O} zyF7HQV%YW!YZ`;tZ9AuBj1-<0Mvh8nLp+(Qt|}2j_ukF@;0it4f8fya2qBD zfN7Rv2>Bk{^F1C=*`fh6Y9-FeHo3BYMz|VaE}#6n?B0J7HFSNn) zXP1zjJu1M(Pt3pnnHFHHOYda4bm&kL_|TIJKPP_ph>WRk*_hR2g0?kyjLb2B(=0OW zQ~HQ7V2dbkpVI5joA&PGFBkygV8v&bdGOa5wzKOE$psCNQ^CkhN^BAZ(^K~8SC`iY z2NIn*rxY4OgNP?Mw?F98VoPtWL%D0z_#msd5}d7W_`oH)K;x7w=;^ zS? z5O+cNQ(I){Sy*is5cX7zy+IsrOflP92x^BPIjluqT<}8dCP+tP2EC0Vv?@{{4A36{ zGh-Iw@^C-g$3Ic%&j7y6KK&a5&^SIe!P9tvUjv=dF{mQgcFY<%x&lxx{rC$Yaq6>+ zv;!D)gM^ICeE4euUrbJ6J-kNJxJU;+!~M;_Y>}U`ad-qk$%=&Fh-VR1>9%DuiWkSe zu#qo}TI2+ctIPuWLu!IA8btpRKZwTB6rl9l3W+8@Y@c`;#uTFz1XH1+n_ind=t^Hc zY=CCvh12NMuOSX#9DV$gO_UCKpm3-UGY}-eaka=hLANF@8HwBocnPjQZx`Amq^Bgp zNz*a*qex#iVCBAprHRSGhrR0~&d0uC^CrCpQNF~1M8v~QWlsc+jp>A@Yfb_>@ksYe z^5CogrPuB}BFfiVgY?KFGW}X=5J`R5B5wj9Fw5gqIO15uf!PkAL?I74?FQ+QlNw> zvMRACfQpDb(7uGxr?Y|4={TFmI2-vP?@v<-_#fDY4-kNGJdUDM7LIL#I3L-ZHx`~7 z?1EqlCrTdt@OC;+ zCsc3*T=raNM~6@HT?s_q} z=6k>%haT$1j`!(THUx$=@+?YTAOA!iSOB9iiZ(9XS5qenCNA{%h;N*p;4T#Y#1C-W zK+}%gAabE8Jm|pB@M|?DIx$)t?D5oda-(&TA*4h07L7O_T6!EjLb}x21Q|iR*UIZ9 z%Rqfvyk{-a(Z~cy6c!#%8W#y#MrxJ|*u_zHlTFg;!f<~OJPO8liu}VC-D7ur8cQ@Pz#G-#?7n>>(J zvF#oqqpc?nzS9dUx(-B!+}t zxtpO~E{|zlU~bb(AFHfL+sb6BZ|ly7yst>DuGXvXeB84;AA&_~P+x8}>E1#jQ?=F8 zfLeO`VN+fwooJoO^yb}j*J`HtERv(%fYoJ3E)~U2gWLt6)M3ys_!`!`Qj~ZK^6*kG zLBH%}SOQbcU?x6-9L&@|&@Va{mH^ap_nuP6Kn`T;6zI>n6_!BN3)iz^mq31?!X3~r zITMyZY-bQN`+pu{h3&t;*rTwdr%u*W;sMC@le7N!%kG0EK)V@GVfxQsEz6nx`z5cz z5|F(NB)9tK7cH~+_w&AjCD8b(mW)uC`*Q$eQ-442B3K42x05paej$%>X4~&qxCoZP z&TXsAwx5F?+x7b;ufQ^pxm^|8^z-mCdw##<4!AqnmJ`it8IB2Wd7G~@W$RZFnX7p&l$ecf!|r%cOKvO(EeTK58MX+ z)ljU;Et|pX8=?KH1um?(&^hvF)e=-)abWI7r?#^)p`Ej2D-h17)VVR)X%CxHt+NP=NZDw>?*BHop9F@DW7QtFjWL@j( z-MSV*MZEhl8%M&z`;5+o)Mk(JajD{yHDcar6ripUn>JtDW3OC1IQOg=Q=fiW>ZT&H z_S63WDWySPf#O?=yUrfvf&cXPKhJ0(+<19NEpWFfq09&A-V7d*k=+DMC;NNAK=lE= zh|3lB6a*%0#Nbcj10Q-s495f23$*f(#-ZA46)iXzQ6Ih$_9jzY?#A>$s?sq!sx**Qrl|(&GNv%i7<6^D)7E=E z^Kx0aZ;V^S?2o6v%JIb_yI5nT_*{ZfjH2fhlfs>&2l#u;7{c}X^KE7Fwmj+!5|vHa z(yW2r1JaZ&+LqL=K9CGU?of^Pm{cawZ{L3W^ONs^5RWhwhr_+u8L`Gn8+Tl1*B%Sw`}-XV2&bmjrM&@c0=3=I6|)CaKad7$ zr%TJTFAzf|^BVak9k&9Ie)+vny(1~A7*b_}f$gw8wWGGx^@AO6n0&Uv>9&w-ygs__ z*3tDDWEfwi%-zKd<65WH(duB+dlqcEv7KRYT&iG7_dUV*9Z^?zB`T1X1DSG0m43-4 zvtDFiVoE7Q8Wznc{Ys0>dO?D*EhPs%v7_|!2ADjsDjV(EwcKyez)*W%`&+ef)!B8o z<%`|j>hEb<%W7wA#p^n^?tSo%u9aJNwF#`~9ZJls+RfI#$2v#wIWg!%?9=^>qt9h80+d!!wq+jg=i9^45i_M9m z(w1@wt8_-jn;64$I`Bdz@%F4ewF%~$e1Mq^<__s8c@_VejrEP*VcOgvGDi_i=`8Xs zMj~ER$eRGj28kwwUt1mJ?)_kYE_pW@faVEreZ(m?`u0Wh6%6>|EFO}DQ*z;NW<&2L@4k6L-3tuE_bCSls|Okvf_ut5gPNl-60-=HW5D|( znCJu)AAbNNxWIByur=KlQB-mqMqHeCUBrysZ7LG^j*hUZkrOrue1d7>~eIc)m0^vYHFt=n<_Q+wjrB}W+|Zk%AHQH zGP+Y!l#aR}oAF9`b#UigaOaDm54}+T<1bv}`i=$@+Z~r;Fbhxffs)DB=6urEifGS| ze~z3}yeEBvH^YiuWRD8@Z~rbnDaLm$3&k`U7Pbi1`%L0MT zO1VbLHu*pAOJ!LX`yfPDuSX$AnyYKWBthw_5|1+e5q+MXkI}G#1|OwR8%;k zVMxs}a6jPE6Y63u&up5||HOl19wfgBZ~YuGjTdR*bfRyahFe~Bio95d^_@{n1scijzAWhQp*-Fa?MG;{VNuF zDD}Z3l-+E95hPHTHM3T5AT?1GU82cE;?FAhL7GU6}Z zL=Kx!5Q-f{6ny)#RcK2NM*ex=!TaQc78%0oQ!KUWQpD9c$Sx20(o*%584kEipz6#xSLDp<*s9Yf1wx@+`xU?Wf3Tq(kJ& z7|x{Zfo`ZlG~G2(TVqaKS_& zffMN5Ca>x69Mgk<&_G4MJC$d#qH@sg#Hz)z&Q64?l_>8P6E5C<{I=PZwa%ce>!zjm zm}b&8>oHxHO>3aY1M8CU+v`duYpu?e1E%j}xbnT{2#PLub?siVge`<4@AT5%V`l$Z z#3ipVKHRw^xXQi+`eL_Ef_vZMk~t&UBnR$F%DvxV$*bk69f=sutaP~$+&K$fe}Tns zy%6HqOqWkmnVRVOMf+aSg_|m({c0T)TIbpnzsRem%i7IrtmW2oj)CkXTN!U$Tli`V zUv1&5Eqpg&;j8Z(^s5~?^~Hg$AUA*P?cBuyGgI@YbczU#<|aZh2j zska!dk%+{LMdeLJ)H*uyRy4so#3t*Y_@#kiEB4Rgpm?o}qIvW3=M^3=ybs$AkI%!r zTzGt~qkrBydRbRWj?y*kTs7ycIp>XV&e$fuO~#o6c`3eG$F}cIZ2O9hxa+a)3rKl~ zL)$gIqiF-R&Dy5iW{PSJdfg7QRLzjAj-d?JOI*Ably0}JOPxBly&|?94uAX8uW@}d zJ1j+wWwa z7R$;;WsCYbg3Ljp^?5(n|Lr10cIS~4Myvw<$m~kATDx2;e&~D%BwTERFtk7W04mEdzOtw zQ4uWKGWC4u7w9aUd09ozp{Ud(eihdx$%F5o9y}y(G2;+cO7iK2@NB}QB z{(>cQ6iJp^#PWy=kFkWB?V9|lmee$Bkr%wU7cY%v6X0KPT7r=6Cgr{;mi7t8(G-gj zA$*N=I=yUu@H7P8j=fJkWPM{S#{)B*#&fnWKnJ>D*;|1Dwh{wR51KTdBo+_5fUOIA z6;u?_HH<1=Ad|Mq_p1CQt?G$G`A~K)aypMWBsfr7H5ZXre|tXwo;NShHabd$Z5dkR+92+}yG7aK)F=9Gce9E*f?;^^r0w=3nGo@FBzW_AL zA@wtw`j7weSJEk99g&$d7qkkD01G+6&jR|>p#o9=$A9_1h=xVPO0NL@V#{QP(0EtECZ;z4^e|I@ELPZ@83*~vmI`DIcc zmSsYGc7b)(io=^*Tbzn2?F><@zvZyWGCH_cGB{eW_*h<#7nlTe1D1e}Bj)2-Qy_Uh zbPrf9Fo|&mtt?&s(inL?J$qJ37++@t{)vi8n{>QS-po8~6z0lvPT=Il(*vu>@gj4g z-s2dk9m`zKQwL|%4>ECs;aEI3*P_pB9_EA&e(Z921R6)_{=u=pu_caI!F=FPWKg$$+6HzL|1->{Ki>7|(An!?=&`TNC5rwhAi2kIq&(yVD_N72y|&iVF>U^i?14Kz$tHf=N2Qo4#kEmby+Hd89F0HO|F2VMbK zALyz=)q$xEWJWbf9jG0tBLSh&4%OAUnjMR_+NzGa)aa~#yV|~{gW$CO zJ^x$8^OneSml*5yX{TE-vt9#DUFUpn$J`2-DSf;ES2x6gT`webMs0^04%;#h>K} zr0IsG(6%ngy@Azb?T#kXj;^ac-JCaXymQ6%(zm(G5SwZY3#CSGA8k!X(|-&`-r3hC za8Nq_%C_E%-`=0H3w!)^1(*eXgj@^L^I40Ut!TVw*}>GI=X+gQlX{?vTtb!Irq!ZS zrMasX#kwqL?c};M?xW8jH_)m$&bwE$RNGh7R!6Rr1KqdeKpV#xFHR&RSm6E74s^ND z*X*Wf86|uhabNUF1zzUg-eKfi!bQF#xs>AI@LZmwLST5m99q6!_M|u}JO?$42=ABT z$yby~O%BH8NW{go3wx>I4&d_dP6Db4%X)nMr&f|iBG7F*uIkm>R|8+8;MjLW_anx6#joS^VE{*h-VyY9yzBH`*t05x!8X~LftYiJ_ zyl=N6ciG6Fl||lIIjobHm8za+!tc%{FRQDb*V)7tW)oW-$X>i4Keh9B{w>cgp=61v zcOFHyg+#o7%JXmgODw~W_n_sXj158KG19e*($`YJ!sfU$|bvd4#-uvyZEy2YZ!TCpb%D}NGo*};ORmJdDUmjZscyaJu zC1+h3+Lv?xF7v$Zgnp1+`y#O+V6#hPTNU;(?yE5WCkriE*LMz;$1uI_Luid~!YeZQ zwR^Z6XR1@-F3rHXBH-d~?}pou%GTcyYfDqNyK}BzI@#LZtc_H=dT!m+PK_^R_1C|U z^>{D&W@|>RJ7!1P?d`V>8L-|Po7xs`Lt|g8To$~66FPivm!11--AKBMYcRXOlz0Mn zV)tCXS+qWf-aaEYWIReNl8E|JT42yXSDE~aZCNkG;{iD0&> zs(!}q1kCFnvJc@p=X2V2*@?cGq1=o_aSg2FT11xHyA!wRd3<{p+pc!YDlug*xv*4M z0@|HQKr5Jgdi^^KGQ5QV!Rsnoyi7={n0bwl8Xqh0v2&HPE~PN+av`mrY<6_r=$a~} z)HFJ>Y*JHUrlGY>%`htqX?0Xh?upDX^Y`6X0=#S?El$bzw35~lI~{v7E?;0dbs?>{-YIL4XLMrwMY!3cJX~yEETA<9F*n?oHeR$V!p%KfJz#Yn{P&bA zMJ{cSALtx5vPXH~={I@U+S1xzp5fa89g`WTUzA{ZSs%lpv)f3ZL?Qu>@$%Lq;xbPJ zr&AKJX+W2i_cFc64UvcPF4LMA-@Yx^!$>o2&KCuHGxh1rqWs#^#2))tF4plr{W@mx ztv6O4Il(J$7>7N9pWft@1?=N5rx6)@7NN-NLl4>pV|$c_-P7t4V&4Jd2g;`gUJc|H zx(Qk~`3Qzsp_dN<&X1&y_l!Rb=wu2oh!p_>d8u9a*Y8x5CURCM+`3$8lH3d=~tw*Y~9mK36-(_ z@k4t?@qR6j^N+~KUu@SViY!&)AH)8~9$pK;H3EY19NWk0$H|Z)7_%LO-YEgn0+Iwm z#A}H5g!(SV{eSuu$c5o;f*=HvXT0$+yA1#%J)S{papiNuKK>;ls@j4vtL+xX`di7x zT0Z~ig5eFC%3Xo?6DR{L<3$bVivJcSq=*+B@MeH(QQyK#orM*#5u-sEO-&d}i@YNq z-e(Bl;e{Pv0tG0PU*kkE1tbW5kJE|oU&Qxnp|8+3I3J!wE%Lp3;c;FZP5wUW#9!F8Dg~ft)Tpe@w+1!_QJb`{WpQ2vDi%nPwCaI)qgL zX^}D_E98ORARYNjtbmQ;mv4g{LkS$lHj#Dmpwl3-_9b4XJpv>$_#6%w&OibJA4?6= z)4zNO1j!KsG|vQ1e0t&7a4sO-iv{^#?U@(4fDH>!gSS~gwGLSrA?4V~9)s5cwKj=9 z{(={|`JGUHnHZMEXBU(}=bZFq4MyJnk~9e3@=4YPAjk?}O#hNR_@MAh$Et#tO23BR z^!7vS?mJv*+l(+>1Ktgv;gSooWXKz`OG%ty+cJ58Sq~&cgzL?KHa`A^xzHGI@1$eL zEu^c^6fml&h_j1yRLLD7#s>u>>2#7Ysa?Fq4Wc^V6&|d7oH7DbXQV#ZhvL#~zGoz- zV&4gTzz!dDk63upJ($=ozfV7*ANENHev>s!f{+_o7x1#Eaql5{aPhU=AX+k9_#7{t zlTW{L&hJ$X3*E@+fU;>IJK0S{W+x0FHD)081u&prBKUS9EZ4ypq&2_- z-jauA-Xy?rXH&BOfA-#OM~);*5PR2G7z&sciHVH#Yh0479AuGM#jGk`z+_eTl)6Ef zyIVwfgok^yUxLisZJ>8%wY`F>ushJ}?4nD0(UUM5?Lea$O+S%u4GrrT`b%_ZICtK^TA zNJqPtx^P}LL1m3>g78Tn-n)hUer(e*pxc+i*pz=2d{O>5IGgkNZ$5%Fg-b-^jnWiq z2U6=mR$fWkk)|F=t)?!wwwI+))0&c|*I5ehSe8OTqUNU_TwAMC0ySndK!c_*y1G4} z_=f2MJBm9EFoJYdT^y>rHo7?1~%4(=Ah>EpvPrBY5}IuQ06uf zm6KKJkt()MG@cjGmVZ0X$f~T7U9;Bw)tbLr^H*#BYRzA*`KvX5vzos;t>p0)O}TaE z$n9)Hp0V4z%#pFP{G7y*vv0SXH_{HcVl_<`8UMr=wrdHMX*`DPo0&Ip1`N}==yAP# z@@eVm#%U_!Z|Xdi8|SI4{eE}V?{|?%x_5qzl{Trn<+*5?E%?MzdWKJ&``%H<#iyPXxu^;AM_$>vFnN@B)8pSS@mKr` zcsEppJ#}C^+#=N6vx4^@vU^3p!V0&=Z=*!V_9f^4O0e?}^T0*372FoR5eNRQ{r<;U zuSG0wQ|Y(}txB-93fM4)cRw7&%l$Pgdt9;#z|h3}|S1 zFDUJ0;c>nb(J&9aH4FW5G~^5O{$H3gEaS;{fBZgsng@-oN(<9o=+6nP8F2fk^9JPa zxp2ijAQKC?|V*DJx&Z39r3!wdD zD!?N9?ztkbFy~$@hRW7}5!yo~`v3s?&@);zt;mF{1cw2iP8q837>_q1K=qsuj_M#9 zQ#XhyO34}W)c|1HV-ZMg2hcjQF31}ikkBKE-Dy%7T7&ETK^DgAhMNV(a5`aB03%%? zc$!}za_>*S{a>FwzfXb-7v2XppohVFp3itlRB2XNKLk~k@C-%~qM8c3g!i6k3_odD zu?pqY2N9Q`ZpjVUuerGn#|Z2w{}^Tv=ax*3AbgO-q!F@2z(sJWBdl27n|6@dt{==J zFS$>M7i0hac*G>P?p^eVK{$@>j)Hif^Lz_P102y(Zb4kLFXM~1k$v$9xk&zr9Xw@z zpIK*a_5u9LR~zMlM6X3=d!F@=wl!5TOrs++$)v3ggVmASnxQI&k^l}0Ff>T+I zxCPszZu7T5JCMZ|r1=>|Fa$<#_n}X|$EKi0pG6=GV=ox}P080^KN2lxYX+}TomG}Q ztwTl9rGu6(%LlT05Xc9#)44+FU>W{%IBd#VTR)Jx?YB*wYEzPBRc*d4!+jAMFdcIC zxLm|o>F6z0>(2T95#J_fw%v=Tk5o7~sygqEU!mnY0HyEDO>MFX@ znk`^{U7Jujs?_Q+1IFJmEK^rzHsv((0RSnU`=z}wi*_WcTDqZX znqg{EPc=+Q>$P;FO;x>J?qvr2uX2oU!cN7>hQy9NBA@-G&_~Q?rV9yX8bUiLe!ldg znBVVvj&&t}^J4Nhohlj-Kw!XHfF2YuuBEgjqu1*wjCG}%iGTL=5qUA7{+OC*n7H5^ z%qw7+#LqcAbuKb_K7Cu8!@kO?xkmo#SBqK0ZY6hKOhU}DwML%=eT=a=Ng9RXV-V52 zF|!0)?=($sdF35A&LBEBsZNsPj z1^FfbdE^8wc9=G88IEl)?xmQ|%Cwl?H4M|V%r=vlqA*>fJ=1~(?{+#yXXY$p{U}rh zy%<~sc7{uzM=n>zAg?#ZGO+zsNYX-{X3yx-a{g=;xpf(FT7V*@Dk}SnBEYx;iD|l~ zVK-f8+;r_hv+qq0Gg)Ir)VKz2rV>w95RP!fX}XhwX68094k(&(psu8>T7q_>Pv(%gwI3IK;#bQ#k@~aeAm)78k7wNBSIjw)da~z7 zns~7vm%QK?z|?c;F<1#)(RclL#{GQ(*j8?PGQag4@Wf~Rc*g5}0n~N|HS<`{0ZqNt zk4xU}d6-wpgi*VFZ|3%0M@JuTDKKJ5lr%$KTww`M-99?*)zicb5Dp zsn?ma_TOZ^74axwVFh)F)%i(z&-~@}Gbem7q+xKxdE=k;!w2Wl7_RgSd~sNOlYTqF zUpK})q}>J)?-=gsY;f_j?3?hyVb(!xp(On0r$763ZVpj zX4djns{(LEr?u5UuHd^?1!#LNY*(bRD{jq}DDDP(x}~1Cf(+L-mX)z+OC#7e`fYeO zCQR-MYWT%k^&Qf1%`-g4aU@k7?((RV!ws(0Z=QYi?D@~jH*3ig^LiNmHM}fIqH|r# ztFyjl%h&9-oKsi7iHkm}Rqd|os#*nexyR}iF2wCF;|^_CHY?JggWr)zNat>RG|?mh@q%zjY!Mrg!~ZkAwPtE8cHa8m%}guhwX- zBz7)~61}46v>N!$2-EF(b}jV5+DeA5%n9pb*A&fBT6b%u^`aw=NkBvPHcVKy6;v2$ zlLD`|s#w*Fj?^+zi&(QGKg9XwJC>@Ff4V!O7SJj|Ox|N7?G<(s2C zhRJo{c${wMm?vy{c$oZLU=o~X=DN~RShAV1^xbNQS?`nS=Q-eixa%9T~~*z ztHa$!b+{z@{SH-!({w+4l`xlFnVfRGdgV$07cfOQgt|4lGLsya?8#{ z{4a-Cmxr4v5BI`h?|vA04RY4^{9@g&^hv(v7kqF2@%y+KSXv=&QPD46b~&)g1K`Q* zA$|V`3V&W&KAi?3uyr;GA}kEQdQq=e37Cjn8ouogDHd>!NaV7BSBl08_M#rVPcJ#bi_p2dD!?spwCRPuctKdrXXA= zoj0JbNfahMHn4DdQu!VRX<#XO5?%VU@Mr4H6l$GKqfZj4O93fT4#$O^Ak9|e;k zgU47e+q3#urW=c=hc2(4KJs9Z;DBJ!WD-naqCsTA(CiU(>{-OVgrzY&$F*UT85ux3 z4Er2=?|x?!%OlAvuOIIMZIZ8`oj0YUDLhR64ebdT!RZe|A1Es^{7bBW-9tA*m`yw` z5;7R>$e~z9oEO8khRpFqb#0fAZ;U$v;~n$SPY`Qi+D-C``J!t1e@$@YPrv;yxP-i# zJIU%FQP7hi(t5p9WN{KF8N2W z1ooIu+wg`ifSY*EFv1$~QyQ-|mL&J>@epW6Ho8kX!2vsbfg*Aep&JxHw29p9s!h`O zV7~!19s{-nJc1X;F}2_S$Otk74&~w!FXcYW;5Nt@#uO2Eh@*<3!ZZ(HJBGn1{Sb!Y z3>h3updRGj$*c4G4e}LqWzG8rIx}GgjNE%9zJ5ww7(WIU&jtGwJ_P(Z9b+9W*SsL- za9jEqUU%kN>@B%>dVY2v7ntD!!+;$)UUWERh_w;mgh_|EvVcodZ^(xLCoP?8lYACg z5q=wtJ$|eLzO9xAXb?;pf$_cn5#GaTVfN)%)Sri6bQIdtxQzP@>4-NROWF4U!|^Oz zqFy}UHQ)@MbY5T{u#uk-Kz3Uk4eGN$Q$i^&G{8HcxRJy|eB>+)z zvH@NVgLHJ_9sw>42M6c7kLYg!xqbzsftdg$jPOY$XcBDIC50J710V|VD&fDr9q`Cu zmM%6TdjM1jGud#2%8(QR!LD;+SUe4jYc+OuyjCOJ+30a2B8B%UpU7k$DtC>JqR7T(g&MEQu&%baS1w1EDTZ5Z zRqN5Vr1n}ZRi#F!+hWW#tRA&G-HNosk9`cj29&Ub!UmXnRK{MSGS(jQz-$u^A+ni0 zI)ZT`$9q0yK=^^a!1aYoBW|7ORtuOdD{vL7_Br7R;Ioih&;qXHpL!-W4$s_2Jip#r zWsJ5$q;_ZfD(DzUkWGMqGz|Vcat-%r;2+774*!$7s=Psghufd!3OQ+EvZd^^bEdyA zZunpw0t7dkAw!NlGfc5WQL#oJdGhPd-X^~u7gbj!IjZ=s zPI`TplU}cB)+u_`)q+PYc$5T>@rQcx2L+R?0LH;pQ;d=j50q7kLEZr3 zR*X|y-2r7@FxxUuj4Ph8)xd9b+Uj_$y^6npb$NI`;AjFEG+RxDLN{c~Rxo7#XENFK5 z$M0EOI5Z!c)hPGyG8&`JYSbX#Iy8(Uxze|}XLJ~f59_689E3;~OwAmiMpi_7mhc&M^iL_~{7k##u z&ydlg4f53XOB;|r$wL;O=V7)M#@vlO-*c&t%5mGZ;BY)3SPp57Y9zR(5S48JWE1Kb z(b#rZ559XL43U&j7gbwA?_>jOIi!8A6yguKhRYI4q7lfKz76EcM-wP2o6xa=OxYz@ z9F07KIw4me#e?BW8N+i}z!fn88gj?(kl&*O79w;JXla7gk$opH$cvC?D8kGMmK_h$ zByh{Xoz1x?bj1jmAB~d%9GHAe!~oE?+sAL|1t#B|zzIfn3zcaTPI#~_H6xzflaJ^k z84#NjFZjrIhDYQZf_ReY0=bbD{&G=ohOb0Tvr9_6aOB&<*65?DDc(m$#lXlX&4v^Tsdkrv_PMH63mkB*7TRgXVV8OO)>=9w5B5 zFVX5g=%8mP;jRQjPxsO(x42doY+8}X}N$7hM&xO|Htgv{zF`QGl zaj8Fop(g7Gh?3itNkR!cBN|NMWC&*pmrdYGw=ad+)r~^xY%+ORf?Zn(+p~ z3{@~xd?OYJ1|4GAJpt&_^G7g%m+TVmRp{)9d|QPC5lI)v8^)`{0fn~MCa%-v>SQ`} zz1X7zwj#et5lOcp2jt%5q0}IfB;Rl5`nJ=KlkW-$HuA75JSj1fpq|)1Lca+#XSn$@g^L3FqQhCTJi|scIJeygLly$L znq~-fnbM{xn}TTF>2kLYz|9`Q$2Rb^)yiW$Iu{9)$KA|z1*-rHRK)EZY7@JB44|=3 z^9Wyo>f}2rTsG+FDA@+|tD(?(L@&{6Bc*Tf1E!P^<(MY33d#?(44o<_2qEGQ9n^?Q z^R8T_@(q(5IG`E8PxLwQ;3$KB4=`mxO6HK^g|UNDE9VoBb%lJ}h(f_D|MHVl2Ms@{ zpL*p@IEVH)g0-B&dcFI83cbJo5v@U@gI$a~tjK}8Kk*fbu(v1S6$Mn#YCCu3$i~|X9{iZeg)FxeNpIarda;Et zyS?7qJAD;=43TzAbhEn{X(#K=c1x`z?e18loggLa1lldd-MbxVx2qi8M-jz$n;PXk zi49ClRJrKDqIv3g*2qL)HKH{YG@uF z<{MQy5F|X?($LV_qEcH_YKzKVT2y}Kgb!+?N^MlBjVf0*s??F9$Lo5%EDsqyV7ziw ze!a4%b=2smxv0^MUE8~;Q8UxTVo)|OFOeTo)%!Yn^rML$U2c~vMUSp2%cO-;77CiR z#8XQ=Zuyc=&|?MF$9(BM{%fhm;MSDIwFdXbHMn(D=aTT2g%rK6iM@Jcr*IP3a%5*7 z=w^|fJH)jnBzUpbygsq@iM^2%o7ii&d0_J}uU3+9p-rWZAl;`3(uD_h-y=xpVf-Ek zkhUbLE49_G)KO%mZMNiYtJ_g5L(}wLtJ~7%RrHf3MXQb=T@G>62vW|>-_a1#uZNN0 zfh-rsvfN{kJBQ3EMKKnh=3@=>&1a%X7X=;mY?O*nJ^^6{aMcbYldI;RJo~JH%J{K? zlI{B+VYM-bmC18y1rATIWk3&XQ5d4>MQ(_~6z?^4I ziIOy)AfCr*JiTFcL}3hqIxzsAegQo>;KLLO`f#M=O_@jWGdFTez)evnJ&MS>}&kG6QeQk;l4n%aKEGg0=5%RIw&=%pKx2w2J} zHfqe-s3zQ(aS!3t%kr595vJI3Jsev=4Bs2_e4f5%IBd-GZYH_1Fyj^`Dur)73p0W7 zte5!?F#{K$PPBRy$adm5L`TlW=aFl1-%m~gwjSWI&&4wYA-;<)z3@pN-n)fue{9n+ zVBMD<5Uy``ZYie+va}pXI*!*AdBY6O3T?9i%rtJU()Adc2W)w# zcTPpW7OVJCcs{So`R<9gawZ4mTwZ5npyPQ$RnbDmMU>q9P&%nfI$kj=(RiM2IRADN zbxGcASj!%@vlKbLy0l(r+^>D7ZZYG&9dMB@b(D&K;tTXW;lkQb2y59lGjGVlL^06g zih<Oyh{ce#cVN?wq>o=rk$3msaC72OIEAL zT1tztN(0TTNWBUaS=JS~Bh?0)N&^iRhkE~GgPgi0qfGLkV2{DCgFk-1x*;ay)i!|P z1x};S>vI6_W<>ra`~#db=Kr=Y?a_5xTh61jJ_tHaThS$Z_d6O_2NMGv!wBF10dtIF ziH>0E57Ll}A?0HQaf1(*R-0oUdE*InFL+5!5XmQe`4geE7ljdQ5*br}#6r^ZI4?pb zkq;LkLs1oOZQy#{J~6#9MhJu#&|5H?W3>4OD}s%A`L71Sa1$(q!V3t-#qyGQf#8(P zi{^Y?(fu|*rEF2K-dR9Fpdq0>X1r2Z(!yo-<#fmb;Rk5qijJa*D~hEh)mKtLNvV~> z=A-C^rPv%gh3h|!ElLD4tYtR!k*Y%f5!Oc$->1-rz{JHm5KI}5FL_avtR^0ajq#Mw zgO-P{MDKnlB8fntqr&@`it2W-I+s*bRXd-G&>1m$6!%1}5iZSfkh~mU$bDcz2;r;Y z@v4p}+ofZSP<{l_9Qq!JkU22|#y<)tQCx2D5)RC}-`TL-JZK8*?s2ziH>r1L$Mx?R z;pI(|ag9B!EHlE8RnM`p@C=qNiZa2CDf9b&kX%SKLVOkpgefjG}2j~dD9hdi2%AH!IJ^W(5TUJsDFoc<0S zLfuh>QF_5B%4+BFx|#7v<6x&sEm*c4EI&~@n<%fph+4GJ!+LkA!#bkO(0l(QZkd{J}%EEH6ha?pFVtgj`Qa}<709%W<9W6c>2JFt;J6zjm=cs7+2n5N$Dez!!crp~Pxh&A8%GT&PjhA_VptgVw|u z!m5PCvL}u==Gqv3k`@_L6{$9+-gxS(9Q(5e#?%6e)W+27G^Q@$p0{O8?QYAMsvSvE zQ_}R>n0m*IscQ>%^Uba?xw{o(s?t=ouG(p@1fn*j-h$z9yK>-1VQak}v35@^vRnz0 zH^!z4kmJ1OWOppD3HP+r@>(kpY6U{AK&TZ6JE%aYJtB@*RGZgs58HBk$nMSB?V)yi zsNEiFw}*XjdsxWr@4eeYqRF{KZVzqA(oLXdGHX++8HTCIs?k2+qP{dC*P@iPu0c0nW_0XHT_n1KWp)V?)e7y7uOW?-9Gm7eqh?29EGiHHh6(c zG$?|jF3IEuM413V`>@geuK%G}#_W#_ITS}`u?f82@=A%T5@29A^Z$bn11=nTz5+kw zp0t3jJX^`3KX%*N#*XDp%}C>nkJi5vpYbODT%&Xrssy1%}f2Jw!OdH#M^*-!uK>8H1t73VWHMZ z*It?@Q8HGPOE6Y4R}l3y8Da8}{3S)5+NjK4YCtSVdVbt36K<$7)$^m4FB1`Gv9@I+ zU#r&)`L?&iSc29^leA2-C3APtC9h`~$afVm=|yu(tFUqJolp$uuK%g_46WdXsE~aH z|q4s>Uv)XY**;tIHD6xmGDRPaJF5V!HSK$|QX@n&qobtl&g^y0Sv;SUA|TK1Sc z!}23YuolV1e8>qy?@-bnr^g}E2wb|{9F4?aX0TH~nBeP^<;;-Lm6#ZB@)JxId0lmW zS^Z+l2CaH3O1iq8vRSr@`+CWl&8(XNS@s_!ux<&lDF70cLI~as{m={=%LG65ujKhT z6z?Y%4E7Fh>V13dJ)5@9ZpRpv`*I$$yo>I{X*It`BXg9 zlo+lf9hC9XDIV-saZEMOCLB7dxt)Vm`ppQ_WgBApWzQBA?yN>9#y6c<2Kj-r!dprh zvzHs|4f)Sj6dr70J8uoatPv(L5<{1FP8)r5@u<}KI^aT-#-V=AH3~F!6eW|Wp^KvY znV%R!^u{k}8D{DeXVl|oL-%OYcOy>dHKwWM?U70cGj|39a+8wx^we>jN<%aD z2%Fh|G=}7?^pZ#EN}Zz!L$S42XLPgpsA(1YM4f3|JkDL`OQ~x?p2d*WdB*ig(cH_dsc1L3?%$^+`5?h--~tq|*GSJjdnq9Ak)8 zoyI(W3>7mbFFXx5W_Dg(ufcYjL;F9eJK-=}feC@3*EYQ2F~zMCTqz9%RE?9H+W)2) zm780OTmYWTpN}Q1Eomln8w~bR$#c=l%vZ;eBdWi1V=>%TnHD{TS7(hc^YvOQEl z4lIyn>_TV;tqx|fattB>h?+;D?E?4fRtW%dfmPvP!0PcZhBpV^mkHy59QcgqI`FN; z3gyI)4u|NzD^#&DRNGrM0>K!XE`xXrG;r zBKK(D&lUNJ2`CDw*!c`aOS2y_BKQh_o%Gjc{nWwhm|>@6MdfDcAr=_ztF>YEh{hg)ZdOxT&Yl*SE1$RDsBac&_?6 z1&^&1DhbiZ<}?dR+@ZV_s!Y*HO4J2^&#b2upk>eYa``4x3*K}E8AVSEO`!V%SM$t% zo^ihu{mJ?muEonzTnTe($%=|m>xH=XMriCgM#?LfB?tbUR%DMm$8!!<7%(%4J+)7- zGZ9AOrI%kDw^h{9oc^UAQtz{wKo@IO5p^+XCIhZTY3(mbjpY!^=lGV)qa5WR5I zS3QEiNYuo-)tGXv`3H*YSagmKYB099V&^pCi=PGqdzjc~nAdaa;TrM1^kD1qX+3AV z*3qdBayyOwHP(tG!F7RdT}5LY94FUMHeT}*+JavaChea&wG}3hga|K#7RyI7xdLj!HJ7yRs?quj zD~qbOHwqSZ&4mOn7{oNGOMKO^-Xz_zD`F8YMQ)j*jcrAc;&o2N-jmCngMw` z`E-Wf6_;$16|5z>QmPMx*6Td@AK3MvaVwieSC|shX?GOqD-Om}xH`{)rn^L+18jCQ z;zmRXf4x|ITVPANu^g%yxi*PQ8aJ;qnC~Vfx6g}H9mUm~f5XMrVAbAVw!b94&NnyW_vtK z-*`ZhM9)keVglj;isDnj3&R|YltYf)-s}K9yPvnM^785EzsBR2<{wY*Zjj|r8T{@Y z^77#Get=3;&;I@L<*gG`b7n@{PGHh_ZQ^Q(_(}L}6OhwlWxCB0QkyyRxo<0_%UM6@qo)|!o?3JU(y)i5?|Bk#K#7;sWT z#N4<1G^rnI)TlSEb?G2A&rY$CfJ#O!uyj#xu}(`-Pp1_HZc{}f)M>ablLA~`Gx&Yk z>){)IF;*$g86aZY+DVi(&(gKMM3I zI$OKzZ+)K>o$HwsoIW}Ykdy8AaRcnY*E>yjY-AxQuqgJNrRJh1D%cC;qd_iU-UYCm zxv4r6feinSet43@-wy|k%UXNE4$Se=*#ud-_vg>TpI9;i6~NqA4s51xt+YmwbDJ)@ z3o%_ERK6N+_m$A1>H$YAPa4umLtjzkF_u_VulT zg4zqm?Tb#@k5q#qDr8B%D-jP2U0$Ddnwz2LSD<*!_oqSd=DX8U<2u<2YoIFS^(vPQ zE5NZty=sS+jWhB0Nv>`XQCRld3l{80D8ydP&TU&(%<Tf` zgr}+!?D7-XO`UbdeBXxRLbbxF%?e0BsEJ?{=hPBh^H0PwKaK>!^Udc1mSPqylx33H zA-gd}ga}DCJ~sYMPra^uuzgyvXEQ#y?t6j4VpKlKO@;U(-+nU5-TVar4}#@yTZxJ@ zbx8cFpn0Xe{5gz3G!npZTB%YC0GnBGS_F==&~$i<>VUs_yN-#3M*x7*M!Gtyzjbu@ zt`>fm;+NVM0B)0|@M0BQ>;k_C1#@gp}bS4UyD^_O%jd-A0trS5m_^OiwMU{ z`|WOS`p5p1$;(i#&&O{nY5zYX@KF$9!MA^M9eaV`oUrC60Gz-WX<{%T41ml4p)^8L zdii2zN2`5l1HS#WsQ7_5{U^4U@-Xn0mJ4;anR3)@K$o|$hIBvsm3=??qPv`k2+~@B zUcIfCmn@L(5i!JlLFu11_F0jl?o>mpQK5*v0Rvg3mM44at^};!39D6zkfg?H~0vN&z%pjuzqO4Z-FHzdl0c-{;!JI4x?rafejO|Gl(tiJV znlQ`lXqGC&l+_pJK9)@>6^(gs`_UDaswb}%WEKa!PE0-VgP0Ot%G(k3l8Z|`sHgbn&}nsX0z zzo!L3N-b4)8+P35gn0BgF*BB>uv#aMnr?Jk1iRX;=f-j9UN)1l0hHepcN}KmG);5_ zU=QwEab5pD0(C;w{~u+2rEb0fr`R?(R?ssktUxuJc_2?1W60%K?4lM*u!%=JHWB5n-U2`wIsHVRf&n1)ug^6` zH0jZf;a7CqBat%#Y@sqza;Hy@*)t#CuSND=8?4SxLLj`6qG?kw0#*5ZyrN(l^}J> zLoQVY`V5nNhP$qS8f}fX^3>lVC>)FoOQJ8i zrJtk_WJ>S0c&7Rtw%-GNVR;d3y^*+^#TxlpEkuv!KIR<5=ISGI1z&YKUggr#z}_fi zV_T)Om%XzdLv%L(tp!Z#QBQOHyiDPwAUfOBTz<|g!JLFT#FcWBH&U2Y0EJJLy0MPpu4OFqi*xujJ=-FPHC=`0Z?Gpu+e4tcL%y!HkVwcNqCx!`{w{ z?q1cV|6Sth&rVd-ib0v~M1A4tk@zmQZTAxig55BR$T`!(B8^OFWdE%6`f`W01y(j( zb~wJTs4id3^d1B6L8}<<3YnR7tFU1JZ`5!Y%_KbKNV0gIUl*T(mW~8&45X7pwoKT5 z^i+F<&~qB-0o%LGD?Mwb4+1~uWy9?(cS47UCLasUGB?m1b&#R2nY*plNF?hJB!)lc z)ZO6FIU~ey?H`Vks2F(b3wh;@^Sw3F_tS+v)$LWdT?!yz;-(Dbid3O>th{qVSdI|< zaj3Wlm5v48C`8VS?BL4;V<|!MM37AQK>rso_Eb<`!f(z7KI^6>p@%X=J?4C5I5?S^ z+Q40UXhmjK9+<4pWkNvk)^24_Ya?@D39-cOdl=b^kdb`RM6W;PCF52jT zyty+q2F;?a1sL^eVFeWUO%wFIKmV$2_}>Acs`2WWb~8KGhOunvD2-bAcW(`#b9(QcZ8+53w- zW(&BqgDOJi%UxNO&7Alfwk^G&deBezL#*q$UjAQ&U!k~N>tS;Efvl`+a1|6$fc~nbD)%g+;YmA!Mc|;bHq7eN-4aIytMG_t$bo-|!q$%e`IA4TK#Oi;bVCrEnZ-X) zPMU{(bm=WEy{Fw!OI1nEzIz8-$y?swU z?=nHIVTkBR&gMX?Hbv*uB+XJhg5`(uVRM>BtMNUDJjgX{UWq@MqwQQaq!upQxGs9z z&!-8tXY;P+JZ1cdCAL=;5U&BXwUWA$PW}B$Hc`BG;jCwI)ogBYmC}@7Nh8hoyZRq_ zLZ8_W6FKInh8q)5A+jufxaxalLU8o9pe~E}ya7 z7GLGLGUYw8HOVC{qsy9nCt_(Aw}=Jv#chZh_q}HHxHcRJ3(Yl^rAa zq*Gz)L^5`2C3+)kpaXjFUW)v4Rs=PWkW>!@5A0=7a;*QXc?a;-)WoaE zBNZ^oUOt+KD^8o2SwnIqt`u9SqNI9wg=!@KNG5LPEEvMA5~qLl3I1To9tLqZ|G`)Q zc*5zn&f5Zhm~Kl*2#BMXRrf#hytL>)hcms&vj>N~pa??RC&2f=QYx9M3JZsqM}R5H z3BAGrH<@_C(f?00DwqbXMCz3SO1U!iOJD9>VGxto;fbQ&3U8!Hi{Pzs&mJmf(}>|i z)SgJ(aDalk1%3t!vaZxI1<{fr|J^(N zM5D_4?VZAjzRPJD0foYg{quT{S(%^%7T-&5a3yPb=4{Z@gT2oVZ3=~IasOOF#y`C) z73_8{1zotU=zc>2!-?`E=LiKQip_kpJwAlr3dZ@@kub(AkzQvKV_elK-0<8HOTAG) zn#7dI>aMRcq|8Kpyf^KaovK}zrO(Jqvm;6@hqCQ%k{YvFRR({ZW7W>Uoh%+ z3q8@ZxL7V+wa(hK4%UQ3+0iUlb#KRBqQXsrO9FELb)&um?LYCFqTX9YLmAL^t2h<8 zJ0maK@_D)iFF;Ba7Dyn5=E%!rWMt9v6CV!{7Cnj(SWeFkaplS%wX51kStb}Td!b}y zygEJGB*D-WN<%_V0T3${E(2J3gxQ_iXl#?%c}rulPBLDvx^8z~A^_tzy+%Ypk>)Wl zJ3Xp_g)*4WKE^(XEo1$9#avl)j3!Dh3XP0ik0@`%;qL^M&?2QkYan=&ulZanv|Eak zUe5;|>`Ho51~`D@{>O8Y+(W{?9Hs|2!$HC3rRn z<{>baI0}_f3#61OO%zo|b!HsVcAI?=E77S&&#F=6rr^l5v2SGJFlszG4mWq2VXL82 zqt8rPGg%gP>4uM9LBe!tovS=P(ABQ@g1V_N;L5M(zL1LYFidNxfMy`vXJn80fNN*n z>>U2si4fmJj+BaF7ms9X0ItT1b**QAP!#2#d18xX0o3w5irL=)@qOdUhIzXS>4u{_ zoGt-e0_i<~iuo_Q5I&c91WuJZGI1}s(rU4kenxo#?Rxo0eMq!LWpIvVAC*&_|D1Rq zw&Zf_2I0$VCIFSl@cG)7$UxQ~(q-=@&A|r=5ef^?W(1$gL-~%`8f zcKor|cEquBOd{UX+`s@ll)bFL)9cp^2skVaA%2K54s3#?l+W3d&4QwvC=j!>)1U)_ zAjcTX!O@hqlty8Qkt?8Yp6Ktwa2_C$zyhWMb_xOd_PP@}>ng&yark<7#oY&p_(hS3 zffBNqE)4hP1R8mu$lV@4;QT?V#Q3Es_m4NeQLlJW01bdyGnijeY%U>QkXWt#44CXa zGz`4}e$fhLKvJg2f!ZF!q*Ni1AyXOLYvk8Pdz`s5*^N?CfA4!fTUSxyDIT#NXE<9m z^^_>G9@K!n_YEc&Yx9D;9g!AO5@To>Lf*Ke~47NRt2}p8Sg< zhQSbgYs4`0syD#BTErRj?Giy{?kxbK;OJW9UpAD`AZ9qZdnNAvMs2(THY3%i!RECEdMSCy z1AlixmiXr{2IU&SD5D&Dm?w<9^+lbL`003Ol?}a8_24=#(K-G`RkuTJP?Cz=kS_d-_sO?P-qY}?}+q+Kf-HJHr{ zZ#Xu(+@H_NB{x;aakar(t=%pP3#&0Ez|za8^D0yFGGfT#9VE){Oc@oNmo`)XbCP;L>rU>)qhT{ZBg0yS~jrzRp7tST8v~O>Ej<4qM+4e)22XYg1<%0r1KQfDDJWvWe@5 z1Mf7|;-OpZKnEd(E}5lT24{gl&$X8ULK`uuAu-S zV69OmL%nhaNR+Uu8wf3qcZ(WDt!(<9asO;I+9oTR&wYlYs?B{I<2Je#p<{Jo=K?1< zG>&hHv?#llSe;dO374?QfvrWvUl|8^y{A84EY6;aTD1mbu2U7Mk<|8EQtAyzmH$ak zozlMCF0tu9)ye%SOR< zH8QqwzLWPC@qle+%p+`R8nwx@P&mWi(yTyh#YUZW?1<38<6uG5?1~E`;F8uc0ds?h zMszUo*&@aHDR2Y&Mz@J(JTan#^8>Kp%a1rF%H08TbJ$_b(DyYA3TqBseowVMgR;Uq|1|l)#zxM?=ol z;d<`C_iH5g>4rrz>iZiZJHLt$Wt*@E&KEY!BW#Q?A~MIJhFPFl>@>)LH{fz7hJz5Z zw`~5;r1dO-Aw!RcHMYH0!p{W8^YCzayowyffEnnZ(AqpUamvS zAm)ScO0MH#%y^*u9w1(W6v>bWo}tPl{W#F)TYH_O;ABz#iudQ zWWMAW#eiy<==MiP2-8B9@U^4ewvG{U!E#{GW}p-o3})=ZYAsIU#E;lvYH_?`2N`dG zZVwM#((t58JTK(PxcB$y!Cfe^_vQsi5q#O1I1~h4+gtHmE(CV@ys*O{&kOSu5~jN+ z@(jbU66K-5+0l_%gwlU-k8squt)BZeVm=$I8;>ckp*s)dy^Ns9K_9%U87(QSW4J4^ z=W=@N2lFxCB2EFtO7naRwc14{5$3k^-HDTV17tiHc3i{rQWGlh?8w)NQ>R zNVdnNhAaxR2qA8pI0`QL2?ciHsLZf}dXNYJ4#fi#!Zw88@*5xE(w}`s0<&&{JC}V| ztW2>lI(|-d{D_3mwyx7Ms5@*js_8RG57gx5*4kRrmtb9|%GsUPhEk=>$!u7-3PI&b zHHT=}8yvH(We5*_aA@0SojMOh&pUt}J6i;2s}l4p5Ly;HY(YAs)ZfDS%Ti!81}dLX z=92iA-7fe3GA%VEbo#?PL#dO@fvm*^Z3S)JA0zTh0Yr3R-8Y>KqmYqWHvnXKHPD(R z9s37tdOisJE*25joxyhcmu=e_<(6Y8O^ZT)1?0|ot7RL7gFjxqP2urF{FERerWMU& zl5g6_7sE8tU55LfBYqODt>+_~^9a3+UU&&XJgwkptt)C6PaAfEnGJ-Xqcl|eCtxk> zQYtl{pWseO!U^aqSaNH$V1LqMXpF-E^2PEm3`*g?=S*7L;&blON|lPhFik%?%B!9< zc_gUyebxY6Js16I5|`B?L9TvZ&bmiCTy6sh?s?V{bpy}8@#kmt9G=aLqcy)H^%~ii z2#dHvvzv33tn3Foxzb;S#7SGsA$9e!vupC^%*1$Lz8i9AzlC0&E(d)WL-K`4d-f}* zR^e)kh*>PQ@wPz4s9rZZsNIeSq;%y6B>Qi>^9n#IgxvW`Pgn;ddaR4yk1Tq$$LFW! z=UUhI<>!n4=Uvsl{PDHx^_TdC!vIr_?Cu}HfH@kD4aY<9pq~{P3f8NT9tei7k0b|S zRT;YpoQKAvhN0tj9Pub<%i}vZfGiS_&k_hPSCptroOzV^a##T8&Lwg>=d$NHS-lGW zOpT=OUMv+g;>cUa z4PZ=^n3)W~^bKSe&x>!+gKv%jXj_qj0drXZ`J6w+QLnBu+;$+h)G$golG4Q%nf8?4~PP-;|!MGp99~uzxpZ`=qry~{d?@7$Vg`hDQuD1PFPLbcw z)Z-^6M&4MW^RN!L1m9GIw7~n!t+RU;fjND_E@-E&d*C%gwoMnI+8k#wr~U_P4PIL%lOFJ>KJ|b3-Gq9dA3+o#v!5H!*ul(7m4F0 zOE{j^bIB*!dZ$4&22+--irrI#S}{w+*xD6rbW;isrH=!*c2 zz01yD*+@}BE$zE7&-jB@iIQ0F&h1fmCkgHG9+3;=&IEUSK@?3vw+TvTqB$CUyxJTD z6;!J{Q@UC%GM-WG1Rw*hO2iJggM6zI{Y)JYnh_D04G@cAevkVEc8Aiwtk3qrs@s$Z zdJMN0nQ9Ke9alO zNuByu8HbQ!J%wmcGz{1&#N4_O(du}B8 zNapg*Al{~U5n&jt``zL@1eJ~9evX~FvY&4;FR-oNynJLEw`02Kf7ElJp8;!F_H%iN zHK)XCXLtX;hfLhP2VvBp`q+teqj8=`e|43n+dBZ?W`{Db;bIi^p}p*Y#F2F>m6UNL zt^jvj`>o~T^)%NMF%e)U98;}e_Av`eadvP+iSmayK8Y^>;Tr*}+K>)H>5x3~^XV}2 z8k@EOS?FvY?<5skyppuNfHrRA2h1Yb2-`!QsHh;a57$tQzF5a}Ocwst7w>Pnb!DD} zED7PkG|*qc-=Kh2*9DRSKsWtXz7=^p*tMEI>3N$XkCxkx!EK%Roq>N!xjlhSS53&A zrRIKuZhzhl&0qrA{~)|B!p6|-eE+o)@5H|a0KhSl_D&QD6?YAUygduO#o{2Hd$yLh z_z*p^P|>?R9MI+YddK0U^N7=T?azk@Ip8oh062{|g_)tL!@RZNb=lk7qX`1C+w42( z5(6rcKUX3V;f~V)ufLa^k70Qdw_tUiRMdQP2CY<8146fUVgZ)}jjJ&_69| zJ{f0vkmFk6@c6j>d^2rwsP4raEGN1StTH7R7 z41Y!EN7>t)Lt|Kz4kqrjH9#dCa2Y+i$}~fl8@M{L&Tc?BV{%^paL!t-I8b{`A122#S}gNUbRJr<@m?Ap z<24w1%^ur#10c|frDHdt4P2XtnGx*poLD%(NWHK#KVE8k4Gx#0F~~0p9uKuqo^96Z z5rJ>s9zxzU9!l<#^?b6EgGxpHhWYyy1BO0N5h4@PfFgfXtzXu4J643?EO)MS!p*!= zFLksf^3fnR6vb>5+Dg}wD68xH=*nnyU;(Lb`+S@q&kf@ z?DGJ7Ig9m6CwEWs!w~h%j(#5WP-~1~8PCdDgKLZ&we$ozJ4@|4&fL!~xD?SPt=^fc`*z3p#c)umrOP)`DH$p@sA`luLqEI7t&FkT;yJ#sX zXxK)rR^o@#LV6FT@uhAa@-J$shWI=*)ys(YSgj94N*UBhIy6Uam~Hvk0iql!c!x>+ zq1KBn4myZ_rJSZ~j{N_O1fe`sk zdEj;O4@gJ$vYU;`nst@Qx-AlZQ;-A60u~*S%SS*@OSoD;_Rk!7WLUt`dnmvAJEn$h zO*Ns;kE~XI3b+&^5`Esh1%a9=gFkWV3$~8pp+miOv2Odlr9|Z9Le6?lc3axT$?dQk z!11Ki)+#xwK-4kX9*iMK|?4-MHGQmw zliR}ljVQT!2u{sVl;VxcV8VV^<@?K1guVUt(J!j1Kpdqehs8BavV9Luqjk9z&p)$xcsM^ zUOp4rlLK?)x})@~{gzh;U&jih`ve+%H4?2GMSE`+77i%!-*38Yq7oQb=Pe37n4E&* z<1jtIY~7{;G6G5z(sZkY=#Wm7Ba3-v!G!*a+MLOrr9 zcSi7CU5MCM9AIYDovN{0fz=Ehlp0Eho3RBki7*Qxvw7}mmpKL`5*FAODxUyv5)fp9 zq*3UYo^ntvX`1a-MglMgj>^TCG(w`^r78N#a4n(UbUzdw(q;fV#P;{7Z+8{@`awCP z3HV4bj#GIh0z#k2hF#dR8sZRzsk_Bd~+uoe*Hw< zEa?5-4|e_g-2 zqv9L`T_3&C&ImQ*y7E`NmjL3md00w9z@9Zv0?~)yG!UBl&OnZUj2`Sg^+p-BmWNqv zMLMwK>Zs?hC|s&~y~RmG=Se*fm?{Um-1yMLvX7FVGk&1HKz@7?czVbn7YS9>p2-8V>NH`xFeK!)WyGIk&%r~yONT!>?^ zc!<6igJ5VUmp7Ca7iy>AJj$`>jbq0S2@@#H6t4^=+>RRo2Znt;lxhFTPY4m=R9dZP zcyIPe@F;LXSkv2A@Cd4xQex>nbYB$SZLQM%<(ZWI_jU+Ih#~37w0PaWk%eVbr?*FR z^4#2=6Tll28j!(N*{Ue2|tW|-L zNq_KOGGwbjL)1fJtN}mpW_`mSKs)Y5$e>X_YFwC0yzW|}l%RWXanJX$GQM7gxIR!O zAr9_12mfU#RQB3Gd)P@C{cbomAe-P2mK-?*+$^7}JdvZ>dF|swkjYXzy|l7`ny>0X zl;ys%w@3g`=!ugA=Pw7;JnBO@8b%jmZTyuCJi(c~&NKZwIygN1uUU7_u(u@dF!5c2 zuWYCkq$3R2DbvHqeeO0`CZ<^@4!zRgw!E#o@wHvXL)m~RVlZ);AV$aif)tTO`(Y_? zIu75C;Iw0(1daatrzPy5Xg)Enprxy0X9pj`6u9wMa;HaW?*)ZL@DKvo{Tj#9ER$T^ zkMI!U+~gqc2)iEsax+HN9R9-kKDUB7XOyz&BtRHiF%`oSiH!9lqpbyhC)@EtL;NaJ!amvxu%44jb=KM}X~v(qL)TG^hsVb_A=| z*EImq{t{V=JF+gCR|)(F`*_ks@(x8>|ZMbtPv79qzD@DpC%|M zQFJy(8XR3FnyhkDf}qPN2>&^0dm#FDC+$^D*%0KqDo zF+W{;VaDr+EBf)dGU+4*VJzxMv{NbhZgKOl(6#xnf!5I=Q&1BI)x!NG$e;kVpoh)X zWYt^TR)o^U*9Hzp>BwI9KMty-WRRVcV_5E!(t{C|-`xA?Be#&S;5zu0iE;FP-FSA+ zxc5<7-Gk(G7fRnGfLWL_0&$=V$#Lk~4e1!3X7OqIEb`yZ#S#ifJ5-M( z9+xB0M3h0g3kSMqAye$4S><4hM^OrZmV&2Zzh@j5wk9D{-Q)=H^PQV0^rK1#svRZz z#27}YDsSTYXa9N{S=vTn=uw!a4ot}@>4BxWDhtCWpVFdXbK5^RK9Oi*gw85)f{7AN zQ4HjvB%JQ+h$et#j@e|cFj3E5p2-(U80W90N+nFA<&lGcQ^zR6TsduzW#>x@hmHMT zf1M`Q#2@>JT)4M#m%EfYEz9Yd-T*!G^3dOiS(9e>?=m+{B$Q%eUR1QkZ#%128zzy$QhuHv` zcN(edv|5D~K%>%6j#Cd6uc$_lH5dy=Ib(%rO43~eB#9Z>%_fX8%#Y25=SNCJ%U7hC zz4oJ|zmbW=n|r0svn+^Yb$Z7fb+7N1lY=;f;ltJ~4u!5JWBPdYyn4!B6OL=vXrIIk z8Ga5BQ5ZO&f`dOBT$dCY*t5&OF2V2#u-1)iNgrB_dQw^V9FvGb0gj* z6Jngc#20?F0)DI*!x|UA-LFNXG3(9~Nc9B1khZ zKUtI3)$EsW_ADPHXj&1*)IunZ36NbEbB2lNE+0Upu7c(*1?N5JRE{vm)I#@KV6Lu) zvx2ia>haQf^w0s6ngX=FSior=w|QvYx@!YeB@0e`R2;WZJ}L87+xL!2Bz2h_s^ zgPfY2rQ?uWALe7|q7Ge*$bC2(w40wYEQQNoC|;i1S06#Af9(3LFI1Fn$Xpj4Y=?|v z9JomwFG0&Xi!M3?+{gsUSnsp~-E?R(sETVr>)S--mD}Dj>XcRUZZ^`M`=#`rEKRPS z)`fboX&+5ebyV2B;|E;p0GQ!R)!6$!r`+aN#*WbINo($((9Dtr4_0LkF_e`@3YWml(ZeGeY>&g^ca<6Mmu@aVnzxJ$mF`!M{ASs~D!vjela zRwvZ$GE`JVZ@_8F1;IqE2H> z?7w0c!Zn3TA(aP|HoDy#C~f8UK8-%>uVkNpwxqXh8N3g-gIKG0b+&iBl=xM2$XXxa zin|pwuTL_-DlNBXm`MI*+R#=u2#N55j%?7Z%)ULQ3bIZPrZs8gWCzcIh;_>f(ots< zMP-r1I=(U6F%=Cg)Wr4mz3xj1^$Ja7#{uc&f&_9)-x}4NVk6G1FH-02f-a@h>;)^s z?IOnv*(}DoVk(RJC#Tc>@qI{*aSS~(Vqfsv>#nrVW=7Oc`n$2YfWr(+s2!F|Ki3!J zyYQb_VG``JPa4M%F0O=zszJu`JnBNiFU-6s#3Ef%s>vx>(U6ROJRDQeA*MwD!_#I6 z+IpU|(P}r-X%WV6L`-qHK>@+$Lb4@|rTD#IIw00aT`3ussc#cv*gBYlPGVhA*TvIy z%^RjoBPFc9BdtQ@ibYwyrlD!G$%w{_r9K5#eL41D$X7^tFfP3*Kfi*M zpn{8*N>f!zEtzus5pwV0qTxsbSM$>R(jZhl&Z;n{OV&zxRZ1b*T*Q&%4hdG>K?0w3 z_)bb=si@VqcclvLloIWd?ET2ZQiU_^yS4k6)Q4syUkVSTjBsbapUKWIxD7u#^wS~Q zWDcPNolgOpRZd7pO|^-21gqy3EBEGsBsJyG=z4C$SB{*8E9BDpxA0HA+%kMM9_BN* z2nN3_;9TwA+qNa#M$lD6inY^*76mJHrpP*2Hl(8~(gLJ{?oN{bgupNjG59%~s~4r1 z0DHS8kDNKMdTtN7`DZzQe9uGMI?|(mn&EvdY$R-2omi(Y|4@;iV*lxL>@*}(NP`^PQ)3NOr zVP}u>kVS){qw*qmVh}S1o=7$cJe7MewlOnh{{iVUZ_IolCXFc^Da;HO=eT@;T$krF zk&BI|By@pNKj0v2%n~!-1KGqpYAFBlfu!!JPhOT?QB!}OS-<5kaBKjiYHY}2&Y4t< z`Xx`}=hYu)sbqPU1u-$;uT2^Sf>FkNz7}e&QAH2k0}oDvBax8JPjWC9aiRhp5qTDiXaen=z$1OM6KC7VbGcXJL!7rP&X5(5k?Xy2c=Bf) zKtF!(LFwwav#qL%%A$^V6g|cl5=Uo93DKguOMo_{N!|GyQyqx&_QIy z1ra~!CBXRI{4U0IR4>gfTh{Dn}8E1}2Y zeF4P8n_#L*K%P7dwW)PgP41o^$kK8c=`dbXlp!)OE40n}pK07&rRy;?57_cf@0=xP zL-u@6-1QQ1F<%KT78Q?+V&-a&dZ~<9bO21{!{Zqlb2UG`SU$`_O=QF4f>c>F_oNKx zaTTu#-{!t3em}#4&VI}#zua!3&{yRuRk4Sn@jRPM{_Q-ynX;mLU@Z^U^5D(NgSFe` z@wKR->U752a@17z)=@K`t{83Yr4d!U-ma^r9dOY(_0Nod;tNcH&xPXHtt0zp=1uI} zVmcQ+u7gQFEj``XGc^9Db`9OwHMF*u?y9|X5#P7(_R>TSze@&FrQPaKU81I8s&cO- zb!ktQRjb$46hktrvpdO3Tb6aLI=|C$h<7W$6VLqpHZk?Zi)2} zOR&=?dBAat<>uJ0;7)%32N3X|qNjVw*qeKn2aatn|M)#E+GJ-ypZnB^#qDQwU|Xee z=bjaoM6@c6T&n;-ro$u+`pFFRlS=@|BDfNSZLJ19QR_Y$`)b<&#c z_t9>|L%A1)A`_dm|8Qb>bjt@FKLW9WjnE_Kze=!*5d26+WPOE&lHQ2{u7o`=|( zh4@8`knjJ6`lFKn!FMp=GY))8%($%B*R|L6TV=9;T3(V?Lt8_>*bU1}yiwmq$};6w^zyx2FaG7< zm^Z1fnex>#{a4ia?dPQrG;Gw*So(n{;T*VmZN(JVs$<RV9xdUSb8s zr3xTh@m~Ea4ZHTkKT%sw>_4%wS4YA7?dSYrE&7AnOVpb-ZQ_kHPUhCxO@5_9TE0=I z-LPlBNG2%<_UlnA#icT7e4$RaVVh#P1wr^R4S8)qic6Id$2aPf8+PSaNxmLISXo+) z`L#0buac(aGFTfD^XThupFW18l6(uYB9_;D9Vf0Rul9gE%dI>Q$io;Up#*I|56Bl3 zbQTb+-fa)Q_pD+tDu{(_L}g+5N9mcJHY6>3zp@Vw|37P13r0c(IhBtjd+y1;k`di9qCKN|Q#h`MKs+Pg5zbM`JP z-i)SIv=)lNq0rJac|z3&8p#GHYp^W3s93}=ja+dvCPR>$iIirr&azAx!Aim_NA--n zNfsVO%Xdh}Y!1hVs3AkXN+xODndhJo(J-$C-WBCLb5pDLE;AI1GK8YeWHITomZLD2 z=CvG0VUWy(S!}5XSoqzOrCc(qLh-<-OLj_8-MIW2pgdx-+zhr@-eN^XJ5fj`n><%a zi``<)y}byq!YSkPwEjzNq8YiM?$w%9rzoaGmYEb~=_;mTEaM3?oRPytQ`#P~h+wge z%8Nma?NYhT&!rW2Ds0&eR6FS!_dpBJ3uR*UNMWgdCraDJbOKwTlhuw)`-Vk-MTK8< z6y(Jq^Nm#Q)ruZ6@J1)|WvY`T1fx}D{H4za( z0IP|^paiolK?QGQvn=ZKrk|lCh{KD<$3s zT^ijQk<9jxmVB5@sp&>Cfu*P~EZ0+Wf4^gSS<4|8HD=zGnk_vCQR&m!l7e{QnCvtcv93NE%s{OLL261$m5kWMBZ}G%L`U=F*7kbTMCOtdFLaE z+8p7+ZCYMgrPAho*CApPaIs1+=v?^1e5^~A)n(56lFrC55v5?I|2*q4E?}`vkxPiK zOz8xhh+r*P$fvgV-T$RUpEw{x=OzfYq6)R1KT!+s<|KN>5U4Mt0$&+Pi+EC zedxfjPlaDk%)OHejIdn<3taQDnXqZ$+~u3>F&}Xh`J^06%A?;YZ&y|6nmxm~Pp4E< zL=2tL)vaEN_jq08UM4n-FM0a7_PXBJwVrNiE1rLprh6{E_wxB8RngTRteb0Y`AH2# zVV=AaKJG{2W^2E!3{-od4XvK8D0YwbnC^lmV^ddG9lOW0hA+bb#fB>g7n$J$yXU%n z*S6@;uvo9JE4?AJO^Xg}*DKKQO3e_Ep0f+Pc+W8UlfRBKaecTx&JnF7SB~&$@Cy?0lkLWhGn_ ztNe#88Ghv?p6vv$d&`yuupu~5yS`)duJ28|KELR~fp)*(dpjd3A8I2vFOutL)YOX+ z&LL(MN$Df5R1KcHr3$ay`kWV;Lu852^X zSdIrduZ5Frmwb`Mf|Q6BaUkclFp}+{lCubqa?eZAALmN}Bx}f;r4d~sS`Pa-ug8t7 zA=)dAXc6vFqFfI3IB$v>Swp;E9*8W+qeOctz~j6*US!QD2E`}RIFB-Y5#w=QiVRsp z$to!2!5t-9d0@wRH7sNeMUm^4zP=2=T|{)8SHnTpQ7rGOJd~qEv=qj1z7z_wj&ONr zr9m8(fh@?@0zlRgE$yu|ZleqtUW?f{ZwmfcN4>Po@}P}sI#w32abAn~SVy(A%Ve-d znPe$Y4GD;*% z0U77DfQ}9L?wNqUIf1KZ;m}Q&Vd7#wJLaI8s z)Xpq_dPNbAhTDft?M(A*NSbGw<<}4qSM07_U>!zRUEO zHKrpJJ4*X%%InOd@;%ZJc}Cn!~N_^HT`u%c*PZQwYQ?aU$u^G zPi$eJUS}OXi4ApAnciy+M900!jsTcG_tG5_n!ZBxYCC760dk!=G5fWR9ebnkxPEjy z%_ny`7UoD!@KO{`c8HhN(BFCSgE5Vxv)S)P@q?Q;qyCOgXAl1SAC<=s?utRR)Z-g6 zt*$DUve8quji+#x`|OBx%~cT-X-s9)fweh8b04X&C@r;LqZtclyZ0wX+MDV9~$kxj0zbzaAlkgq<9!jeWuwzAY#vec&d-YZH~+mXMbX0pZy zed9&owut3uYU7HS=cbBzo5QKAHQrVR0o+_L+Kq>e9iWiZ#|baH^Pp&JRppZhMO&qG zo-!&L)l8~U%jxR_X1o2KV(7*|rH0wJdm2?5QPDI*RaRl5ZKQX*FwqWzTJ{W1k1=R* zUSJ3AQmLo%)e8O>zO@0e>{;;P*W{u@{v3pLy2Yg`sMzASpOcF!T-iEMn+2WI5UmbxzkQBGV2e+;9)J#snfR8N1biA_4Cx&9 zD)u9$lX)=0?nH4ilLHG}|MG9?_*Dp(=IITc1kkontd}6-Q!)>|fXMNL^*Dnr`#hGU zacIf+-xu5P@;y9+7f2Kg3%|C!Kumq34TvFvbCiVXSA+wB+*rI7vrn z4JL+aS-o|oq(0}syt&^YZvxmt9-SYuVpLlgDwjsmL--`J=xS@`IFU4jSN`qr2M|fm z&U%*P+IDZihC`e79F+TpgTYWY2kO97d&Yi98izl_wj1sffi7su{lDv5{8UQ6?Y6}? zBtGAXnZ2FNyPa88(QoC<-i_q;4fcz@lG}AmM%VN=igIR}XNLKEMLAq{yK!bHUA5P1 z)7x)Zdb>Cyc1dpExO8uKa{EDr>mp^nK2Bn53Az7G!(c7nJ>n6&qP^=x>M9ln)N<0i zG@hQRv~1UM&TdQ>+BInMqsf}b5pAc85(+DUsYS zd~(<6Zq@ni)cICZYBi;brnD+CWu+qD9!u@9^wEx`1LsYaNbYwm9g+HzZ`9Q$ymqU$ zTlJ%DRrBy$wV?v{)?4)eNfT`~`QzA#r=B>m;?z0iv$^{<~dR=iIjQy5rwJONo5fwG4XkLB;A zSe;Yugn@*r?2@N3(uxw$QEl0*{L_d?5qJXI(WBuAQ7J;i;gn5^*%UfQ1*+qaGtU(e zpVULlRe=(bevIZH6ZwuD@e-nBDuGLFMlOPBfT)r|5;ZnZbUzGc;uJ3T1}O3A%tMYI z7cdPEZTc6WlzHX|9CcGheUadwL@*n$ioyUfNEvkGo7xotR|%^j3?!bXjrgaPo?stV zHM2DAz|BVjVN;9|3~F_7pj85i7RDhI*`Y2U3Kka2S;QX0_hD>dhO$n|J1+2c?#Y(g zA!kFFZ|cH95y1d;$>&@wfldl#i;ExChmLN-HaKS6iS2>X%ZheQW6 z9}tnON`5ST8mAT>abX=N;*;yjiK^vzyJJj=IKhHK*|B;OA(yo!aGaSC6^ugF)p~~7 zrzUe9$I^Sopl|6z$5E)O4cTzdOo)9?m6cb&bNN zHHIpqhN2JkzS38QrbS_2>$FaI)dX+)(11DveV}N4P3u{j+IMWlq85zVz@gN#^!j(c z@n`hicW^v!QTJ2qWAe4{A)@nF+zx5@PKqlJN5DZ*9@tb8^l;}N2c;W2JPScIg5hP3 z;wp4#_0{1}H(>-6rEe<^Fvq^4uwK2}FCRXCzPZe2<8nW#Dxa%2>^5xtnu~mbi0d!d zajt?uta48-^6pz)`CTWN;_A3RbB8X>t`i?@ja+5dUgH|MTAmSBETD5x3{Z<(G0R^D zsAYRQ$-1Vg7^dxNv}Y=+HFO88*Edw!H%;9#o#oTv8z12cee$a0>E24HHB=Kzn(n;e z56SBrC-(?X|LaajI|Zd^)5+0UztjfDr#~IGkncW+QJeI*tqQ&`w>o|m{LW9UY4m&B z*Ac&CQ+o<0!(B3$-GvP04W7cI%uw!i6-#Y1ly6yvvScN@q$qD(ySF<!Cc>RzwZH}>=L#wD-(hJvx`wwbuC%e!@X z-wBs@o4I?wP0&qST=GE0CDqf-&b{g4l5KA8apmUj7;8}tpludTgb(e=k_4|L`W6Le zgcrD4fu^z0lz0=1O+Bw%7#j_V*e^)OTQS&HX zOBzsM3vFWWsd@UK?7c+uYIV_88Zg^kaXq`DO^s9*yM&iw>i?eQX6!YOYn${%-2>`x{>heSm8z^#l}eF@@TL15VvPSD-wM) zm7#als>*9TQ@^$KFm;2G6%h7yQHh z?BP8G#v79ZTezpI@UxRRuTLHBX_bxar174Z+cPNDsA?Ieu50?hVLjKKj>cxP1VhL^#JSMSS{ zC{ZFtcPg5k_(@Ddf6S-QpkVIfzex=2Y6|*c9N=U~(BM+v%1Fu>BI%6d-35H*wk`_0~?-7M4g8K2Kpy{(IsEI6vQDA zi=u?MSfhn_!GwxdEtct57M7Pxu&NBAHp5?IL|JyRt-*pjFz*s-IUyr43xB2{Y1v6J zTN8M5I>s<|@+A!u3_x|mWJJXcq=e;WThRVRS@QYv$djJ*}9cSyT z$^sBo8s;1RNjrs8jiDRJ%_a%?BAnAWnk29+7o}lj_&Elnlr2@+I()sW;xwg<8xs6w4ma{?MfZ-Esz& zYumQ9XBBbS@M5#$>G~ygf$QFT8MweI@T4FS38Hxz3(ZRM8eU}|DPv-53y5270dYCl zV)337{}@<$Up*1_v9!`Q;o%+0&StKrnVPLxL#OY$8Yp#o-&9mxRfdjc(%ybM`*|=* zE`P!z6+9R7u^`6#B@y5?_s8oH@f5Ayzo#t*SgJZuPGuV}Ex4oI#?@kIs$pohqYjmx zZViWK-x%n9Z7>+9wr=;#UVl$;OHp{H&>3o4+CpeF7jOs@8}u5K-502TfWC;zR1ycW zv`&&K6QJLwXLhi5(`zTW>6;)0-GGMj%ZAPr!-cQewyG;8e9TrH5TM)v)p|W;3D2qs zoli>{lh3IG10nttpaYmsB&-_n*AN>f+9(*#Mijy{%U^J#2p)@tCm|9h{dwR}`~Fk^ zk;vuW1V&g1(Qf2ipHb(6dBG06>!wDniJO6J-u0siax(aN=G*?+MR*3f&6(C$wH**; zZm?5ZAW%WyYFJY%dbY}Z%`!X!^HZClyAmNtmztXx2ztOPUA&*1&2lUv-sN^QQ zpNy*6o{YEWWZad3oyNJS_6F+U&YX)`Z`VRFn(eu`ovLW{E%RiK#Tqj{+GDY5@u9jo zw0f56^sOE!h-}dBJ3Y5&=siuLN^ehF?!!=&GYxDO2f!B47nXYoY2EAd+M{!PDwvYb zFy*=1T7Ygl{HCQ2dbiaGbSJMd5|EyHTuIT&p6A2OPO~VK+-#y}Cx?0TLiQt0Dn9MTbW)Y8hUJd4~V0g+{$x>f1 zOtRXKEKG8#2lC}rnRMm5tAeu!TjG1R5sqzyV;kXk0uhcaKI!=tqS`mWmR^MOX5QSC z?tI5_OY@WEa z!bW#XgjUVk5M|-LcMIk~h6pdEGOx}0+$Qt0Om~Uws_f5t253#i!YFr;QKAI(1Xy(+ zeqp%?%7h}s1HzIpm_-;v08h@xEEd0Ih&&3H@@-H){=|Y=MiLJ`P+(B_z5E-f25gGq z^QpSCfQ#?x*his$3Sv4U{=2uXQKrxq1LFA{X2BIK0OG+Ys+F;Pl zKpO!?3x_6wlO4W{;&V0{bU9<7gCe%j5LP4#bjj-&27rw?>L2$|&kIn-E|1{YMy-Sd zvKb((RGD>x7RB6~?;czAGoVIooSy z&G<42lj*%@d^UlB2Cc@YM~WqM>kN^H%9ANy#%)h(>HsfemY0mrU$R7(P@h0&321K(~!rx)CS3@23YeV#WgK^?k z3;HsN=&U4dYw!>>iPkWurzG*ITx`w)WI$R z%?Vvl8K*?kJpmo|)9giIXsp?hS`}y8tkO+3z86d3X}BEg%Pt{L&=Z87iOcsaMlXV< zc5&x;7%4=FE2h;OqlW^H8!ns}xTQLTk_fo*6a$y$Y{A8D%J&e77;KmrmSTe8WOM|J zOKe44Z4qm1Od3Cj0eCWlm1P;ME101SkGlL@f`MPPO_?%Zman*s!7z+>D+N}~!g<}U zrYVZvm5c>@m+q+q35?Sx$6`}wA-d9eBy*9dFC)^hvq_i8!vL+b=_tndtqNya@&p}|{@Y5vn-P!AR`R{NH4pPVufv*G(7l3&c;l|t=TsWo zm+FTck!;x-tadb5Wl7*I<2V9ymCV||R&!PP6IX~K+VKRsbxQm79+M*SK7jX-e)o!EyNl+Qfp9u`3FkY^wo!(_4vdYGr_gWodw z>gT|w$3Zo(s*P$mlu_(TzzwA+hf1)cv@nR29%v(MNzCU8$Lzx|BISjqA0_Em8L6aP zRlW*4RA{DrF$cx3sS)JsG%sgdp+h^Ev*j1TeIjiK6g?5|C}O0h@HJngY-R6ZbR^AN zO1tDmVQ7+_0UbO9of(A=Z*{XfBJ*mBu{_em!bKuiW@>^SaSxS9=801c)$C8^uy$jQ zewwxq&pU4npUR$r$73%cpSaMsVMs?)5$MRhAm*PIgMhhWn6Z|dw@PhH1Zikr0FRwv z#+v5`9TLnyq+pjniMeqxjVqXhEXH>+Ppm@{;8#NP#YGUw?=Mm;Srd1=}GMgnAZ(I`qDJU3hYYm-bsG zyp*p)o8t|521K=QKzP4{?{5nu&5NgS9#4Lh7cIXhr!h zkOM&IMHfx@j(kVMAw?7#pS-zXXuv}w-OCU)9b9IOwYt8ZOD(`3^O2OnDe4YVlonb4 z3s9F})EIiWKwQV12Z3pU5;#mdd>iCpcM4){;0jkLjYAp{^lFL_((U^`Ssh;k9-X{5Q#(#4pk&5tn*m+z^u>Au|?6Glr@Vy6*xT)DM_$ z2Xoibxh?gjOP(%>aC0SDhM8ALnb65au!UbH0)ZaQ}F{gp> ziTdTQz=p}jh;UE5hH(i30|T0l8%*>C(L?a%8N=NcxY9TkjIsS(I93<1D`B9Z1+bg~ zAJf=``UTuUHr1yQqH_2!o@)zORWO)Os({sJK`=QbsKWfzJQ1+MozS^EcdBlcr8xu5 zw2Z#X99wgB-L(1x#a4!@shdMa_ryAFJdA-a%>&G{b%P1q^x0Y*Z?y$AJx%7I3 zoD6fQEwmHCC>{)Z)KJYH)0o*)nW3w$VJNz4nCehhsD9vm8|S#7LUfVaI0#XSTS&OA zGXm<|EH3VYz?1HCG{OQ|w2hYRt|$$Rf_V^PYI|9<0KsGg65o36Flf2MiE@XYW*Vnc z*kh-Xmp8~A0`~{`f0n79ojFX?ZKdz%ifO~Pbf{`tnrc~njk!i&H}>NW--=xyWdIS# z7n-&1wL4cS4XF57bdmPqbu3Uj!Vx16qbP8Av4;<^#ftzH)d6Pd4owSIkm}N&q7TfW zV%ut;8q{J+eIhkA@&tSd048oXt7#9;2Ybu4Eitut(?cZUmSHgTc(ugVld=(_A zhBFF0cLf)C7=rfcu?>uXN0AQ$fIyJ(*%@^Qj48c=$qZX{d#pcnhicy$ST^nV46|P! zg%@JDq5BbYk}$_EyV!3 zL{Rwo%T7YfH^bYs92n3{Cq?%0=qPaQrFepEa;Vm_?D{%;71%6{Tg2`LS!Pq9$tE

6?Yo7EHJe>R;B!PZfIibPh(I2nl!fijc)YTs;@FWc4j-3Xfx~dGP#PGCJc@RE5tQc( z_YL-#j}VX#6UALTT@V1%od%abL5C2vFbojGfoFx6u<$3)nxd;F@L{1kBF+m>FzgT> zV8yxBCEo@DUZ(C^5+i{44m^ciTskA`Qr3y!V{MLmBbh&9r|7y>C)2vkwEARHYvYnf z1m42K3DZqu_G5g$F~;x@_w#3~N6(r!#Id&KVG*2ofYWb76E+-x`+up_{;MCdQYj$v ztUXd3zC6hIyg9%)bqI+kR_K0v1d28x<86*ly_GRhTgQ#B#u;lNHEsc<#@bM%(}&bZ zLriZGqy{xi8|GTmdPCE2RjMk=KpWbcGwf@I<~BoW40Ovhdd-L$n+fiVsDV1$Ed|tg z93(Ml4juB0hJgn|UB%R>TrHz&r0;DHt1)FRCt5 zDfK8sKA)%f>++T_z|Y+f!S^V!*`m_0tPL?J#%NR!nD_#nejfTs%%$d=RVl_$I~NP@ z#uG+9@!6;4re*PUp#I5Oo;G)_n~kuj`O9273Km@WeMTVZ`DsJo1348$aKC(?2u(C0 zi40g|*d%Bt+TA_TfdE^Q;^#T2O-_;-3I(7eBm5=e#-qkO3TXxu=m`ZC3uT}NUg`|u z_rq{bW&EJeWc^QcEiBxS&NIJ440I57&rPt7G$_-_g-~Q;hH;OtV~i(^iZaijJEKmU z#JI0fEE#tMpkjzIi6UH146~@9oMa)W6y0*s9BsXVaDYGaZYP;tLW~IN&fhI>Bhdg{ z3s^Z?3^JF4o53TBCc?LQzGm_zgKa?*g)oVbg<$>@5yLqb@4^5kpy%U|!Y8L>j-ZWL zc0<&-2<$|lMT*vON0u*v6(S4^3%dL{tS;j755IC@LxE=FEBP8?atXocxF3GC1w~OV zfr+^T38W*dU1y)eqS%NxnRyuV&rLx%Kxd&x{{GkhMB>G~o0Uj`*F?)e*1SldHPbPR z;s7Si<-(#-?ArVrxXc8UWiVz~ZAA#2GmMT!H*JLGnK3$}rpW@H$3K8>{&o_Dq8!FPYselvSlg)`$|mgNS8FuiUTVOK3<+ zWUMnf&Sx}2wty2pMrJ6hTucJDpot|mxG!yW*h1hoX(jW#TKgQMq;pOb&GLpJ9%yp; zH!iJ5>+oAL)aEVlTcjbVg5FXw1eKE{cTB##LVo2@C0b`<*s0`Mn>*!M9~MZ3N6)cv zxBJvzlVM#ga?{iMFeZjddqbzEDY`qv|7Sx>HTrhXHuuZ0-cnN9W(=#T)h`}C`|9E2 zhvfN_$4|d}{p=xm`ufeQhadj=>9-GGeEIM+Q>;6ug5z+-)qHe>&mW$a5sAzsLtz?>Dbz4x9x4U*+1BgSD&M_@K;QAJj*q z+3|e~S^qW}P10w+Wk!>9!*5Pzs!+bt>9YF!YY&%wKd{vBy^gSIo{4aT?V3OGaeM`L zK6hj-OF8x2kyWDDDW{IOnr3Kyr*F8bZuKYw9h*{1R}G!Hs;f7qjx0NMx1pl;wZ5*m z#aNqiN1js;)L4W;csi$kHE|@rQcfCy7o6XI=FV^bSj`*h)G0snVIzQ)+96+r^B}GH zlHV;|D^t&2D$~}MX@5l{>UYSi1mqK^mhSQ@Sz!B?(_O?%Wy1P0;m^?FBZlKKo#Tj= zDHoS2wQ0pGwec{PNq@m(SbQDwEgJZ1Z7y9ala{ZP>6*)QA9K%UAvPGs&a1aLzg9{7 z5wDc#_Lk{BrxK9sOXejXet|KmlE%wXX(3eC2)=wvq8gt*Ai(($N_-WeHuaEGyUQIVqNjqQ?CdOnmXP>@#^i!VB$|Qn zE1(7%hF~e}={rUcVSk2ZZRn3bA~b6U21DUUZfwdB2GGS0Ohq=9ggST|5McaKmpqpj zFQ}W1ibF$}Q-@5G5dQ0k4uft-;M_zv9(>(_F#^q(g-(Lea#4q5$CRW4qZh?Q+G+7w z2h>6FV~BXZ{s>-(_{bN&CK#J&-CF7pj}2o2%6J^24ub~2kUDhEHW|Xd) zT${1@EwrT|6_LIe0@dh<#TCKEuD$dYR$smr@R`zju;C|m7B6RW*(I+NJ7RxHpylNA zr?0;#nkz={yVs9i3D-G%?IL)Kjl@NcB+4s)-J-{qAO&bbr>QwbQ*ad1C&>h6GZ3mK zb_or@A#4yByAA;n5;zC$lQ;mq{?Q8pPZ+%p`8WTzQ`!=r{6$r}nht->RPEC)0d>E8 z_i_Kv>hC+`$yYC;8C-j;8h=*rKi0J3>hC{A*L(-tFI$Za3syjN#BnpYJPMN;G_Kxt zv0&*tTpO^mVNEh@vx`z_FhgJl=&#GnEC4UM;e_<&kqYF}!BcMHRSvuun zYN7-kVcD8M2a@QOndH0Te> z9L}&Hi3sd`U#<`50_!he2nCG16Rk~E!T{7fayXw9DS|BvUH`B#pXtczQ9lYY@fdw* z;z&Q8AoQ|2Z~{T-m8EpoYg1P}w0*;HEYmTZo@-l1-{|)YI5lnS~_XR2jFJLFhx zWKD;V?%seH+a%lJJL!>{@pxpR6~S)BLK&>g?s}9Seg8TlUj)&N<8kn33M<8KGrK-s zX4k=zx2i~DJzH~nOn2$fw)(E6z`!#{HB3tT{h_8V)#-dmG4L5qB$|$>vt8zvu&5e6 zBi&c)9vUcCf6YUhWg3H%OYTy3sPwxbxoev<_5{T*T0t>$^p|KBPX6r@t3Hk?)~#>Y`_|VMnN$3?3c@S7he7IT2R0m zl4e0NbLp38X72p+YAC=OdcATJr7ORj@twNy&#NH-Yv}b0dKLeDWx}QJ{(Q;*Uk$;{ zT^h@)qIB=3i<@W0^Np$Pt#7>bjkmt>);Hez#t-5fZ`0Aw4;$3BAzhvMquNM5@khAO z+HCb&wt68EoLsiLQ`i-;h!cu_l_oXaKWx4_k3@7qo}`}si8oN-5OD*kTdjCz=}wwI z<#_LgB1|QFw03px^!W6dHb1^GKc3vBa(27Li!_z9%XFSx4I)x8>Xy}^6RA|z==pCd zPPRFuV3oeLSJ})qv2Sf+pNP5GrBYe?!dAI`B35Jbvm#{UdWxErT6=9Z&FWpfwhxuY z=?cZujwr!73my9%g_G>TQv6u0Hd@LNGqc`mt6pleYA{)~#fdlOI_}41Tda(7P|M-V z6PM{8c<;BE83)(EZ#a;5ZSnTC`#UdRoq8o!Iwhnhq{$IdGkm8;?_Dh34=VO6Ta> zM0LdC>~iGXt<$=%C-J1vm$RXJf$@2h4*9P8yxXMxt_dw%lA?COj@}LZ?c>FC-gyML zwXuGujsTZMz~3SSINKSjZqHT>mr+||mJQRaC~Rohx@}U|X+(fCw7zD^l(?n*g3Sc? zMSw%C>b3&F@o^g5{8{3H##X~TD_^WYe2LdKqriPLW?ATTI0@?LWZ9)MW${Lha03L~ zXLJ(L4tY&OI<6BgU#vj2;X73XxOK2()W=xA8R%7L4J2u@Mf8B^kI9@z3HA6w46(S9 zhrmVa0LZFPV&9gj;2HU_`pM_%CxvY=2q!|3^2wuMI!oY7_}S-h26(7;rjL-97}wVf z0QMBk(WpGabDgDLAO$M!$=E>DaiP!BGU=qByf=ki4X?t#z)O786XE+(3C&t1Y8_Q6 z3iI!rVLYvmHcYfIGq+1Vd$-(%%2;A|xsPdjpjg89kgy{@SJ;TXH0&0|2UJX97lu^C zaL;HM%SNQll`*?EhXICf_-Kc&MlRyhZ_|;s1NU92*)EfDk=Sn3b75mJKgvwE`6n9R zz_Fy*lV~RM6kN6P6Uwf62$tz@5btXAOXt z99Z%p?UJvh1Hz`MH6x|srhQ{VS}ApzdKhpnC@kD{~s$w8>Vi&3HZkT&Bz;A2J(LRDCTeI`+2m%@RL0R?2Qb^1-ixmYodhQ$-!Lx|`#Ef?r*fwjn@v>*sw&o{ z5JPu)z7W$}bggZ++SPeaV=aO?CqLYMlG0iV3@{6us5S;ZjbqOG+=H!9Nl*l`|DQAVs`)tXp=Q`FCZWt zfqOLljiQ{H=9yvs9!M&OLIN_*6s_CWdo2>sEkgp5tYkI3&(4@m;H1ApAzpHz)iJCGnt@1Q7~rQe^vK)WiBW3~QO0wdVq*{h-v z93=hYqrxtnkRd}}idejb_2OU-;pHgs^_GAeGk(o@InsDG<#n;8@i-ozEih(yS}$t-@yK2gEYc)rZH_}J zr{%4-(YK7?ss~*Q>bD-$55s1*1oabfINRd+NxE0!H7mcqmc8stzuxSPCZp~l@xlyshW;y4lH2l2JH{@0dp0mn7skh2ZrwU^uF7es;O!Uunwy^ zS93GPHdk{gSMvp%2vf(GLBwaYn)6t?R!(`uS2ky9envw<_!SIETq9h%R?Vdhm$Nk2 zWmAT10@9TuYRb$!KBxYebqI9>k41=I!#l)<7pbUE0k`FoP8c(g87|-Zz`Ed(V9q1_ zNdht-vk$-8Kql!R77d6$AAX4gL3jHM%$I3P$b<%@XyQFG3q}ar!xv@o9jJ|@AK!z*M;J&u@lqlC=R7@f@MIFYXqLT$UCV#?h z-=VdpC#C#?=nB;h1e4T74;?sHI_ZQrA`US}((}I^%m%Fxjj{=MG2v?X;Dme?tZghe zXw=6HS`b48y$Ww?uw*<~C<90Tgwcd}L+6L6QDVXRW2~P@TAUUJ-rIJU_se|VQJ!B9H38IeVM{fEsz znGL&C#U4yQk~B@b#{eK1-tsH|wtuq0?@9HM=y97A<}_%h;CN6TsOtKiWyI48Ubw3a z@D(AAz_{9_ulL$h*4hXoRp|>}W#Yto^V78V!&nbe_o-wy0TWZFza$o)8%6lv-f5)f zuz^1Da{T*bK<->tpt4^r` zX`J+qA25qpi3r{2gaAxp^#*#*cIFw}5k$C$QQ=m>rrdsD^#w7YW9VUQs&*lKxrIJO0^BFhW;Q3iLJa~%-mT@Yq0mH zvJIW7l&dErUcxNK7(Ia#2;r z$@+m^WM&Wgj~WO$|4W>hU}tU*MjSr-P6JI?&kw^+w=Ch;*r5t~`;Bt>tu}8Q>y_qS z)sxT$1rrMd-^pYrW{eA<*g7XdWfZs{$T;Yb-9r+lD)(3Ih0?jf8)s*arh#KeSt-fU z+IXdPAix4-535$}S)L^rCl!H-@A>XgXn&YkZ-?dOP4CNf<$#1*$A16X_j_g+{_f9+ z=%Ii*L4H<~n8Ry|@o)S?^5m(TGA?8LAL~FAA?IUR4KEznX+cKddr^0V%hm+3Ni!;bVhlt z)rUM|J~)@ONx~~9UlyKS_L-=<@hzWCPqsXPFP<2Fq&UM-MVsb#Bd)cr7n!@>wSzi< z3kO^*ob2pA>00FuiO_|)x0I?hETfD8wh94!^RXFq+#CDfLi^yO7#iHAK1y+Q_~roZ zS5^F-#41O(qCYyb_ucK3iE8Ih&>vf)Cbw7p)mNri*EalqE~AS*e0Qr23%Who)9Wm` z<%=zJ4-8Ys9QZAuk#9*ySltDiQ*B-7C~%osNy9Mtc@;WV`b{WsEeOIS$@?z7@++Ng zcAO#o$-Et61w!81TYoKjC%osGxW-*$-GW@~mB;T3%UfPWYsu?-!_Y)U2r(RVqf1wi z#7N=8O7;2w_MsdvN=6`-BCj=ZYA%ip}5Y;CQ+I^rPYowh{>Xb+994032)QxT{cr z0_H8G=5MBE!!VRo7#Sqo%dA6J9?|wv`RHJQE!|#p)*0Wj#P(}4(iQMZ%jfc^n7msQ z2i5#+gRCxu+>C~sY%kH?UeY|Tu$79>pDzwc;}HKSo16aHDg2Z)gjKjM5JQ{{MVm^I zD58bgrO_e`O?pGS$^AEI#3rWolvsvXj?YU?a&TOqUB;LbuW>&WU+oshv0@KXDQDsJi$iylLH^O^gkf(I*0#gWH z7(Rtt8I01Y^D4P1?IX(ZPqom38`uOVYmv8W09lMSi?w~ewC`E98Bh{)-Q>ue%hJ2t zd-jlS%M-yq!yaP)>9T!M_G6<+*(4f7Z=7U!U49n2xIpu5-xF4#j~?>*^>v_T;W+g> zBjmowbU+NRaIT6Pe0caqHS64Z?F7^mx|jSNyuUu4zBZHE!ea#p{T2>9jQC-X7Tg6t zA^J~(5~Vq4Vi0e>X#xhpoRKz!o`-;6Oc=@BOx+sy(pwx6Jv?Ij z_&?UMqKx;cUn8Ckr?f**k(r4aV41XkDR}7g-){S)MuDZiRpOMLqr%UQO6ZP5aS^vl zTVM;k^WxE9@n=3Lz70w3q^J7P$NDW2Lx4#JKmG;lRT5z1ke}SF zt$QU*Fv%ww{Z$N#m$6OL?j~BM$`cW1XoUHUJOxR}L^0Q!ft{lT-K8B8?9C!1!$jm;!Qu!XQvu*y?fQ!WY~e9{ZVdE^puRo{t-Qoe z>H!2`yhdoDtr>${y<#UT_^PU*6Wu3qBk&i!!Jg=Z%*ZM~b#?v7(+qCr$%)~AfO)(# zzcqfBe@6CEI7-3_x^`^Cnh?}c0?cDT-N@WDMcJ1_kpB7c0FqRFGUTLKh*(jdF0hR< z>7p;&!v{)#afhkz%OVs;i5HNgGOrj3dELzm{`>gRz|? zP2X*pv4nG4D7)vm1hN6#@pa=+Wl7cX3nj?h3_*hiUC(YWI?dO?2%2Hzt@Y_G!wm`W zn?!v^JfGb6I!Hl(<8woYvbz5!t8@XCrP-UMIb=_fyFr(Lx zih;G6I>)Hk{l7SXxQzS%!2z;VFT$64OOzp-{Rct@^5?HRnlICeVFRkxi0*~jRXju% zo1X&ZUc_05S5C;HtWz`bxT}qY07FCwZ4wN;JR)0k4Kz#>^%K!c{-vc|V@bL=*oV{# zQnu{K3YZn!oXryWZ5EV@S`L|h@i8X8>mkz6;E{EpQR%bGp1G(Uc)#Dw&NJ!>cuErTd*0ozW9#+!h~x;kzb~fB<$8Zy7KU22G|br^&u>kc z)-1D5%&$(3F|5BZ2xeI`tzJPofbllC$}MhD|4Yx$rRl0Jdq57gFDTPdvP1oCi`PFo zwH36ePCX~6(2=W3@Qc>S#N>h*mLvc%itm5B>=qDn1i@qm#W7uQ;cUxDSl>nV)h2{- zwpY2UvN2O!cUGS~YGe%KX1A8-&$;nU8-BjvWwPd%!WM>~e@BmRI(o>3-UT!84zCBE z39cYwCZstd@TGPoN1+}-s*}+BvPV1rP^6{Rmt1vTomXHnU{yCMhR#$hB~h)-Aie<0 zYYTvzDvNEE^ad7JVNd0Q_y^NVfr}osE_eD(8c)YE9p=AQgfiKE)0i3WfQjQ7xvDm# z>>3Ro*R?M|UNrt*UMc<$bUXX)`0qA*=)_hQY@QIvWR?vc9$UX>;sKF31 zj2?ivjNu!}1~DFNYo9SH6wC~lP>@Lb%?u18RdHM(mLh_1X|KL0D;TW74Aw?I*cmD% zj&cUfSM7j)luL6YLj8dwEsoDx&BhWQ)$ z-%e~)8q9dL5*^v;C9Na(OQm{U_P3xovZIx*3H!X=j33I`8%W1;#WzYxGF_p`uCNc+ z(cisJC%7d5Ux$04I0W+=(Jqi88Zwh~d>$i^A7bEK${f{|tIM;yK$Q~1M4C;O zRhOICc`(FkOX|zVNzRBx+R7mxEH0;~v!;`?Vwi_cAlKge;~7Gt>yhq}LGdgd+lz^Y z&VcYE^ImOx4uXezFpF%P(!DP-(~b@bJLOXl-$h}pq0hUTON1#CoBMvA?T#!q zPj(rY5s9ns<10-x2?{h1^ggeT51bxH=SyU_Z{raIczCsDqIc~`{aqnX3lq&hmW=upK!CVL} zT&zf7)T2`hM zu6Nq!0Q6_ufGmHIY#KITKosv|jF)4d*snm>+=NJ1(n|DZC+c1r3PcPPZc@nth7+@} zDIMSu{oN6r)kW9~-QNTH0EzbAJmJ}PR-ikYze|Gb&x8EX>j9!9USKI6F;-#gZNf_) zsawwT{%F)s|NWimWqY1}GGbrfK3-yI1XAB)k`u1^-q7uXUnAL(#HKN=wGRng@gs z&~w*SG%R#N6uB2rSs0=hQbROKs?1^xmBcrsREk|cw{Uf~bF|DMI~-~i<}+udaGRG} z*$Gbx2Ek%*b{z(!v|~VOi1sf#cdz(qf8sDJ1DKzqlQ+B{JGx7DC{9Ln$kc{Mhy9SV z>VvZ-E%j7~WU8s1I}Tw!>1#h2GT^iSq(TwPebd~qlBw|%(x}DNO||spy3&IK459e{ zvEQ=-pS>7kkInSeduQcbO@0w}qhoTXXvN={+-1yH zcID`Ra^-(A*Z@w}nEWe@qHd#Ic1{0UpWMF}?J+ zL0$SR&TH0>?<9NksId8^v|~BamhVgA>_v|@JE#3vfYot>bh@Vk;2pT;?H0u&JI@~Y z7^3_K(ac<`wt+`CAVJ;oUf(yjHK zJ6~`^-Ox_hcZFnopCVaIqH2%V<+cMN$a$Oz6^QwuS`N#Oz3*%Svd%RbH^GMJ8vfQZeo8BlbqHlszfir4`Y^f57EOyj@s3v3zbozos z8+iGT@Y<`qi$DQrZ;g>y3vMZMD7w)1U|ENG!ott?`tiJ$5v=4AM$>LuOF0I0K-j3? z(f3H>JL8Q1pU>6GfMqNbcH_3?2&X2+tELNx%?lx&7t8K5V82(%KKmqc`0v{=Jo{ZV z#pLR8>W{bha&sPpwvha<2!pM2qTJME=|HK8A1Cu%g)bMpAtIPE5x(?@Fo)1T58ZW_ zZTi?_v-bolA9>U`rcGyV29_XqXPpixuatZdwYliJi3C?=Obr+Be+|ueyxN~ADyfuh zm0(?3gp8f^HbMn{`q=M=a)YWy)jA?88Pna$>Q-px%&@YdfMeGEC9p7frvvo{*u+ia zm=7IP)DVtx2H(5!o-YH^iXC%91c2_?Izu``ik_O;#E4&rxy!n`ANSZZUrAai-9^G$ z*R<{5-9Pb<=wp2LQ+)t9QAiUPWX=uM0aQh1^*}295P)u;XKr%+_e`JruF&oqeepVi zo7{9GyTl*oP3J$8MY4GH-Bs!0ANZaKYlQ;}9;jUv=q}*6()~4R0ibe;;EW8#UX+@*m$A-VH6QgX@|XS`jUMd)`#$^X%)~ta=_UJZF<5|A>NZaGYD?zDFDDYK<~&@`7*Y6eHc`ZE5y)jsbB-e z_e9ig*T1ob`w4;*9`*VQx(TxytUG z&RXjOp-6bMBc$up*clz{@npcr149ZToN&oMQ3!BSMzF7~L?Qy@gEelE`IxAK z5gW$}Y~A@sf4U&q3s^*8`PN zwfTvkJ{Kt~!#`lH&8{3UQTNN?4T}J6Iqud~%CvaA#%*6r3G$CCKa4>c>kk@UJ;fVWyl=}}pCJ#m$tga>nLD5>t~q{=3#5dE*XC~*;$@}_ ze*((FWP%%%M!YJDN*Fe(0Sl?H3T>Q76@Ty*4{fw9BrG@Ew(iRRoNaiNO!eVYy5`aC z7tbVi)t@>~_{@LHnZU-@zV9@3{l9)H^JM!JKXn`l+RdNZdh%=@HMRZ7W;jf1g0|Js zTu~v&UH=YH_gS&VFXqfl}itNlyB^QGYi6k=oWY zb2)4y8gjO$OV70WbGaCdv9Jchr%h9`MHtetHWUwBA4|;SBT(C!%gm@xk+S;=w|T4G zPLC_0$g)dMh%4TURQoNJWEpW_xi*+dHn=nYXtia(am(s2^KMVuo7W?U};^NuAB1r|wc!2Apm19hmwwU|QRqm@WW0#%m zfb6T9b(#>}bGPrMmPpPqm4T*=gAwn7=4v+-oi2Fj1c9gPw_$D)yUL3#iH+TkjNdU@ z8pKRLtt3NNU zYDU|cG}lA@6_Czi+V9mjtwdD9uTvuX)th1NmI|ndE^zsfT7(N$Zr&op>$~QmMwj+k zJTdpu$R9M|QWaR|$mxcw&3jzsM@XlBJU`4QEw}SC>M7#xIh5~x^zcq>U{9~{{Qe5O zv%(KMZ;|Lx5>GF|<6n^Z!`+qV)8F7RkYH{e0yAh?q#<_4kVrQiHkl>yxr9K;>yg!3|cVC^%m@0b%{*4~W zNzJa3Us{8SB=W%;8t>K77xVp7pR1c@l$6A7)s_|w2^a9cDly*UMjUE{xWyO8(nGb$ zp85S^%k06_$16((1oB1BbLN5S0G;D~L|gzp#Bg#&j2L9d*MCKbl-wK$${Sf$a8+JM zA|Cm%Dcaie^Aa(_!k2n{jNgi-ZS>Dey_gJ_Bl}FgK@;h z7bPf=`u1H(44bR5#2V9cIAKog`jyr$(^o9i{S<)XJyvjlS5aagSCk9Uxd?0YG5V=N z+0#Yv2?Dpdies(YaE+vI&o;k=Pyy{BIDK*sI4gNj{?n8Fj!=Q>QZ5W3N$jAAZq~Bj z>FT;(0O2j|xx~b#!k3S}#REWDv&s6^RdQa!mf104B{AT~zMQYzfH+WNL?v&wNu3k;C}z$k5{#DI536IpkO*&sRa6-XVp0LpD2e3{h_Ixg zHoXG$Tnyx9k&}Zi{(yqSY|=B8p7M1Gbu&VJOn=vRgC8n@dqA0vcSxC;vq;W(~UNI6qOQct1H?f znfnfdo6(kD^>*O4Q$>AnMdL?LO+o#RZHuA-pR}*+UL?X538mo+<{r@9HV*W-&Xu(n z^COh&CTJKrPQ;8lBXvIcix-R}!q0v{*dcK8gi!?Mr{+C!f$VIB>^B`-AJn|gM}Md*;-Iim zw3hz)o+HcnPmEHB!;%B`J*PU*`s07ZC}ep5lP9w)4=vNL8)!=^?OHhsL6)z}o&R$LDb z_$O-6TC9A+<0i@YAFl~v%hV50BJAaSqm2Va{3EGqCXmV3)nSPf2qt(!dh7C092on8 z`2hN@OZR~w+uo8cm~(ag^xg#~LRY-J<{b+vX!)O2|Lztba?ZL2gy(^hgOvwvji)4sE1h*K8h#6lgb;Lms6K1OkDhn-s3QEg#FCKC8gn}36)YTel zwZ$lv>vskzm<}<{>O>Sm=98p2)Etr6;a`~UQ+nT23uf&5D&nb^A}CqwpDc^Wyr}@h zWb6YgVu&~CG@*kUSWRd9Drln^W(gq(Em3Qv7*Gj8z$FqdT^+PAm4h%}x8OQD`cKM2 z$jt3uEr{wgcZYHAeG1P3y{X8Scwd=+1R6#TVvI!04ftv#kHHCRE7KW54ap}PJmnPE3< zkBw8_CqMkvlEp0%Cu=Da%mEw`5%hm|$)~(rDyieqwfi9pfP?_0%M^$C`c3auhCPs> zi{WQ4CEPiptK(SJ^s_VUlnPX~zl;QaRW){fP*dGsq0*B*0d}RvLLN?iWGscObhsb; z6W~X9urqR*;=SaCSii|c;hrGngm~@zA!2T=SaicZ2>Odz8Qnq^^TULad|9ymQ~Yy# zcgp0vOYX8n!XV<{IP9e1be4+nQN{NgYF1)&>%v)K#0rrO<@gIkK#A6t=7A0#ijbg! z+vTHr?Oj5;GVHX-StGay%JmI2plEarjC$VYD*GUbD*4CCURA@iZwUo%9jGI4JI|>czMR)0=XP!w zqZ`+HVyDK@@e^%xwpf;Y z#h|Z7;)JyUQc5A)Vr!HLdD)PFPPj{T!z3QeJO_|}GA9X!agZCWxJ0j*cXvf4pK=HOPAER!Lb*#7 z8A%KRHQ^obUYsxE<-G53D+e+A-(*@^rsnK#Ag)b}`ddc-igf>-^@uO+y{i(P`pgfx z>X_p4oTn>=bk#f8<^9unMwfoi(Win}&bKg`Uj&HxcAZ`N@6F{K;oyo}AWw+5TFTVD zess4sWtDft>%*&cA*dl{{F3~};uT?;Kp_i}GZ8?fY3l%lQUp2u7M3b^FIlWv2>34Ri4QYe2JV6LKH&^P!%X!Y!X=>Q zUHk-r!VwvNc05Ld;+I$2zH+-*hgFe%&S*il)2Q*9xciPi_>pr+&C$c2Yh zW|UPYV1VmiA)z6;fg-Vt#AB85JrK4dk$GyyuO?S=|C+Rk?WmupP0(<+O7~rP_5TP~ zVQ1wf_;gA5jmEl}EyR^q{NGbv9E-iZPYb}I5}ruo#8N|;Jc*Z>MnV|kQvUVbTu=9Y zKHg_|!HA}KDR7X0=NbCD$A{O|^sy zN%QEqQw*bzp>yctlP>2FN3zg9uL;@%R0JJw(%oP1M%+=e*aDCJm?dXuF)s3FC`Ddk80-^5k<>c<9V2#g(vwC zV-eIsVHHX;=nOm9KvEWUgeWOXy>YYbEtJ{HR_cRmL|b$BqT)6jQ4Dor&@rHh8}auz3oUkVNe!7klU$lBRpX!3Zm$;UWB=q_;cWmekm;(WkU%w3ByT@W&-#GC1!5h24?%bv|M z6%xF9Ohd&VX3Rw($S(~Wv!#Qvy9nax@Ln9bknCR6_*}%cl(+fEMBWm_fo;twqNH^N zTk0BFAHFoD1hLcKXz`_aLHEY$42-a|aL39+s2kZ^w_b+#g&tDZ>1e?=(YuUWP{SuY zGL5{?lf8Sc6Jx>sK=Js-`_H|J2c^lqEYqKH~6OnP6KA=1>=7Z+tZMKu={x;c=#O?He9dMx+p z2r;-!b05f&CccFhMLNaHj1bF5l|K{!o2C)1R_e|C>n9>BdyF%c4Q7 zR5Y8ux&8G{J3HFKD^tYr(4*{%s0{aam`M@UaRB?f7vF=z?T=FW*tHai&8MKWhbu5w zrSB?j!Oe14p2ZlBo(~NhBZl5qZzWE|cE#-JWCC!?Re)Ge*PXD&XY=Kr>8%)-rQ_na zJ+%Kv1!CR$UU4L#Y!=ZY&`VFzFZZRd>|_*V30$*6KM3Z{AGikEMw7_EAx4Q>98&ac zD%h|#cCx;J((;Je+@!FhnGU60T}{`)SHvGY#&HY2RZSx`ABxCu>-5U@{T$DoZ5PZ6 zz<`JB7norhuo(JGX0#-W3eJffH79vH&n0E@stwzT(^uRehx2B$bO(pwm|)ov^`0&7 z8n1w!vK*rb^uqrDGDhKTsJAeQ?gN_4yWyg=gf@~)^J|-#KbDA`wA9f2TKsQjhy4C| zMDa1%=X4tD_x_Y#2;iQge!`oOsd_schKWRucyCLz*F&Emj9Pc2Qs8cjyf>f#a6;qa z*I)?k9ElnekUi z{}V5xB9r)$G?8=JdgT&$+(|#xs-sXJEnR9U{Q*^mrRs!?`3b3EOUh>U1lwxTibs3Q zna1wHsuqdNZ{)mdv#T==m<&+5Qus$bPgAOrVZ{})udv+UJnzgld?Wj-j$O`X0-&PO zWp}RWGAHp-b%3!qCJRz@ z0%wK%Ij}sa&_wBhaqt;HMw3~(%4C%cQq^8y$qh{;Wu6yN9K0_NsriB2hZ6_YA&>yq zVJ>L=Dqlfl=uAjNod1uLha#|Jn|>%u?<~X|R5y9#{8kQy#d16;YsjrA)SJ`bm!fu3 z60b)vnIaHq^2CfJs$3zHU~%TK2_U!cfp33On~ft26fO7K?sF7117zA$dP@yP)>p%g zRGSqn4-~vJ{nFqTpi|PUpykK=rXld_RD>(9^;6Uttgj{Q{{(X{NutwYc?I*?pPQ7?JOAk+UmozSdZ1PExQ6vSB>l)Ph-GYHy!izSPBQ>Z;iqN`B~r zMosJr_vnxYJWi^tWZ*?)OQ-F%i%Ncc1~x{SMxV~c#AM?a%K4C%0N`| z85E)c9kBSrJi>0n_v841L1$_QKX!FB$qXcCb_J5%8zEyWXs8rhY3BYUs2|n=>zD1G zCS)Nhb22hGMPxklCCq;96bc35WttIU0lSD6R&2RmT&HCQ=`O40UnWKp4K1Cs=Bk%{W{P&y}2iz~6~-r-U?!lF}A(CfLu zz^f?(XD`3~qBQ?dn(%KuLLIvpMoV#1+ns+1%+_t7Qb<|+r-}#Uy1)TyB~v`BU@8xx z=|2*c-*g|BW>TurN${DfL_ly-YcA-o_7D$&GHsd$9!A&(GQ?w&;#l2zgkhi zM~%XOL=)|#8RUjcykR@+tvzRz0~rh(;NB0om5-XGnC|QJle2V!R&3K(>4gJ>>3%}ZapYg!aac3|MKTpSbY{Wpv|$Dh5qiOc2GsoD#jOmb;RO+JQvoK z<_*WwfTLh~J#fcU$=U-o+i2{7dho{bZxNAE9@EMS>Z8=Kt@ZJ28QE-W0iiv9ZMF3w z7O5^13o`hF2=-rbXu4YiHpp{4=;)6-3?+>Q6jS~=y)WmG(W=5znxDrsC9AkBEcc?6 zrm}ppw@I`RQd1RKmyt5SUt#)5s$Xj#B!(t;;g&A2V@ah}QB>$tsJZbvvKo<}?y zm()mlHKvejI4$_-mkIw05o`&=O ze5rauT3%1wtp_f|cg17W_l|m3af(W8pvYS5*HAInP^(Q^0=h)WbB#p=#G%C)^V_+v zEOWp{>QDa~B1a|$Q+x+#i6zUBa3(uKP`%Cl8^Kz%_reL8ZFug(alhfDW2^C2N5bRj z^-(vB{Yi(@y2Lq59|SzjX7hMo*uO%KE-+`bL8)J%$r1Qlt~v1GEQkwPP>Eqwn0MTJ z0l*1!ZY_QxU+o-*&)`k-IXZ7O+4RY8lSL<1rT!tqOS$wh`JpFj$pkvkVGESjsJAsz z_5gi)w}LDn{DdE;SdbrOzIV+BS`{PM4dNUI?=H#@l_u>bGZj$l8}e-s?@Rv)OvXS) zywVFid_T|K3751V_KrwE=4RrPp~|-|Py7{mM@n}s+TO13kb6u{I|?q;Nd_92iVuR& z0NJ1CE87(79xHMGst--VJH;aYZw}H#nR1rK1qXd51kLXT_m)ty`noSC)Q&g82qb2N z;N~IO7~@k;+s=KYKR#sdY)s$~5BOuyCz0GZf!$9|UiUZ>3Ew`Z39+EIedO+Ocdxxc z{Pb)^?D3I6mJJ*3pC?wX7y)p7rSRA+-p{Ed069l2-7vZ%p(5Os3L)QGa>JiG2#&T% z0cIH|xONa>Qw1ZSy<^Rxeetrw6ff9*x&USAF>WGO4Q1{s#d5e0-!sIpSpnwFVq6G*<7*(z9+N`*Q&6>S!Dbl2Q?jEW#oOz| z%^YD>h=v`ULe}tzw;hCoDQ)Q23JB|YW~Jq)4`mkj2s#Z(-KOO^g*{>%Tsb1SvRD*!cDg?1Y zP~Gqdwd3QX{LsUCp=;mNosPz~gkESY5LyX~fDjcJXrde$efEE8UcsA;{9r~QgfG7^ zpb0sKLB7Y6vy|j_=s6h$hBk^6^O7)Oyf6#JoUQ^FZwc~Z2JgilylR#4ovBD4^L-&h z@KZu##XS8{9Sn{(uF+xF(LYCR@G5nu|4M+sS0j_rITO`R1hW*AgR#PYwD87D=JloG z(knRko7f6jn2VDDj!`+s>R<1Yq6VRrQ`8|?5w@l)I17S3BN#yFY)3YE97$^I?gLDsX{3Gi|Zq?^wabGHof}fKK#Qz8xEV6Gq4Lb5v(h8f0rU47%#; z92B@fmj#iD}&dSV(Z`og}?^gAm206p4|W zB7Y%y>n}J!xMbfTDexOyz3%1!FFc{sWVvGFveJS$`aA+gY_kBoskTsvfRF+`i0o>0 zXLR@&u$OuZ#Vvn94_q{Ug%vU{niuLOX2pQ!#W)jm880kS2h~Eacv!llyI3$W& z^q`wn^@R`@vlxshAa=Kij3&}WJ^paHa%ss#V#62du?5(10PtaiS#;$cZ_Y+-K0u;+rTq*c1229s%yyn_( zrN%EJbcU!ben!ci{Wrt1pLnI=m4KpPg(0Oe)ox=q&cSc-$e@pHdJ{;US0Ak6D^bwO zLL}dI`uF||U(H+MO-az?Feb4IzUWVWBSX?s@Laq6RStX+ViNb(?vS(=l%<&hn`?`c z8J^g@_-vyB8}iSHBi)wmxX*?+3^5V*wQ~h6R{Zffg7sP6`3pSoA?9C0Y?IuVoCso> z%RWZjgTDsMYmYbhq6^&Ng#isKK8Qz7Rd?dIrT8L&M3pKaV{(+FAVfwp;0ygVeu;vM zS15FQV~HTOktUzR5J!=1d^i1=rleVS1=aBr8+(MY?-UWmVh4tCU5q|y1QZg2@B2|4 zor!@U4Iq~~L?5Ij@P#hrx15aJ)*AnoYVwr(^Z0K%w+QV0B3UtLaeMM_aXbzSKv#gU zD>n73`bO51PyOehk7xjv0T?a21WD4~I+=B=_fWP$8j{9g@@|Zi|y9ADzv$QBU zcr1_pF}!ezDRD&c9D;h=#_AtQGrMruXBds3PYdUc1W`rkg`}N0VkMiCdNGja7aafK zY4N*mD~c)m_5m7T#NxP0$SUO|DqH}VR8S>|_%yDuE}>2lP<+s2$@_(e%;N1u{~F!b zlbHHVux0n?9k6tmU*<#{OVkD-raVpBRX-WYB6U3! zaEjy?)XT(z_vQ7ES% zD~6w9*zt-aYnLP0sq-3djn}$?evTDlKFjE%lq3={KneH>$4cidD@AuNx@yk@5b&4l zJot*$y%+yL2j-|k-5dD-2vb@eJ?)0mc^Mg%{7tRb`cyj^I*VcthooC5F1#-TM4iH0 z0o7)uW;f3Dvm~NGt)`vsHimuS0=;E8?vVefEmlRJ(N*WB`e*KL)XAUI(7hFtIPAmiw_U&Z0mP#cncjASxcsOAts9rbC z)=Z?{Qk=Se7od3GVu1y-N#MrX^Z_CHQC8_X^XkI62YI^xgp`~}nlMFtmMucZW9e6~ zv*;+>4Y*6GWqWl5Tkuhf^<=YZsxIdQ`_Fj8>__Exvy!=MBEiyawC!Q>e4khUxu5cj zauJSyD7zm@JHHz6R zEx0zxCX|a8Md@8uq~5p@>!Iz?mtr)VVawf!2|lPPpnnc+{oz+dWX~s{kC)->m0DA= zre2yoBy8czfpzAhwPyoJ3*x-6(VoL{keV|AEf3Gtde*DH#;*}iWrO%Rx@)U2x|Qiy zC|2&aN~4a*qi6-eliqj~dt!y9iS8neSbo3?FW|DWIS z$2@38HE$-wG_YbJoA9w+r$qG5x;`nFlgk z(rkSmM|S1{H|5_^J9^&l%fA1n%U4meUqSQ~Lbceab{kb&*?bj4qH2~E85)QF$n5H^ z3?I-v*670_d5O3eH+$`52l_?K?}2g{a6Ns!61ByLO`A&;2RfN?AmGMhEI3&s%6t4j z0AoO$zr!E$jx+@G7%@{9Y_7`CtEW%A7peXXY$3uEtV0?{AUr@yaKTeZUQ=f=m9-hr=6F}>oIO#;0B1v4G~E&9n|1s74h1Ze?(WK+wy6#W^C z(gc*%(*#y6{*DGxrU|~ zeOE05T0L{%x(>Egh+a)A_&&`3_QbDT14UxzTjV(?mgrD=CvN@AFJtBnFb28qnSp1 zOFV~z!u3(u-JRToFq#rzA|GUIz~%_W7oWKT6!3OiGQTWZsE75zZdFgmbJ+nyGq_r?r!o1hC$V^ zkdN9ljR%vavHT`tyEKjO-@bbL?l<)SteU&{TH-VtJio$-Ti2mHC*WzJE;m0XqhBM% ze6RRDjMOZLSgq!9-!iarok2E>g3R=NG%OC585Yu3m@yS5etAIv<{VR94i z{RalqAPPLZACJ+O_=Mle_`-AigrGbQ;J=hlHTy20c3y{PNET!X?Gg3+ZHN0U+H0OJ-^NaV(<Mx+dTgngc1e^sD14T4Jov?Z`>Bb?tj zC&43t0#?Tq^O8Bni9b$77(c(jyi8c~nEZqK`!4xD^rF;(tsY!ruiF59WLeM^krDZ1 z8kNz<)dt$bbBQlj1*}pD)1Ae<6WGo2wTw6L*Am&fuON;vl0zUIkd}mSgu?8A4s}?MyX57RTu>OxNmzVd(15R~!$6FOqRTfd+jIof;_ID& z>3NW$ny+_;#+cDiu5=JSIvYIZ7n=O0Hk`sDiw~2GQ3DnZr=$98GwODT-vt{|z8j`v z2TXtp#nJQbdnphsUq1oD9l^4KjXq*wr$*dG@FeO}dX~TDiqf);I|#oGCL>CKybeQ= zEI*n?Trs`OW~M(czUpwIb574G{}0;lu>&�Rtx!QZfc;E)#TMc`tInu=EoegTw>~ zHF+?N&;#8(VLx0XB$^3JF0gQBqHy3MUmU}~ z`{LI$gk$S0>j%$YdIkb6h%S;dAI@VIUtJ3DyUA?J`gwlM`*~(+&CXNao)v9RBYBwK zofz+ms(1&2yellWEXW&P;SFLdOaELngZ)kv@G1Y4d{cipc{%6xe|!MZ<4Y`!jiSem z1I0Q}wNHw{dlOh)Q`dLc>!9CNwO$)Me$Rr(*EWLZ$C*oUS9<}}t^|)s5s(!Qf>hYS zge0(oamre^--mDe`ild+^c7wdB!wycQ56uAXo4C6a8DIRPwio+g_U6BX#7CYx@71ad(B!!5m4&o%oVL?_JdW9@vg|B;34klT;$ZrHkZv#>ujBthw}!l%Q7pBF9Y* zmIc7|0=RIElmM69BF7b%N(Ek_#va!+Sa_}UWA)V;#dh{(?&>J5DtbAZLQfmkQhYqm zyPMul7Wz$WPUifbq}{l42=3qRqmo;<-q{)s(d<$fC`~J2St?ASXAt9B`*$ z_G<0v#;G0hoi?{)WQZu--tm49o19R5P5uHa8$;11Qu#8adG!1M?!m@ZfG39F1sr#eSH8s)w>1L=!bN z)u!}7vpna(a9RJr)->|~9(s-mHjUNOZ@QAM=Qx(aR6}+9j-#p6an)Y0r}qZ-pr`dQ zrKx(!@$Fad$g6yoy9`_Tt$!L|7>{cqCHa$b&`0(`HDF~?%*VHk2B1nm1G{ow zkVdES!8Z(L%D@Nh|2-&Y+;n^ql6T?s!aqA>Xw0tz2!vA!&$dJ;1_H`b*3(W@8^$02ik;IVMEPu|DT0FD4rSK@t)2QH{Kzd9LDo@7H&YcLbkF8MZM!~v}NPXzJr!2iAA{!!v`M`w`me^ay& z!TLBWllt#DfKils0sxMB9!2pLoLhiDZliNXG2tIfM?d&`xwBlf9RaPk@G3~S52-%} z{U6k29O+vWDx(OD1a}lg0r@751S|qlIvh^%ooF&dmkglez(hQp8J;84WC|A05ZLzg zjFCq%hs*zX44;6`0P5lpGz@n%Y7_p2s1k-R*1Z5T{5jpCUeQO5e&yA_jEbr-~m!%1;2Ysp=B2e51c}^ zo011kjmj0!1E`V|R;oR`{3>U=+QYY2CM|QpH#u~cyYuHYSYKo^&#yOZ z)aSp#p1WW5&vtQdvt_xo-J!5qb+vWbsnK1zJy&0|7qs;&F7PY1PPK>aRI9tr_DFlY zGw1s5L{)pfZ*{&et;Rq1GNP%v=e`7{_jIGLcnVVnHfYCv#{zYj+NQ}YV4=Qic)DqO zYR@)l-_zB3f$5t-q5hFeL%=^*RL1U@Vun)JrAMl&_sm08Q+no2N1m%j-?hv>%mr)$ z$FOwO0G{jhoPkTNzCM^Kufu0SvfdvuR-vJbGMpS?kbidt&vqL~{b|G!KUph)YA z6n;4J0yG~%s&wI0Je(2AYlWQc1jYRotMh`>%3OK}Vx4>Tz%rSmP+im2fvM;g9rR&^ zY=`x1TODi`>%7hGpHEoDJkV?*A@-6j_U--R^xsiX_zt#o1AH%{v4GEPM|B-fF$Wg2 zC@7KCRz1c{&FJf%r&x9PZ>alm0tX*kKQR0q47?+$yc_)M@_0AmcxSk+vMaAY>bgBC zcYIRr>?FIFVb<%}xtoSr>j%2}Nl}h8_#bl@Vb&Qy-8#~>E=-~gv%c;^o%J2r;uMrB%}Lu5Oj{hrUgyI|YU6uKkP z`enyoxjw0I`W?@>4d)g7oORzUs`**+>eOFry7iEXOyjM`0}QCExmX`~cwvL@_<>Vo z6Q}qCCs~8H|AAAaevR?K$x*qVdEm5|v{V2p$Q!jK;p}NxC&Bljhjo9)SRA;44i7w# zJy-SaTp;^Yx;86#EOzWhe68nRxFx%*WxeKPsgaQfsojCTUKB#u9Tg_oKe z);rJ4(fR?6=4fsic>b7W_5tCJF5YqjP(DC&%Ma6ZOadPaHLOj~L>2QfECW`%_9HB{ ze!=A9fLpD+5eK8^Sp1%1DR^F~ej1T07Druz_+rIzUd){FBKaIV?+}uqe8Gb(6OxAh zA5s=v5{U~aUc59pkavQo2tcjjlb1{9c*qjz1<<4@ZH|A+!uddmA;XRXG$fR`EJg+( zKx@%2fxFasYXClK%=;{WsqO@nH=jVH1ed&1xLL}sLDmmn5N%pAKnL_x~g ze*juFfRkMm#2n?cI9+flMMnZT%hj6K@6JuRZb0R`_b=Yo_G;w{?plH=OLthsOxL** zn-@nZZ@t!wqcrMql%M8F_d6dvnP(;}yeiuw<6#B3o24Qriu~CxjWM|zs-}Rjd zGT-4ju&%$gm5#O4v1<{Js{r0iKJJkrweG>3-8?M1lxT|GzgwLUcDJ?nNNU`-lcJr^ ze$QurGLKg5qOp#!zf+wwssS+6bO!M69;2G64EjCI?t49rY2LtcSVnEdGE0+y5EDXXG2|#6Amv7Lf-` zlCrC68WX(w-~XR~{ht=STTX4t##lYO={X3bXaXCGk}<#zqahVNx>n1{7_$^C(RSh{!1$XRavT+z%aNgEVw=+Aam^W9qn&y4Vf>rufe2u{8P>#GtqOc=!LOya=8oUaIgCCa* zkny>FU7lNWhLO_861DR(jGj@L2d>_8r~%3dGYmx=^fbkWKR3%Px*7Sf$`b4+1MS=UZ4T0X%t2a9 za9?*0(mI0i{^uawy)$)ZXWj$IL0V>u?qd#;rgnQ~zs*6qlQ~GMJ5S~(AFb_CxraGO zfZ0yEk_f`@H4mTm|rk zyHnct8#n~Y$yxKfENvGHgBFcVuSrG7_5nyiY0zCxt;nWxz_;E7DWI&!jH@+l7HwDVNDGDCY*`_hh z(Ir*O$92;Xo)WH9`l&y_BEJn zZ*IGm4WLl}C^}8L^fU^(=rTG2rO9^#cBon=+I%;g^3d}Jrr|1P&!mdhS9L}Q_Q19^ z$2K%(DXboybTYu=3nBR~!VA41A@4%0@kSnyqcu7^x!H$uTZK*TQlw!_Wr>oC%tsA(>TQ;e>UVTngq@ctBN^0uT-|!ZI<`;k8uo8#uwz zyj}}z9kf1W@e~02)BgeB6S43E6r6xpI`sqp3K<|mgOUbtrb7_ixB{DBcD|! zn{oXlh;poe@g@P~Ca0{u!#6E#Y#C*)aK;3|I=XI=fr^^%Jk(4Pr}J`ozh zl1OO8=!YL7j?>YR*E3zy@|*!2-L7h>rZR91%d`fjVjF#C*AMcwsM8oFjS-i*76U4t z$3Ax_qsig$bV8HEOshdPw5-~!6+69i?F2QBK=fs0l^mN&X!UbY^iT%A6JZJE7pu|M zO4k5GRLr=0uByJ)l~Y|9dtKg?mDuaaUL+eu{tx2P5tqPH20F8QeaBNlt)K9LUo0}$Y`Uo1@Usi762%CMj<ZMl(26t1Mcvl zC%Ub9?^F#wTQ^PS=Csv%yz5$S9+IWpuH{!M=MBx~#OSq2=f2Z>o)ojLcWld4nazgV zVS3y3+3M|Pt8KP*ZOOTt6DzV+h?F~?Z*X(gahS9_bK{*^4Pl2fu14Dhj_oZ%+K=b@ z!Lv~GJMGON5y|`+_|Mj(vA##6ZmSk1&f;UEY7AxGv_=Qp?RitL!SA`Qgx@2}tvR79 z4-RNj1D1yx%(fXD)8P+(-LDOz#j z_UR+T&g6|84RZL@Z1A`-wjc`IRByQvw74WTW{2%u*QI1(})1x;CP24uvW5+hU4Ea>V_t zkZZi~2MjYyolY}(@{J$}#T^lndp~dk7>#9^tqc5#N9Qws4UFKCf*tbKAA%)Kfzw9t zJ(#0I)b)WhQ|69%k9&C=&S1RgQr?0CiiCZh%HlK}GY>XvUcrZ7w8MyKU!MaZ&>etv zBoR;imQw{Sc~RRYa~%lcfK}Y77V}}rG^dY~u=pInz&CYb%o%w)%Ifo!N0pT3cD^VW zl;*Av5A5KCQIMn)tF`0vM9b!b9kv5GM;KYp5h@&+&VBl2AuTt4ms zos&ph>t~q3p@Y&PLv4m2IZk4s@@baG9QcXwh;S|uHi3#Jip@Pe<;8jF7Ul$RZQAV3HCJbPczy0xl zL0n9U#|0LAqXn-Me@aHwBj@N(C~#L*fz_Cv@$yGvF@xv=*g|4Zd;zdH!iW>E%~V4y z^NaylGs8#Y@Vn&Yv^JGS#;NR4J{H%dI6>geSY~*MCHeI2`<%Tqp(Ez{=#qR%RIO`T zpUJ-^s%3X|?Xi>+85H5o5-us1$D)*F&Z7zWRWN;`bj`jo>J~;&fspWKzI^9zC8+MFlQ&6e5nmKx4psi19r*+yq2# zn#(`<0tFxkQ$DHqS&)uF9*Yy9yC?}6BXz;*(3=wRk|c7AZE+%*DG#)P*-BXshvUh| zw~uGil1o-p{c9d~(IvVK60S>B47||v$4`i=s|TtfG8Yxq?jP8_EFoD$3k%2;Z80&K z!-+T8X~r_DOH(*&Rtw)hE$Q%HbP)y-^`xqLsOgrL<$yQmK`)?CW8QO#B(r(Xr62br z@7e9?p6)qR*A{&+MsS7xz@XZ|Ih#>L$|$j?q)upiei;aK24zVTDDxjeO`D|7?{B z*1(GoG?#ZzQf#kZ4%gk+RTj8=*|gUnPH!pEu|0$D_8I&TQW}>yZ#PP5%rg+RBgFuL z($y_}S1FB}airQ^wWqczjrSp?anW_d;{J|nvT+4eW@awECt7Y{K0VBV-NQV|+tJqV zN>-)3vMD82mac1#Yqvd1QY^D$X{8j)_KKsu;%Kip+AEIxcf|pW|74%8IV3*qRmbwH zjyB8pcsHRJdyx6N_We_1F7W1eP_tRU`2|bqCt>g%*Dj<$BNxy2D%gA<(}YWxR(YCm z>Di+7Ytw@FD=m2WWlc4z@fLGXZri3TpHdjca7w4i(d1Xd>66y9dN`(4U0bmj

J%ppLC;VqJT%;G{yNK}m35bHP3s({PSK={Z%P(!j%;0FtiOXxr)@LT^iZ8mAnWg2K z@l}x}ig?+ed}gvTl6Zkv$dwmPfEbQhEEf^$MION+OwqmNFp8y-&e-EF$;g_Z_{XV9 zmJw4crhvdbf53O!RMs!zJ~5Se^s+-V{7=F$)x|&Yi_91H6>u^C?Rn(_xWli?$9@pV zI!uYr8#el&neIXG+SMi@zh9``MFMl z7;|!+#E&=wJiSghA)bLPFILHkqwtCq%jX7{U#Bn&gw9pUTg{RBVS-ad&w<1Gy`BR) zJ2frabS=*@27S%62Gko^GbimF`mGh2U0p%0y_K|#`xu(dXWTv4G|>3vRF}1!$@l)k z_-y!n2tS4uo_g-nG02j9af+XkZ|W~6FXz0z2iaExV_+Ljf6#M!1J5-a&7cmLoK|1! zJ6eyqn`K{Zq2=IJnUA+*GwKfha>#$W4#j5$E~%anFJJ-B(Q^Lk*SpHlYGUE7dpMeX zOEGP82G*|j*8|P_y44CW!mJNJ(_ma=9 z>#(}vR*v^X!`aKGjn^-?_o->)J>jH>t}8!;T*hV2-i>k@4ZN;VwNHw1WavkV)irf< zC#j6)k)|D~X4h_08Sh6b0WzdruElJWQ~tfo84Tb|?M zW@v>@l6YdRkmlIlZ?yLt*LS~xiCx!z$I+%l9bePM?jGbwHQhutra|TRC9_#gYhF~f zxlfztKKTj1EXY%nX*-=vcMq2m;e9`GrB2{ggwS`9-pd8(a?f z%~$qn?P+sTmbhK3CS{3d%`DTVW9?Tu*7Dn%YMj>XXP+!o%F@XuP47Zye^YT$lN}AFZ!47k!p=cFTEi zGhW@hmkhUaxw`2Q^nlY*uFKE&K!p2^dzZ+Jdx^CErQ7x|&E$P&>5+{d%sn!q_a-y) z+3R;7Up%kPycWRHoEF z6+dE|$hmqV=dH_#2a&?ru+#2t9_M=6@-Fi@SJ!gya3ZInX{v7aVB;DNGX@|5DYJW; z-Lv|xVaht}O3;8__FA?sGE_qdhTRbYm6(5wL>J9jwM{#%= zb;#GVV83nzE59hiHD=+yio!rr;r%QX-p_&+AC#e*vrs?Mz#pUtwGMK2wfL)Lc-F$~ zTSZk&_+JYwvCyB2+eBWAvZw`G&7}R>XMwO5WtjwS?0v#IU}&{mlIl&f3|p-aSSy8= zl)G>3a466XrfGbQ11Kh8AfH4j*Q~Uwm;8Ls z91PC_e~e{bpN(iZ1dH3FlWAUO7;WZXN5Pbe!g=H?3RX3K^%fw2CK{4&;4|8skDNhv z1!T(ICRi3Xh=lh9{)}R*-{uA>)~F?P82hr+T$g-gSnD|;V@83I!Ca>?);8wvdC^3A zFR&oAs6-2D3+3=-q0+2Au|MsKDyk$qEoHsjv422g-{U3Y1a#DX*=NLtd7=;K#D$7-<6fESPD@w-xXZSB@6Qjs0E&3pG(}WD; zD4h_`PvDOvs|rZPDgilw_$h|dU`0VhjvuKd9}+4AGCo$6GE^;vql+VqBEVo2;l6@b zZ46@bqA`}5X0rOB8;!?l$iql7nu=BSvBjX#kc>zT{keDv3iU8m% z&g?WegvE%$2(3q)H>e}xj{;7YyaCN8#?X>Smic#&Gmlom6xp5_)kXm=AGLd6dn`xp zqvOle{x16-U3I|Rf(@{sb{S3sdO0wTXdsvuD?Za;&;;|5t^%?ZN}Q}kJ90Yj3Zq1MZ?Xf0-~GEr$pBdXK>&t*erbehc3tnIz+x1!iEyeFs4J` z4qQMOvs@`V$%Q-cq9_tRhTS&g8*6U4@~l|7U^;X-Pu-Ch4FQ|#cIuQxBA>zljaZPJ zDQATVjE#&hRwInUgCSUSaW>~D9>VFJ$U2!c77{=(?ikEu;>)98THW{;QOxVmqqG`x zBT;UBoQxlcb7^Mp%i8CINRE?D90(fVAqWGE(dW%jh49@=gdK1~$(8_F(awkq{s0S| z3VQR4SG*8CCKd*DHK*K~E{0HPVGOc&PT&yt`R*>4OzX)?5sspiM(D zgyP3}#Sn&Y811SQLWPW#@jl=!Nol;Db+PN)J=Jhj$Fh57-?Tl&>3J?Q9LLb9LXAP) zid~u1YvqDf8G!prGFQM}B!kV#(5gs*2UX30$HH9wFw+TQxK^o6*l(ynKET#(B3IOVWsXj#M?gX_u;yr0~RX% zurPM!tgVe}tEfAY@ViM*?OQ#gul4)Zw&|&S!*$O!%il${gT95G*fd?+Gqf~awfdT| zbu#a+sb!fa2-`hdM7!VDELGQf`-K*+pQdU0zM-XCx&|Un-!rrfLpLqeF!l^B%LG+Q z?eCj_FnWdxi0vD-G+Qyv-u}rg!vfKz?;EwOUJn$HzPWE`*@^{nYwrYv4MN^f_f4FF zfavLVf8WqDRaMcAy;E!~yRVw6vSR^p*Xv4MbNQ$1Gu+oH{&PEqyY#yx-E`Om+pjwu zEqC8oef?1H*>NWsP~l|4uDCAqRdMHWkf>{BX>02Fs+{5Kh#yz-geW&>~YigmHSdx z?3z3z?Y3KK+qu;7fIZwPk0{RnI;2j*Th|?Gil*o)n9aT37N>-%8r$tz#oTUmeo*$+ z^*3x^y)XOhdR%_(K3llYR^>BeBW*BeaAnIQQBh__UqdpO+n=Y3a;V$tb|m0?+`d2O z;sc`q~CNYhKvRmRlW=In=jOt2YY+*~e{nO?R2?w%euuQf?t@>n&u7Ewr8A z(b~ZVv|t0Sd)lG0-oG!~?t1R(+ikaN=SzPZg;2HEXK{PGgSMw^=Pt5^K7DWQ7Op4w zq}^xNV$3Om`L^OC!(53qYfNA9^q1EOYJ2$IKo38>GrGMFenm;BqJv-Y zY*kikd-~maPro&57*$Jl??o%DJbJ#l71nKGm6fm9FRZdu{tT-oUzx1=w39Z8H{!&t zcSuW9xm(TLv4vlJuPY{KSx&5NAa%>gM>sJ1^KdCUujGx2_!jMo7gFIyjk5(bRd-Xbpi>_vz^1-6RT!9rB^_&VKkhvy7n3 z*O}q>To_>AbX)VQJzW(wuhV*xZ@TC6k_-z-^ZMv>TP9NP*h&nBQ<| zcW|by%@eog;=Fz9t-^&n!ebluy*1sg&i9*XH+yn8%fj&W!x=P>Bpoq~etffsIz^VN zxL>!rS>5>k!YZrtvtL+cRpa*zD_M|wzp%>ch-O&bUgF#F4K%_klG65r`dPK){o;wN z|4ChNe-p)C6n4n7)~ssHs@*y+vtnesa#&_H$#&ssY(#2{3T2xbk4FCK4fs5(3v!FT z&gx3ulCN>MWZ0T_ZqPb4RvL4@C9<3F_V$cv&zPOK?zG(m7OK1Rzlq)>o1O`#y`ww6 zq7%u@I^48dj_*_V!1+~Xfg2RDOBVc91(t+5P;)~TyM4CQ$!Ma zmUS)4eIU%tt1yWpETVyK0x#wHs2TAA(LBk!KF{ zjDawWD_-bj0bp1OqvoqA3RGZXnv0p~SY)6>#!)>bz18tr9dEvlw@z!T9b{Oj#%}Iu z8>l(dZs#8{P zJj1FzD%+!S8+S-^f%F|5Rob>GD)n;doBPVQ_1b5}({IEbvS>kW=N!>)Ew`na1#Z;k zeQWOiD%0)aAoU`@CWaQ&+}`GtYADgYUx%pG>*;;HZ`N38y|%BPO?UO%1*56tP#-7nxngouibFzZt?(IDOLVD53t*H093iGO~-7- z|81LCSwrF7Sx3CD@GdO2ZEv!(F2kE0P`ugjEtpu<1-Vrtr?ir{Wa{186?m0*ZtR3C zG2KRoL-8ET^6^G2jR(eU9~U>AdwL6})4F{inokk69wBfYN65AHs;O}4yj`;U$`*6? zO4CjO!@63q^66o9A<$ly`ut^nA*9-0wpCCvV&J#XtTJ|KW?%BnrPc`r<$Q z?hDZ|O7uzdMPsmPmm!lWmaUq)W9fs+YB+3{ZagcgemM)8fQP8(aD68ueuzooLSG<(PBy*FH&7vvB4V^Q>cmxq1WM5}!J(FmavBDM}1pejuvc=Z35k>`J*P7Vj{ewK; z#jS*OveSTeFtjIiqlEIv9y%3`CuXQ2B|;uKBc#(;)IGyqHKBJji`Ky_`0t~ypS^nA z;h{kpCKEUy!zuCN6qp!eqkuf)C{5NuyGve9$p{-2ReD%tL=v$`F31d+!xWvXS^qIU zp$MCnBvX{L*wcp5ge5=ySn9%)PrE}H_BX+Q^7tsvr*RCD9`V55CYR(A);H;rZ=(zB z(8OmdLdXW}0wbm1XLlqqf-xp>ZAE#A77!^zS`6)GivmB+tOa@W?3>2~RjvS6tSf!$ z0P~3VSef9kyrT#98>0SL1XHoklOXFxQX5+`52-UlBDmq1#Ne9@gYFv)^72tKBURpr z*svpFD&*sHGWK0YE|3g)Zz0b=hrNPjY3$@K>Xg3M3H|4(FHv5Jy63rnmm54G+iJh{`Im$UcTfn zynOlcvG`aF`s^FNBLq;4P4ltwmFz!t83G|Oj7>(yb;%4c*~bY<0ZUK4G4P|vF_hi7 zD#*Y{T35A(fBWNqCTFlUjIOTbf7so=A^(6?as_!jddV44Q;`nX2M4JFL%|Oy2twJo zs7xdNv_vMcP*vP`7a0(umvOt)Kh=J3Wd1FMzD7j9(wt(CHG@!VXi|f@y&7 z@g`fqHsuj~%;w`gaX5$Z@nN%ba$aF`EP=Nl6DPn1KlpbtjmA7e5a0G_rqMQS9J8 z0u`NJK6>%`<;Ro9aw+S}FDvlOFAH+!`_7$0vVbK6^3Y*w6hPY)!6DXcD41i^T zE|V?HW|?)))R3`e=L2UE$~pW_bhksM7~mO=?>6GaCCwp<87H0MJR9FEBow?V+gphh z4kNk9ASMRVRw(z;yZt!QpES*<+Zo$Fm5JY)+7)%EyAh z6a>~eHb0c43976Fdk*bfj3dTQT)xD12?HUXaEgy33@Y}qa|r;LP?T1e1TZc^;|o+0 z60^MLvLNWb5(*$p;bxUkR5weOl4x;*jk6WiI#9F&)#h4Xo)K3u&0b7LVU($533^QlnyzyOKz257Np2YC{}2@+DSEWOD_5=~@Oy+htc zMCGkytjAA?sUKLv7*Ti|z^U2q^?;T#KjQ)=>gp$qJN zj5@I}-%AvfzY6oa{v%Gi=x5FCsTo2R237;j%C|gXR)QX)k-hZm&32v_F1sHli1N{q zrLzHgJ^J22VS}Ehs}@sLYFZlWn}%U5@h#LSw>sf)0%;sMZ^7G0Q#`N_k~m0TN8ZF zI++KmcA)4B1ophu#}Bx_5?3wXi3`HDI^|uR#a{^q@1Co=uXb&u;fByORZ*2C;8qIT zssPqBtKaJxx~5+GS&h^{gvX{BDa5%+Q9a|gpEnQI@i&{+gG#HG+3FJB! z)9S-L!%klyLze>|A>9#o!s5>lQ7SDI z2ad+8d&@^VFdEijvu(t{HL_WSg=QkGcELs1=<>euP)QawAHR1>7%k8sR#AwzkVdN< zTzLrMU1gR$c8X3%<3updV%rrILP6S%K;)7O!C+jJQh~Ll{97MV#T;*%R5D1rF3oHE zj@O&o+FNax3Si^b+ebrS8)^R_)XN9#Tl)P!e$n~j^Emk8=!>U+5C0{+ZX0jEu6ALE z{XNEI$*?RQt|jbB0QA0vxIQZtzSj zHr6)5OcWlJsD4t_BU3-pfmXKqTLEe2^#)+TyI~?6=n|lQp!5LdC#8R+Do3W)HO+nn zVC|DEn65&NrWX*Ccr3u3(C*x4f5@OW9E;nTy?%at$Jjy|k`S~)~@ zrSrcWqk4;FrAgLX9^T;H=HXerAN-dq^nvFM<}qHl+sAy;y>UhtC9P}sXC-RqcW1R^ znaWwcn~-ynKa253XLu&SWBElngp@mP0p75P8_6fy6L%1qSTE5kQ+7(0MV>NKu`G(Gb(=y$_(4DzWXb+Z$}O+*)iv39H%o3g zQ6O~yAX7Rl+swkyDbx7{B9e?6=$w7tOwaPKhm|f3s|J-;+cU@{&^#g$CX%IX%YWQJ z%xP&*;r2SNMVUT5l+X97WO$ZBTSKPVWG|l1b&V`6u+j7uA|dZr2s+)rtb~>lHv4Q> ztNAp$*Ab>BMK9Cw(`+9f^IeSD#PaW_vw5(`m<5YcYUYQhd>uzNPUwOQww(YN+twS~wv&!+yOWOXq+{E*)v=vS=AVDQHFtZ}#dFrVdUowo zyXr9KICSWMaRqUs$~@2_E-3UW5xgdfSJ}iulT-SL1hR6+dWzPWh;a$Gi^)ayX%dZR z)M4Mg&U7XgO^0fQM++se-t0CrwZLJuNWi9pb0^hnqM6{_Wvj?J{+0fffURY7pd)Db zyjq11Fd)BtciJ@YIbVfyq6H?(uD48&Sj}TQQ>{Kz{V5RsRPMSN^<2K9LT(pZ&&fBO z)w1^?R9<4&&IfmnF|KD;>UdDCzvd7my;$m+g(uHs;C|#&ueTlPHaaxwA^FnI6*LGh zofUCTh`E6@)$yf$el=06TRYVS4f{czH~fl^rN6=rC$>-I%&l@ae9lud*yeK+@KMDV z;7pos;zqcBS#CS^c4m%GZqHC?_&we1<}md9j<;F6?{O}PVdprU3c~7etdxs=V{VNi zcqqTVOrN(A>sP%Q6uR__Q(*XO_+GAcjCWOi;R$}(ZiU?MzvX-k_MNo6 z$7!M85YY|%4)!Pcx=5XCd0N%lUibE25<3qyogXg099FBXj{kM7sVa9l8a&HTZ6vyY z_<1PJen5y$IcNDfh6OlZU^HdMn=yqClT)FEUfE#23O4(kLq*!(L;X>e;A-Ipn zq=_>P@mN1T7UGi=icS1e;RD>qV3AxuvR_Vm~8C# zBrpgJwgvWoQ?3zu=7)nEB@2-s!&1WirLsys0}ig5FpXLef{4gmmJL&-j*;9q6Tlts z_X2q;jYCpd3Fi5%D=^O+$Tm3^-H&vdj1==SF!O8c)%0`rNZ3WFJSEVa{5vpz2HRQ6 z>W#ho?QCTJfpBR%ySJi`jZb$XdfYy>R^GS!fUwnKT!FSF%y)~g?xW7 zoR1GoT{wR<=S^YSj{ST&7UMZtX3UWjHf=k*zl1=68nsEYM&K3BASYj27YS~Ny?!HCCZfZF~hmZxMpJ`8em zyp>6n42yEhmN+}f_xJtt#B;pEe>EDuZN?&eMI6soU9|7=;SENMCts2tIn$o2h)kdg zY4c%hy+~TUfGQL?GlP99CEELzXhZ@hpejU_xB(}5M>*%}$_?QeFURZ;qXMStlMok> z`BD(%55Y~euhDMV*LAxP|AAj09q`4Nq^KgtDK+2Mt9T=yKAfkhe2m78DhW$QK8`I+ z+DV8`2D2Anh2>R7=kz*At}T4U%x^Xd`72&6>`}&YmA->nl4jh#0X5MbY$rm;HZPDhR_Wh?(-Bjqx_U`u`Y_kb-ESR3?a^adL!wno zK=*~BAju!>26@(oSh6*k_NfI}uVoy}jTe-m3!wxsl21u;gJ7rB!lOwu_E|8~M-Lza zfqA3KKxwf3>rz5lV{Qh!(@{57I829}CDK==K@27W3S~0=pD0|Y_OMQ%jUqW6VCJMSKCHj$_ zD2&)N#HeHwA}FFvg|1vAM8q8;B89f5XyPWA;s=U?aT#dQzT2;KIc%8Gdc+u?m9!6O zj)@UFX+y-m+(Pn*0gzs{QGFyxd{W9EnMeCbCrV#KCTNx*_kUFWwR4%QqQ#7?OYvwN z8kz;%GvOt;K`yKcKJQu5O$g!aUV*^`)>YP(B=FZckbsQf(2>nZ{(=TwS0F*{Wik(| zb-@Wd-Q`hcxs13le-RWXS%u-b+y8rK?vlAbSx=78f(REEwK|l0^+tjgdJY>?Nxc&; zzz7s_(xsH}<4-Nu^=bz^{wJS6@omm)KK}Iux*qMZS4GUqbxP?Gbc^i^|6BIbsta_3 zWyz0$F90VP&eGZQcfg_265Ylu=2Gs?7|RmuwC0%>Etdhwx!`dOx~_k$Gos+hD^Ko5 zxtKvTCL8*ID_^X3N#PS@-c(iI@A*8de|`(iTWca&`*2@MGFO1gJ54v{V$ga=*Grc@ zMwis}7_La-4=jxz9aKDbr#^v@^2dLy5;mwcXI%1rjKZj*e%#$@(6CGkRwtPtW1iSn z>g3TQ>;U=(=Z~DUnus0F7l=ohfjV>;QQ+XLuxUZijDuuxfzKoTPm&ueqTgO@nNXGd z5ll;hjng0j|Ki)DjbZL`cVit|Ci*zQsayWA@Jq>w+abNm-~Hn2XEn74yurg@{tGLn ziCW(%aU|~XJLstPsd@UqR?7apq?*!gcY&`RR-!fhr94sRs`vB5r*A~W2sXOJ)Rw5V zcybjzyT;U(s_upnvG18>aoq|Nb*UULS^1S__M}$5vLOfz+@w;?Z{n45hl7_j{xjrT zFEL*Cp=uQxJHK8b`A4(;l(kNvGI6@WZ!_awIJdyw;Q%#Y8891^Yx(F?vAx0ABqVTJ zVXX7+PEkRlG`TI9)@ln&Hn{v5>rPJux1}cUGUGY1xR_`B{3H(#)?RG$nb&ObhH;Z- z70kEA>IYk92C{Nfmzgr3)3QVCnR4vj^RdXl-@3#X9vk{X$skCSwy4?UcqtmuO|_#%I^Pa4$(v zck%PHoE67gRLw?Ky#Zakv#c_`>Q2W1h%b$n-7;5D+xn2@N*0 z?+dX*Z@V5aFa~2qWj2#*_rh`=KPBQP`06}so2cfWmgThz2CN&VotwULWE8`jTLlB2fUEzvw8#HX`G3$S(Ep(J zg#Rg@!2eVJU-bWcjIi1U#+Hy8yhz0u0?1WGNC$gC?xeS0;rc2 z83_4+fFPZ^7xmHzj0ciTMf7HijK4h$MjZqxn;jTk86?=ml@Y(UvR4S}yO{W?v?ERJ zoWy-14mw>JfcT?w7J_(2&YLU!{)eA;PWt9<8!iJLfL(vh?9+T$cfgQ0klYt=1_$~>hd$l|EfwCI`?3!F(fZDP%jI+P zHNNyT4zM}Kwa3d#cQyXWT*)w?&VAoN z6+1Z#2R9bAxqc;B}DR(GgHI(k`@!z@{!yuj^&RS^y+ zj%e}rFeEwi2Mj5BmqsXovpPk_0~2YwlDs97j!mjbCWzBiRw%x8}lS9WooBn^` z)-_vfwB;msnYaN|n#zZWfSLmOOj! zbESg*xf07h6WZCkU6AuXL>>QvK*evSm}x(Z z$6CK6BFB;$$t28keszMIKREmSaADyJ0V z)IVZ*JXq2S)HjgBMt08w>H;Bs75sDIPgF6J-9E7i1#7QXkn!St199JW|H^+TqdW5X z3cj`b<@jH_D~biFS5@X`M?P)DCl{-CwIkxhw7V9BnVyVwK|qEq|lb;)P3e z4KMj#xXO#|iU!D+DX$Gx?X~%lJP^tf&NObOmH?};q{YeXg zyUMT5m6Y@>yDS&i$Wqx+(P3ZOVBUI?!%Y<;C1ac_W-Z0s*ccCasws%$e+w}U(LxC4 zNnqQ{Rm%Z);r|?eD!ALT+e{)Ag+`s2QYLF&&%BVEG}=}bp;ceAgTge;#o5XZ3^Zh> zrN5C~JH(E7cb?gi??T!iGl_0p`%)|$0`_sXu-Fbx@K-`;RPa}qWM9+ob6V8g&v#be zu0M%(K}{u|2Tu#$vUOM(Gu*~Il=*2pgSAy*e$+!0 zDP}0E)b;1&{h!pa)|qeqDn&9Ap*ibio`5)2T=d$Yw3vS(#y|?8LVV%ZDQR|q=Zh4H zX0f&F!A1o>n>mr+Ek_oz$N~s9w@UF>d)k>V!5B096~6fKXP$D=d=iNW6R0UXd`6eb zau+Ns{$eAE(SVE@S(R4?eYr7N>i~D3!x=(_*(}r*7wI@>j>$KjZvqZ_^Mkg=Ox`%r ze&*cK*D>c3)flotwZKt$Kpw^;-{d8_Jm9pxs*ZYS6PuT$b}N(@6hz)H+6&QpIoEWfJ$Yp||dg+x>6KZ8sl-PKhs-TML zX^r-Q(fenVEUO-RU}M%Pgqn_TZp~P5#Hqk4fJ3E%CM8GvMJhtqwA%?sVejORdg6^bQ+-F z);-(I?+kPqqz{#L6s_V9{cNhInF(k!*%}7D{36}g7zx0W6=S&#pD;;F^?)H>@n_5H z|F(+n6nx05&YOUO%&kb$gKjZao3deaBE6bblpA0dEJ9-9xa5* z*;_;bpo=S+0n}9W-S}K@qN&NxJBLlhs#jO|5q%ALSP_6?6fAI`n*{TD;cOCLt5qYG zsiR?q0>1kR-@z{5p^Z^)Nhx*doVpc_xo7Q3CK@R67AgAR^Z^`-a6nKlHE^n^3jgjT z&Ez35YI3ygdn#nYjASbJw~jCqbOpMpBeGHJ(o8)^k9N{re2tuF4+d=SdgiX!B?(2uVgg6RYVEh~@81t0pF;zJ zfa(hrp5rS@9^)B^nhKnC{HJbbj>1d>Zl^@7IG;Z9(cKmNK)}7v=J0x zPR;N)B~$d;v?V*wJ*KK~)v8cZY6@BlI7T#^Yc=)}ErKc_Q|Nf& z;iDayip0;6*rU-3MB8pW0J>^KwEv#9qcP)slT4{d(a<2`zR~{IV^Us???kKKI_Zvy z84R62NPKCJc~8u_pXrYAj)@=Ai;5}xl(U@yg4M3bgg;m0sm#E{b;D%?XG67&ZAqEX zV1uCh)%IUi=K~#v(^I(Mx2|B8fNoa|(ENCzzu%YX0SII4@Dj>ERSuoHTWm=w*A2Kf z`-;e`8;>_M2zm~Dp~uLFsdN-*HE_!gSkKWnx4qLyb-rT9wm@~q!YUoXeFMo%AQ-2s z{2Ve9&k}Vs(Fwb2x!XKXft?T6fvVtel%IL4_M`9j+%4_A*{U|Tv7B%xGg_#1sHMo! zZW4Sqo1RE?>{o$w&MCVu2hq?RISZ*#0}xq5DaM*9WKb{)pjsWjv^EhWfOMG5ur9jqwi|hiq66h#7ezWgyg*#HvUYst$L_ zsN)q2P5fyv8$gO5MEOxlpCk)Rw1I%~hDHw)9N=5Dw&@wH;SPElc%nVQ7|JrMK!7`u z5g8oKzv$^8I$gLg1xxn^;iXRLUNSVo8g3GGL-cmB7N#_j((I1!i@$c|u93s@mX;zs zUg#_8fAcN?Fw$B0kef>F@@6q+P&ppc4?@xZDGSQAx}y&VFJ$bd0578 z1l&#yDB#sr-l&F^DIgUP6A8dX_lElQ-+qmB$CJZZV%?ey0KYn!3wd)P5f99I)ccR( z(awkgYj5u(-|d4S>buT%GNu}sl?Mf8pGq71VPVo%WgWTd8Hx+Bf=2 zEB`S`v9geaHVz65s;ae-OOVrTN&{Rt9dBuFXl@qROQKP%B1SA$dmo3|SUON*NsIN5 zMZYD9Jv+nD`rnJ-yU7#-69tX>NlZ&SZMD?D!8+=$dxvm;w0gCdjK>k2YVQ8|Heb-> zL9D!^3w8{$11RyASh7`V@iN?F7;$Y2Y;hxvQE65g>d~ridql zSB@x?DJR)bMcY7(6J~hcP+^&7935wg+mLK`IsoB?RXYC{N+Y=2OwRdpo2quH<3MAz zT*G0kr6OS|z6$Wb;ZBLQKbs&?LIK;aRFp-!f#ND*2WSe*BCU4!2-W&@l-Mt)v}NK( ziVuzK&orS1I;z~xFuM{u>dFb^$S{IspHeh$@8ETb_c&frs_5+JYgRA6Yieq9WuWR6 zfoQP*2yd}YJwK`7Vj?TARR6-VPIX`@t6`MgULF5ymwP8ueVvF&y`$ddPlZ0FdhT43 z=4uV-)@qhpsZ`9vN^2zVY*&6Iv52cFQ^?)aOkFBhT(Oq3xVNCR=hjEKWvd>UxXmpC z_u}qVZ6>iudkLW*n^4Y6VCTkPq`4SNj55^#lubk%QeZ3_7f($sm^n?`};XYaJ7s)%iLv+SD7-48lrE zYS^80a6`A6&yYe_s~7n^dch)s2jk5KS~&pT9&)Z__TC*iVRo9Gyy#oymZTL8ncSrc zRi^E^eu`ca**qIpLIliCCE^?N6~=W6=A)J1kxORDZj_amzN~tD?{b)XuOHdB`)gy` zI$xItip%35qP)`QFfo2K@EDf#@|RO8@_Nz{T8z{OKJ@qY`4L=|4@)CL;B`tYFAMXG- zHvThawuZ8=dBoRS$QXv4#U(}P7z1~#`UHJ+JIxCq$i@%o5qQ_X} zn%+*eP!X?p-!oEUSn4C&1?&T-i8hEi-f?kmnyyw{p~T2FyFH?L3HDSSD6a4be)%jn z`XS~ryc+pBU0oZNqGg?^BAGgK=e|Aft$FbAHBMn0Vlvz9DnNPR+C38^j|8hi=3w&k zzub}dwO`gCd4n^U&BULGbQ|SAtr4FW^t2K<@#!`QXE22WVYv zsxz0;@KdS!D^)!d6eRu5Ey_y1NlSR;9XL!u<`MZuCvORs?l~G$FWU<|tlzzrZG^Rt}5#pnfav}bd{t%XwaBabRVVeG~HPFDuyOWdLxdJ5CT-i z00m4dM`@GCk9_!|Ju;rcSOU3UiQpq|11P_h_wa=y6dm51F#JPd-i^sp_5v2VchB+zx{m zdNBl4v#aBK?&6H-eEhILGTH8PkS0XjT89F;^CS=xl8BB2f)EEoPRBMHJ!x5Yf$*U8UlF#&92;cfZC&LDwp#_B< z^}lhH&k5YOgP4x+R0m7r_|1VKm;^Is`0P{gtHD2mzQaD@?ACnqKI)FcZsE3exkUCG z!jj>UROuau=Taad$H+}hb#6Gg$ND1)eOGl zf|iAp->9fgqI?lNnrLwC#CD?l+o|$p_l1&=7~Z~^#dbJb?5h#SYFWIhJ2Sx~oLN2h zD@8<^BW8h7gm5xSs3J=~3gb)kYj_GRL}Ozs+9^C^I|8L~CO8IvRYX)!M>VfYh2KUN z0&Lb|X8(NmPXy!C2J2!u6?ux6Hq(zif<=wwcF*Qf@BKbxPRW#Re1+T;^R8b$)lQ5~ zkOEI_hssSXW^HhlNShwLQ^x{mn1~w3s~%gwJZ(*lH_xuB=(ln0>kAwaR>6>z-)lT^ z4A*(0aYQ3u-CB%h+f<`#KW z_V?q7YtdTSea(o|zEGzXm6cZ+*%VrhP+Un|nkh}kX{cyj*_BkBaoleZe~G57W$b(l zx%QY?Lrxw=9gxDVhaslAmqO3;${g%8G`BO?W*v+nipi6uUf(amBq7Ni^Xg&hyqxvS1Una67a(c;N&#aB>XlM20{9TK}9))x&abKI-u;A<U3M2cWCt9 z)Kw6jqyP2PTOazD{aS$<8Yl3VRWN7M1hEz(otD27Ct<=E@r-t@pF6(~39_Au9CQ`#3wl=2D48B+(2!yI*Ma@VV``6QSnD4t1j8Wlh+`km+Fh?W|& zWr!m{j;5zs7{frLe4@WiFg?_as)-ku#ux%i3ZX#wQb1CWea&xDdxhsFo%dkVae8Dn zZ7w(+OaSQ*rit^6P{fM@G1PsYxa0qtzZWnj@cP5dr2QRHN9elQ=(i*%fuWOFyA)Bn zy==y=v{$s1{=PRb(B7)t?IA9IiP1tZLXt#@7%&|sGXSytDFQ^!PJ*!}7|O3DaHavRN!8sIoZ}^G1l6#z@oGpX zkdA1aV1%3_*QfZR{93vU%83pt%k_icnalyE`3)+J4^>4l2~l>xIVbpdM|3RZ6xLD{ zLpRYRnMO?$WDPH`Wi~M5@zL8Y5r<4jWi$-Tv zR!&-hlhm-j_1H0S{R|j3a~b$2l}b(%PiIEPqlSE#SOxo+NkzUwHDknj;JigvT@s=h zZVib{FNsYRM1mbn&RPaS*DEEfM7Q(#_~A=3VSELx@Yma>UJ&NV+*v!6)p0u!7z8Ea7zTQF&pqgfI+>`e}iFq zkCGua2Mgbz9&VglW*tmn{MQYe2zS2Me&^sx^J?8DHb}P+5k<_lr<2-s^hg28)<`ZZ z{2junvgmB0i&|{RJ@ZMiA(7mm*6@Xl7A?5br4IxT0svNUyQ98bwb4}f3P^=;qu4&q zl|mdu(6iV|3De{i=tMmJD+17jqi_^Ync47e8NoNY0b zeoh!8Jelg{{0*Tfj=%b*WU`27s*YEOh$p+)%1EwkH>2oX@EF|*GkT`!ovrEQkhP_J z7m8OcY%GR}5yLpB4TmccBX7ij5evSlh!KMZi4_N1`61LT&5kJwxJx2@{@QHDUt7Ak zfHAhT;cnf-19Oh1Qu>V!IwZD8@AlEK3AN(ak$K^2wr_^OKd?k+AW{ULK+Il@I0N&=7|j}7%&2TB$8V;_p)||2w0}IcL(wXO zjWuAXhmG}`O*;U_jH;pSYss{o^6+JA&MfC(bMp$U3iT-J9qU8xo6hkzwk{hOe!kCN zLo}LWYoE`mYc>70gbop5HNP5@Yjr8n7cbFWyk#SG>RZ1^4$8kj0e5E1<<=gG?yy5= z>o|jH>STo35Ik6L-N$dnHd7R4__ZESX(e77f0Jf-1UiR7LiC4f_D`qfI(P!ByNtVV zc~|4s2v%vG2ZR2+x88X89wXmoijIZ97Hu=(m|HA2s0VB~WreF=_E`g>Z)kN&y6xs`^aCaRD#dU&pvBunbdJVfna{G}cLx^5EfWYykYT)zo6!f|7R_CnS;26yx%*NyBoD~AnndS5*t6jS@ZL}nFZb5Nres-*SJ&+!<05I z@uK%Q$K=Z@r?jm%#&LfYtur*w-vM%{kw;-}IQwTX0M<*xNVu{FJoy*E{lSKW9%L|va4eMkL;iRz`6pfAfE)A&lmbq(JxcNH{ZoLA2>-Ev zIO7QdlLqA>=s!j)j3gbLHh(mRG4x^f?gAZN5dWcn62ypkM#a=v&R9l$4WG42BdoDQJM|s_E#@ojrvZEw+Bn9NmP?BG=tFzv7fBB1 z$Zu>_u85s*rva0w-T+_O+7f9X;g6x#i$<(~uRZJa0nYHww%m?gr=83Q8C!-sUEW6L z*VxdbW^;tcd1{7dZ0%aDKfDs`y9^Pz2DkHrYzN-3qpldVVZF4K+**a_%fD6Wu0li6 z0lls&ps`DBpueVvjU_SqVi<{5le5n!vPL^qd5m+?<^_*6fuJ}bGn0m#8m&}M zXCTaBbU-{ANdtPxRR}tepN|LY-(4s~g1ns(H&AFl_P9y~y2BFBoluFMUopb|#(x-s ztuXTOKVC-(3MzyD`sn$_w9HWiI4~^$@@U#)gXwVCNwd{iV}i$={6z)zd!gZlK;T91 z4v+3jLv9nzn8mpSm{?c!=REg^l=sOHPzAAyURwjESdcb|7k=6j3(M)Dx-3sL-|3#O zT(0OQ)R&a4ID*ZBD({UWsI;wN4)dNbzkhGQRX2-9i8uE^+u9#*9$e!+U-*Mqc*+s= z++f2H(?}V58|T;7tcU_7QdrLM@cDB$@M@B=9x2a)#;y zf#nVS9C)UX(Fs$W-(061aj7{Dr)&b-Tt%Zg0%u}l(vTuTv{Ai^UWjpAg4;JWWev&r zgtneHD*@nz+?9~6LXx%#`0K9V!exOYqvL;p;IV|EKX4H^l#GP*NzY#A^tL(>2x<(z z{bA4hH4s@1%xf@q zxq)Mc;S}g>rnIHxGC_9aGC^0ylvVl z_pjDyDCI(yUOqB`Py<~i8h?52M+`iJ<0)qV`G9lcprqsLWh72fp!5S92Z25TZ6Gpq zg~FCD!m>(&e!0J>3B4m4rXy~@SlazENGi$|TpVP@R{z(RB-4*AXvuBfyW8(t~ z_a^rE($`D@mzYw zdo(d)5S1xWTJ|{(MCVo|zsep=dLg7Wio46By+N31p3qgTA&pZY%^wK;=^TP~DW3r{qryr>HgKxUz}>$WCm;dppihq?)tF3Bp*?oD&aUIvR;&%GN&e zjxzCqmMnPCYO`XSOG#1g%6_M#X{O4;Xys@o?B;5xDgWSashy*z?3l5$`{r(`myIM$ z_lax zC=s@)CDX_96xLM!EzZVSf|qW^1EiZxq=6gp?#|0<2=Y{>(wQtA+El?!9WEe&3(ac& zta3kwj)nK$Y1#*JxiJ+j0O}P+Li*(mOOhfeEQM)|ZaBFjm`6RZp{tk@G0Jyu zkLt8gr9u7}FhOOoejFrIY*XExQ9}Qm}60KEiJ;lC}By4hVTgxXWY1OJ$TsSaUg>hlWvyNd*qS)>!M7*LMb$FIV0x_m~b242-pm zkmW6(frf6J23@6~J)VQIZd3NsB(=`CgK{Z=7st0K^=Fzi!;Oil-OS5(i!F z@yO|pV>si=Lw}=7p;R~0KH4Ag0fvC3)_>Y1UZb$>;{E*k8Xmp$28d4IeKJYKJ_Pc8 z!A#2+TAm&_A+1J)Z2WAk`v*Yvf)Tf8YW5Ggu|)9_QFgyj=%HK;z!N;x4{tPt_Jxe9 z8D$8;k8~4f!0ifXL}a=48ystBKvmOu@NP!}Nw>L@{P^oU)ji+GvsuH#ly`3cy$8K> zYf&gD#vlZ~)bN3qQiaq#rzSy=h;iaX#Ow-%qwLJjCs?N=G&ANKNT%{kV8t4^XG{Xh z58n*g$XO2to?c1^ZXOV|;R1~k-WQK%L1Gxx{PJ;a zR1{i^!3HO1$N;HlF7|zrY!zihE5A&nM0}GhI45PfvNFilJX29q?O0Y~24`+G^@L-m z8yWIHUYN<-Lw`J&&-CQH|F@*XP1e_|+uiWsq$SZg2b1Z6fp&L6g;}kiuN2-u-61H$#nqWXD|`PQnvc`Xd8{+U zYQUX!7T|e>k<7HKRFn4|Z7vO{d-|fuTEj_cR0TaX*I#1^P;U0|fS=UbAMp!x92gPW ztAPWh0^Q#fz$uWn0iT?X-8Cu(GG@2XJT1hKGifBSNje4+Enk|0;T;PSA*iqFBSw`@ z?`~3|e2L4cY?`XDTsHHNCox}93*{ak3=v99rkP=D>0jKiF`oEFtMv zIjhP<$y-r<)_AFXg%%)K{iylf?6fOAaI=k&;h!6*DM->TC$K@mhW7U97c zV$=mBsk%V{<$%m1WAVw<;(e!-NU1yQ!$ZXBsK&$t^giJM3F_@E8|-r^851yprAPwX zxTcD6mZYH~)?tH}*H_f|<#*^We0IC@g&1Sp6CB=%jy@0>5bjbO-C#wxnVBqM6LM(3 zKiGIw4_OYxyFVyxjM)$D)l6g`VJ~X|!zzMftQ$#an1cA2k z;Ana}UASu62W9j+=Q~~`)IDw-Z-W+OMxm5hpq;CoLQnQ;9EC}iCxod7jCwjKAR|0x zqJgJBK;t#E7lJXvOrNTf4U%iQ%hPDhQaxq)88-cS1ITS!o=2~ltfEJG;3`H*J7dND z$C+Q{yEdfxRPdPESSH^LL+3DG;)E%oElA3U;YR=|rcnqTt5j zaYL;yIf6PGU#Y}j!+Tcd>>JwX8+^JAj_fK72H&Q$4aE|9o1tMmwa=(nS70#A4^Xjk z&Z?$Pk2Ze)lnLd2l4y)d2_M_xw`U%Q!{U#eikqstTv;mEOyPb03d2W|OPt~9`8Hq`_^$ANVPhQCbrSF19$xeTqLEMxqM9g~^ii8)mlj6J5) zEte6YBs?I$mht7YKuL$mzo~F~|0-67oc%=K>ky|63L?5d6(z&D=|*_Q0H``a_#HW3 z_k98BD`FgW%x_TFlW2MKSp<^bnUHd^B^OFm0YmT`{OeUu%K!bS$M@mdvbP~Kx!Cid zNC!1VyRGR5HDHmT`;lt7(r;*_?s2~(au!i3ojJcHiAKt zpgz$gO0Z-fViNtI(^_6IERm=3eg@odqKQk7MR#LcCIK{Miq#SZ6#$DKi(xiCQlPVV zI*84YB>OL`9HD+72(wdlT{wT;rrq&Pj-nNsaURv4zrsrtgD1(E-35a9T_O_>s5HjS z8_O1cNw4#vQN|TXF^=L{$j&@mfXA?TjESd6vf$kBi-B!&T1b${M*^;F!ngY|ev034 z5q}aX$5lz7gZvea?|~E9iy3W(OA5t;eramxLQ)wtlGx?FS~~b5EqJc;5hkMk*gt9I zlz9*GIY`WVA66@(P39p#l+L7)UK#mGqeD{KR4)?CG_UBlpeaLZ8x9zYJ=||nQ>#s8 zTx438tq*I9Errm~3#1Us*DCo^-J4)wT`{Jbg{+IhPHj}lsTUMp!bzr)O%x@XQ~f3B zlP?E3s28fCDX3ZgEz?LN@~UpUFMf=}=Z*c_U6~&}L!0lYuli5>I8oj?1=KVi>8e&k z*=4_o{mU@)aNOs3rJm%did&yE8ckG%Eo=o5PXp_p2D8;D>KIq*4W&;WkqzBin}~9Y z8@YR3_7?U3H_Yo`^*~M^Ux&Bx`;5BQ3K2&!1AyM59E`UP_f>KKtu^eXN6%fOWc9u= z(;u(M=KCkhc;KMZaZjL@nsj-^=!YNT+DJqG-XAYuMNAd^8+QN8RpCSdsX-oGW_(Rh zuiMY{5=~ z=k_zT9J=t9OQ;ypxjjdsJrJ_#nHRjE9^{6-S5=TRuR)v{`^GD}x3)pJn22A+U!(TkHH-NS@k^g=4oq`8Z{xYwk7sv8Qi zrIEEb7=434!xp@U_CQBar{wju4G&X@;ZzD|Iv{{Ihb7CI!;nUT;11{)s6K54>6<(yUV0UdaFqM z%j_PBul+T1Rh$5GSR0%Gr=nvr*3U+ZmsEl zyvno#B{(sYOWSD$o!;(&O5KqATu1I7lKV4b`fKV7OH@n7Ld$^LU`cCnT7mS_2 z5DkG=HYPU#{7LilG)FfH=(mjDL{!iVP)f0m`TBHIdJCuwMjZKJY~O|p-J68;t~u?w zwx@xkAHtsD-s$TQb34?Uw=`K-!1Zjwwc=&KyJkv*y;7b^5HFlqRU`1G3SpN z|3JD1+;}9&^k+Rws&eQbdT9>VO;5GzfSDX`@g$Ub0b5TRYN2WPX74)#ip-jQY&0*G=s!Fau zpC$yuwGtMyc3A-37V<@E-$&O^!Is?-bze?K1NGbHj?s=6o7{4^w{i8;YK|i`zg1=l zKSKpOEV_)D)p(wSP6 z&?A#D4mw)U*ekq2-xBC2zkt;kW+Nn;M@9bKSEr;;7dEVP5P`%DX1!^CXtW%{?| z=9d;X_{=m^99ToF)4a1uh2m}eJ`NxaWj?U0bXSa(zAP=_(4imv#WBT@!02sEaUnXX zEwe~fs);Z@7XXt$Y`?CmvZ-1|A!YSX9x9j~`H*o9 zw7%rMLrT}Dm_WUEA!C=V_?7af$Ed@74GGQ_ZlK)~oaJ*#J(uKxV(RTAIE&|^ajwV% zLz24$=f{xXT;;r6pU>Rbqq%K{bM=+{xHFvFvG~-6BRyP*wv*#v&-NnY>fO+kx@_uQzNy$< zdy)8eE0VX9OSdw4yK|Iwj?&Ii+Br%Kj?&d=<~T4Ub9X^KRpdAdslZ;}dLN7XM+@@`-r z-BSW%-LbcrK;3xK+htKU)hA1>Ss6c8$w{jAifx0ja+GqT8;tv9xjogsgLPG|HZ4Jy zJiQU~WX12)GNMXp>klr0xoxv!BAu+Ue4UV>zqcgLWVYoN%>!j@=FENHOc&2S(?xhb z|7N^+)*1i4Cgb0#Md!o;)4I$2E56M46Ar||p)3jl z7Pl`XvnuoTeZAb~w7JKFv_71q>0sUQZvmm>YbZv25;y(hyJ6eJps0N{K41h2V(tPokfrQ_$Cjd@Z+oi zqre}{V^9>K4qh`w)bj(-Plt4|K*h4p@!~Mp;XV9-4Cn@-&3+exl6#j>dj;3vtxqn8 zA;U8pY)oa=`hwXP&61qHn}q|bHe_jI$>Ck7S)LJ{qu@$LbcUcOm=SIJz}oqfAzrPH;7FD%QpKdUY9l8Q zl#wawnhbnkR47wcASG!;%VrhKR#u+hLot^<>9G4O2RQZAX<3&IT^6vel4xkAMGe!| zY*{pix?mf+wq5G<>vuoC_(0yi`|$B4d4BovQ-ytJQQxENck;RECwTI<*sV2oVLOs-+f~0CO9*J+?{-#ar)g}b`3m3>s(ci?!6QiL zigK=+1Fa==1ZYn?SEPYpnjJdABS%MItfT=tLiJAGvi^H5;q!4LBs6iSB~_Eqr}h9a z5zxh$N&8RSix*3mWL4&We4kWi{%2i@CsNFNjTyZ(g>`T4$ztAX>P#!mmgP>kF2}XK z9M`2|GtF_$_%nA~`7RIkwDMr9J%9N~pHs+mT^bv<&2&9VwCS$;7>R>_o9TpINNTbk zvC01;WA6X`S@PvFB5qf6_YN_u?R0&Zy18H35o#*wo%WGCtq9jGRH|Af)YhA%_El52 zs>dz!NxDR_6Xyi6UHJiWf@9=^H1|FeK2gYadunac`JLvL_=SbJH!%ty>(eaUim5g`rE5S*+92Z@V)mGCD2 z2Q{U*K=fq!#xMfi!}Z2#?kt*mFlUyViQY0Z(ev0$ln!r3zrffXe?E2MGSPP|lYPYS z*QUNVW&C`;}$PeNs%LdPp=YY2$CeF}-r!fN=hskwD3hOeab4Cb1S~cpdWgL=G z4(kz{naWL14Xrju1U6_A2Fh2B$AV^{49YG*?oW*Qctg+R+We=l1~ugb@6R>I;I!}0 zV?wLsJm2kMs+2U_sg#Gb%5*SgjWxNgnpvl3=3Cr0e5`C_+e*Hm zu7#oOw&E?(VjoIN{oX70*mgI<33=Z32Rd~qT&d%Mf~-gg%m>rK9@K(8I4!w*lLh;6 zT8`VqM`&~{K46Rls;HJ}Nvb*;nxZ%|Rq8kr6$RNGii%)34KWf_0U9rdNT}|qgGk^{ zej7N{>yhvC00~(CWCb9BJ($Nw=oRo5rp$ge?vdB|_Ffm;OP9;GhM#t51?`J zyTFg;MG%JED3jOX<>gUKro^I{WW?PUq^H2mVIo*L<5d_&d^UolYJF4H6&__Q3A)i& zMOi&nv;{i*Tx6;2*q+IzrimMY7B*3=6i8u(I;*wim&E#8F3$c6HW7@ z_5>Se;Y7U?$6v54-BFB@ZX2dD0wQao2KTvRtCnVnAO#lU`ZrU!S=fGdsoDJ%bLW!| z+0UNwjUoH+`xt)oa$DgV)H>i9DZ!RMh2O0{9KKrf_)$O`<}(n)5?Vc;z%0+%e!C8AHvsb$e*ikzzWUF2inj zhR}^Mv@KOCR3m;71<(MAx5X8#NWeJa=|ct_;tqg4HgoKOXARwfH=PYUcQP3Jw`a@K zQj_Vozd&0b?+wX*g293^wBjxC5E8f#B@`5P1K%Z?O;INyyINrERj&)>c`zwo0s3;Ckze(nmo#SLAa+8>oudN@A^YE~@9UJdjkW zORRm|iM5QAG{~#1-pE_#)mF(6k2|GyYjf!X$f;ea${kyzFA@`Wy)xsk^NlRc&=(o& zbyaR|d?h`^+}s*wlJZisiT$*gJiD`TyV&H?mfI!dZl93rhRkZ2X2ZrGfidGPz>c1U z*6$0Hae?8ke2WbLW94?sB;7V`tME&_%dF}Bh}q4PM_0ZrtZ)H-DO?O^u7eoT5%!e>iWKdGdK0NU6|eu zQpYLa11ybJ2|hqpf?JiepL~4&?bDpRjd@5v`@ZJ`O_{jQIdYFofXPhnF;DB3PBBvp zbc+zryc2lwvG0+)iEB+*8y1a1o^@rrBX<2P$k_3mfMRTy4@4Wo(S=4>3et2MVgA+- zZXyQ*PRWnJ${hluW5onVZV)aApU4ne^&A?&SVTb}Skf^hbnN$tyC4&I*!QMbJ<@_6 z;SZMPWV;|8U@1jAO)SHtEBHUFum}qa!~fze7NN^3kUCg15|g96aON)JLWYIOV!G7~ zxFE|jgZmTa#pmyacw&? zyL__-`DUkLl6-Wgz*aNK>PjdNI*ZISEk_lHAXL}_JguoiRn#S2bVimWX$`Z;;7wUn z)}@hEb=9Sj@h5-u{T{h_na}#VnG~|V;JJz%ve%B&BR{U>qy6}~&=NeCPaP|?bmREb zT7$v|vu(-6wtjGDJ#xL$)OE2bcrM$LQf%pJ;=3@a_ba2C7n;Hcvu&wtwZe#*65};k zMlxQG5qrv-_^_SsLpXtM*dxHxf-&qk5b}Fu1}p2@uoW!~jIjLKJmSbE)RxGx4{?Z{tk9{ykAz*^Ck}7b5THRw8cl==)k%!x;w zwhYRz&8*!+c+~fU0l8$WNCS$-PiTft@9}M*OJelyktrSe0ZNhgyd#>fPRR}+Pp&>b zCD=BVc=R47_iQ#SdV_K%WJ= zP*MlR2qP}MahP4y%$L~7)yMRU%$V3%ou4SIJB$?<0)B7<*b6S(zp#?=%J8*-?*@ct znE_S5g?)YxlpG=9mirvrv|Wlj78+%wDO*!?)i~(Ofh{te`aZ~Q4r1RS_F#{ADHkYy zQ^FSl#uuKvef{BSk6giT0h9f2+1Xh*E@)LwaKD%Y{BanVJ4;*tF^m`iXA?tK+NvAK!iWPd#D`MEw~t z)qg~GG{W!`i#a@<0G5LM$E#QHj5LtU-+uX@By%81&!Fo+#QpZF3Gn3IhaS-es>=Jg zdetN1K*rW&&>EMmiN@2Ijd}!jApr*Ok4Ahu;HsDdleeV|Bq7_DAVOQBK2YVfEyaAw z36J~{EosQn;&bTpMc&962XMb4=fD7W#{BlaOI+r`&rUp^GMffm*C6Pu^7(Z4AfrL) zU?Z1*aXt8f3mY(_wndr^$^J%PQ2Vk_l2w{$x}1f33ntch$Qt3Cguf$~Vdz_K>QsK~ zj?w8(p1irfx_tYzfCCW!o&#q|gcuNXkegW!51EvBRTC9MCSl(_BbuW2HI>(+6h+a5 z0Vic2cTD|H;}3h;?0*C&rMJ|bq91*m{D1GWH7|iwVG7rosEN8LZ9)L3MXORIfDCnd z1t5UTkJ+T2>U7Q$dTf~ygd+llZjM7fKi70sH*LoehK}l}BalO6O&?jhWR6tP&nrZpwy6O<6Fq4K>K;pP)jMRrEHi5{Xu`6zKX3zh4mM5+!e*QS$7( zl2;Tf%M`1!Tje&R=OmsLY+1DwDR zWI;5wBG5t0Q|TzP!fqZidP6meVZOU1CgN8P3-K7_hz!hg`SxxsHte#U_QZxO1Ol?xd#2kv9t_hS}ZtteE%_)Sv`X~SFk${)%T?*Xy**^6vq%%wG+br=_K%;Dq*4PRf zML;=E5>0x4W|Bi+VdCpW96VMJzXe0VGA*z$u!}foY(oq3+A;9xuZtvF zY(pE$vW@F}8!YVWA`TMU(1My7#XQ#nkM_DqqQj~HXyS{%5`d-)Sn2|nx`3rFU}-M_ zOLR7W_E>_JGINOomvY~9!Ao86(h-A~R&gidDDaGMm&R!4guCUpt72W;!YAIsD9QV) z$Q`Zo)}AEywh0TT|BI~;-I8D%dXV-V=lbefg1J7d{t7|RMKvi?2| zWUG4C8t%RxcOy31mRN$jJG1J}TuasN&RnVJzA^T4$D6EH>)Tyi-4Uo38-?;B+NXU4 zP6-Gqi9OpB5LCYT#|iqj46QDUi>MCN-UjFGcxSyVCbbk0t1{NLgI--dsD}^bZGu%= zOTR2{?hrFt+jm8L@75$Js>6*~;@J)?_gEWc&xoP5cUX(#INa83bI`T+^dVQPnX#rl z!QyNuh{;3YJ{~yAGEt*91ZkDUfIUET%H@XVruO4i7%12N_^1wp6`U;O`uM=Ic)1n; zJr5Wud0bS_lXH)x85K(x+Yz?^opyZ|esPc8ayshLSFereo-i zG#XJ|5+#)?+Gq#|uQAxE1{Zoc)@gN59oA`_==Qsx;m*0r19)By3ILvO$$!WJc$R^k z!czTAdt?Pz^kuOz{$vH>>Bqnw`VlNakE{TweqU@ceYgVp^tCgko=q{_>ShJBv}Bmc z!xbo~Z+ve-$DUJ%j>)t2!BB62ojQJK`E!tU(f@}LsW0a4+{emTGrF*_>{yRHr}s`s zl3+G;D!p~0fQ2wGzk1^jv3^$I53!C{ePq-tA8wEkR52d!2{$X-kD)W3F^5zVc)hsu zF?=y`WVthp7CTRze=hKf^wDGr@yPMwMz0?ZnuYz+IowhdIDA+RU;4@;Fj}}$7p%*f z%U!Kg-@;&F9%YugZ0Rk!{^GzhN@f`{Fu$3L&+;%<7d@KZtqY^Tx%J4MLnag^GmPBn z0$q361qmH5^m$n)xIU+N9ijh`PN%G6IvoQ;TTEtQQUYx{_3v=V{t$zBhF}}LM%iE)SiKWLoC={0gYX{w+=E$XvB6yc7MaP z2Ot3XD6pYJ)8~Fl-XfU-+YX^U92A>px~TDU*h|v;N1yMw^r0nr;C1SScUa?!uf$&? z^tk%oIr;6E|MDt~i>qO4%a@VvWDbmy&vfC>87W~|7Vt41;qq}BH-I-JBNhv4g~_8Q zc#(X;CMpBkIx7?T1NFEG8MH=V-ZNY-W7pqRSYBm&}$tCr)w# zSN5-$?B9iBO*|Kgz@o{SXE;5zENO!E01L@0(~TV;z7BY@0f_%mzzCiapYL4mHwMYa z@tiZhOBhvR2rR6tm+L8R^*$-VhqubmNlWJ8^OVRx$*kch<|u*t6G4Z!!%q*^x}`H7 z)yaQ_8G(vH=WLK{v!|m+aWl#HM29@?^4KAtPKEpTUD#q6;ToWmpFQq|xOwjh7epzq zB{-wrJ;}sn!hVSGAcdR=1T~Op5<0!gJd$tQ zFuUW5dmEV_ zm{|jpwpf}-oKpf9ev(iV@}Vy2H6YI;SU*ba@tsX%lN&mR)AIFc1F$#ub4>L{} z)$w}x5)X-85y;n&dHAeMrEf==0c8i-!vj-XTFuT2?V~=CJaYu7yFEERkXy z1vIXx31<@Xh2S=UeFno{Zm5voXlzqC%keUrFUx5rDI<71;!v7t=(CkxE(c3*f+_N? zms`js4;+vmUi_K-`0m5?%{dv6@236`3uSm%@D%#~hFs#ggk5xFxhNQf7;Zffg(uQe zA_;V@2EqSSl!eLF)@r2{_$zSjx!uljd1gE3$*vE?oDhBKGo#q{#r61^_lr zB*E%Pd1SaKu*h(Um2y{}K3k5r$Ksb!z1PiC#w8tpp3|^IYenVgOqLB{6Dnx~dac7I zO(@hW!zN9BJT5jVcOaaf+q!D0nmrtW-erM4uL>he(Nt}u3Wfrj;;I(bh&qj+K=O;m z^?k0*K1L`IG@q~;4{k-ll?L%tH-UlVc*3b;lp3;Qsg_NtV%n-?7`h@*P0$@tr&L#{ zz8yyC38-8()+W!dt2Wo};CAcw(vH}em=U0~8L)jNU6M3Y5_M}R znADUE%djXlO}PlwH>dYMf}UEr_>Qb;Fh129*}AM~I@*;fu)vWjI`CJa<(HSQ$%|VC z{dZ*K)Z!R-gU(DA}};8*1NpSh8K{a(pcVC->sDgXEZbbJ2H zmP`5DrGc z8T+a)2%=!9nwmrUtn~(LGb>7Q^_#sq!Ybmpk^A^pKdJrw$CI1vg!x@}9 zjClpo+aK|-`=TJjzZ6B%`XYSjBYzPzK^mmChDHd)M_8rfW+Gw zXYkq`gY@UFYkzg8Y{%wL{G{$6*K~(&t3$UN#cQ0Owzm6T6u_3w&p}|oABNbIwG#Gn zSg4fiV?PbN@mc85z?z(Kcg|!noBq+Y?LWve23v`WWEAnTxw7oo80Qr-IX*r*XWxDQ z{_=(eJoV8Y-G_5a|yPfLaPAw4oG#LA1(q}vO&o1ttuK)|aDw=gSEO_x;G|mNm zU@CGuSn%4pVw}s;Kvg>|_{WX~k4L!y6nyng-VO@BQf7SIK=4O(6+aZTb_~YOV3hNK z_KUY1o00{=*TZaonQteFWxq(n-|GRlf1htA6Tp6vjeM_%;C{0*re)yzMK+I zp_UQr7nwl!3QYIp0Tp=z6kT4Mv-@5?QMd!Cz1JCu=eD;Cd=7Dwx1Scdvz>&9!((8;?}5k z4+r_Kk+nXH0aD*Rum%HUqpo|}SeK`TztPN!R|i&Ooug$>0O^~%+YSZm6pAJ+ebotO zQqyBlYP^1#M^tL!oZJSLv$jV&N8I{v?SrfBNTSEZt&V}rTg$s&1~AzyqG0tY5Y4l-}8yJcyEOsXD9T z?6@$HrLic9P;)|7gAU64Pzqg#alR_@Y^hAG!e2LuN?wcdy>)Q%)&f-%!0{Ri>#6CP zjS$S6FJYUY);q5KA;r3bAbQu_Z6^-4-Sh%__x{D|?UT!feUx|4SEX*pW|Rw-ANTvj zs`8*%@F5Q49s%h~&;HsRP#!T?ClRy^ExkGyHH((MDX4C3oZJy>N}fn0=F&yGdDrBf zxerG-IPT``tg5ZD`GS9_sS$6DQ>ZiDcBb2}!gQP5eCuM_9}Zt%=D?S6=jx35jyreU zx#P|qcRoe#yfzW1G&G_GM)+{peiyrSssH%xnl1rv{d?PBGikgR6%z_`n%^n5p@rKf_Erk34rg`1cj&_r-NJG1}vTq^E+0J0&h)+h@*CR?S-@I2;6ojf_S^dFkIdn8S&&SaE71G-{U*57HVWKD} z$Q`b4kI=;eW}c)@UwR9t|CZwOyk07g4EsLvScutWG={s1NT6vYab|z=Tn4klJLh}l zBc|M9lP~>n&Tif(e|rA$6KIp*ayDFu|aw2>cfVSi#%LwPW zmvAXLl&3YrcxE8JBqU(d;~nCDpR(a$3f(yHqd82WFPZm1=O?N>kpA&$50io8Nr*TS z34SpL8veoq>@cp62KU6Vqd2SbiK^oRYyntDSChaE;VBQAHPwOgkC}ZSiO`}hU|-1) z^Gm^-i0QGJ7MRvcL(H7!S+IV{(?)B(oTCXN8w_XU(15w9(kyxxlj`A$`~Uu*{~O&Y zm>_#ko+QWpDT{T4-HK_h?6^q66G`HfWR3FdT8!6(12SeHl$40t6Hy-+_)y@$$#nfF z&VUW5?Z!c-L-@rRg+x@Ih$<5J%y?o9OrRV3)3}>#dJh{r%z*^p4kL3P*z#dJ zpJ&bUENGU?&3hTby@#Kc6B&RA_5)8q4$sT3uO9g^&y)B~*IUo?2VBtJRM<)s`-0gQ z#F9cOZycME2-o47V=`R9(FXQL(M^0ALm9-;)5&r?d2@Yr`SvN>^=E&2`|A0Nx8ykt z%bFA_VO@Gm(n0R*!m9G#N^@{U%)u2!UY$i$O|1{A+DVtZqQercf=QxhsxHaKrixb^ z=~7zz` zS4KN{jG%@vU#6;ygLtBwAPsZO;M6Nx*;LadO&B@4qdG%nsL8{jF?0-5b8LgE>UI^a zj-qtbSdC-Uuqy7I9cwyCf^Mj$Y$`iLdRGkXQP1>Y)~gVUkPdl5wt@ z14UKZ!X#Jqb4edaVuwloC@{$zJ2Ka!lvj8+x5Xr{x}+aBB>91L{f7cV{vCq~NU^n3 zAMi!Sb6gLG9KFApmNK5-i=5B60;oH=)hdCyJ7;a@tnHk&owK%c)^^U?hw7|dML0SV z!gAZr-%?2C<__Qz?B&F}*Jm6NCRnbxzgkY;ENy*D4E(L&j1s%&8q;di&6K>k*1CEMNc~Hn`PT$$dt^Z2!cOOu z?-+K+u=ki<+sMDTvjT83Li_O%OpgdJcr34K2cJ+MKH+4T=~n@mP`G;^E?hzrhTbMx zLS0t;fUy!rvZ#rQWl6H_s3U8nQ{8kdYout7BiI;MRR=4fUMp*(i9}I1G^GPeNP2pE zM{U=0$9JHbUB1g>8(jJYY=h$3K#df@f-vK+z*2sZZfQ!PVIm=gQb}-77 zCLIpl#OxK%6)VOJLq8bu^4n~Ty|~)(W}-d0eE;#)(>1hXg?!3L7gMXpOL))y&$(s6 zK;h9#kD|-ojb;SDvl;RE3SLJ6ib@3L2JiDZMj?0Jr*q|oLm=KDKL{W`R{n9v9;;sL zEvEP6$^4t?;+ZfI)Td+u0ulWzv)0rD3MD^&sCJzA|d_rumt9cUH4}qh!3K)F(ti%Fy&e4OOljir7wSM zmX)5PLyzl2{No2dinqhqpV~X+yVyScoIrAd&TU$zZ7Pw;i9!pJZcUDrms zF2R;E6^%-|VB3Zf=S?+ESwEa8xn>EJK0?zmi3kjgt6l5KD;XDd#fQ=nhR~OCqlc`nO6m;P63PKq-ecADOP=ExD1)A(_ zS_MZCmq6fy#;OS1ob|Qs8Q3y$8AvO^)X&voy!`%oB0%?7Dl!7I)XZ-cO;*h28*7)t zzQ2vNf6Rb)ENf&63_YMM@q+)daZ{|YV^vVpp)FBO6-0An8;+(cBGpw@HdP>Bk#b)= zw3~d$xLe9u$4`R{_b3rC8Z(bw?z&yqxJ*FWig$F|U_~~3fZHH1+a6{c?08?)!@D_( zj8Xf-pu>!^>FUDHC|7p zMDe&0dk*31r~}i(-Lp9o=5F>qBlKY8Q3h8h8+CM%k65ZoCW$YgN@SfbXs8Ps+A(N| z&gRd$z@aX1=oo=RCjzRnHV~;C9c8Ovq!NUbB+{lsNNEotWf{G+L`YeFSGjf_NXlt} zq*O=dk5v=SCH6zSmV=y?Vu+C43awT60VTd+Ftu%Y|Y(KysktS?Qe)hB5)X5;7zJ^XvCm(?%$1PhuobRvyi-mLo_5IiX ziu$|HzMa1WdOOqDkcAJ#7A~I+1r%dO4471G*7Dh`CjMv2{wOhU?Fuq=5Mym2!@kms zO5%IGnXh)8n7YZ3+aZ?h&BIbFcfH&LzAw;{O}%X=4S)3qt^?>?)$DeQo@xZxS>r$! zTWcqlYTI$PcgowK^S<##Yc5br!)SA54@jAoSn;+Ns;`!Hw`S7&dSsi1{K$K@t$Cv9klQ_$uBwge=Mql4B*4RyYt&KI;V zUr^5V*t##M(E3(Kqy<>X!$b=n8By|t>|Ku?RI*H3dccs9jVx}hf=bra&mI)24KsEh7a0DD~BwN&xO~ud^ zeKmRLpI&|Vc-b6DGK?4xiHEaH4*KFcQ|C<^Pn^M!gLMX+YXWoB$+;W&$hTkmb|Jaa>ev#&cLRaB&U&%#0k?w2og zN*m0`nfhO%Tf7lqTFqZ;PPQ+&8KNjyOS2;K=UW%%vO!aGAq9HoafMF%_`#cve(gCVn#z-PP-NH!rSN zpy;lqJ_dtl!RUBL)BA5Dtih9<(_Rx`w~|&136R%5mexU1O`OmTD-t2wVF4v*6Q84K z>HpVqZ4h7a&NArJ**3St2-RX<(Y7B%1PzKI%hUk-_P!vEq?yV(!nQDJ#jHZ$nm0 z+0bh>k&faMMImt6iqNF{!pnmiuY+P3amBHs(+8R8iEV{&-~-T?Zk?j0dniGvPQqs* z2IYvkBCxb##woFNBsXoTts_=AGDA&e&)is-->!}_Zs8MeVU!pWD?(B0ytT)@v7%4? z{wyBP*GmzuEOvyr$ZNtKK`z#*`5kF4ayE>P9v9p|(-Gog#ro7yu8eFq$9;e zsZlgbmxHF<2Ot#QQg2~1K!UVGHw+_Zbs^F(*Q!I@a=J+yil5jceY0vnc6O(gh_)tG zUl0UsWfbvO?I!8FA+s{3>GSbN`~kB`nEf!xyHB4f{1T_VS=0LwGiE1`u6(<7;(q+h z0WCFIIGCEgA9LJlP{qy-Hnk>fj+|hwgdA!4H%Ez)sA8CPy&TO#Ck_-j z)ilasqf6>>#WNUGa&*noVe#l_WUHg0ZNtJ?LrpZ{3~V3IAeyRTu8n7?>~`N3Gfyg< z0rkYSe*BVRml%_Y5LrG$$+~t!D+Vpq47gvS8~Y#TFIYG!$LbsWCK+7+AY`HMdQ?^0#09OIxh6w;iwi z?U#SqR0|H*&`j1aUkmO>MEm4L=8>cpfeRSgCGVm6*NYYk!`0iw&GF=A!7zb^ncBlR zVJAJ?77CxW#FHO<8M21e!s--ssdvw#=mC?Nqx5lpZW@kZXwdP9s-~ol3|mrEOC9Ns zpqsi-sDV^l$f>Ffwr-C-sOn(b-jinOJy}vAY{wl@uYk)jcf^DSZiu!4Ivq@?_r+oQ zet3H}&+2y+L;9xo{e|C$TeEMcw`$)X_3z+H?c*I7vL6@hO6|jM@K>Mm8?{hYg@IRx z>1{T!{2A`f9R<#lFRpTk>tUAyD-TMYsi~;+HpU(6hAy07-0=x13y*2bbQEby3qw@DK}YJmB!0 zTE#W09VPkbgr<-~2fI2BjySaOV(L)7HlWp&tGug#o*PXn?ZMk^8cYiqlr?6t8#>r~ z-&;f>`55>^Cm2`M6im$1l{E#wnJ%7vri<`={>^ytjC`Q>gnkJ?dO!$wls3Po9{u75 z6b05NSY42gutwg8@1L?u84Cxp`g|es11xC9%8&tr1Lkb8qIz~lAcp@%0h!ae6R^sH zX!VT?&4pOTj|>CX8G$fu`7!W#_FKd!~c^n$tXvTB192ZV0=#UargrC<2$2ASRl z1Sfa4oZTvhc|yVOU67iuSV(=AtS2N9?5(!M!AeG?w2mf}{NQ?C6h5O>{u)7>|l z4Y6gw(RDAUErBGf2GbJOaT~Wn=;drjST2K3U9uv8^+*#1F zc(@|11qGsy|#EbkKv!v6>2is)|Ld6UD z8VKT$w=M`r4-=>N0KBs+*s;nE4-_V|u3|>xXqg?k^FgQEVi}1o>C{G*%j491TgDS@ ztOU8MKe7cB=SP-XcwAlb!9z4xxqCK9J>~g^9Ys2dR8iy+%knRB(AQ? zYQ&kCS~RK-1{cLMO2dJu2&%j}2z<#W z9&$u)e!e>-7woH{O|iw&!^&ONFb+e~K<*HA4fvNKN<&50h9fEt1=~`$%UwM&Ho0M~ zF)ExC}Z_fOB?mgKnZFC}+5Qq^`WbnZ*WP*9;&Tc)BtR5e8y z8HyBhtbA5s=!E|*4TqIY3mP>1UV;t%5DOVY zZ~frT8N29`K*0wt2?v!;a>+H$u%tsL3VbVYL+Cg}$wroJosOKLWjLCq{{QT~-Hzi( zvL<-HPvJPg_EZ5O1)cwlu8c-zWK?EkW@N@>WL2H+=|;jyIwhtkQing06+JjW53mog zz`5PMtDAE%z+why?8RJd-VWv=`bpM|q$EnBL`q^PBn3~`sR~h)lWuNqKcAVKF@}sF ziHf5#1w4uc%Z7O-*eAvLO&)raqOFxDplwa%vMr0IWZJ5ti;BT@(=jYb2bz4$R z;6+o^?S}B=0ZC`%E_1HT6$2)H3_{ zF-UTssDE#Uu6=Btz=0ibU(bl17-kQAuzkN*AC&r#+g;x>-LCHqyT04+_JX_9sby1> z2_8U*lyby_nqq%>cxSWHwaX9JZgNwa;(eaKL*E{Ibsl$MEymG5m~<;pf~j{9JJ8K0p5qe&OfTFZ>)h zpV{ZJU)UYl&Ia8@J(b z%%xofG&aiAI3CGXKB0NHg_)9V!T&mzYAYRCb!f*h484PAa}|*(;2cj2)aB{!krj-R zD9v(r{jc!oJO!@{^!`I+jh^WW6*UnYYF+rgCi^gsUiF6jOGw$@hL4TBtp zOnwq&?W*!3w<~KWo{8$2EW*m@jg_@i&qT4SQLQa&cVuPl#))}rl(nno-s3N8w?BDg zbt>0o$+R=Xo-uA7tZJRuR;^pCT8ByCZ9mS85VWn|*JS-Zv8~^?;;xeVeTAfd!bi{Y z41fF<{#J10^dg=DAa{ZHEKB)6uaP&=migPiGJn-M<=d6{+liiCXu*k~+w@%QI_TXB zwXGax5tqj(TyP8!%9h0pj)w%hDGT)|f}$Fe&7{b>$nPe`e=9{6%Ac$N?6)m7QaSY2 zHK6cR(p;&hGIXO(K2ez}tIk_mT9#s2s$5wmiRv`GVztIwcL)~MtHcKXWwz$%_^(UP zW+A!#_Wy=>N`9MKNRJ{gHprDJTfFKNomPT0AweE7acfHf_2_TDnf!aivT*x|KR#R? z-dt>6^np!cG+Z9y)DGlibZ3OA?khUJZ)n^7Es6FjYYJo##JbHuHg&yzb|Fl*2Y zgT%L2{X8wBTwE9%-Zlx%^e7Qf%_q5&n$icn>>m2{1mxz31HF@w!)m~6(yDylefo6$ zJr>>zlK3Bg`#%E*{sr=Rl*Z>~SAcx?;q}Gy@5u<(49oR($)~;><*Utq9eDxVMG6~; zSYQqGcv{!b^n$=nYyZI>^f_L_(I$gsuLQ4|G8ep5cdl;q^y57FmXhBZ+XeMJfcJelK5C|+j0X?|EniPSWG18|NDO~ z2&gq8&ey}9Ek)MP(-h_qlKkSM)ba_7WwPe$_{^Goc^R{O>wE$`hP-1gjD2t>%EKCz z$7Pf3q`+-{SmS)t^Bg~nzdn5TXjW|nBP`#V@V{0yC&-T&mNHnWirqXZwA5O!Oi5kp zYSqa^$sW0G#!I;sa0?%Y2Pd)~W4V@tymBkZDy!(P$obx`B7Wt-M~p@mPdt<{cJ&F0 zHhR$nYOr^wZMM|rdpe)(ObGBSA~JowgVT8rYZ{~joO)|$z zI=8KzK+cAv-!CSSH`xbdad8`}4Szgb(L6ymXQyU*?={MYM{JYUDu+|n^P{@%&I&!X z)Z5>0J+)IfsZe%n8A4R0!yZF?)@*Ds8d`J2EU7-UOH?_#6!CZwN}kp|THOv~yR5W> z5Esp-7POXZ>&_m;d^VGMdz6M=DzqLlL+)_8pFNHhgazzc8CO%rt**VewaBS6{C0rW zbKeGSNv^x(Rmi_Y`_cq**1VeylV7uN0E+%am)s6Q*STZiiq}50{D7Ir!wR#m8PvW! zQ)rjjhV%CKLR+~)xzIf^yr-W~ks6Nvr5 z=Vghs&wbw|Z}#3Fz>Mx_ml3RH>)z9a-X=C66;XnpgQHPiGv+* zr#z8zTNDI|iJWp(&}>B!1t!^=YMKg{$~)yPlmMzgRlmrnH%fJR4pCFU&ll)Z>y-EL z@|9e#PhJ3aI6CYWGF4c&w;rXsSnpo2PljOlFCTib6(7Dlxr)eZur$1Ta{22B8!^L9 zcYSi5SgxPuBIoaYk-j%(=-&h}GB*gj_aKQpa?8VHNM6i5zr3v;Rd>)<@8(@_OFq26 zEbW~~4N2^%q9Dz5$D?$s?szBVEi-z7w_U>wFoD0Y?9K?xVDJ|d#RTU>W{l}nv8Z7= zOm;-B=!(J@O0Q4oP+4_N(&)GdKz9U|ZC9WQRg9S$V+*bF=C=HSw#r*R_02nN?Rd-; z15G@?P4dn) zLzZRI-eQk>i<9qg$9~^L%e=Y_Sfh1_tQj&@mAd`nMAgu#DlPPvyLQns4V)N+CUDgd zDDWy|qyF-Fh>}2Ml?viQ2f01oiY;}ELTStPdb?SOjOwJXFF%*2F1-7@OTU4+_ zQ&k+M=;A^{q?aF_zv<*=L|JpX2koae9awP`No0~^VMY*>1<8>RRdp1DX$&SbKkzb6 zt$L8oEM3yU<`7{Ih*Yyo!C;1{Te`)VVJsAQ{>dM)`(K#nRrc*;svxSmrdgCxO}16R z(Wzr=)KNv-p%z^*tMK^a&!0Y#A6?&LzFkQo8D3bDMMp3U(Ke`Ka7|Dwn1*7&H7+bL zE{t8#=3ymEbDQbt!YQUC%cdm@j$~Sb?5K*VF-ck2C<|?=YIT-tqNm(37|4g!L2h#^ za$?O?70-WyoRU*C>W;i1J2{Ad0C`XIN2cize{k(*tJdImIHhMP(4EE1*JZ7(WaLA$ zR)ZDJ4~|EGMR*$S8V#+KJ;qNtEt-I9H$B)|(3dazgPN@m&_`<!*rp|mg$@gXqkv^VwL{qza7srSla)0)GLCAY@$BjDu6q)JpF*4I}{mw8) zW;!`?-pP>j4(p7@z)Z)VK`5)#a6tqA@8XXu0nc&q^o86a5aDIw`!vHvVnV%?W=Y^n-Sd-0t};rP?m* z`LM5DD?*_=jwAbU3(R#Gk&uIi2z-)w;ITy54w<+hC9!KXHV{U+9i@6;5@?Lb#d9hu z>DR9>>Gq7Olsn(hFO#aHEnQ{Br zz7v1fiALPQxb5ORv1jlY<{+~n`0Qb65zaL3+2j7WS@+}@lSh!rBXR@_6zXkX;1?|~Lts0A&od6;awKDp~ZsliZbp`J&HW~r_Y zl9rnv*Qz_Os#EvF>Ga+LRLJ5ljJ##;$PHQ?u-&YWnOreVEZ+w`wncKq>_iT~qfssu zK6su>E?7AF_U|)Z?4~wfD_dIGvf2<>ez5Zqf(JF;=Rl$YrPj#Yk@>QIWYwqei{j!< z&}Y8S$m@IGp9j7s(3tB+a0iQH7P7J%Ub8TC$*t?c4eqiAE(1oMCn1}E#)CGP`NS~3 z`Zfqh++zo4IXy<4q=g2y5rr!Xd~GMwDhJ*U8wV-ilN%O?t~DUnBiFaxgWD|+3XHY3 zz={pvB%T-ZU*p>T`VXg*pRDKTKQNB4#a3!tkv+Cu{v>)!EjKiuaKo*ZC=C$hpc|6a z_rIF$O@^01i-imWkz?VPpQFnNiqNj-MrlLO46?a@HZHqT_-?xJ@U4rK)3kw z{JMl@=4G*OvS ziwa=l6F;;hD$8S1Q7}|QUD}3JQaP!lZjnCMt_{ux&i^fZNWxM7p8N=}&%crpc$}jw zb0Wfg5RgMXJYI_)X~;(Rn7%Qe0&(TXz#YE;4S+VUIf$TsI(OywM1#rdz$bzkU2K?tRJ17sj4bThT~A`*kEwz zrfvWm>b9&gQ!pJZ+nT%(+8OV!cPOILO{O7#rtOQ@r?*^AZ`e84K@f8C@f9*#@D(gQ zq_qnMT1c)hPRQ*5gmi?A``q(zVSa&a`7<&r`8454OfE^HG>2sfV#ghMocL){W}FOx z3Fnej5H+x2wX(C!g*+b%w2AA(y|{cM;9?XsS(*xiCHL-Jiwr>#bjhoGADA7~p%4GY zRTOmKU_n14vLJS3LGR|vMRfr@R9WAKS(S3)!8UmfrY-pyX!AP;_y7Ji=rj~w?^)!A zE_r`K-r~-Cc@4z%*npFpkGShYWa*bT6Y@=xTnpJk^Lv|-h>uu^$vb(eq>%qh;$kM* zG+!r1ZyBX?Hy48i`hYMLMPzJQ=)k|^34s(r6jWg=!KaoJNQGy>A_FnN_9r01ehd;n z{;LbRoB+Q)4;hv!fT^b+-!d<70{rkANOS3+kng*o04{+Gna4l{@PQlQ$DD4@?K`B7 zjryR+rnge(HK(32mnt7HI3KS!G6$Z%&GoN^v21Uzw4J3vEB14wcjx5tD zC3O|Vq*x^x0z36xu;V$lGW7HkkWJ~t5A3rtuq8`OH}nn@}dKmwqZh_|Q7X<0LAZ<0cXU##3Rm7c97esKvdl!q2?Om1GlKI!$+G(2E)llR%3 z3o;l;m+hLX3Ed|9Cl%Te@tBpG8SrS(nD}e(7FB zvhhDMW}q4tQ*GF%B8Q8rUT<@Ox=$;FZnC?K@}&fW^b<2f_pdH_{xX$)z}&Egw;R8>oKA%$zJP#}0z35$iXn5h zTG6b@kXrq?%Waa$jD9A_@x?I<72Sj`xlB^mU}EtK#_x%&z}TaaN5JxxMXe)?`rJOh z6%57b=z?r~Cm7#rf}p7#Va$<+rYY)n8=Wq5g+Qsib-fWr!&2J|1tMA&AHIMlL~}Sj zuZMrUNMK3AOi6|ew|>qO1wjn`rH(%_ zPBdL25gUSq7iGD>R8z=A8uwGHI-~t~B~osQ(^5)kxN+b{j>mK+W}9(FK2iy-Z$a-E z@V^i~!HSI%Grfb{@)1zGTebBqFl0Y7>vJ+li^QmgZ0oXZbJY|KRWSwb=$gRre3mL< zx^0cEFRF^Hh#T76bcsJm!6Cf9$CxK9?kBLKZ|?60g8-NCn)yENGI9f#F@XZ^_|PRU zi&_!)X&89-pzu7mhhTE-+Nk7!AF?Gw(`AHy(1h;UE}ptX_z~@$5Y4mPN{`TJd_CUd zyE^%1_Pk8R86VB&c9upiLT23OJKsK9W4;4u;qarr5ZDcPn>KZXUeIDyZ{ zW*>+nPC;TIX$pM4i0-BLOi%$JrzVxy{%cTgiJyrZE{lhfH#bu;|C+H8>)bF9=ed?B zYK)syjC?BUhg>hb`gDaRTjrlLuNMHX^oJQ(lpi0JXOw?Gg`_f{Xf^dEk*EMe*QO zkQTKiNN2EmQJVhr8f-NXkV)79BP6igbhdGLKZ=06T)*#{m=8g|`}F+N)%Rt<(~F|% zK@A;ro&xTpB*?GJ9`lBQC+ zp|D)yKBDFRr_62q61D;+=@Q?I>UeU@vx1?Cvj6AALck~x z)(2TlKBCA^=K{{D?b`Q45Ja$TVh+3^gG6@_2#XSw(!e)k^ae2-oVm(EG6IVLRJC{}r_YFeXoh9c)u@`+^ZmK#r2Hh;@Sh2cnjIFu@#*#}3zc zKZ2J9oTunrPCMy;)Dkmm4eBj%cffi}VV6wqUx7En?yZnHajH1kJ;|6 z_`{p$uoW%fjbZl9wFRDI5cToz0{r;;QYPR1^5e@3@%ss_(chn9kEw)PKpH18g^$Q` z8E>iJ_k&#n>QO7`M^TpyL#0~<2$py|B0ME$qa1zBOp0sk!b^HRbkl!%1r{-opvQAx zEj!hid1hf6s=LcM2=rSLBKHtH5XeCk$En>vTIM89k5}e?`kbSfbI@%P_6m+VvX2DZ zLnVoJ;9qLdcQUloo9DM(!4#;(C8~>}BsuWnW09^)=vG%_=x!kmfP#;vfbTAg1{n2* z1-$D}*%T~8H!LR&6V|Zk;Z@`f4MTB>wuGe2u@}ZdPB54g_%LemOzP4_j8hc_CQuTi;_-tRkFzG|7K0&> zTS)2ZmSJfVz__r!LbemYk?K&b5E!ALkWar@qb_>UT|pBiMHw%wKozk0jrEk6W;sj{ zsb+HB;HGZNw&|EEHD%M`s^HLi3oF2rwG9_`S>z5qH8*CyOa2JHcft|Uzh-fi4Dp~g z=(9h9Bm(aXx9$6&2d0*bHrEwU{JIdaLn}X%MstUDQ<>C5{f6g@GeI}}$^_bf`##3~T!7AA?5eLJDoCop9PDOtx5MoKL!fM@zC?3g=G zIYH-RL<`3K8Eg}H<*D(Sni=SgW$Q47Jn|uN+ZBwwbW{)1TgP$~e2VTQRxl>6CW5}u zK!NH)ahE9Q8H1RVsH{z*QB>CqWzGG;bW3wEpW2da3wE0Znn+L`(Q;&wJ8A95>it0t zsw#@S;Rada*okk}yCO$j3V@-B^?AF-h!ts5x1bb43=j z%reNMk?Pw!ZcslJ2b3Se+Gt)k1lIQqmM4__| zK>zfw|J#55r!P_9|I^t&{pUabQ`USYYpCRAJ%;9bpqP%PF=kU)u^mS;B?;_2YMFu# zrUheJ6V_=1ldKC=jrJ!XXk}TUde&NOvGq#jW3xu3IHynJ&P+3hx3Im^LQ!$JEluS5 z*OHb}|LH+`N@cH!tB?_SdLFIxm9c(QUit~Xz4Z71d;%YwJ}_7NU%+qq^6j_q3sy(+ zhTnbzizNYmg#X-s`+JlGU$~cqZZzO<7wO;IAxk{h1-1w8G4SFbe@Nk>S^a&Su%Q>b zPR%t`$=)3{Lw57UyAQXQS+{eKncT}RdI4(MVngsR<2t+eYky3dD?f7mZ-4*x_s~s1 zyujSjk+Ams4VIJ1dtC{FD0I-!%G>AWRWyD~d^^1(UC2WFvwR6xv{fs4!!25t7pfX6 zf~=Uf$PkG!99!qACQ}CHxMV1nZq4(?PyWEiWRj1CqA>OP@l7|)E}Np2wkexqn5s5q za7z!q_h%OFrOUvEw{=5LO!mfZfYPoV!Oewkn?+YRJJEN~|ic+mi>&=SMn1SGeeAmQR0Eg7T?c~0is#gFFUASs)m@n#b<_Ud^$>v-%cOD@Z)Zs zM-SoW5wCveQMQe1-u|i6GHz_$%#dx)c6gY!aZ{aIdVZ#)6O`l5S1|a|#UOdx3997G zMS;3)C#WTzpxh`m(5kvkKcyE!0P=A@ODw6ApjHWBp)b=GU8tNPFfympF)NmLOX z6|7jxP<2~36()(ADF~b^szDudQ#4G)(qvQ+rE)Z^`mx7}hUHB#^VcZ=IenmSddK9a zlTM)jdNOk>7)ZcSZkh%V4mh67tMfA;r5_AktLE)7Q!t;g{MNrL;(AfU?$eW5=}i7Z zg!$fSrV#2AxHCur={)QIGJTgU3*^Vibgm22DFgIUkGyF>Z$}{9%fJGGi}Ce7kKCvN zjB5<%s>I(4+jN)s+vGTK@mzCC=LyBYEJ_EM(m9_4e1S9g>ErvetJ@dv&kRlP7>d|Y zSh+{P!bW~es7KlmwwM{BLZdgEK1`;j0mWU5drnkBLXe3-vav&3hZw^;JQZL>SA^& zax+8P?8;u2t*Z+v9YqPdl?dQ*g*7B9tH9`zV<=R&n83IrYK{Spyd+uqAX=Z=J`ii&S34po6Mk12cE=OXE(~DrUTFMRvRjK$WTFr zplvEtpf(pOkcFL>FK#RaEGoR*UYL$^G#j~RW9DE z*1Oft+W5E;!|AW{8~hQE35a_;SW1CK2=x=pe8j%6Z1z6ZevMo<0Fv4qi|ygiI3hY7 z_{k?g@_V54Z(o4)>8(j(=LHOoEerdGWkt))AQ)h=G^@`>SaUAFv_Dzdvl3wMv$CV5 z5j5J-C847U0^$&DTJ~OGtrcpjDXdj%2x}FEGg<8Fy4-@T2MxAXvfUP7>qKC-Y1u8< zx&%;a$}-w}aJEgyURe8Oo<*C1d%nzVYDsgR`GhSYQbMRL9nNRH`IaE(B~V2h0&A1` zGh7?{>db|}(n|*&0&4?QZGfsx=HKIF{#sb%@n-W+fsiAL&&h(Ja;|G4(^N;LOi*M) zH+5T;sm!>g)r-&78>LRkP!SXv{QIfJ=VB&Pxqc|>2Ei3<&+MT;8Q!^r0cd7|RV|s% z$RP2)@HiPnAZODYxH}fI2;5IJ2P1Yy(%-XT+Tjeo9_7Og0vHXhvg|0m_wBcS4C8+L zZAB$;Lu^A<4;!+Y+Z2yn$a`kJ+i21QMw6<9>fS|@=$R~csn8bkKImvtHQQ~Ix<8g> zkCeKv$g-ddZ7J{ljwkKfyxH%#(^TiSai=114)Ul)L>nx4KIO6&5p5CC77=X`(G3vM z3T1WhC}6>WJth=Tf^~B&%a%o5rJ`&Lx*~~8lw`_PQxG?9+^30#tmvXpUaGwMu}=*J zq*SSo0$#iEx8J}T9u679nwTSZ$l(I_dDwH=ANHZN$0wKDytJYRl@+aH2-0p#bbP?J zzQhC79VqC#in3Cy(!pFOve-2B=SRo{nk<5Yt;6x zX5O}9M+^Fu0uU9}%zk5ZZQ+h`j1KJC7NctmcVvFPq%UA}>7|3l=;jJ{6x^D;T8DC) zm*z-)p|&DN3-7fRISvHtn;O8!g!Rb+(;VGmDl=4!>Z{aJsiW$;rK_@|*Ub2{@T+?=K|3m8xu4#@egEN?_j9bz2=AXQJHZEX5f$)QBBk0!VoyZ)zG;58APcG?NO)4F%#Epj?51uE zu-N{3F%7j`yZ{m>!_*iBX7oq|leSV10<)lDk9yfM5EhXL&o`nf>~nG-Bz})r`LCn@ zKt}x_4tgOQ^&>1!kPpXwSlqye{SdF};z*k>DPErHJ&XV4RzAM(^U=t2VTc#J&+gnH zgsYe1KJBHc;UZOqj-ltr4)SJ?bf(v%Xw0^Hob}zB$rZ2H&JkZ8m~({he|DzZvZHgG z1E*4hQ`^){L1VhD3#w!omOjs+8xpNn0{QIb9$T|^;HKs_6}qxO^;7o?EOi(8M4<4$ z9T%CXQ5XGXRvD0d$WrKvH=Y!iboqqRCII_9-iqSwbe-buNKw!A2T@MLqFxz`n~9vx zgy6?qES^m7xIhh^-r`2rJnV7%%AeiaDKA)(Dp4*;Hls4vC1yLE zY1CpO6}ZGqy*SXv6^a9=vM5Ub@wb2bm`T6ei7>rd5T>^UVfx2Zm|jeU>CLP#&4;_3 z7p8TVBYokhpVQ&$)fIW=1!f+;AEkI<5%Zw#HirXq?; zhw7r*QSmheZGPBm7p=m;i9zZDTY?}XuW%WbKR^@&K~;2}8v8lrc4Z!9Wv15JQVVT2 zX5-1qgJ6PAO#( zQ4FRrMPl<0TgEH-x{Zyfu?E@_d*7}^*A5Qum046vg+mR)syP;WU=6g&WKouxETdU_ zhhm95L|jI55ok&n}Bbzy1L(Jlk5JJmds-63IlrurY#s7Wx@U491$-4p4Zm0D6eHf z54i&u+COtyW$C46v&!a*OB50#2p>Jm(jf9O6Y0b88YW7RyTE&vrbV1@WG^>Rr2;d2 z+Dx;0MJjTe7fi%y%cN)PN7iR66o6A&pcsrbN0Jp@!WJ#4&X+kAKb*pC%PL>2hLENu zE7k$2(yYwI>K{4DA3Ilf=cPKvQJGSmUi9Qwm({L>)E9YYDxOyh!6i4IE-fW8FM>x8 zP)B4^AY_e5Usf(;rUc1~A|x|qNY;MJ+(VX?CBd(zg-IS;HY|w?j77vGujYK3Q|F}xt^CP?cJ#qEwNH?1tmR6saV8M{(&E}9@4%jFvHvQiRnF8#|Vv6P ze}jm`0l`yPA^zG<75fmJH`jK7cu}fa5DNzeNJ5kOIXak*O~ya)v#9X+V@6>7$rwr` zU|o<3FtSUYlYRgu5xgA!5cxL&&E<4Pcu+piTDc*#~^zCI1oMB!;)YgP(y_N4|rZ zjRGIdqD~L146_*ViYX`T6%bYGHcu0F%4B18`p?pVmxsGcg>YpseH&wF|CLp1S01+B zYS2!6Xba8?IRk6x{CVh-*GbYJupdsx*Y?D%F99kx^AjI$Lq8N8d0hMU5&Xd>Gb6Y2 z2AhAA)}>p=rCYi+7Ro&=Nya?ljk0HXo@G$YHtC*Qs^`63&&vTj*aN#3z zky7=9JWuILM6r`mwz76pTQ#?>ntNo&YAOU@Q&n?~rcTwUX9Dd~mA0p94(L?P+WK~z zejqzg!cXD2&SR+Hw$kt3gu>fOzip-8q@P=df@JACEIQ8u1;my+$clN zqBJ2n4GMzOm1N4EfZY~^IA^hI7*wQ&blE|dEwcr2`bus^&kQjNMv2ElV4va`z@4Hj zQ}MDxh3YXTuko3o?w<}*dBm$9dX(OHJ=-%lVizOv5=mD3!SUCKYUpi=bWv;?T!O=Odbd?&YlF`C6Ym~`&(urFS zeR8CFDmKDu3G@YroAgHRSDw4KoUocTu6iS!2H?Dr{ z(?bhseYEg4@d6iQFbNVn7`X8n`H>G@A5U^dAbr36Moc&0vFi>PupjxC1ol*Cd}+X# z|A$pbqS`o;co1-3V@O&h!dKWF`+)GNEyA}Ogs)2KHH7dfLURb8Bz9#@ZxOx&LipCv z&TEu;t!CawK=^1^k~K{f>y6POe2;|iZEvsaJ^H7%=wBK7M_cr-N%T)`(ZBsd|60_p zMeQCGwcC2Yb(Ph@19b)C_LzV=l?sNeO1da39HuC!x@0Pbtf{8L6>4k>)JZCpB~cQL zB|u%}V>boru)O!d;dEDi-!&0Vh?aqMCo}gy_pG2Fc*N%Ydpitz!2c1i_tT32Tn0q1 zQ091^i=6@L{O-2nyD`?@>Ul}b;yz5N<`aKJpWaJfm2dLKDKL;dd`ih$c;+>?-DKFY z`smX*4AzA(%eAAlpj%-c@K#}3K<+N9Dd$d>12Mj`nG?{!Nv0o%@DOYoPC&S_vUMTS z3RF}uc9dUv(Z_0OLFa{=+Eo_wvZ}yrtXE62D+PQ#Q8P03k4D+v$-7YT@yG0>=UtG! z5Y>fQ7_}J;c9RZXvcweIu|!d(s%Sf+Xo9pc1ksU2P1PiGq0M9!iE5#+vy!i*_7>`p z(=T1ibi26zU4eE5K|kbXUCU;)akseL4Y;D8*v!_f>?el1()9;XPR63*9oJ`>D6&jT ze9ZO6$@Gp3G=%g*wZgB!6CZCLJTi{&c5BMd2Btt1WO30My_7Y4`9fm zE-6A2ZSD=KiH4-h2eHYHiaJx1Z>gEL8{6~b!Fnw3tV8MCgLo1KR_I1H{6Ek|6+~6n zG>bB-$+jvuI(2M~I;hO4Mdu%MgQ)n=pFWWvUEgEAT{#n3HAG1k9lp2d{_erfn(W z!cmV)AjgQ4e8Z&2Khdm9IdE(sjr9%iANM0ZEAu|RyiUhFO53YBt7?CmlW%IKsHJAg zW40yVLC0v2x?neSeWaG}4ZFVE@AiVb)2Sj^yTf=>JG7h;YG}T{?9Mh{#Zo#3>+^Gl z58su1lCiN@@+FVVFR)j}b_FggczO7P$Z`>ivS9;QaazSOzjB(F>pva2?&&T1*63Ob z2K*(7%CYnce|XdDq1kbN)g9VK-MctOxPkkpqaxxeDVC}z3hPkalsd94svXnRdB>uP zuIqwn2>I#iW_4%tCfi;Jz%^X|M`};sf?JObZtaGDT8Z_b*8T(dfQQC_e}VrUe~xTL zz;UA-*-1_-J)_2%q;>^O{!>#)vf3Hdx?-Codu&Ov6BW8nl58dS9)EGQeF;SDd&=lo z^tUOaQ{I{8R}}L`pC~a(lSf**oG)g^v_v^Clb0=7&X?yjwZu7}PuQx+)#sRkHbaj~ zzvs2N`sKO$Q}up;x%vz3;qpt(=IYOt&M0KD5k7jBr>x<(_&1K6UBp8JcY*gT&tW^S zKs*{KvVpm(ZMJ^BZ2cT^01tiIV&vKSDNwU{+vWaX6~t3k-J%*tDmI7P|E74Aa9>&D zrD{xZWrsX@{v^*NFnGj0@oI({%X-yq5pCSA$7KB%Zm5Oa)-&C{{d%d_L?rUv^Ovus z?@6CU1XdkWHW74SKFNeH=w}l0UgBXlIRqd;M#kLrCqP6dTHt~kN5o0|OpL=2rAWnh zuWsLe_&x_L;03w_FThKUK!>z{*gJw66Re3*8`zi0y(*gLk$9-*(eod;+gL zf;V^nibdzsKIQOoAY5U$rw}y0Fa!O_EwmyDPIZc>bjc_D1<(+%!=d#5gEvygu{EfzU3u0zU6QO7$|$wz{HCD zv+tG4pszq}*ybJJsS!{Sf6^tl?$DiYyFP9T_=fqN4_@=VR1Ja6!cSBTmpyJeD3Jq`Um zf^2=ju>K0ET{8EUA}vK~P~_k%uC0J!54qx+X`3cDHPfw&?MR%l+;PojS1v=29z@8{~KeHdX|^=5lX zER1=uoh0U@!ee)mSm!X?N!v+ci&JtHTBWH@611s9ays^km1%oO?vM`2t!>O~)hl)) z%O0hZgxCc$R?r2x?If|!hYK4rSL&P%%#4v4O3fG!57tVeJr}sJ>~l^)dv@o1#=~jp zalYWbwKO?Tebbf>=QECXOONv+s?d_-e338OMn&5;DmvuDk8Pu(wo%az8Wrutzc}C~ zMXTJP!w;EF!Hy#enQ4~6G+1QAlsPkW(X<3ZG({%bvZ*={mZ5-;6id;5nGwpk&%CSCeBT}>2y4V&>YZ5c0 z2aFl+reNBRF~hm~Yy~eUFUmHp#0I|5`8IA>14Y_esIu*;UJtH(@B?cr73Se%ms11z zh_Fi?mRQtnU9n*KxMczRIHCksEkRdhZq~yt>y1+Fgoui$X^Mn#r>BQq%Cv6oy;?W- z9*5QIk1E$kPk)LA0q8eiZW1S+m&9bqnClIh54M9FyY7JbWWb{V=pF7QAYS@>6-X-W z2qgWKgo#gXnI8;UIqE4$irS%)^v^rRIX{4l6cV5u2P*-+&Fq_fL_Oy=>0_gwQj2=- z2KB5G@_UDR(laV{1zBoQ&qGE%tJ!V~)N>-s9tHI@y5L-DnyR$0=U&5}n;1KLk8VmW zx><&9%CyZ5Jx^VXmJTUH=q(*mevsQ#;U-~!>40H>Nne2d(@Vz$`xjt<9QGHbF8C8- zniYLMgCn#sehcFt5sbgeGCX(~zu@s55sYsrlBiQt)lES%B~=n6hbbnNL@McA(2Y_U zzm9*btCVW+PddPYZ(jY_rv~FoR9p(!KJi$m8Uw@KLJ z;XT35j|>zCi35`1#Bc?49QiOpv=~boj-CGk{cenfla#gJrOF}b5?qAWTFESs$8wxEjoJP$7rs~(C!-P!wq z;%hC6zZ(?4N@O>M;!91T`0|+~b|tL^@ec^ZU&pdoqtARb?>+*EPrHgt6@${Ygw8%= z`1@#T?R|_uYhwh9P$-)C6*YpbP$1)~7o2 zt#WV_S24=2q6aq4j|S7=e9k{dhQRRw74NaxoGzDu@q*PLMiKicPf4+Q@} z)cTXrD}Vbs81jzEP(O1gRzVz20#a#CE=rL#_V^Ad!5X5X|3 z^+O`m6{@l+gj#M2p+=CptB6JmQ6CpXJ&|!*QQ0 z#E&&_i$)jGXo{IkPd-7p+(&E)b6(;Aw^TWw_wL(@HVsB{rgFNCRGU2S@Nn7a2vOl!XO!?d26@t#P9y`fBg16C~@-s zrjtPcF>fa2e{b5@UIb$Xs z57Joh$aB>vcI`1iat3Fz4qlO}n1ZTEf&)f3(->EDCYxG0B)4e4*BYhLcc8i^b~FWJ zK*x#5%>rvT0gqiPs)2j`^#1Mp55LSqUw#l1cE?<}f+IErewB>cNGa*Ypr&0xS}k1l zkZ@In#Mnc)N>aO2Z~NOF5?r;mqS~g?^F({i4RVyIORl@7xXE$O^26n)tBcz@=+Hz( z&)Ksx%Cr#&;W3H<>^}r>lm$au2;hMsfIV}@9L10gE|`);6^W@L7i^P@oHNi@lC9Kq z#%hgHsm=^d?-+^%_79jrg3>wKN{h2j8Jc}WR^vXoy7-iK)w{%34g(+59k~enJn@@+ zcYSs9@x%8gEZ$V2w1?O)^Ijkw`{n+F5w;HQ#pZE!LVRyhP z1cZCve!Khjn-_okjd)3Lm&7sq!n~NJN@se-Ctn!3W1vqMXo>8~AOxAi;a~gEKK5k- z<5=#nydu3o#g0bx*&;@|AK~1(jHGB)dO3BY@=7DA%XB{W#oDqW`<#}4WCi8h z?wi}?=}+zJBg)e^!OUlhsTl?r1jjHWRTDHO+lnJgA~*sY*OWI@7{Cz4B^YJpW1kvE zsjh`lzIHu!=f(rDsokN=(54S|k^W1qptItI|bqlsfR*`kR>G*NBQ#D_?%6Au_o zJj!ISg^I-!e<(u}(fgUNSe#xPmn=qO|2%cDS{Si~5swE(T&Wfh9!4w}%EtsF>LNEp z%@A}|F@d8j$$_P(T(LANih^11gju~&YNSGFvMR|sRi7S=C}P~HE{gar34GWp!(_nR zi2GnWM{dXlz$|Q-UKsJvWh{<4Uzx(F9$ul{8y3bSaEKWM1A;Au`fLOSI)YC>=Vi*Wz#chQGDd zbyru{U8SfF9&RhxoW}&WnG!WPQ>mbsR54V;(oL${q6zjp81Cf-o$9p>*96HBg(X>g zm5<#NZUez~cxc-obR%5ncp2h$GIRSdi9C1c*1VhhJ{t@I4;VVA`I96X1b!TRL900l zK>jj!kiXdBF8V1uF~5^d5MJ10x#lc8v27?uN0sSp{-jpSpVZ}< z{7H$Nj}dSQC0w^`1Y`6M>v7AGeZI)!eql>j7kn=0<|@2*pe#Ql*%OzBY^G9iml*7%5oNKNX1Zh(;;-ns zc~V@+in*to_Amzh%ZjX1VGVE^3GBK4Aj(NvRJmc!Q44%TG#yFYYae+oqdW#!f z^RUP5D}Q&FZ<6?HOYdd)-&Y02OJvX4mFpgN>?&CJ9C=%1);DVN4FD zW;=lonxA*AV2Dh0>Li}mI9D~e*~__XefWJTaV+3cpQz{@qc*&xgZs=+eWz{ zliUr8xges^)kN)24JCuf!kM6TsiC#WAP1BTvJO7H&MB`-?mhm(#t#T$?)@Prq0NJs zXNO7PpqLBsL}@XOinOFSp9%T2^f+G@&uU3?j={Q?4CiSivZcfMvM_7g$-c=Ww88=B z+Mzo=mur_^YBtwyzCzwKMxC8$#fbF6nO5*b@-wX%p+2t@5pUqYD`uRu`F8d4?Q+N@ zFu%AHgyGrx;mq0WtCA$X{Be)nk!3#Xt}Ylz#xia#;4#fnEnu88+g6@Wk)5FNi)=C# zE2}r}k{eGSJKb-R$5#RBXPJDnw{MQ^=c5&wa7DJ^Ouk{Q_naGeHs`SZbGDt}%Ga39 zikv1t{xEWjtT1G~&+feTt`&z^WLDQk<+Wr_7Qa28hAnb}I2~V{etiqqQ818OW;3UF$0R}Fd@(s8Pw7s-EzLAuDWrCWwCr9+t&8m}m1zs6 z$Rwb1W3S(S#W+z05q>ceo(r0(T z0BAs$zmNg{)x|ADKmZV%qwI>vS2ym%t&Bu{79z_8e$0Q3Pl%bs2*rk+z;)sWB#ILo z%x<{Iz}w<{K;nSl8ITZP%T94yKdTyvVC~|b#7xex7Y50QjI+rm&yV}vZWoQy>AdD; zF)ZC22d{|CZ6kgwcq5a@d*lI?hDoM58)cPBsxv@Ni)tkZ8S0%+!JT{OTKyT-^B^&r z)>5vcg~S8YLi6u1+9wtC*ji<|ykCmOSip`B%5Z5}$`6fwKWZ({X5|Qt+|AGPw zUH1K|;&MjgCoy-_KH^YH%*`?IW0#mS+7ffmc9T|IApn~yF{d_{RV=ApL29##4=Jm7 zZN0lyiMfd^dz2D$B5{XV?zNb@<)nl7Z{cI)+P-@G{HCa~9IN(&b zwya!RR<3R9uotD}Rw=840@_yk%|`{US)yct8O$7ARAq@t4i&kg=&E9Cswm3prlsWs zh03CAP+DGEuKKZ03tV#t-+ue{n;+j}ODn%#v9&>hZgAatZ|U7`?ecIjlc>D3Lv6{u zV!eC8xaf1AuQ5%}bq9$bv-;5Et!wj69CSd+aALbo&Kzl4kQw-#L@`|G#eKXmWD%lt zUy|X7hgl*~l*a7f(<+>jHrfQEyjoliQXzzRokUgw+Gc<}50~&g-}594@cS!#TQ*jZ zCneo?si1c>s?uWhIML9H<>MqlnW-N~^ZIcJI%KbU0Tx+S76}qi}Db5z`n5T&gksS5mif-RW8X~)CHUARI?4+qLxD)k!$k&X8+XZWV5xXg~%3U z>*lm4xzEQwTuX}ciG$kG;ym-dTS8=n5L-f=7jcu8 z9Oq+DsI7R>WEhYha2Rkm6)%>C0aHgXwF@mo$<7h60vBlR>aLp0PvO3-5J!sb<&`+F zCP!L{BgOynQXGgqoo@nhwY})~IcRxAy|6aNU`c+!VLb$R6R+;zW1Iyb=$K-hmTIdC zEUu0X*CoXJI zVlqsc`0(_8 zY6^%9X3w)vIlXG0QJC2yj~o#b)~Qh5M#Q#>h*ikgrXpf`QxP%oOwzlm&_=`#DI&JE zYTxpVLN3c5B_bwusj8`(BDEp0y$*>rVb4sVe%WX9rZ(0E%g%RDO1#`mx7}tF0=`q1MgRSE;x( ztA8<*{vUt)zup7`hCyc%j9f(9-7k!w5RVxnZX9tEfSXHVb{DvD47xO8|M=VgeMY=w zfcgM-Q^>I34jH)Ii_>B7w_mV*!Qk6(_c5jeaToqWU|_fs0m}{|7e4dQy^rCGJGhA7 zgMM&>^b}`)7+^^+cWDQi`EOx1?g@DtBp$XXA#WJ@V;|-bO^+%pTuMJ?v{fS6yEIByE`Gn&Hulrt)KHCb&jzXQVj)-gNQ$bzqC-bcawR7KWhr7cUc z&xqgN+E)9VR6kbBZBqTSKNstvsBKQ@dFI@-g<+cPfhM(x@?Im#1*bBHk);WCjIQPR zYDcw2k@p!zKBDZ+GE|_n@O~xQu!&)C@c41TH8~>um{Z%9RBD*EAaE`zBIO!WWx*Cq z+ky42mzP;@lxp`+)Tt(c^QApS{CH{gmPJhXgNKc)-6%=W^%#LC!O-G~UC;GLc)T`R@7kTXGWyrRc0E>qbY?RAmp-Fdqq!o!bzPU6w>^ z0omOEvQ+|g?*Li#OxC+{TPOOE@YuE0_*OZX6H)dkcr5KIGF1#pTR?W70ojcVn|;r? z9IMS1Fr5WV#TF-Q^5nSI;)HvR6BgW?^RXY)f`9uA{vBBg>4O7+D;3?r1Aqks_Lu-b znK^>3$-3eg5|>TRR9)tZu1QQ2Bt@$U0M;6%#z8Gf&?S-5Cz(K6v()pOd(0RZ+-u5Mv#mW&d==SFd|vZ{1Ri@ND#;2#(OKa;{6|6vtu zsqYB3Oy4Jh*I^!;%x~eyxumEaD(7ts$4Abk*;G@ht90?jcM5TSi2FPwo{JSZ@Q518 zzS&2>b4~y}HsGnZfai_?&k6<75a5YzS(^bowKI`+HL1-&J#56YlI^xRbv~A5kA!$? zsv#J9y)jzE^N|qGY6EER;Z41TH_PD7ws!0ccem6yFYM};8s`Nsv!%xQybImNMw*O` z=m#7dk@SVwNP6j*Vj~6YBrm{G$f2HMCg<}&OdCIG<0nTHKUrl-AAJ0z;0_;A{KQdo z*>NOJO(v?`&_pT-hA2^mnufr%^1`;Y(xqfk)g-Ew7vQLV>{E-Ms8p#Rc6jZ29!!ss zJLZxh*aO^yE9?*CXXXw965&}-f8hFLfc(QFxW~2e2P?x7^|Pq!RK3_qIf_j}1C62X zHPW9an_wTJ0d4y8j-mlo(yggz0BtB5pa>WZ5EQwM1{_c{U>z;IMm>sZ=56zz+h{;J zRI#~TvdpqeP4J9ds7-Q6haUEVpm4+!56@ z*m}Sxn8iG?Xb&2N`*?Zr;oarMr|0QngC`_ds;W@N6)LN$CWxY;Q`2!&m1=nsW=R(F zCZmAY020# zcZb=bfAEP1aFwRRFGKS#U*3|UBUUd9`P~@BVg!^G&JqJme$V51@q@|{6irabJNU`B z|MBg=Mln1dOm5!_@%^ouW8~ct#;;Jo*6RKKW%j!O*XNZNBo?Zju=e{)4c%ABo z&?W!r^8H`Z@i+kz49M3obtuP0to9(E}c zcQff&hMwd570}i6mZ+b~d`!RZF`wM9#5*CMP+s=&zu*3c>BF01^OcLBOTK#t!v+2c zc@=UNlNb0luK#@&HOTE>TsQ-K>qJl+K(>U2RfSA-CD`Rb)nhPi!E z6=lG2K*$fGeA$bJ61WX1qfN|^Oo4yQRr<;Fjtewh(_7r=nuA=iul(r^@M(gwr#FFL zmyAAtbv}dP3r3S6={lFVV%VCcGtCwp-IO&^k!)Szs#qN8;|jY7r&31{*PvR%(qojG zoJiiuE_j`?Sa!5$vUz5hEmcZS>Hev?xYR`48`JirY}*mQGNSVj&J;;bg0a9H@5mLx)(>LB?wAWw9cjCbxNYF&=Y?t1j5KZL%{JSD ze`!X_p0xV($>`I+9Aj{OJ(Odk;JVnMVn+}^2|A2_rU+eiHWt2aa9vD;>oA^nCaPzW z(A8zV4X!_MaJ~M1D@%K<-rb9*@z&+cRj2WmgK~4fJH*KGVytV%LjD~C0rUIJOx~MF zu!W{H@Df6PC_jNG#}LL3r}>v<5d2hTmqPFbDCT^hP?+Kx>^~HPp|vR;n+U__;m!%B zyF4N~y`{+wF%KFyd#LEKXSa&ek87r@LHOgE>+)d1am}+lsBv6#Tp4^huBk2w5*^nh zR|K<;ZlWuKZpSs(MZ4;8bMPim_4zh1b)I&WjsvTdC2%wr=mHAmDIw#bCsD7$SD7z< zJ{R@+;WWQqdP5chE8MgHS$aRcfBXK!FZ0gR^lG!YdK)IsVJ^uobLS2!SuRKH0`rSI z0l1#6AJv*oof^5tl(2q|P0vv!zqmjW*GBeR!k@)&w@950i-%U{gyIZT0?#`y%_z*t z_lSVPeD>ci@Tzsui6X$iI!Q1?{ujAnD$`cWj3qaoK6aMk4|%NQybve(;dFd)COdI? zbRo-5oXL1xk@Gm8^;k9T+_1~?9`UPb=Hr9t>QWqb9Qm4Z?K)Ak0`%4h-u7rv@gd^0 zE9Gnr$o43}&3SBbkF2_N^;)^PHy62o5aqVL5NK7`+Nx`;Lsln0Y@wX$47`zcDrW2F z;l9H;7qSa&atOxbKHF^@96nix(GU9YiLM!%rmK=@tAd~!mS7sHLJi%)iZ|BM5-{mX zB+O(mWiRj<9v-)BN2QL%WZU8@<05l($Kr}=DuOEKW0eHXtKOhJl`?-S<(h>N^LxPw zSVZ~pgvoc4D-j-cocqD-F_x40aW9w`Gt*^0jQk4z<0Y{0v%}9`E8bRpZ_KO zivQ35vhbkdFvTazi%-0EJ6 zbU&K(F@HMwk5ls8 z&X8x}$jl374SHda`1Tq3)aS&Wjy9ePvEV6)29xw%qtgHuI>5h9h?&G`YQzvl1(ui~ zCd0Rb+u=I=0soAGAoT7*h&YJOBjkzIL5dEXNsXTq}EJ683|k37=vP43&Qo0&De9g<-)s_?ThZxnZVCkX2p=_C-~BEQv}@X zxdTp?GdH{ki|4rlXMPIb^}&GUNLCyFJ?4}yc@4{P$HSAQ>;g}-lML6+rV1pnf+0Nl zSDx|_I(T2X=S?~3rw9pSA+7x^za(4aMYaO(pLcA~g{UqUnGBXIfD1tqkr-yvLAl`F z!JR*Ty!@2CGWHY+!7_w*NvEGOSSeM9fuGKq#6k8eU|M+bkdI$HePn_K7t`Q7?nsjp zBsy%9!U`5yX1x64r*w0sNymjR{FGK8<8AWQjr(MbXW3<&1I8Nj>;L+19fmerFdq1X z{ICDf2}W+<+8J#w{`umgJb7-Ea}(HV1o_p%jhF#0pyA*p{1|4FOcnzk=h_K8Gj%QS z)m)!&X7$qvp|lI!yU#sDlDFA2~VE~Jay zho$}%Z@U`jYfUmy(@oMTGX8Wi^H)L+=uFr(+r}X>pdX>egsG)u#bFEMFj)T>d5|p< zGTTMv%LxKt#H~~Sc;5X9$qc55L`kp5qd4QRm+1z~-Um0^l$Y>LG8(3p;_kpMgZn+s z@TXsar{I-cAMTva^DLu?3-W{nezvnhH{Cz8OSEK7Qi|XbfKA*R(;`5rg+D6LJ`;cu z7?1D=KFRg(U@zf=mJI!Kw!U4^2O$qq84H2x-Zq zlYE0@q^z@gUKC`KJX^vt;fcKTweJG0k+5YmAXlBMt4@IexjdXK7i{uWPBQZ6KZ9wM zsRsOWq8_pp^ry>JAX=T(`T1(RMY`k)l#^|{$o6vt!P$;g3;YU!hV(6i^tMrebaFvZ z6!&mJo@R2x>cc05Hv^C4zr-a2HlER>>HAEFjL?AgFu^+)W5rcS?*z}MSmT`OZVr@gfAr;f7`s+}gl7*=zaUg++jbWm^e&J$M+Z)+u1KPA7^h$6 z`1BA?ld6}AKD0X-{WRr8Str@UaOW24R@FWhwu8x>& zQDK^X5~VXr&m^TQ7~qh6QQkMMNt<`(}R zADXl{dKZIG-D zlC?pyHb~Y6$=V><0|&`|_b*8h|2|g^*@rezwtk@O_-ctP=5OLR)r(K(XU?*f-fwmn)se&Ew;<~8g^5dR#jy#hH z?UA4OTpH%x^1x@sxSLfYc(mZvLF~6(5~AHe>u1R1J8C!EZ_T$_xgQ^S*YA-%-^zg% zJp^0DyVhy*vid*=r;*z*?$ZwA&coni7iDRd!SGJu=B#F+OB<_~^2oqRV_e+07xLn|^<({E&(r~KEE z7leHLUoZ>*Fxk#m_A_Z@dSk;BcD$D&S%rnlcpWzP`($Y1@s*i1Tl{u@&Bhb_IG$jY z=~eNAT!~LnxASZUv)5+Gy-4GIpnK%22Tu2N2J6wib}<{g1iug2mG1;&*_MN@lkte> zMi^tsU-bUjIM2KA-4xV=Z(TP!179eEpgvFTwI`_5Jfe60!{nv{AUu6YiUj6a;xlOO z4dwvI$&rd9$pV%co@-Y1N?MS@Psq>k;U!4p6Y{Q0p2LS9UC;e%;DHXhhEHz-Ymi!fAK}BhpdTU7-6e0}Q;}RJ z?R-oT8y8C_l_Zp^}%{BuT1`TI49Lr^{7_5}mgN&FP#1ye5#{ErC+{BFUu5xm@p zrxs!~LrzdnU81TdM5a=Q$dbHBQ*2Ielk_+U(2a*yp+mbZip(}NRZ~^T(J5Co*|02) zI)-90ORzI;<^{z(X32M`(^9_F1WgiE33c)M%P5sEqqNAT!fM)PR%%ArDLGSOro+t8 z^nQ^qk%sjDXYb8+<5;o;!T0$JM*>xlsw0BFFH9wCxQi0EBzdKKZ&gAtcQv>5#bTR2udQUxK$}Bd@(mW zcI@m+*H{K6)yZp7C#=Pqx|TjVN26|YNsaK6G-@n|Ewfm2PuPlk3`eWg!HMe#T34EY z9`mnHo4h`)*Elik9EdN4o!aa%Vu%Gi45B>V1t*+M`s8WrpsE1ZNk6mmyqTZpbKef=Ezk-$T70XFmpFT@9BoI@6_P*Dq*97*7}Q0M zX%;Ff4$A%v4X6yk6lGU=wC5nLfz!0FdX&-S(du8rXx-2hr)Xw zhx&Um82(x~k3yhO2cZLxAJZG*rzpN!1~A5Cz!;!aCB#PpTF&%!gc|I2vA-g)jIsEf z0&PdBTpoEiBxyg&^;;YTj4{x&2-P`On&I+6mbR+l>S*5p-N%~f+GK`KYWiVBEY4NGcJfQM6Z35!fH z{Eo$~#$-r=vN-};NYIL~6K9t|LJ}te{&o?$nd`&ea=BtFf~kiMG6KhQK*H_?NK7$x;Vf-(h{#A6bHfS- z0CLYRLuW+tA&IX*8$-Uu;D8~l1CTNOVd0mic{q0%jpKd~sEn}5U$`c*kA|eWjR{Yo z%`Gv=dptjnU6fS81#q*k$&wJ}hcHA1Il|cNK64$H+m~cEtA;D!;HG= z6-8l>QRq_|2ZJn4G0GNJbk2mnFwHn36kB>ku*5jL^15Vk|D?SBo!WhYoG$I(2Dl^9 z%kFXah06Y4?&9_6)9tWN!`tBH(1$}xJ_E;>;1`ew{FB1O2cWXx&LuS9GM*3<*N<|! z1r3hwgkuEo$H3O#*2!aiyX)`ZI6#srX^Dmjvf&s>bU~pVlA*PvJIqp0aKeg~=peJJ zswIGcsVyb6OD$5n1Y-Ls#y=3iT?TbLa3{Rt^-gr-;=T%dh0M*WVl)24d9)-=WJl2; zYDTYf231{ybvL*W$RWq)oTDd>U*NK2fDj^gwWVZ3sYTG0P|_urmwv=`kBW|tic{ro z%H29G;X9r+oQ!I-At2v{S_|Nfbd>6PH_ji)1W;XuKE06!OKNQ#!zf zqdc@l03Bk2LI;P+<}I@}_ZeKyOSo)D4neTM+3~u>1rPySiMix40CGy*;8wuib(A3S ztPVx(G=%h$A0H<|fZ@OKa5N5HPB`(HHg%4!O%hre^P8te({-V%EvM;9EpoR6f;YEo zGgcLbQ3mp8fJNoAm|s0lBi9A>v?P5a46eYohB*Lu)Lx~H;wc(AK`D>x?q+{lCd~`hS*y^;@Ebd4nXKN;gFf8t^$oQO=BrRO==PNC6M^{vxsws z^S;lyI?nqIf&E6zVjR2>a4%T8OcrC3L?gW*p_k0-f{vr4nFv*|*@RY6G$(XeIrWNEpMN=?ES>|;+Xh5pl#^|$@1XoP4YgJ@{BsqO_4J8E< zGyz!xqz1q;Fqp|k*SQtxG>#k)G)W2|+jI!Zxk#XY=$-dAkxt1#s0`Ht4cE{_aVhw< z$oZ04gWnIR+`t*zMXfFd#pNox&S7{I*&yRP3}D;spl*S!*&y__)5M&Vlk*4>w!=Qq zo|5jjb|e!#R^>OXh!2erBR+7dX=iYJC z1f%-z`M;B>l6XtB<(dprV2~y>aTY~wnBnK>nh(f~8Y!$ZWTsR!802ia3@j}uB-be> z$+ft190xEUIIqw--YD-BLG{iXeX-+XVZ==jEA-#nf0VlW@00|fy_30gWA<90% zHB?aIyI($oY|OuM5&PlqGn(?j5ag9nx*#Ikz-+rYvn|PU{&$&$$L*uJ!oZUOkr>zW ze7T(tFPz{1(tD!(7fjl(x%tghClO=h5 z-8HD?QiF=NX_+>WJ=Ife3foL?rN>BXb%tk?VC~Q-G&q+&y$ByaRfaT#;MC-4*Xadc8z6dB-?O6w`_mzlA>#J9bOU zqz*C)@>3s$5w=m~0m0nE8W(5^Yc;l#YknXx`y|Z@v#shS?2VU~E1+dOK~>G^nPp zxzvninVMM?lSV;kC!knF0F3P|shRr;K=}mVKKZeB9nGhpo}fGh4ATcHP*Tb=C)bWJ z)xFG;MuTHaFEduIQLt}|aW^N(;Ce?;3{?Ul9R8A*5<}${l}`qhZ#TI?ih0Z=PTS^U z!V#z`e)o)q(V!%aN5t)CN#^!*l?9K;ZVYIQzjqjyqU=uhC+3=?Lp(b_6pRRvwluuy zFB!QeF<0~oQds3Q7jarb#uC7f@MnOcI@gWS@s1*B;ebc+lSLE_(S!?x?T64$1+Awq z1+*4TPX>|&(i0{#3|A| z;ux2}x?W5vMrrXJ=9K1yo^$!!xyErz7ST7!Y)6yqw$nr{LLEXqL)VF|%eE;hlCFp% zU?z2B$#Au~D1jy_mbD-uXi+3(9pwOQ8-LYdc={H;Ng1jk{eO!-t}BA9n{CoXpW*9X zZdoKL2x$rB92-@E^DHDFtP)07-XbCck%>632aPO28IJppih;NIbIcY741ft1=cV#K zal4Vq=OrvEa$;j}B=T+o=pPUJeu(jZHQDNzu=iCH5L}U^t|Cvs?`c~|4OcQP7 z8lp(32^*)Xl5H8Ht9iOumu2a^J_&6#y6 zPdBCLO`BZa;trp25irU^ju!8Mf+V>$%V!9%%>?ZM7a-Z0eVg@jgV*bvzh7{yxVKrl4wXvQMeZE zS_bXHqf|@rhQGjCv~=L#d96k z^JGmmG-Au9r+cDFGzU~;#kA)-+J-1w)}o4Viv>cqdgH*npvfTj@8Cx4`$3`A?ia^v z7&F$Hdle+hGNwS9xaiWk4cGx9NolPDs=6=K&JMHeNtUW~;g6+^=t_&;uYlhF4(Rso zFyS)QX$(6JG7K(vlVk5D>^YK`Teh-RZpE1ot zMxJTr^l$vjTsjH>8_;-BoV2$wVf4GwkdQ^4pYQFi3!A(O6YZme@|qXbLI(>AJM6Yb zOkq3@YB;E!<|e_UtYXHZa(I3ONTo;HD4AlyJ*$KTfRxaBY~GXn*{-HbGIoLWK$5X^ z)wUGP)>M#jEWAa*kIi8K;c@#WAz??&So9Bo0;#2$)fWjqsi(=@o%J!pQ)2WeOiG1%IA$ zN$V_1nfIQDTFfK-$gNhzF+q3KqWEU#!gof+TcfKe@_l)rWhq3lY$9r=Oo++MQjS6` z-IfhyK3jzj%C==Jhy*Qawxae%fOr;xIAfD+z(miZ-k8OAT>NT^PNRa=c0ckF%eqZ5 zcBaNMzLOYY_u9Yg^AdwRz;P_^iHFDtG`s|vmd0gNiaek4T9ala!qnxN6B3+gjUi;Y0hdWAXQZQU{eCmW6bKl3fk zFYmB_=Jw?J@rOg1u>C(`@-Z--G|g?3Z+$c`UgxwSCk+k#C?A}SbN9ajCQ58`0pcfi zhdB`Z@}ErvZPKJS#EEQ%o^Nx9L<4O)rS66(Rrogw_fiX4IF)~rCJgqu7`xkI3RXs98S*%hJhhJ*eC!xSGj^F5>tfYH(V3r2aAN}} zl`UBYeFFYcmXmi|6nDj#VTRpD*{31ej~QjPm%DEM6B=tlr%0pFy(nF*^82 z0lkW*w*eNOY=phBpc@`n%gUbS(WU|5r~es`ZdpG#kI4YRIT;Y4h{|BN=`h5E*{K>F zX^>I7igZue$dD}AmH{ti_y77R`!X3j5aZDQE4(mL;Zw~>4_Os*5?3U*Le&eGeai3sMVDOuz!7?RqmzNuE zI<{*OMSc=z!!#KQ%#yl|*sLP54b9dP*a^26^JOpT-fp%6?o&>jL$>-1Ryk zN&$}2-sbu477M_^NH0@lgvpeb3RO~DG;$R*@+}YzfMs=e6DOmniGk36!qk8P=4k9; zCS>T+&$)(2VsZuef6dq?KDU4O*DjER&)xHg$#Kji4P%l3!y5MZDR3BBj<`H|OyX;T z1LC6?)p-@&Bv*O3(_TP6qA+gR?1bp23cuRU5w!%)`mx&~alD=%}SUxZUMJX27gd;s-@^>}Yrz#pLXy zMiAj27-^L!mTX6ZK^6jy#PBmyWWnysC6KkK^s0hibg%d}o^#31|C|kceuj2~K@#{h z&rk&TglV6Yk+kQ1|2iZn#_^E%ckGS8Pq=!p8v`ST^H=3G=6;Jes1g(>ZX5Y+UaO(h zvRFBH%sYn>{4`^HyD0SD6eM%FAIWvPR#CDsPc(?yvZ|;?7yi7jj^?VGZK*m13DmYd z+jAw=lSI%miQ-zIna{;|3`3X11^tFX zyJwmInX#5l_R@9TDKe+q05c8`x$m}$1{lrMf?dkh>8R%NC0qa#0)GfFhk?m0`o+tL zLLg|00(z06OcJ!FI(S7JBm~7VY|rvQN}v{Sd()%}C7vdd`La)en5qlQ2rbIAilNmc zzA9Zs=V&+CF5E^p&va!(ron;?&PY&gn<<+Hxv{ zD#fj!N*zJdOcSHamkZ=nTl7&C!=fUv zoV*ykA9L)Bc{mZ^Tta`()7gO4UWS7rl&NXA%?ZzCdu#$H9X1D%AYsa%VkuqtV?ylx zG@QidOtR!D)FGa!*@l6BE?uNl_dMIdIIwxBhp6d_v7m)VZBcwxrubF{-S>p&WKNCh zEG)H*xYvxkM-J@f^=+w;q0FbPs(Y7Uzboeos%5ZBg>vqI#=V-NjLyf_)$@5$fV^d$|X*ItSbRh@ma+&W}Ffh<@xRH-6ZKSyLxR;|QWX5zz0qKDChynMBntQ75xGJRkW{22!J0``&z>^Vyli2E zwRvGtF|>?kSMv0fE88Dj5CPzs_v{Wk6aqz?9Njc@z2x)76pLvkT1><%tbt)HSVT zuP57RUr3uTrXqa2_07-)Aw!Deps@^`XC*$swU9|8a*H4nj@k{s|Nt3kP!o!kK8Y1x0D9wEZ zSdIr%ZwV_Fa@H%Y zp>MNcnWh z6e7pThAdCJ*ySREzcdUNd<(V5; z7!P%KGi0cKnm4kFSP?Kx+f9kn=bhQc>@!+Nuq=$~wk&0?fY92NWP{aW13C+;| z>05O}=PSWI<6Bad&Yrj`R&|bd#8}5Rvmr3+7{vz?I|OKOc=zvseX7o|RTs5GsYc?LDP9N3^d|LZ}Nc3R{KRNsB93w|*%QxJ;(bB$?C_tRo`epaVO8PPa#efF) zvB4BW?j4^DfTp~Ou<3MAKQhVw$g{~o%hP}O!>>BO`ZNfDExr0j_*=NalPvk?`hX*^ z6oN+s7_S;)2)k0rDvN@>5<2++Fk&@8xwXCGK;eeNBaQbGf1MT1C5I7b9CHjH6t$yq zj`e$dq&2`!e5?z9_dX?VDILEWD~QYB@XZR1S2Ehhy~Q&SHLJH9LoFz?jSM)TBw>AW zadBnAK5X-qpoJB4S9%~m%aWz+cCk?xuk8lcx3#>sabt*KG~_;$+vXUmCTp@~?si!U z{ER`$d3+#2S|4YQcdH4Ez1*AU(*t-tp|d~w6dSOOcU_wzKvr$lMkKqh-5_ABEPll| zC)UjW`fWW+*?nWO-2Q5-jZi8a znXli;(@@VBY`jvPvKfsizOu1YQ(aB-BwJCbL_FCuZ7RB=p-G}?P-CL8v=pH+6B7gh z_{*j1>z=4c7O`wk6+K-SF@wsIMaMCSB3inxv#};f^0Q{JsIe5;%u}|Nk}T~G< zZ~+ZO&4&};iQs@>@9O#9poxQ%a(OsTnz{46whLPK-CySXM#518Q;1N+P3C)xgm-@e z!57`M1mE}3>@*h;FxCzxC)^4HmNZ2KipkbV>vVrNW|co>)fEwvST>|8FwFZRgi+kx zA=kc(HO$Xw;s@L*d=9XrJec%+NJ5a>g@mBX3f}`V1~ND%ab~%hyw=2x6BTT%2OLF6 zf%dvl-mE!45{`fOSCAS)5ZozRny@w<+zot820*8_kYph2f-;Krvfxf!%*$un3kb=< zfTAHfUSVL0+kTt5B!(Gd%Ov`{k3LvI!(vQb3P&y$j)&O=3(7fuqB+aInuE zVW4)|m+BwJ)%#dx8;i6vo(#_nvBqVDQMV}N`Im&v2PSY(W&nFC4|-+5rm@Eu$~jFZE;C;R|s z0OyS{FvXE}c}X{zHexbtupc5&aDKPVC(FZ+tzkSL)k^r8f4@+~0c#l+6agN!x7_+v zc;`x(ezM;DN^@+`e}cQe;x53b*ED7&#Q7E%_*QmRZLGpweYztIuzq1o z_%`$XoeLZwV#au`F=&!SkyfKOko^T#GsMs%*bA54VU-PO2X_c4PK1z+fY=QDWDvb% zJF6NU)%?4BO{|v!a74a;oEX#&YxCo`t?-U)_$=ZB4pYDmeiA&{*LS~gu*}@wdM8iH zcPj^_(8o}HJhFg=a5mT>?t?N*7>MD+KCyx1-U`?F4O{}E3eu-|9=h1WB6rM>fq;Z3 zPC%%4Ba96w4JUB*6zNb9T>_{5wQw2`0Nft+KSn?pleMt-Ypf?FD*4O57A|Pe6aI^$ zS|-SJs-)__7WT-e+7%hD*jdEL+y9=0est}F>@Vz|pK|GMk{=jjjcZ<@x_h@Y_ndM0 z6rk}j`Vrjon);{<+qmE?F(*WrM7u~SegLR^J7%aw{w^#qGVW{>#E5l~U;9)#5c@T>IL zv4`ONXTuLS-Um^X*Vhp^t|ZBhtO1lf%dkb8>bgQ5R}mdct`BrJMKi;td_B36$9c@N z16kGqCLq}Mn`Dx@W4RTBAHx?$&IAap9sO^+C809}bII;QNbe5&fDK&Jyp1}Am$;AW&teGLA zum{RRj1QxvSj4?W$(m-{j^=oVZD^t{s)j3Cwr0E50@(R27X3(&ab6MjKFD}~vR8)T zeJipDNS8Z4dIOM%f!r}q)u%y_`DdR~gkL{-k zwmKcuA+#o~RRI4x+4(zJyzaptvc`Xe?8o0JHvBbY|5mlGtVs*1fDb8FOk^>SPNMzY zqhcT{i^Bshasv9zsj_@!LjM$z=IhB9Uu${KSdw^(2=-j5(u3GRfA>8kSWFLoSt7wg zx{!=@S;T}0MG+q=!9rWt-VBoIk+j*}6D>O+m##M-ACafuskagOzGj73>ja?QjL>{_JT#wy5d738)Ga$5ztt%BTE zL2gx$8)lnTJVJPthr`#nbQ!MQVAR#smSn*)uD(4}uzKVnWsp2@|J7PP{PS2;Ey3V< zt!yk2;(0AEPuO@~iyD<3p4aLou#e}p!1hF)`Z~XUPX1`d{CpwQQxfLyA5UEyY?VRZ z%%EQ|r=2IQRXui$pOE{;qODECcrA1fd5gAaht53|i* z{TwXD>d_7JdFYw+Mx;n?4#aMMcS zy&wN?kCR)J+b_tSW=-Ug@S>Fv-FoP>)uEeL&776qx1=UH-|&m!`4fuf;g=Gcj@ya9 zc+z|oBvf}ouEJ2h+c1<-Z}gMk7b_9&DR7hT(rDS#%?9N1gFouQn=4^(DMhW!FDw+Z zN90+X82{d$bUEn&&ra!FiDe@?5 z_p0LucYQxuUoRw_59PD&PJSsFJ%$^vYHb{+ch+oaNxVhq6}IBWW}%0aVz+K=KKv%a zX!G8i;_6WkJ9Pg&Xl}p8=G{I%cV6_(#7J&X=<>Gc*j)1(1E2FA>DVpvE$A|!!Q?ri z#no9lF(^i;PymBKe82ky`#~QRA|bA)BN!b)60NQzX}x9ojkdH4-pXR$ly|dS^(G(* zW!=o*RO`N&V-=Nhxh`Db62eQidXRBhjJK1wBL^wBiycM7g!9t48mH*qllDm}_^2bh z6uAuc1%-F{dafx0eMk{R_`~ynyZ{V~Nc4Pf4no(5sbZJW9p5Xkrz5ClN6=)Wt0~e_ zAZ_UryiHipTY`<3_e-e$jHwSXhctGCF|29W1T+ZfIi3XJ#cwl$>8SZytF|r(9o~RC zV)9SI`yeN5w<1u+VZak|1@qY6To=yCC7rO^W_NKn@{8u;*j-$=O`Ynlr@Pdl8kHrG zT0N>MRMth-GJJM-ab43bl_lRV(q6o!`|oxi`+mRgy=RcL%p5<5Xj}k51b{@Tko-O) zXjcG;#SwA(*wDV#$eqG%7BYrB6Pwfa0AGZC+?^v3K%)$H0^WYH>-Js(Re*~GLKnJI<*7GI|Mt?5C8x;k_}N9;!U`P;=SOOb-m{Y zuQ;G^7^_iwLLTjbmp2hZfwS%)4n)Je*E%;CB5&vbh=8P7Mpu^Pr2wM00ECe#q+dtn z=__a!_Sin-@KFFhwK;Oq6w;eIIYPS{UB4&s!(q{4m=tZgH^WapO=01Z`9cO{GW4#w z+{%r%m7@U-TZVU-HIbo1jj&_#73sMi-TTuqW7?5e3I9JR%ca`K1pHZfBF&3m) zFI1SfY<899n2bXpd>_b1ApQ4Us3jnv;1HqwgT5s#jdI?w{OxUbF!88=(I zq=PkoaEZ~lQ9Pi@nl24pZ>g1F#?Z@yA4sgy}l*4}OFAP9>OE2d zG^iBWFS!JY(Ae>sD7Y%;licB8R<+;!BVWkf|K7E#@mf2ckpL zV$nKdQ|Ku1uKevCxf%bw?`jF*vcgA9WtR zvF6Up81)i*R3~0BEUj1BlW*_2n)B_Eg)XDooClRI>{pUk<@r zM9o708HWy|+X`1^xn7E*i%ruTN<+Z`9SmkZh>)anX_5|xu&%fSM^e)|^2gmuE^jRN z#Bs7=ao#zxIf*Yx=zlIQ*7XcsXQy1#y1KO-5L)EcL!eBAb%h0-@g1j6)70mvkh3?+ zh1e~Q422;DP9>*=&HYD##fjmg4hprosGI8$mzmGhRLYf4FJn~nCW=99-$n~wU6%nB zLwcd$(p3dm{gNcEMQBAid4T+UzsB^o(6L__IdUb7%OYJmD&2a1x-Pu$gLVmM4FX?; zfgrH|fGjaaUNDokmjwbzMi?T>5dy|x#lQq!1NmZcKWp{t^KXZb>A@Q0FK;cL^G~V)4f>wbYAIo0AdBtFkyp{#l6oBQjAPS({ zE*Yk^vJY4U0cc7h{L_9D36MjiSmZ3I@Pxmd7Pd!x?{}!!4QY^9hNygGctYTRO4nFU z7?fM2+c|)eABxSkCA%!TCMj$YpufSs+#)U?XCX3U1QD|dBDZLSR>9PB=pVF0i+75aq_o)7rrX@%P66c!kylwr^Ta%&;j;N$y?l z56#V)bqmpvp9<$e1V?qk2?bpX4Gg-})o05UA(mfCDkG)j$%^5qnnFd(u?)u}60r@* zv^-DO^tp_bhAOItx|moYw+P4Qn_CB^+_ytmfe;j|CMoOdYikLy{hpnazGhrVm5r{tq@XRg2yf>g9re-HS0L(7Sx5M>61W?usm4?1)ITcu znhDEQv?BTz0GH9Kz;%&Pp)8j{q*BpXlE${kVdq)upe9V4_Re?FWd_ReMef33d^nes zP@F{WQ9qmqA@_1^lL3vnm2?-7HcJ_)&JBWi>9?65hG-N|&?=ud%)A+ggenX2vBhhLRI%|MA5#z90*&a zxC?^1a%Gcv8(q^VLE=~1N#Esa_oxih0kjAsF4#`Ei8PRcuoq#R&}AImr2Ph>G)?5A z$E_ovog8SQWGpGPv`BCl*(GtA%kyZm)ncnxvlSO-BSoiN?pI4d?WC1gj-r^4`wV;+ zO%^Z>G%RzzT)OM>r%=ZIG^8hwEgA~nR1|YF_Ykp2GKaY~k-?3qd3F0U^Dv+k%PbRYwZF~y%HgF$ z-=o3_9C4sv@FnYo32WvI1FgIL-v<$Li#5P8EV{L3tx0QUPXNt(T@rheWa?{@Y8YKZ zleMnZUCs)mw1|0};+B5SK08itD%(8KXT z@wxg?%A}zZEd2;lNOQD{Nu#kx>jSFe1D+F50XxEWbGYsC;n;)q!L;$gjxh8B275g+ zT7SGLJw90@zdsjr{e>b57!4ZC(ICo zBSIW=r#a@%gh}Sl)`zqv?FSs*zz6cb+Z=Lt(w6XN8`v$5kGTcPebj{sV4=|vDD_@* z=)LiwM_y@sVU7dzWAg()-sb>G6Hd@c078zBh&me^if0>RPB=>Ye$a*g%`;^FXk)|) zXK5QYV=nw+4>m@daG17zAl(59&ATHr+#QUMm_OVYbi$c>=LZ3~gv}-M%(HsBG46!p zwI9<<;E*7r>^Db!b2$!8{%m8&35V-6Mt|34bGXe3bPk?tjAbz%gQ4sHcIW+6@v#?-}FA{ATSa#1Lx5ph3e*93&i*tbCt0{zeyhC!Bq z>Ym2YXLKIFIOA6iU$Rg|_!kDQGD{g5-hy~_@se4iu}E)_N1he(TP~xWpB!EsbR`fT z*0xX1HV+SXcaMc(pMV&4Z~$}ib9z3xOE$>uZH;eFK7o4bma+9937;Y;gcm12oWA5B zt##GjL<<_5)Jez+qjCapaDINgE{K+_7@DG%m*~16X}V>KYSHZ@FDMsx5B4ukd($KS zPvfYx)1u&z-u)S?%rfI){xDA)JZL7y?bCQbKrOn!>;u6UPP2~@2Toa!?lE@j1}VHy z1ERdx;N$F9gNalgS{XoE?V+sN1AJRMEee=pCEqpNJGN)OSHH6t-}Fc_{Pd5$`x{k8 zE6da9S<^T_PoQ_V=!v%Kd6wxK5_LRRGl)Y(S0z;Sh@~#ryZf1u%6azszO-_N&hn^q ztwoy@uc+5Ere3p}txY}wE2MbVNb!BNVVhy4Tc{12+?C|6C|^h-C|&EC*^?#h=LhM- zX0B`Ux+wQdO<(n4dtQCm?rlJB@4&X8Rry)=VS7H}>l5hdh358Jhb?Yu-^eZV^L6Xm zY7Q}K7u*=L%}*ItsfIml4$DpJ8@VNZ$`EQb#Kq_kxjlWOGQv+8NUH|gZVr@N(l-jz z`;t6JHyQ@=Y2($-1KPOjz$)z98#Sx=w4rLJVRMA$Fx&P|EGI<>}6Z)zp%Bm&GV{3^r+tA!BmU3U> zRFJh+;=3wL*j|%oDrDOW6RIQdnPA0x<|PUr$P~C_vBC$P#bRX(i*q(bA(_8|KKO&p}_tHY70N)W;Gm>{*cW z(O(Pqoh-(Rp%MPhTp!z|ur3^xLHU?eCg=*-!d|`v7i(|0vF~mZY zz(6x6tfEssu{cVRMI=H3%3}yhB=&T%QAu-Zo&f@Cq?6-I4aRxmE&+(;*}1SGBnxqJ z4oQ3%l=Wz--L1BzglUPxO=u1>Kz_i(FoWgxsnh2u#pZ@JfQ2OcNK+08P*7QwUpOUq ze{!$@4#%4`VoL+{6=ri85?%x3mMu1K!~@z6;B$yoaFL%8mZHosF3Dh$f$#YRrcT7$ zo^TxDI&;*=dLy{@h|S*vI-lWq9qbU4q*1;TykkuWF9IxZUjc-0fMVQPSV)3xgu&a-HC=YVs+mWulrVNHBG?NDN95I%ih40&)Yy*> z3=GE`&cV>fsf4(r`S5IEZr&p)rvPxaFm*O)%+1U28sZWsE;;zhkKmMGJhpeZYmu9M z=41@$-QBve`U6(C7#7H0f>Urwc*`FKx41$z{s75AaS2o9l~Dl%AfhJvw}Ni6R6O?`wRSq=M5l>jimT%MUSZ8Xa0~=V4DOH1|u_H z8o(}ZH^Cu^bM%oNz+I%=C_f@lINfFFj+U1K2L|L8SOh*!872hytsHaVOaP8WNu4kO z!UI;^{W1W0hPw~jnXn#H5hx@HQdmS!ei;Zvz>f=dj^d4^s)b^yqy+IY!K2A2xEtlA zhgeWvZCZ5zFv*iwCg6i(+bTFxI9D=(g>?|2P^*Fm-55*k^pWb+jsx!97@aTaO^7rI zppA7O3oR23;Z(w=Yu+>MjcP+*jkHg=gYL^=ZKVH3MQhL@Nn_C@l_U|v}GlZ(aM!q!j1 z@wRZxGmDqX+SnqXN(a2SS>h*;IXu5qC)b!dx%M>T>_-kkBhG$!ugr@A4!lk{2)NN; z0B4uQWas#itl78S(CLQmn$i>3;CIWmpP*-=>gGrVerL7$-)!?AL(-sWDweI760rawTZ2 zq|DXke*r1;d&uLR;Zj^Ek5}!AT30qMBxPOI*JY!psOEBMysMnuXO*)%!Q!G#7Cy+qf@Y~lCzb5$XH)cfyGy};>i{{33W9F;1`m+I@I# z44YWSVOpTZd^HldxnyG`YU9;fe*ID9gRW{Me9tu!X8YAgY9y@m=BK1LFQ78dsV$Gn z2CG^ZZHZ_zEK%)wz2*pf#$1(OF{@V^cZ&w{Sd#uhF*#&>@(mC=LQ;!J*S$McDtzx(dBORn;Da@9KM}XcMEM zD&lpYFsl{u`hJMlmo;!a2_gNlo}86DJ|d4};(v<0Qd`1^rL~LQ^MmsbCl_ZYAC5MU zf7m_z@aBij;}3_MXJ@;caOvy%`42eK&HFEcCC}*L7XhJe@dH)Tk|sDzOQ%MahwDo0^^`uSyJ!ml!(D{7cZ8 zqfpq2{7<;W@TE5-8;leesxCG9u8zMwe_^3*8sGiffQ5mDSWI2$c7-GAqV5DYLLXz7 z9OiAw+xX2aJ+wO*-u-(F92JM!gh79w`KjMtil{SlK6|i|yZ@Uk^7;7p8D$K9&bLQD zKi^xYQ3~+7?Z9$bVBKPG6-#yK;_tsl2TB~4TS#CBTT4cb7lFFKSA+z}Ll6L268!Y; z-&kapFC>}Uy@X%?@&5)aFz)a_|KfZ7cNEFp`lN@Hr1AP^FGt55O(nvWz!7Y_>?y7m+Q?ytgS5W#aW z(;_O2YnJKAk~|!fCPNCxI)HzmFHImUP;Z#G9*DvuUpZh;GO~8O)M52x{q!@&y1{qg zYy^Y{M8VS>DlC=(Zxa@@7st&{#Nzwvg zsqY58J_+4`_JGx|NusRjYm#b;Pp0u=<7hmAUc!!Tsm5hMS`wo`;Rxw*&HK%2^12u6qn8DjE!Ej2s(&?Ev?@P6VYV3g<%*U%o(H zf-eGsFGF6x{G=BHnz{-L;W0}R^Il!$ zt$qIMoaj~lwrSI&iOsW(nd5oRBhJxPBJ?T|`Z*;+&-NrXquz#oJ;1M@n`=j)l3yGH ziSHu%wcuy{?ib1ss#!N*t-<%6N@~D2Y9u*TjeHvU0$-~u@0=;TQw{FR#rV$}_kofM zXJExdv*SEI#7TkjvwfBIX1MbicAlaMa4Jpvrq;mwQqTK&VjKH$&sV<$8UC_(yduC4 zk>~j2DC{bqYGsnXr%C$zqPi>LcSU=@4DDUzSU!cc!x=~MG3$Nz%{;s(qCad3B3`1O z(wH2CUR3OOZbRZghVg{JPtcg0v+&#B6d_l&dzDfHcYQxuUw0!XLl*g=eAeB`FD0WI z&Bm+pp_+ZG`UtN$&I7g@#;I2^xw*hZ`nSg?XJ>$oJ>i99$=1t`z;&eaQWCfZCGiL3 zxmSMd^lchncHjBH*gL{LjR$DB?g)|suXhC*UN^du)dLcdk6*XJ752vI6P4$fs3sQ^ z?Uobm!hF2D|0_wb+;Ad*x{(5Hc|-bfKMO&@O+lmpG2$W`!bfq(!j5pBW@&l}a>r6L zRwmE*aP!^4&p&UTogH+9qn>aIGnYkCec8jB*>!pA?iW^i@F2x%>VlgIVHCR{)101c z9iANTf@0o-)ovahp6niXV4RMyb@s#I?gc`lBfQ_-J$$=+e2yRK2rtl~5XA|=XP3mm zEgzMH)rR(ji$3t-KB!d8#+#$`b5m>UWbY*?H z%~e}Kb+UWD{d)T?to592u!xxW7UjQzkHkgT!#xDqPk2oNoEN|#3Pa(=k&lImvJj+M z*!Ec1#Jg!WNUlKo@)L|!6ui6t6*G^EM*?_EdpXw2oF^&oxRBI7SEVhvGlrMY`-*3YB$Jbf~VonnD#*wp7n1 zqHZbLDgk*y0y4e(%iW*h*byu*&qA5*e(8-QA4duO+aJfXk7@X~f5p})`OEXC9%H0b zf;H-4#ejx+aV`fCL)gRugHDD8zK2-x)V=$2l47;{-aT?zKcUMojw1m`2CO@bgf+e} zAzyB<>PR})Oxu!LGAc`t;}IQ1=sPTmg1jS%LxkT5yIFDtKSyqlq*5Y*VCBpZYS<2d z2JkQ2j~MA7pZ$mkFZRyQUykQovOLR9OYY@^=RhtMEB~Zg(oK@k6wO$ZG*eZcM#TIG z+CG77N1)BFu85ih6ki3Jt-6K*07AO&+PWw%nC&>{(Vqv1^JlWukY%py!TR%0DSPmq zseIUagQ-n0U{BcIhW|ywTdr;AZAC~{b(p}P74^Se1JcN3#IgYe>T2-u70G$LIbEZe z{Q$&jMjqgb7d>@e^gU#6&CqHV%HEQ@lH3*L3(;H`m37hTX^Qpp1Ep`N*19bAOx;|i zZ$0<)t(mAfGkxoRTKMx!-&VivfO-8&mtQ$U2IG?gw&&qCe%(W z7~MoC#?y}oU&v1LD5AsfZ*Ccn0bkVKGafSD3=?0)k+0&&pGF+{Y#Yr;L#=yK$TG@v zChh%ZqSGpryuEVp)V+O5O&cCVOiiieKX6RV$heEeGd?1Y^c02X>9OATQ5~LmrcMnR z7t7T7VkYGV2*2D{R@8q*{hu23Uqw_tg^0=-2lAVu{zHUJPQm1-+_gcssG=!Eb0t}} zRZViJrkbYdy1Jz6RGK$vYMxPuzaIM;K(m((uX{9E$2e4JO_O9(U4=%ChejQs@O7Ed z&%#9n!Vu`C2bUq*lA6&`^^5iJr~__dtPL=KfYim^FHQ)vOoZLMa^>vq2MC@xa#=Iw zI7|H?n(xI6;JYO6-;8~xyGazd1p8ah_x}0)9^XFLJ>EXp6n1un!;^F2X!BzC>|pb7 z_gvT&4vzPCj}P#-aI|}|dHBQG{=xR*1Nyv&tsCMzLs^G z7_jp>EHZLXn#3;dQ$IQIL1k%vm<>zjfF1q>aAkH}v^_7O*YDrlXQL7R~|FS62Jh;dZUL4xrHys6d>@KxMer_9g4|lEK`lmVj^0xhKj;bo~A~V z9%pc;03aNLgm-@m{KV$~!0zc$TuzefFV&7GQ~<6luykXN%6w)8aN(aJnz8d~0`f4> zvM%t&vpDA6pMRhEE^{6LLyB?9@GyUZ!Ejh;RIGLUWq9`s+r{!3L^9amyper%(F^D& zy!M~}@;~@8k2Bom2rZqIwXkP+)DeI&5AOcNW^)ZFGrsh4NjbhnLqE##12*RFPqod$ zNxDAciLeehP-#|ntmTkmNQa&3dDK53x9!6iA*L!u@P2&BKLcyB$wxp00+aGDGZOQ+ z$_+?ia)=%9@sGd^N9OFY8Ni?}=$k$oQ%k6m^;7B;Pmy*b&^2GL3ZnGru%G^Go@e`}9ER;No;slZ~-B z-Rscp4U+CKx}otf^7BqX3P@nGqCUn5-X~q{x*^p*_IRn0^Y8>>cP&RY9S`(JUGfZx zNTTXlt|=Q-l`KW{ti@t?pM%^o$Nnx#@9tArzr_DJo0&YH=N~AXx~lV`5LxtenTW)b z2$3{dH$OM3dc?j5#aEat$*u=xkY_WqptwV-}b^Jwb9`>(T=Iz%{NL<1Utu_Y}>*Q&Yp9i$%CFn$BleUD-*$ zke&3YnMg-%ZuceYONqjh-D!Ei!lc$Wvx6Vjo_?xojZX7a)m2ZI`={$}AD*1=?u<-u zBx>l($X-jM@1!G`$JwkTtGQb!%pYqusk7enu&pHovv~$y<;p(WT-h1i^vUPSPQm1- zoGWYSmPid*aYfa!Y}=9)*EMWO)u`lHt}|CP|fBiGL%t+YFLJ;?29+hKAt3=re ziLz&z|GB#rp}F{+gr7UYPf-BqNSb-F^^5g9*^#%KNwS>@-*Q3t|IBKmz=Dn_;IW&B z-G#5)>(BwUw|hJMJyPT>0q`V_F!AnZJe zv0==u5P8@^Fcx@Y!PCxBoCY+XaoRe$3M-N$iAJ|vdK3B*23V$L02IP z(6&n-U`iO6*bm{hgHU0kqmqqugNG#KUs|ud4{!C+3IP&;J0nd{nVB}jttG%mWK&;gm@)L;VSDsdmpKKeNCsA23HHSz>{-q!N8KID9cG^fg!T0F-D(y4Z@vJY5#^J*11x(DfHe7gNy)EJ+uVu`Y}2 zirG^zre^VMF$L4lB;`U<*A*B~?TMPa$`*U>*b6U38WYg3$NEMsvlbcs|oF_l4 zVq6VS)GmM71g+8-R!h(tGlXrl8P*z=(VO3pt8Y|Bu;xcb3}72=Cbd~9V9J~+U@NoK z$}IKN%u)}Z1eK5Qfsf!7UfUUt_<7#?Lgco-pAQ@N{^USWTJmRvz3+lNbN81h7=c9X z@Oz~ws*L*EFqAjm&4a_64$>CLnB%tF;?+eMrH z_CkjG5>nPg)#iS7qDcW#vn^9K;dU+|^`3-lrMj%54OdFUN{Qe~#HcKO-yqblqw7DG zz_w}pVilv>lKnH){J&fR&>TZAmIL%8a^Sv}FtR!c4-u_h^X^Ux$9|dz61!D%A(> z`hK#$?nX|ABH)MeS$8MDl#J><8?VZTYMOWT5yld7k8?w9eif6O3s}I0)|GpEXGcFA zZk`>SoC{rHJ1Pnil3am*`ukn>)ROU?Gqe=4DCsn44sfTj?lo6R|6!C zRQCh~J3w%j$IMF{!T)&mMQ$vygUGpZ`eBt@h=r?ja49ToOKC8yWrubc2;=^aoi$;=4cPwH3>? z79}46#YlX~o`COO0u958KWNxs9>o}Q&#!UG%(UY>S2Rt~0E6FzODRPI6H6w?ry}4} zzSBq3Q~oF`NQv3@73ko7K=9+e3*p5vtYa?=FMT_p!bNli^7zZC5%aR#8XCLXyXc+w z4sf35z}#8Y$Qq)-XJ7nqdi1hAf_^gwspGnG|G?Ajq4A^jk2v?4uZ2$i+Gl@QMBl;w z`Zjjmm}sS>v*O=&mU} zaSeWt#N;nH#{B~LAK)WTEff=%TXLl}#+BBdO88~t@HN6OhmSA3pGZWOHN#L0qT8k+ z>#|EkT{k4zF-=|b77OqHN<3%|_&14P8|``?IRR9HgQD7wZHTVpDV8cbno31mQw+~@ z^s#0n6QMEojhHGZ+kW8Ra8z3cxzd(p)uy_wP{VRXs!>hTbWc~L(QoW#F-($#XzPQ` zWtPSr7!EiCItp4+4WJDvWNznn9D#Wd?cr(!gAVD&0HU|J(AT2gmn^WIbPCa#Qxxn1{=B+ERD& zj>1zm`V8wv`A*8mSxFO*TAFy&hjz8?Urvbp9ujzFSh5yM;9&_jqIw}|>$0`3DE~!- z{~joV2L@E7o~f&=44&tn!7~#zr)BVrSYDrd>dqI}3+KuCS&$Pp0KVuLUpAYlFhJMx ze8!B=8BjZ~t_#o^SGCzNA z$7JLClr)bCI_Ls@3P$Yqs|1f#g2z)#@R)5GA4%|-B%RK**m0rszGlHiYVe-zo?o0D zY+sDk;jLSFo>K_)R0-s11b#}R6!EJ>ePVOG$whcattm^E-km($*9n83OzB6w3t!|o zOmZDeLj2h-i3de|RULuF#j_1`ZbhQ|0<+MWX6sZ>QI-Fu`g;DDBILBGy}GMVh&N$ zC#@T@$skMzRDpQ%^}(+u%gRI*aFcu?Fjps55MC<)XXAkUalt+HQt{;jpzbf zI!1#mWb!sv;Fzy(2{uy9ePnaKaTeKkf5qnoTK8NL_~B+~WJZhk+<8nWad&NkJ>sre z4L~d|BEUBy7&B?fLc*5>hp@6N9U;k_J}4q6oC!O(07%~v-tn={fl0t{FLvIYzw8Jk zrh@H97&0^TIB7klZf`TC+U4XNR6%CL; zxFmK_ZGwR?fX9n5Sf(dj^eGlS0JH}s<}zgNPnbDzui-Cr4+_yW-YZ5ZmxS05+=5aB zw{L|$d|q(#mBl!L4pw#geyx4%m=kGgdHex zPMU%AQ31jq;3`>%R+nXy!rdI`B)9~+l=cVM+%!a!tc^8qKw+dV>}kYzBcwO=C)qXn zX&6L)%4W?#0~kDHb!cLi8Oxx8@W2NeQw~X*#cU;bi*Gx_iWnA&7|{bHhI>9XCru~; zsS2J#_@L&S6%Lqb&c_-*fLjr9OO2aN z9s8{IvA)mHcNh+Z9eBm{;md%5=}m!q2cDr4cZ$7?95eA)qe|$S^4|4hW8Si_fxH5t5{E z_me*W6`y^OodldG-#Vae(M%nSht2&j}J2*3z5>H*;E9x|`s zw!;9&G;sJhB5V*vR|U+y4EMGp1h8!(Oelu|U=%cbUq!9Tid&GtoKe(}=@nLgkZ*!)_euC(y1()fLGmimkYYV{4}FXohZ3 zotmzxDMTUlf$ovdH;*hf2LKnKiL2DSDjVpYsdC?%*nHhgt$Wgyua~KXCqV6@<#@i# zy9JZAX7^cIF^osXFur2S)@tt`+1`IAd0W%1gpaN95~gh_J;hozUV3I}TNCWs?TK0U zHg-Szw5{pl|M#7ARkPRE@~%c~^&4%r|5;`Kt$DMkR)+7D;d^EHUKzf>nBjYsHn=hF zN&U_nV%1&5-#IAe@O{t7G(89}gS%gn5fD{ZA2`pn)$BSk!Sug>2$aB{0TQ8iJ}9Is}|c#4(|vDySux>*=F~UrYVgU3G+a#JwE=>Vjqfn$fk>ZzhXQzFkimtp;mmjCRI4 z+&T2_F9h`x&`4luZ=&Sx7vV6=>-(3_!FBq#KUOyyt?q!s{q0}NLF!+W$G`Xkz4AN4 zYd;}jgwcuhuQaaKKY=gR$5qD1eNC_gmrugL4@c%+yI3C--l&hMPR*wUp3CS+0vHGi zFYn@s`u%)KvIE2?Hh>G9!kIT4H!qMQ02#+RIqB`3ud&C41PB~D{O5aV`}OY8!TH76 zPaWapY=86kV0*sz$S42)Gwf457WaP5%R(o&C;$zo=wF=|k`Y`5g2$~VXcCz}DS3x& zG_$rOw;`|jf|ilrXC(2vu1`P`ifIJw!*_&yr)k333rrZTM=UUeZ)OKL>m_Rho8nat zJ~lSnF04T)jDZKlU;;u5CD-Ua#;42BRGi7N|Ki}_<$wOme?MXO%AdzV%eu#Ms4ZdG z6W&vS->`8CG0iPXZp->fay#JhFdbpY>LB4CDGCLb@G!8>o4$``x)`R6%SxkaKI|8t zsK5zif*~s|!UXP2ge@AW0{RJH>-xzsAS~Vn*T1FU)U z3&QskND2|W&$h_{eb&E3k%40uNK&>R6u~=1oK$Tn+)OmHaR`Ne8sm+gu#?duZizn* zBVRIKVV!iqHpieH!J-mWXo?R2KLq|Ugk8dg6~HML-QnRq^?jcRr6G@?O3a^o zpGrbmi1anh!>(9lQ+>E7gl7Ss9pS}Um;6KLsN4UC&RN&_hnEa${NibOv4i%*GVBN> zQWVq@R`)7jJBY~~kE4N|BO$kt!Wm%he0+A|BGN|qwP4fqhEfjfa(#16QixC#Oa%Lu z1(%VE60=Wq8wZ3PrS9?T{1kOh_>rl>iwK~W0$@=eB|?MH3aFO_>=dD4^h!8*yL+~^ zdH&lUc00lrNw7NEUX)?(aG}wjJ1ASw4*TKxC_p1MNU|Yn%N$Zpd<)nZfaD#MA)F_H zm19Ku0?@(^%LpYvFIeDC3<%g0&U(UWbVCCmHAUD&iY95{&U72lH2ZGQ10;0=+5@S7O%hE>T$3c-5}!=o%{_YCeWN7Z$p65j zE|^(dRBcJ}B!wu7qI;U`8N}9AS&~diBbK&UlJ0YqO@ggq;ngdvi{=j`?1yg2k>AE4xiYktd^>FKd&4|a?Yc(?RrM|p^ z{7lGKR*9u@=qsO|DBF@vMOQT~S#=cIkYv?>zhqss3`rZK@SA7buMggx2s;P+2N#=% z!s+JOrf|3^?3`@x9&aCP3P0hmgX4F*=NAY2o7)E`$1Umf-z1Npr&Sh4Fq=|MAw-c} zM|NymQ%pw^9og13k7%Z>j(&LOWb@6;)nKm&Q}!%f5k*@kI#pcRlx^ZFl4I*s9vyTW zJtx5CW)7@*a0ZCtIjSd70c%_ecdw@YEn~mEE5o0(Nv<@ zw&}Q{r^+VC45RC>*x#E7?OOqp^<&V%VK2A;fA-#N$&D*Z5PYAna7(k!BD(;D?@Le> zJ;7jzqQnqc3`)w(Dw*uU{Q_{(;f{2N2N~>2vsBktlv%f#H2qMe2hB`oTEC>(*qA9^ z`4#yC^9ys%y}rX8a4fL^fvQw60&u^}xo5j)#XA^WJue&5u0cs*j;cDov$P_(4pMJk zUVqzCnOM4!?tRA@OcCy<19ac;0l2>t4}kTFgM-4`*@0y0Vpmfvqic5ISbEuwZ5hN` zi`QHaVy#!@Vb?JPw2d)!dcNIvd%idB`R=GU3@#3G9c}4~Et`@G_4shDN+s@`?bxmGZ8PSAJHXgC zQo(HR9@#ec$o@d@(t*}Un3g8rCR9-O73!9XPrMTq>KH?o<=2vQC`yNl(laz|U!iXI zD%7nIb-PRzyM?H}^;PU<^G9~eDc+{i-i?%7XfB6W6Omjfi&uZ+O8*?r;TXpXCw^_*z*{$u(UnO^Gw>CF(C)us-)#ZZi)(-2gf^&5Gn`sfXrMu(GM52ugzX0H2V=Bipc z_B^swQSWZ@yj!U++{E_QjIUdhH*RG1)fniWROYA?EJ%6_)rs3Ehqx%fe@zl;H~eZe zFma3U8onF}T_r@kt|sf^*!0F?f;>Di zp|vFg9UBKNBRyp7eEd!y<2 zzwkXibN-=XEBWLgn`Dc*b4sC20ix+|?Lu9*f@*KkRH`bK-DYXVJ4l18*d&u|<%CkM z!Kk{_kmcOvx)D}PVO})UY$kKl@)spE-;gc2bZ&Dw(Ot4*Z*DDhn6ja%*Fv#tSVI$T zZV=M#ka~x&;kOv^uvaTvinOJ3{nrk8SZTiX)WY`ZRJYaxuWsUo;!Jh6+QoIB4rXrL zlktp8r%>Q&jVt!j;hp@Wh_VJcBekv->Sv9ev4_A*Km z-78^#$%i+%hjxw1N-~!1X9F$CZYG!WzV{8AXVBiNzF|vR9`+5yY8!}2J>gRty8e)K ze)-{X@0TB*_Z|sf1o1cy@sls%uM6V;^1~DAIn;+wUcocR@adF>zx?oxOFj7P3_kk; z{`$_5>$z4<)fDiK1~3iGaVB2zaR|W}9YX zP1}>}wW(dhnWo%O2q>s5Qg=ftOP z8Dxbo$rR536JOX0VRa__&+M93VR?#q;E%~vR&;fp6>0S=(tu{-x3SwvkRb@&1X-T> zL5yeqTZR+Z@{BT)!7CEd7M)Jns`2!JA}qt`DwZjh4?pk>0Ig!h>nvTbUpnB~6Y38c zEFTHy0s9p(|97Jx?jRw+r})e78rBSVc?B=@dd-?mSzc?+M*W(NUOk9~&vQnTP#5Dj zz-uJL3)zs(u573k@V=Rds%lBbc%7x1^-Bfrc0O`B)pwDTK6ib_Cc2+xf1LAgeoGka zpY*!06Sx*6+=T<3Z6^V`YAR9}eiR`t$_EsEUHF73s&cD7jZJ&8>h}Wlgr$4fnm_M# zk`?dy z?iC~Ijyu9r8jgv7i5)IQ>5(v*{PKgWT3s0kaUD2HVuOeU1U#Pxjsy4OL_B;7O9SBk zJ_f4u6B%@dP2EAG7*Pk9Fs=<4o2~+LlK}xtSy#&Z_f2m~r-%V&Pt<#&qOG;|CiMXH zgD(h*x>FQhh{P1sc80hp65%Erf_zP+eulQll0_gxqbnEKGTeX;Cd9_?O@KW7&sb~C z4KiN>q5p+DW;}w?-BSj82R`_QEgF7#_Uu_5|2Ro#PLiaY@QfM$p=*jL`)jSeK{!ir z$me847@v4Fi#DkmaSp_vC=-0eiVj*9h-0}Z1+9a=E*g?)^w(NzgJ70GGCvC$w>#tX zCcGR0mD(e?TDj5q5V*h?7 zYg-MLB~l>>kD>7?Azb;I6o+EtP$%_|B)FJ1B~n9 zPo;`?q9*HxEUmSt2DL%5m=5H>C2qpRp1q|GNp$7&ct|4h)=hBkHF-;85c2F>H1y=V zd4-$;1vDTV2tn0S;R;gO2no^5Zy1+r-5eXlwhW92;|_)iQaw*a$`?#s=3hMSaWs4} zf$@CTew&;fZtgL0-?EKy<&RRvV-_+CKtXhP=%7tkkIfX})D*hbZW{!>41)f-n_OKI zoP5o89rMFE4+c|4Hkd<~$Q)cTu2qqYir6twMbXGwD{Z>c16cl%tqPY4Kt!tNm--<2 zE!#-8`eWy!j_74ege!t2DP=ZBoH1dVl_jdRlm>sIKcq`_Y-_H}9(kWaeqQo5XMKcv3-;(ietN~)dk6G5G8baB$k>VA~Kja^PmIrJixmWoe z;k+jxQ1Sw4=>09ZgoE)U_MIrbi9N#<+=y2SI~2a8?}Xn5;oA;c#MJdrwLxEyfism+ zuY{#F$6dUI70e$%QYt7u7HD#os@7b^JboKfdvN~{we=3ZL0^!~#dKHQ>~G1KF!77r z{KaSx`-$~r?lvV!G(dHb#6=sml(jb6phV!UvAU7EqXKe-)l3tmR8!D2Wmd@%zl4jC zE^G2y=F|o~K@pcd6y)y6^Ii;xZYqp$`BGO~j|h|o;jVyiFX6#Cc}s?g;>BPK3R^9f zEToV__#mk%aG7c`%=TtlUD>d-8vG&AB{y40QcqOYGF>SRI)WlEeI^A|g7q7s+BGDt zhO?VpNi+>hw$@sAlWksUSodS<1#g*8;zzf8XQycuO-p%xk-fbGe;)`cHd)y-jPph-!( z{=l*x0^2i{?pmHIr9t~pnzWDO3)c%S5|i84v&Y3Q^5F&brc4{;&z!mFbmhnVWi7Mk zPAJ7hB}P=b@I$TcTvSEWA*~0N@etG7%4T1Q9-6PDSP!|yxz2DNFBk{47m$GD#oxkfv8KmWUa;j)CN5Y{dCn!<7;y1 z1!01uXARppe)Yw(uV0)b+Uu7Ae0t%+`IzW^kk?YE)CPr01%+yk&V7cFklgXte9dSy zucBGS0$vM0jC%7b9+bJ>Av7pXs=VCB9tS`X?vUBf@u9ESXK**9$)P=Nk9!zwU?To8 z-Ak;xT>?=NCDmHX5UV!mU22ow)yNNy{ULpqt{xpqKE@@W4}^0E1(je4s0e;kjO3sc zNnOP-ln3}IQq)7mI+SEA?1ir!9v(k?(mnq2*(v^oMZT1t*gJV4Jd(R>T`Se52#dwq zREe<^5TwLoDj-^Dp9Zy22jQ$#&AMoxRy0jt%Ra3(s99>0n$<>iEK2|tIHjelS{HuI zXpNcfB#NLr8H%x<)~GhR!YfCe^o$zRrLJA1e{tukQ7ss4Qz>}iyFwPwHDi;d1z7t0yRRt=6YDA2Z{M| z*q&5nh8i%mPYl-DOoO(84)KL5#_Nb-R_cmhK502XkA^IMU3KiAR+KV+qg8{r2C7!#-aJBDnO!$U|exg1nUP|U;LIj2ddJKbABu>0Fs|x5# zQq@@*u}cA{QIaYO=aeQ5U3iTBdfRHyI`H29ij2ltpbQG=FWqS5`T>_>6%L~?pQLQ! zoopt=2Y8I4T8gz+l#w4DKxzY9OCDw}{?J^X(wre63YI{M@w zOBBEagCqs!Aa6%_ly)#I^(@TNX#N=k&3N0%U;W{K`>(%x69xXS4uADu|M^$E;~{U_ z>GH;w)h2j-rLQ?U{NtE?S?m)X-mqj{q^7PJykT|T2ah)nt!bs#W|Xv}A00bsLqB*| zF-21y5SGM7WKmbB;wUyz#J(oVhTZ2)Ncr|&r(Hiw73;9Q=S{}w)x*jlpQ3~Yu%SKBFdKd?d>$WtG?$T|mCKa)r}4yRW?+cAhMY{ldT&ehLY z>o|z`GP12?og-KS-KLRG!mFR*Z6Z)?Rz<@}2}!9ZV1aTB^8tq7QTRGodd!1 z@Wv1be0Jdgm_o>*htX3B2n*21yr-l=tXS+7oM!VP4P*@*cmyNKS~q-6BG}PC2&ZxM z7XGY!%}*g~)bJ_x$Fi|Z(mC<%Q6Nwce(S@U**72DfCvvhJwJcQsespC;tNQ-$yYi% z@CPgH5IMTpfs>p9OQVCwJkqD${6@wJa;((@`2`66Q1IL-)}M+&?DB86rz5hW83(c< zVo2^C>xkUY*H~Pl#r3y1j_3MsqvWiMLL;OeL#>SSa-Q`n;bNWqi4)BKV~@De83poo zp85F$@K>>MJ92F7`|xh?9jx{6km#xf;+@)*3=~7sscxvU)t3i-xG34WQXJ^j98rK< zS(O2_0cHCJ)CH7bJaC!kb2-r{8}5`$O9?(&%q-5~8o&yV!+>~SzzmYfCC~GPsgU1; z*b74J29W%jOM=S-V33ZRf6X;mn0(K8?rtmgDrRqhWp(xv>CPg?MkLa=fB04BSMSFj z++sfYFYvFx+IsS+@}Bt5zxpqD=Si+S>?)KbR`agwPFQK%&Wm$*^Lho?8yyue)=eE7 z$N%%MR&Dv7-pxwQ(E3;*`s zM^0vO_++L!0E5G=6`LsZwqN=>|H(n&?b1*y-tj8~-d@?=P4*~kt$pUjad%_mr*MY_ zQh?JeY^`!5z44+h<;L?^_$%$VkXTF#edoEg>#%ulD;;t*0HnzN&Z2=|s#ca2znC)DCzxBmz=P=667PGzKm^51=L^lk> z(N#w>)PXJy6pdoFMoT1CZA=fHFi z&n}%X_{$ILRi%I&=){nc_r_K>ZO) z0te>d;mB-w;zU8nmdWD8ZDc2)m<}-xKJPWzVe^8L6~N>fbS~gtG^OK7dNIOhVDKoy zKLERiu)XPsz&PYB#f*S4?s4og(An^`>o5>Z1B~;BfxO@n)(6uU?82-%Z1se&ZgDgL z?U&^abXc|lrUii32cYg#)b%_#k|1OSXIZ+!J65RZ9p!+YhAu1|_5o0H$Aqnm&c!RfvUUi? z>pKDHQ02py9d|I`8!BRN<;?gli7(Li02H6&2FT374Pns{$GKz{SbTN<1Te|b<95mb z3v4@{076~D)p+bq1vqA07DFuI!RyZCA?AWOELPxkev(zy!{l z*dxYFUiXAEtg?)g`EMik=s8T_MkC=2L}*%P>ztR;>HwBvotKPZeC4|=sAh`QSCP!o zqVahQ%iz{rP$xi_5{7UJQ{ma?CoHE^R6tu&yU#eZ2QFT#5IUIE1J|hpiG|<9z;|$* zrvykAbdm>8U!4f2PgF(1Bz##sND_pqAB*}!JcA*^73CcO%YN$z?>y=Zse?NfUiP}L zN}I0C{me)2f*VCF$w+Yi_Q!uahWj5J?x(1ojgV$w{PCCy501Zl)s^(_vnQv&>7wTU zP>4ap2-!FTc!nZHrvsnL=iRSfzC0F2aUbiFP0;ksj^g|o*A(kR$mLQ1Y1*rXiyXM2 z!o?sI9)CS2z&z1vg;(hf6)xUM_ZxKJ0r>fer6&r?6Mljv0bMc-m~I%L8Z)KU`14k; z2oJtGKY5r2n)w`OxsHcU%4`JMJOuJ)5LZ)zXTrYw^y=pbJ*dZwsXKgK>kbCLzD#`- zBx(~1pWLjN)fVOk@U215FIJs{doPr{f;|95xmrv9np3MeaGp`n%Sp@tk{-Bq=7+gO{c5H z>)(I|-VA=T9gD6x`Z+DjE7*{hxEo{Fgx~jY;+`Z~s%f;%5-6b4TF-CEnm3dQk+;mV zHO+E}+$WZ9(7q{)n8&GDMAuB%m~B@1LL1>*lDu=|R+e%~-1U%E_Q&!?rOMcf*PcCCDDftWLE#oS~z!DACxw5BR_clbsPT5UjIx=Z_8 zU8f=*yT?yyNoBbO)Ltfi=?-A`UeRB;^sPj#{cUo)knEOkllLOmZoZ>;rCd9O<=UyQ zCGAkQ4i)9^u~g$axpr(oNqH@)IH0QZG*R2<+U-`Z-TW|x=ES&fXwLv3C&9NwRrc#Sj@qm%yR%(Mqj+#SI%2Du{!{}&Jz9JaEFzROK zD7%oVGu1>G%JU)&3lSej=g%PY-NYjQEB&W1_>zZ`iqHNo2S%9qrMQtGLJ%>p(!uOtoBMV(#P%z?|KlHAC ziYkD}KV7e0L>2{cREU6p+jsWyY)OeFg30C;mDp&wTVbL*ln{82t4AF9AheSyD&H(w zR-#Nq+h1W~$pEWzwpW&n-rCI2(o%4HhPInBiDjT`WzX2(+3)wAedfYdZxJ#V)fl+b z#Thmm-H|&^mWelMNa&qwZfh*yfWFWY3^NDz4wh_4%t=LauxI$btAat@s?WP*2-POEE}+%l22 zi632)T-#5v8sTmQb=*>kioHbJ6tP^FK)WG%w*65Vw-#Y@hip5wFt{rUb?xHv#DKa3 zSe#~XP`pI9iKS>^TU=IPY-{w&UIuN84BA#9vw=!w+e%#bz4+N&s@^r-WBZG_H52#P`@t%k89kZfk$ZDzxokfBYru1;Y=Vxga|2P>|%To1s@>m$eys6*69w zp$Cr);d{S?T~AzKNWPO;^ow6bPpc0{&l(>8!#zm-Tc8Q7l=`Q3MZGJ@uSNAxRSreH zry1skGXE6uP*!@9YVI@tb}#dOqf z;xH}V&F7zaxz}duF}xnp#gpz_I_UOXR(3kKw2=IlY<59DV6NI-Q!B`=3ukY7?N!%Fg5vB@&L)F*G?wvSDg z|LO6IQ^D{3@>f9-HNFU``l+OH)9LxQCm~Fll(zBc(2YF44otXWu!AqW zWU2rb#j|T8I*dcMmMFji4)Z!(NP2UJ3j|RB7r)tbU2Glc!?GTnMiX-NC-Sh_XZiHi z>5~`dE2IP`b1Z0jk*(yb^W*0=uFFbC`Zn&!<7rmUqx)sxpMdH^Lxdyz1z>kCB0=ei zuwWE^)TuZlmwtP>o)T*fztuHBD?(vAU5r(>-cSX!M3r~5rvyuPemD3I^tHo7st;6E z(@fK#ifUVqZCH*f+C&$r=E$P8T2F~>3T2_=fN&qr*d8GF4Sk2Lo!UilOTLLEjwoCC3z1(bCCG z3x(q_5}tvM9U+$iu9%s#aSj@Vb) z)j8h``L>f;>z(LcY27QW_px;S0t?ukD_yUMhoae&bafxVwp#&gOB-jK!`K#BY-@e{T__G%HLV5Q(>-$j~D}Sjh+HCe=D#{lYM&F!~ zdS7ezC;`_!53YJn-Ytlz;^wfxtH7Y}@y6<>E@7OjjkwyF8)P;RtF2FDHt?$A6WPaB z?dz=Gu~?_3I;%@M9m)Y2%hn~?Mm^$AL@_PUANDm#7S<#wTst;fl%z87YNfC04m>dy z-lD>|6?N*G4aY)K7jrqYbQAr}JB~g573ps!`~2D^8&{fdHEON3KXL1Sf^$py5gNmE zQx8*JjkCqxs-4UQ+g9t!z8c_Wu7RnQWm%id&e=_umCp6|yo#IuT152vT&z(bE8<=t zw-U(LqY<|n`L+u6ZR2h8ekbcJxaRI03*Y{1ojIhw&FMN0AR@>kSye?xwJ5a=)3lAg zs8Y+3M73qQPODLfdl-g@wpnQ5VfFpnz3g2!FD9A*F$fi;!m*@ig#1T>a=>V4Z2a%KX?7 zJ`bX+p9<@cx|S--UBl2S0D%YT$M2ihfn$@8-)Do9GDwu!)tm$&sNPta;~eYxz#S=B z&g?NsT|-uku4W;k*Q{#MY4NGB*LXHpadGP0OsUV{ULZ{3@uX#)#TOGSDe@u5(1mY& z`t1DK$+2+!@;UtT%j0w5x5C+r*Dqe3o(r!}U!6TWK~1~O z0AFHVJl0Ta0Gy0s-`bY2c>ONu0s|jX0fgX`j+ZDqc?=8_1ZTwBQ3VBw98`vJBr4&J-KaqLUm%ewKxF#fRASgR=u?8|S3XFlExBi4*-3Ufx& zw0$x4FWfNjlWt=m;z8o$;xaI?)lR8B^6_$e$yy_WKw({pgLfpvYi#5WSud_QY4GKa zvFq6shXYL_ZD34&8o~f{FmPGZVcr2bjwsFtthEa>!d=O`ES-_?&Yg|~FL)(GuTrBaSUDw@rid>6o&zi5yK*g3h@NrGDYE+ z|N8{zP8x*?(KL2k${P$1K;{JRSPL{94b3bAAY~4@=&*3OY$vY&h6mRP57^%b1Nh-Sj2`m-Z31>)^Jp@>?S^*j!j&M1Cm!XaVt;EA zzX$e9!prmt?0S~8yk?+aNC&8hm4nh|(;03?Q^1^YRKf}P7$CvgusPJpyMQO)%v<0k zD4#ASq(eX$c2*}ZUNRJf{YGgE<>K~>4Z?AFY@3GwfmNn0>3YJasE~MdQ}xA1t0co(FsMm87Vo?i3ihu@^Y9avH$n zh?we!q9N&4Laj`9Q7TrDJCCM9gCktoLLHxn< z2PMJfE!of9-=nZhG?;Fp!EoqK=k*?cIdbs65O^evM(!kQ9hf&f$E^lfN!3nWOkWd^ zsd~Lt^I-Ev7tLv|uiW@c(285fcVz3l`l-0ktDkVC<($_RNv-g`Bx{B&YW#OyF;M?q zoa@S+V5J1sk`rvXgHtaKGskQ#z_$x7;_J!S(=|w4P}8L`k`GQUAnal4D9s z>71o3fH{r$Dv1H}cV5n=TV_;6RICF{lqA%>MZ*-gK3TTZAJ~y4NhN)_WE++)O0sQh zMA2kh6is=c4(MvhvNgJDbj`YI<8er0J5E}YgK|CC*O1v#Anlud{>())Trijdw*>sd zj<-)kG8xU;uViYnp@G`64rN^rz3$Ipe-1l%4jW4C&b@~Uk;Q>76OjxgLL^Pr4Y5!9 zimi+CfF_khilUwAlnpu7w~{fwY9Y;_<}0^1rWhG>Wb%_0WT z6jB|y34In#WD?s{Oi?yuS+``#u=}D#EZMY8n~>DdSD5E3e?l(5fjetMf_hRJuPp0c zKRM$@;%2@1#AA)-H=sF}I{+&KqN0~x_x`;Q0KByC{=Pb9rWEP zw`LVDWe3W^nD{Zbntlh$a>(%NyCj?VyEFGc$)X*|@bhb>msNmT1_9UM5;uT=EmHG~ z_?2998A3&X5L0II;gvlv0-N*?p7|#%$8{<6+gQbo)rDl6iW@g_ncvK2+9&82)v9lA z_>#X83tYN@M4R$1S$~|n%FQsHJYudB>bS_JdUh~ANIuCzfhVqea4t$J=sThTOMuhW ze^bKN>*0!x6L9)Dm~r$kCBrwV(bba19lk+qtQO9WTU6io_D0pMG^*}ZjC-YGT(@QD zrd612vf^z>-MSy;mli}cu2gy%%D0i~`izn0`f#XMBS zo@MOIFYSE!rG=d4Mo4vAiOs9g>EEqxVa$EOrU9@9#Bn4@X>N1 z$i*bI${OT+4KoqJg*8;7fsd95JDxMSl21Mp23(lDHU#)+xuD}Yt59cFG4C{LLw=7| zi#gt0!KQsY={}xxA5XfEC*8-Bt{YFf`I2=>DteX%(wwcHKe9<(E6+38Y!SNK-d8an zX18`$$=lhj%^_pltsTwDgW9c)*5zyM)(&U#$`(|ur2eq7w^Ps)+-Ov@x0UNv<^i_z z8vxJg>$}lRhGmLRrfq?FB16p=EURH(t@4glt2|h?DCJBMgl()uxnx9txz<4a1)f>& zYcKHiHq6A4Q&*>Eu~X<^Zx`6x1*#XAT0AR+v(*^G7YD~3tWf>w1QHheh zmf}xs1AI6nmJKFqE+f-S`**uIRCNuL>!=Ks#1hh0MPJ8ORQ0`nsd2ZZz)h71Oj&**N1{m%Tt|@Xno{HXmb99T z*$UFyz%6H|6t0^lwf3=`O_bOiAeqq9&FgQAmNb{1?vk3A%E{Uoi{^U5u&@dm%?VIk zO9xo6^VMnt-wq?XgC$1ytZGpwjrqK$ZN|ZdyKr0+w^?pC8*C{^}}!RGN3Az z{bo6FJ52U#rqN#b?z8Hl4$IDq2$vYDr65?8x2n zs^SHZmTf{!QOX7IY@o>g&4h06tKv7StXX1L?oPdhi^(SAKx&gKKRKw`=+16`o~~IB zhpq1=Fh@3Qb1MO=>y|=ojVc4VZx0+rqy>@g6|;;TOeA3DMla^X7V8uj@{ z#gg>dd;Z6N4L+2+2h0{jfBv^W{@d}J*q;&>(+#_O3=*4qaLo~%m_>YhL}1y9F7_xp z6woOa#*=A`oyV}H<2d%=s|f*ZW{iOzNn0~e2V)ywfrrL{9bWx_ZnpY*=%=2L#mI97 zR+wJ>gh3Ai5%3np#uo;_H3j+}R1p#iK==Z6-v?3tK$HZrO=;x*@`DrLfx&CPkEwtR z4~7E9JiYpP8hFCr{`lYjF($CsIRLo8{*Os@WlXc-AE0`r1Q zqmDoTY!T*-&_3}z7E$9)KagL3m;?9;DtXhi3FyqS5i!Vwi9R3!;t||{!_f#pOT(+5 zLE}PH=$4_1N;RE;iX8&kod@>S&m_X|{|JY~Gyt}=WP6B9?2v{a`$8Xqn~zmEgJfnL z0ys|$(`Nuepd~K>p?3rxoQSafK&pyM%4JUD3Gm7NT)>B(Y2z)+n zj<7g*j^gYnCJwnkN{0y@7#FAFp;b2NJMaXjX$AzG0wL-FRm4Jg^&dlbT%0lqluD_X zxMLQb7161HK~5Mm#T@^Wz{RB`2^$%yVvY~I4#fG*2xiUKd4eg=CqWGO=CM2xki!v= z2phOzl#(IAGq&+rJ9(B2FzSm_AV3~ZoJ>bEwD@`y2Jw*RmEhrArc1;7i+e%YQtOw~ z$!LzpaG_?xV-PaJV?hY);sE#IOM&ABdxVo429Bi(=23?QxA6l=;@F}lzyZOzcDB|N zPE;w8vF(;unq|V|G{!&%bGdC@gTZycq4=>CvcPz{8Yp-}%7;j3%AI2P z9>9`h6oq?Y6va-8om~N+67s_DI>I}+vL{0q@QhDTpev5+sGITCv4k~bD;VYH`WQ|U zf=-c!VrHcPQti9$2x`GJ&Q zVeH{Y*hj!bWLyxDF{n_qt&sEt&N+j$4JxP3fGZ-6>jWuD3R05fQWAiXcA_pxNxw!+ zIzc8V1AL82B%U;_K(>~6M+A19%gGPf z{?g<6<%eN_nyg37=DxtOCAYd27;sr4`5m1!$*Rw zm|a=t74^~);69X$aFUGxqUf{Y7`;V=rUL+fN~D#pz!?H8-N$3!uw%*ca?E<4Txfc6zek<;08~@@zBd_7 z4kAR6-~1+Xb)dd=083KzJ6A)lp{P@dqVHyJ3zh}*Zt$Hb%7=%#O69(#i;klYRN2(1 zu4{^BsJfzSx@p*}^|rVj(?g3B*(i_QK&|;(DGJYPf29VzKBbyMh$1<*Z1?+`V%m~u z%Y9uN5Y3d;>cmf89DlyJ#J#Q=l4?-ZB+@{Ub;alpsA8x}-=>Cc>odCnmMwG>o=pop z8!Z(z`HN+yU0d6I-8$TX=WMVHH&ry-Bx-*k5tGotz;qN-8`uC7jY{x;Re)r|^3{y6 z{Qv&n{|Ch-{1am5sk#cDH^pIZ_LLXVxGcBfy2})By)N8v0`ev~f#~oM^8=i|B#O#_ znzm!v8nIMaq?RemL?tuz@>ASstm{wSu&AQ+j8)UJ2BthPY*AMYMK%dcMJQ0b>==$A zwmSLEFpZ7K&0R0SyVsHKTj9$E{oyq_7JHCD5ft`34_AZ`UAq zI?u+>QID2!4*-ZJbq+u>i+n*1@|8@UWD9DtCh--8+Mm_file$9jRi$LQ^jUsQO|u) z#6Bou9~7|S<^dI%4=Vx^qy5IT{1SKoW`^jWjWeAw(Iik7rW6LavW0+@xClaV}3rS zD4`BR@2|t1N0z*y5D<$Tw;@DiToX6@ju9iT)tYPq)G?8tzU(ZcO zzda{Pz3Fy;A-%5`(wQ>MiBn*a%b?c7%R^(huv<3BW|cxaw<`6sjV>7gHCL+di*2>6 zy0dg-*xF^6SDJ6jDqY>7`{Gvcf?KL`vgZ{`@roPP)7P7FlN@8M80A`wT^J>Ou&EjeHTl0!-ZC$AdZ4b+Zj)%Bit2p5W+rNzV#Bt8%7>{L&9Ys0yq1!tDhY7IJMo` z2)0I&F2Qs+~UHycno=Eum8GQA%Kzb+j6^~w&3`FiQ5P-=Nfh3=jP#`r+X11ev zEDTu0B6Df|B}99TO(x_6?tu)k5tq;3Uu{FM`e7JP3i!u@1;+#($|EusFw^tu->?1` zhy(#~HFkgmVfVOx7zn=lpMl^rb4#+f2pfa2Ko244!ygxXeJnC48A_n3VngEcssbRgZNdwWO!8HIy{NiijF#{i(mE#a@ ziS26<6w3$X?&U=09STp}$<<##|4sDa9LAA7lKZ_~OHThW^soXS2l< zOX|%orkLG+8oo{Bfv6#TRJa(Lvg61EoxCdiMipCGEe-z`MPqMXO1`KxrsSW&Li(=f zl0$(a7Qo_wEx*Z&vZJW_eTG_Q=0*I8ekOWxR1;L-7PQrfWQrj^ZA^TsS`;v`m$xo)NlxO6x}dv zYEo-JRdFzrSPPu^&&Y|>!n1IqPZCE;X0AAXyf|V1dT#`)Cg-ooK(GBQXL%>)0)pHQ z@M2iFPiJ^tzER7jqAONwu6hIeX+yGiCqMf}2&XQ`-f?;{>9a%T*Y*QRl69qBF8vLz ze7lZwukhT2!n5lKx)}n!sip4=AS||zu2%rTJQVdqRqV<7n)MHmYg(^CUzaT4+C9UN z_w^5UtNy_}(|&WQgQe~LJ6Yu57H^wdka1pcSQN6(6KBOy)!DJBMDx8ZdT)!qqqgX3 z5M|<^D^P6a^(hf;{%J$(ROabx4xGv#*(3v;aV36okW91_+naHD?$-8ZT(i5iy;>LY zZf&vZ>fWv0m0a?>wY91OvRk`r3UMgz^5*lOXL5&^aqiB^-@PqQZ(&JyC?I>Ez?SH0Am$nr_E{pv zg1+qcHtVC&ZL)4A*!*!lJsln#&Fjlv za<8sK5sSv%=albsPWcVoto9=KK3y*M`Pln>?3>NUUQ&o{b<%a83{*H=O0k}wG#H46 zV(a~ZHBjw=WYi{I*Th)086|B(C47GAA2?*`-Jp~N8K>MBCA(_q8~cK8S!Jxm@6h`7d|UyWf$I#KYs6j{8##+oTJ^D znJ*aAh+O@N2rpfj4qk~{6}Xalo_t!I5nm@Czfa#UPNvjskB~9!F2L&5)2p9-geR7>7EVa$C3)f5!__a! zkcEM;X$c$21>lxJtIkb-R}@hbUQ^pgrSIxzlITh&#iG&eGPwa^^`4}lb>E5YDX(UI zY*RS$mn0-JqY4lBm#4hed>j(*VcVHRt*6MsGk=H$j{foH%hZ7K%7m5>AOI%Q$wCWLMi~x z^XmR=$gTBlRps1(9;Otwh`tJ*%Ps7H2idqK=FL8Sz9WfKb6pf?+ zDgc^0xGWe|iI;#WTO1xLTI93xUK!~Ld0vnVibM1%+a-1QghUvicbp0jqSzjxjrduY zJ^OHW0k{{|(js)>w*X7S^HbsL;8U=k3~Wl?#=?)$(u|-0_`+FFP;huy2Ywt}FyQls z4{*){K)R59lpk_Wc!GLGFk$B1tltA~eFX=Uo*eR*wu6=q7GMQ%g%id2O4e5b5r}ng z=%a<(CLy4d!#bKI8%@ch5BPB5qWQX!KCnFFgkouYSlyJoWoM0*7LjgZp8?kmdAE-t z1-3m!+q^v@XxpZY+uATldN!BkgA67C zQrdJxru?L$jPQ#D&yGDF_bJ<+eTn4?8CZ)T@Z7P>I%?P@>OxP$Aoeq&UAS)II4*Cw zaX|pP;wYP8N~45c0pF5VA`EkZbeQTfO-94?QQ)9-05tg&Y0SIqB&|*oKgP%f5=*@yRsJ^vt`f%NfJnfWBV8Wr z0?=WM2r$|9VkbK{bc>)@Gs?P(6yeMKG|5uXa}u%k3{%cTP+*i_EuCghc#(15X$Tb3 zK@U!K26#WrCrH=gbn3AKKjgeMeUcwL$S`0cUT0Q}WT%oGUkdif<@$YjR2$&Y$FD*9 zCYzj0n5UgEW)t}6)pA3p$lDXOfIputIUo+K?aXDJoj`D141QwR?HM~_WVo~>m@Zc-_Cn+?+lYjUc zDh&qql_)llm?=r3r5q@-&g*Oy^N+^P#O(oZJCH0(?@E@c9!TnetR5JKyrZ3wZ>4NS z;C{!0oY)=Q6lLA$i>j^@&9-gFwoO~r4cd2z?hK^jKueUZ$VK*9Nhkj_VNk~f5q-{d zj?Xilgd?fm5^lI+d%V zpI~SuF#kmZo5VI1V8DhfgBB$jc3-rJC7Zy439+mhczoqg$mKVnzb}L|b%-ev#nJk3 zz3ZDh3F-XDpsGW)KdDQsA#eyQrqmSKxV*}$V@4Yf(BY&g&=TQ=5`%}TUsMo2ah%{ zu)b|6LQ7iAWVR5+eMF0e`f7e*SO!DZVhGoRAsehjvVBJlCr#NUcrvz&Z4jb^qPntl zy`Z(ZbNyLgtBQr2Bpz#CG^M30#SL<#_SJFLV@eyVX|oz$RfJH`9f zajs4Z-jRxFo5Zg9$aT}`vPGwU)9TL)OivIz7c&Q#ABcZ*n>zCQ-bU5r>%Sgv6@u-q zdbK%OwP9<(9l(!`6at)y_}RM=?wA{4Wxc2gdE|W+$$Nk{8}UBfi?WpSLrm8yODTzm zqH?I}J z*{(rkQTp5MRu(N(j7k;GasjAn>CrWV^yd^KofolRc#Ydejw%uRpByBg->GsO`--CX ztfJ_W0r%x10kyWPYCuq3O_aq!MvTQI3 z_T=8KQS0d1a~^U_)K(PqDMWPCq37ZNk>+|K%zeHu+0$6ECkaV8V&N#C9Hf7iYwk3< z#*$h-rH2=K$kr?Xv|urF3jnpBrR6SGt29>Tq?55*RnZDTmUoYprjed0BTbE&+&(O1 zmg9XD;(blQYp*GId%S2jSkYDxW+5<)Q{J+1W^?s}J>|`zylPe5g*gsQb#q(1xDF|{ z9DYwED-y|8D$&(7Om0nsZVq~0w;bkzfj3wRw~#?rqyH?7I6m`!<|flAf?S20D$LC-5Xr&~=qez4_ z%A`m;)tjYVf3Ym){oTC_W>IMKE9PDRFD-x-vkHLvWS{_@Kw`fg)L9!Pa$Spvg{j%K z3hlyNrgg>o;;}a<*(*sc3HC0TC(|e|H)7qr>&}NZ9*2y7C$Wb@7L?ebU3@=@xqr**4goZ^4Y*+%Ip* z!>$$G%ce`$U5FQ_Xy0mUi>)C4y#f5z$^Rx?+jq*-Qk(SN3*j zb>EL_-;e6H`%%p!n=NifC5bXI%)T?w4bhVAf#VP|pemuZEs9#7v};GzZj?0l8WXHy z-*Kmac87294!`)0F)RAP_v1Lq5{G-Z5HclWs4?ccBDK~+tX+ZUgTw=OR zI?ZQ6IHi8%R=Qo4KQ2xO@54NP-E8XYY>xxa2|B{5N2~p`@{fw6v*(LrYIECIonuDX zn6TuaXFki@;MqV=v2pv=j|0G|cFm=p=AGV?ftXM3{1PW)wJD9H3!S#)Z$=kx_#vtKIoo?F{acW`H6==b%3Ax7wGIxa$L^>pCu&u!vqPk z(ERL}1B}amh1)~J2l0~p)ijUt1WTx#1Tip04?jC2PMGkenZ{tx2`&Q%A(;3Ci~4=c zob~{@OPF^6QXuB+HKM`@=knoe4;J|W#9|~&f-qkgJT#f~ga>_|@eE(vB#wxWjp}(q z6-8k&rVeo!26^mFOcQ=6T(YUzEWVKDq}ap>FvK8$iKc8Fn7TZGB_(5`^s!;-$Qt0e z7=Q=E!9rjQ9;R6bVaT5ZZ3u)a4N3C!rGPGM*eXmF!8G_7y&$k50Ov)3nWGGF!Z~1v z)8VTK!Xfe9Xv`Kh0Raf>fK3SzJ|j%xkKZ$S(h)!rA`CdNOFJGsMloFCvwhea%eO|j6kn;H6K>UZCk!5psE`)%w@Day?x46&$BibeSQ7r}(3$hX zlQ_PN0sZm``QDRVqjAk6A?EB(0MXcL;Th`S3AsUOF+H9qIi_y_kRgLHtjLCE`QU7| zEFU@WU~Lzm=KxNzl-%U(LrOfvt^jawLf8=rMnfMSk>biqrO1;z10HqByEY#7=1z zLl;xzVo$-5fC>nN@jRt=JA=iLi3C4A)rjv8Ly)*2!e??4$weR^xww@Q4=95;5`ZsF zK^QQk8&OaY87-0`DYA(Hb%0Di5XQDL98L#@DaR9F?Ll&C5oe6-&0*{!wFA@=KT*2L ze94#mNmh~}Wc)2;cpU;vCxBucR2MiWr^EnmctGFwsLK3$Oqd<#pML!J;N!pIc@o$! zP|WX{fZ(*(PFlgSX7UPT0iG~`CxFF#{CE5%6;=2g2rJ4^3?wrd7x9E~tAen?)dVF` z1*2iHo>Ww^X@T>x4;#c)@(S>R`QqxhR$##l!*t3_V2xRz9-AEJiU=SGijG`*vh5ini!yJ-WLZI!RV5j~ zG@;Xv|BZ75jvE8&;)AIm8_)6&{yh7Ri8B6oCd{ZH6=fM>9g|=HJJ4f;baf`KfpCV=u7rL<)F|4*=r=Oo?4As*w;?t@Mhy_)4cw1=u9OU`rna z*{4B>p_pA@tns)g$;xsp;Uz^mmK=Ujkim?}a&sXTB(%>27(#^>fq?a9O5&>q8QI}@ z7$%ycWGc8gaRRR8Too$}q0~wf{Zpy+Y2X6qB>f;{GHW~|v&L-ssl4JChk65^D>qFL ztkhEySpaxgVuEso6{lCJM>z)yVgxb4f+(a$R;9R!do51FBvw$CSU~^CQWAlc(_m3z z;T*xNs7g?}R#>HwFVcPit^zQnlC&ByX;l$cv?{HFlCZ*Ib6Mq+0;Y#-MWIlXQ_0@) z<{oeSzlu^S z<%Z0R7X9sy|Ay=X%key9F}!&JqAb1N5lBMeQ-l+@okmG5FD8yIxr}1>RbFEWD46AU zzk~Y&8#fm@G!;gy@KagzB=xN_kt1+B8|WTDVk3cD+4y`JuZEFWSU4*z=CZ;-JjB`a zgrrmk-aA4NODP=tmz}IG6CN2Yg*6X4Nm-~&%c~p%K#~~=00No#;d_TMvr7}Zx>pZm z?o~V8eiB0k`xt=|gYoEz?7{m*cCELW$cK+vDt*4ze(ZYq5ihS4guv%8036U7!vXWM zPN2KtddIolfD0)EW(OHil)#bk0$60Kx6-uuR?;{NzeABTX5&d6CU=UJtU~h6@1&dI zHLQU6QM+LSF^0-es=W z!M8wTv@kYk0B`A(V`voj-x0XGO!)$ki7^oa?6Ft{EL%w?T4*kIQj3k3RDms!tV8LT zNAokt;0U-r;3Zu6U|cg>Mgk4Q5S*${@n~$u@jr8N1Fl*A7nJGTT)@@tG|xx2WLhKP#aK?^GU9LoVutV%}jv1_ea_8ul|ZfzG~*`{qonQ zsi<)E17_!7GQ-usU;QtXTw_Sz0l5(dLh}Jt`be+v{vR@Y z_AWmKSg=~~%%Rv>rVaIqGjChF01N;L-bB3&aaa_H3t)TSn~Ww0VMNj#m`@o;5uFhu!ZtuqcLLrG5iySJYX1 z0QFE+4kf)OvgnkR+YKlzVpw@CsfUVqC`vs;SNH7(cD&udV$59FZeY2Pm&$-+&_h{u zY1~|7SiqxV4B?`NDN4(RDSKbQ-WRa<1?+tRdtbob7qBs3 zK($I!sYBO3NNOJ>wGWco2TARNr1n8lcQZ(8p7`45r|k1n_W3FM{FHru%0543i}F)8 z-w84q;X6LUAg#u-psIA6x30|0K+-g4TFSX_D5XFB;4ziFOxYhKI`^6w=G*EbETx8-720b!KwHAWFRU;5q0?8qS63RKok1|&9+(=&uldc zW)e6~FDCsSNV-Ew)8X0GYhwZz-qq5WB#cHK;TzZUK>U@*ry# zZNROWk{49X~E7P=4jREU#-y zrowSNz525saHIBP0yB64PPl<|N?r@Olp$HFgT>ov9oLrHKfdax1C1Vww$X+F6*U6taL0Q8daicRD4yp6Z&mVxl z3T|z`ztKLnpF?Js;IUUmVz~DH*akP>KV+n!~$rD~rjm zHfvL|Os$l)IhUEVVS;&c8OhWMTp#{>U(xE0mYluOeGQ{~q5*-zkY#>_|FHa>wrtc)>AsmC zY(=BR*0+tiF06~MHnj%w{C=|UHMg)q@)9@mrZ&orx1V);w$C4TwXTi#O9iU$PJjJ0 zd27n`vdrKP=@47N&u_AW@cm}c+sa}0y*BS)5W!aU&Rh~Rx7q(Mi>fh@FpSUjuz*7t2q&;t_e}t^ zknz=D{BkS+e1?frv#|i>muI5_n%Xda3ff`NvK`=A9FjR55@&-Cp7S(4KfrXmi1cZq z=#BY6_!pnQCwv+vA)}!FV)k18T}K#R{p3?jbGsDaZV&}LWr5l*CbE)bFelBsop|H} z>DBk@OEu0kYq1Y+2Ev1{o}BbfU!Na8>DugYeU=o+DpS!n1P|hp5gV@(0{0X4XO!%RXV1F+)lZJgOD+_$2r)0x zXSubp<6@>O%fVzxr5Ig3!NM?zSP|Bb8pF;3A{Q)|7xP$?Vh4C=SAPjz+z^}v4Ei3k zOi6V0Gf${w6=v+1hWr3A6*MFt_;GqX6%~&LPz0a}ht3WN3-us`gfVpxIQUqe2y3&X z-%EjB1YXQiC40bN5c%xsq41j3K|)lzOv?$ucCm7S!^-+#k)RZfvT|6M2rTp|U>mE6 zgfFE9h|)nwIEEoR_%sm)F7+H%5eXp!Bfs^7cOG?y2w^<<34F@x2Td_+HJYZ#bD3jQ zUU-Y64#Of=xd+}E#Y3854Gak4IhY{Kdu)RP`cA++cEKT+S>?`mtUy^Z0F1^93gw$m z(B?g!z#E1&3_`ecEb9el!+!-V!moJ|tJr7w!HOg0wSXeNj08tG(qIv_C@cm4gXKFcI!> z%3g|qwF582AEV?WR+efQ2JiT$5_ryUWFQsmfY%0nz542;C%j58`!1_MlG@C9-4P<3 z1YvyjCsGdlO8w0KG<5qIiJNpf2mo9$AjV@4R?-nZ52CA|io&d6@ZJ*{xse?EbO zP7{#sVuaYMq~_FJ4HB4b2VYgd?of=}W4#xRe4o6}Vh2bq4k#N)ZHy`iW$$22v`BzE7n z6+mJ~8c3!?ba`O)`?5~$S)6XiCht~I+)Z)0n}BOgAk_Ly?q(Lg5`}!b8Q!>dK-^x~ z-8Er%AA;r<7@1ZInv+@3ocvle4@KipReP3V{BE6~IT%pt$}$GcX%e5UZo;ceR=QYvh zb&=*aTCC@nPrnPs>pOI5Yhb9?EaLa69d|IbV_D9=RjDA$G01Js1W{Dou{7Ejfd~f% zC6Xm;14*(?OH%rlR+|aJ9GKL`wnkG{^g|X6xD4yMEb*v_S!e0OY4*t=)nt&D!G}P2 z8DMwuj&Mf3z;S6g6QuZIF%zWlX8SCVW6C=AOe3M&6<$%e!#_9yO!H#@l00me98wzM z7&}*C`!)cbfA#0FYqO8DG20EP%R*N;^#T1Z(F)NMUS|yf+vlLLb;WR0Ov19g9SX$1 z`q6Wvr2S-!ZI|QJZ4r&(&YOsXNXLDZnAnl@Zu(;9dP?2fe192dA7bw`yx}lo5O#2+7qA?z zZ?Gm@cmlK|IIt6J{De)kF6amB4>i8}G0MzB*wM}tfM|x;2MSwy!3KTw7P%A+OeoW_ z0~~V=MN`*joiSPW!2r9>F;#`dwD58Bb*8WOgi}wbfjxGrg8AOSG#)o(c^+D>NJMB*vCvVW42C@{E?xT4ru@}B!`XBLGo3eUjL4RC)^;48MASY)=CP#bqwj=QKVttY?qH-zS1`hx;f6R3M=^vWqLY8Y+I%@6-uz)ajrggh z*o0oEOO8t^D*ZV{CDw&AqeK2tibqq9MnMf0b+A)%firC&%H}n=D2_LQ>sl9AE#zR=d0`q&;KsvC{^5fkGKQc<=_7JZ{+uk zLh~bGn1t;ptC)o2Fg%)mfL$le&5(M=aKLDW#BTQtCHza5_gcHaR@LvXeyE_57ubUH zcLZfl5tFjl`ow+D5Q*UtUm<{%qUE_GM1RLH309G>~XjW~+$!`QU6Fp--{fDZqcqn!h8ip(}b62Jvu| z`J<$VY@DR5C5>3Puga$YQb&S>s1!I9WS)5FPIrR>4T&TMEea z%;(EXyy>t_%G}+;(Z)85xiyR>L)=E3?C^Udww31 z-jK%x>euP3ljGOIgRH~|XDqL{lFD?pvM!UmSW`V#z<2*L|J}Oer(#WO)et7VhMcIy zHOPQjL^f;N@^`t$!=`+V?QP$nw0ViPfsXJJusXPl2~lUjGAN5U0nKWQ0>z6Lm@SDE z*S9MzO72^_rir>C5k-_m)r2i6rY-7{Ol7zfu9g;c{i=Nn=(o=5C}LTg#HZ5p8_ImT z8MK|7VPgE3Sp!Ko#lB@)dLM9@_Gc&Nrv5w>h6R-|wjB7#BR8M(v@9dL z*4?#9d0K&kUH6(-qll{^zibsmnUoTg>2 z&%Y+X%OB-;=y^F$^;hdQSxtJ&@YB67VDAgq`vTVX1#qIi%MJm)!j-)O$?JR9z}_{m zcMa6I26iuxZf?lMZl%{P6&kWzTb~oca>ud_mvM#7<{K{Og=?3unDpFXezkoEo_!kQ zT}xwJ?9{5HG4Ah$`+H$_FRW@b%UT(>JD;&=NaP8PspV&}U8vSDy2vF}mw&L>@wHFq zS`hR<4?~+ zn}Js&xI1*c*+x&?T(#pyoO1hw#Vtu#oTs~Nd&1&8UbM{_i>ho9*&cuZmHM>bmvm>K z6AGUix~lY@IT?!%n$3D}fwu`dL3+cWj|EsINz&y5$udo>6J49WxNy#$N?&Xnw^s(- z<}x3Thw#_cUxX)c>HYvWK*DbEm@|JuLLi+=791fTq{ zL~E>&0?tf3!pYT79 zhYY!isO78pLNIKcsHUkG2*Gju2Cz6{$8vUZ`Vzz_+M0OnC7W&C29bI}>N6zIS&=FD ztPkqN)nCQ|t9(#+8UD^ELEBRXLoCbb3*+D-DS8zNKuy5~ae@%tYRn?$C2$Cah&tIZ zj?ka#1TgMp!N}DSNbKiz9!6M$)C})5LBTPCuhNlQVwXw368=xckf|;H`#4s?pA&!wD4$~}fO@SRxxao~sr$QL_f|=&< z+c9NMSl*-Ku?rLyOsUO@<;?Znh!q>d^_&*r_bJ-s_#ZFBU_dc`l6a4Ret-=L=LGbO zA?*3;XTKw8LRU~QI>w)pM}lmMk_sg0gs$^Qcul>Y@b`*pNrs`CARZqH{cI8cdzI6& zMPgpT7%QQYC;>3_(!v*7%TrmJ+_A{zHAG@l(6hB&`8!mi zwyxN^=*WOe7F8`rHi+K0`?}c&z^~ZWXn9G}0-P(J*5ZtuMvdq0r{rurLuSumZX`Ei z2{y5I%FF&pcS>m7hfJ9TH}jP;WfYbvqq4*n>rhet9u!+~olF@vprl}XAVoY>tezoC z`%IZ#%#>+iVp=9`W+8T$hT7_QfHH`9K)}v7;Jj}0)-^G*SqH&pgV(nXAdQ}ndk&Y9OXr@$qSjY+-(iwC;TEKX zoJJ_@0q)Y)jmJ71jL9)1| z)jTyny&x=emAEy-$Q`2GEr}w&odCFb2-h4`3>$@9C6uXwaGx@*^%#0LzazM z&Gv?m8#8?Dn&o*Pz`Z2_-1Fqut|8@iMWin+)wVe%U6*vxqLxBULmfDSfnx*K$^YJh z4N(_?NgPPBsViE$2(R)B8;fawPKC!F;BZHH8oQoH;xR3Uv6nwCMz5!@whvo>;zqVR z@jwq~7XgmXqwjj!?(x(2uGezplP6yt9x6gNJ<%ws4n-sQ`2T0`?RwlsvP8l6{S|Qs zxZSR8eegr1NL96G)0r+;R@*MGGpnk*X9k5J8C23mks4A=JH5C-*UZE2%RcSX0QUk@ zFT3Lf>Y;B9us}bKUJLg}^e@?SB1nmpNQwGLXG%$<%WX+yM4S^RPMq(9g;8FG{`99w z?O1i3+N&87e-2y&4%USGLQvSVQ4~tzfZxBh@Pa~)=s;ZWg;!z{hFyp~F}=;FufGp` z-!^D&^js9&;!~m+i=`tzybGN$9^gG>`D{c*Q3)F7*TmvZd1U@mALLVU4t+9V@35=% zj5x`hj;RGpf}_bxAHbt_h_)e-SNGRH{nv^`{q;}(t%Xl#OXIURb3A9Fcwwbk?TG}X zU|uQ%uhQv{0Si0i1cld$C)QiYv0J8%ll;I~49icQ|pa9;YdR#>x`2j?6joE5gx*u5m0;O;G z9xjYXRP~6-6*imXnhAW!z)kn|5=(e-mXsCr!u_n+P6uw&I7XEQ&@6I;vpvPPVM{M? zOD8wkSPV8affx9e@}h8JPw{sr<0zt@*X$t93dqfNkw!NJh}(Ctg(e8GyzCg(Cc{lk zX95BRsJwVKbh>Fbg`ow;U_sdgF$0|?m>~J_p&E3r!a{W{|IHHr z$ljWGBhbcTdLt$k$c8T%8}ZU+4v=`@@Y;3)2pDM+hA323=QSv5K9(6DiLcU_?2`!t zq9>j3m-Ld2DD|15N(ZBf|N!Q7wmWP#6Q6g^_WSK(^j$IbW zDuQDR$7Wq{EIcxV-|@~v_XywbGmnnoAbuI5@a5;JD1Nr^YT?%gUwH8}w9|q63QUoCbXETG-(E&7K zG{%aue3>vUTr&Jn5?Epw;p&u*Y08Rhs!!WEihvcqKh;u`Yhj3SI8J-;^h+;yYP=5UOa!{uOnQiJq&0EaTPw{+9F>hqPh76!&CtRX zxPq>90O8s>+gdJ4du~^ITwUCd;$ZpIjM_^7_6{g)-S?;ZsrHN2~Jog1W z@Ce(PljDKofqQQrI1cK&r5-qJNP_E~rS;&Oh>J{)N)vS-#kpLRr9@TxVPQYtfOyqJ4^(v+C&6MpXnj&SI_5e}VM zX3g?1DJ|Lljm!Ow*`o{JZk?}_KGP)ZY@M*vgzDbAP~DXVv!in~98{?8>d1j3i`4Du zmTqX4q05FPsgh3vSuc&$E%6K186|U1CTK`b31~22nqe50W1424_FP++ zElF`)rPrfoA>zW4a9By9hqLwyE6|HyO;&v{@yuN;V@%4+RF9+C-Ok{=S+Ik7k z3_1>&&=OkcVc|Wzc#B}(&CEzUJrPBUP4a6+6v>DoLA+!EySz?&TLo51Rd~$O3 zQlywIgN$M(MFIjWE`pEkfC~ge7)~^!(WT{d1Q&ja{mfvL8$98J0tbQN$)YES_g8j3 z>>UgrElhC}*<-N>3lxo%7p7kVUpzJI(U!WkY>KN7U_4N!fondCVHnp?*;w?H$+T$f90!C7PcP_MPie;_*y41Yg+c{2A>aj^0!8Lmq7)6P-yBx?pfqhwC_Ay7VwvJ(V`Vq5ya z^&;V!6bc$*j)mB68%~Q0S~w9+x0x5A2+r~ZKwUSx>-b!NI^N_OWKTdLK;I$HvJBfZ zacEXXV9V<&%YS`}+-cVH%L`KxC@i=1psk5o(VD2~*;C}v*QcLkA-e%!%|e&8Si%!H z%nYujZ{RJsYQ9Xcx))lq(!7}JneShf`X|Uo{BYig>~KP>O-bo0s@A`Mk=0e<(ehyx zoOex~*|N(FTeW4q@2Ik7^&Q=@R8uxpTiLeY{LaE-4fNIzZqJcXt?b17^G95M{y#p< z)jU$r;2J)1|1vZdiE0bQb!_1Lk$s9GpXUUueTLlLHC*}F zf@6dXPr-4)RgBd*F4$%N$Crga3DSF*gk~?pXtk?xtb)2Ni2yt2xGcXYN9CjK(@quueQ=!^BeUg$*rTcV>p) zwDU-{)4RY9Ry=k@tw~mHX?1H7V*y!9Zr~$sQL@sPyhDSM8sl!)p5z{hCx_^XSy$p} zA0DdAmr=vnao|Z@If@oJTutMshC4rC?%ediIgrHKRY&pQ^H5h%@Z^oL-`{9-EV$uILS^QowO0PmL{#0V6g zf+6ZBrT(wU^rsPuab6HcLs-Cye9Psu$#Ymz0O(1z`taTKC-@>HHw4d+pISobWcoAC zF(?#5^aAFC#Al(P$XmtD?Ja4k#cQld`SUx^3A6H&G4}l(QF2QE`cMD#BKYe+{qt9U z{ipw<{XhP_JO%SPf|Y=HM$w$}4$U?*69q`A5&s@L;Q6_kM&7%LFsO`ZFr)zhf+hxs zS&Z8shJe>uBRy_gU>J`8QVAWy=&9utZELV%?vCZ~Ofc@z6SH%0sS^TAjnGzsbUwiv zn!sTkF?R@y$@K~ZnheAbb_ko1nOa(~a&C;d;`J~cKsefGc)3m*MQ~d9+zIT#XzY7v z+M=+-{7c7+x#;R4nkUQ-Qi&cuSv008(&5!>EWOFAdJ4mp2(3VCRIY4hG66yV{oDXF znSQX5miU^z3Nqn%;2}q$510@HfKwF_fM&7zUX0Py1jvQ^npYi)TD+boya6aidr`ub zkufiuB$N=2d(jX22DWKJ{j3HjE}g?STX>w1=}&><*=z~hDT-OT>u5XLQR6X`Od-0 zrsFUNhtJJ~ltdm(`d&1M43Ug^0iv7ylWE^c493X90(hJcAQTRp&jJSynmESOzxWex z1QeCf)b$X1L!T2a<$9NWCCFicKgCFn)MDnRwq1Ci`ojb_>BlQCzNDWYa$7 zX7d1;Azb1iD83w%{I-J|FpOc_JDpXzjB>xSn@rw=bc)oSz{s3`k{8s8P;{xve|*}0 z#ifb-dmVBTfs}O}me;;IeNjw)tl1rADFO0X7`&hU=(G25wfGkHS^YAMryt0R|M-9T z1Yh9YilvQb54O@S+4nI8@+?Fl27|cnb)E`F#WNv5b`pEkVR6@U(r>ywQ|e1yxqI1@ zWL@jzw(h=Xt`rVOA#-JX|1u&i$8;qM!oA9L!*x_!GE7xhm}=Ozt}|`BjL3T?7uMLP zZJ1cyWHv?WXp+>Cd#a{%WnI?fT8Wv>Yq$E^o-$pXkVb;pNV+M3>fC3p(UV-s?)Ru6 z_n2)eYTweCy~u3JuDcYRLrAB+jcaKC$XtKrh>NE7<#HQKT*099wqmZmNOn$a{g^w2 zbI}^xv|zR?JHxlWWfg1PiSe&UW|ob%WVYpxlgxZTLgM~N;lAa$Z%QGX^do#o6!Ngu zhzF7VzS=LbLxV{9Qtl5FX`mV%wWsYR`Mp1orGYGW3`1#>-;X}|eH~@4O@80ZjdqmD z@At-HZ~$rKGgn|Pjl9AQ_;`hXu_~#2g)MYdQhBwBv_@KacG;~>D{owSjZ3d_=`}9B z#-+D~OYivdv&+I)?lx0z-q~lZ6QQ&X(9D&&B8zV;M`_dIr^)dl4CNIiTx6RKsaFy)H**2bOJ04eR6)tzS&xvmnpK!OaK{Yn0xeaPji#`Yjz4!A_ z&Wtf!rD4XanK9=7OxU|g6`w5QjBad4`2#sU!P4GOU!VW+zi3!F;7ve~Ktm6>`pz{_6Etr%x}wLj1mddiM1x$9DVm)31JiewL#C zDg5s_fql7r`r>4^3+?~$@0}HpZ1KNNrzi7O$+#}zFh)3EGfv+AWzug90CR4(rojWCL!Q?fuF=ztbcN?aPVLAp@&H(%+Ut(ICv zt>PQa;7e6usZXYViT^QS4yPHHS9J$&1==}tu(Y2r%Z_LSs($8=4>JqMY_qd}?Yh{~ zRd+glvGKh{%(lOpnHMsQeWm^8fBbvBor5|0203~8^!Hy~EdI1=Y3m=spJi1gC&dMG zfA4I^Q>Fl1kghuOCA0CdpTO9cGxN;TcVpl;qF5!1JL^EGW@s>WpPa(jPfKh#H9#+n zwS*H*1vxBb>Urm7=BQnLJ1FJ1!{5pOK;U}lkT1QUMJ_w!iw?Q;U;w{GUUbL{`;_DT zdk|kbuqEVkiY|ZqYQn#Gjlr~6E%KbdcY4KOj0+nDAJ!s@-fNMbtdd8vY4$$p?6P{b zV)Y8H{uwO9dlNVy+P3h+OFDs-gDezvT0~Y`!~pK{2n3}a7tyO&go=y!veO0>JZFrs z2?0LUDp^L~+{ZGsie>b~GH~0ThXJoZny%ypuH-egc6gHbe!3c2)j+{jbrbieZ|t@v zy<$xouIU6P2N^wTrOUg#3VGbr87HYPnd|b-CA@V5{QSEP`FcFY=q5b>9VE01{C|MT zfj3%2HG#4eMQtl^tHJ7x9WmOkSiO#`{|1O25J!FbJ;>_Ue35@hI{5vw(D%L%uUq6C zBYM!REcP4jDS)o0z+p(nW4NUeZIwpFD*L#~-*o^>ufp+|`5?I?ssWKtks!k$ zeCyfoIqpvZlQs?fFlZGioBRV zd>i^M=Pj>xTV#cRegUGZl^%{0-w$tu;oAWWsTv$qjT4W$BZjK@zlj)noj8t)jSvae+LCS!w$e zCJV2z+Y-m+i_SbozCDYv2SOcn#4+Zq8E+FH14GxPHvDC9iZyq{W!0S`^GhGt)sxV^ zN^mvL@nU1wVant{458PEO+4f1032EP128`z2RLaHA=MD}CG;t1_MI$;1ujB7;`0WCF|1FwLVUhgv z337vYGW{U#=x5aPeYlkPRxk6v<)bJoxZ$#F7;X65H1+~i#ntt9G8aBxa3Xz4-%xi9 z443N>Pp@DUIBY{s&)<5?#-?QaS^)_!iI7l(A$yyx{XCfLw#14H?kV1*J(J6eATTPR zp`i@E<9Svp?QznI{PMHcr<&VSZlPkIPQ~(O{PBw?dQ{<#~9#SnQS%n(D12L(M!c6FT`d_siN3#r8v_M3MuRp8 zIv!hqqWFDbj_mFHEWPDfPJEkA|LlpxNe_72;x^2`%ny&R1;6~WS3r@rJp8Na2V|Zn z5zR6)({NK3FHiK#4?wHG9 zFY}Y7LkKb;XJU-}wEdMOZZAgImy-tG2pt_)q2E%p@aE+eOgv`E#Iu3F`<}?9 zuk+@4K&~+thqv-fqMC1=BtxF}<+NWj>H{OMUy1jN@GZFnc_83EGr0L(7&OSbt>ys{ z_TP9wp5nx_j;i?BEBFNSx5eXdV9>@S8k1gmV{ElLTOmnTh=yJcVLHbR7e@g6ZxH;& zAIr=1zGQDPQc$cNd#zs4Yer%}eCyv}L)dVH`I(p-#~YZ&H}(~OGM>bL{nNkj#;qA} zA$DwKD7mM7UU`$(Nd`|$fC&IG@Ch8ayx|*ar9Dvb_K!Vug8*1C5f8z}Mz53%!_XF8%d&4mzIop*<2^Hcj zUf^K#J%zJE-(ug$}^;7Dl;g zPu_-U4x=}whzz)$nRm^TWw{q8h&|j|pud#t>~I)huqbx^gj&; z57-Ad6%bv~SE77RK#zaRFh}++PsS9p$EYs{wx94W!wZu!Ho~5|d{W-Bw{Q&N#J<8F z&C8qQguKZy2vuntY1n8AtUAKc0yf0v*vv=#rEH*b(W}e6s+AQIPwB{tJZH#eEdya$ zuV6JXd35^qt52${yv~~s&R+${zoZEq^Ydak*U|@SRe0UKAZr$KSu8a4Mo9Q^$IcMI z3$KU}qb>$N`}F3-!h~si^0z2gzM@0++c5CGfc=Vk&TkDx`4or+C~v|F5>u7X}wdjhB(SvO3)cwa4zLv~B2p zC-O&eC;JlZJJeNVx9{4L)^jyQW|HIf70uL4TiUL)?I8(T)%&QlWo=uoUP8nBpu9pE z*H?-40$@KsTdbF-_GHOGdti;SuW*PoxeMpmyf@G14t3%A7d&t(bSJ>M;h>Cwe4e2{ zXay=wT~~Eg)0S|z_s!T8n1&h}YG|mTp@t*{t5lI(U7-?nWlCjD(fg7`E!EZ~ z#pN=|f{5lNFWID`wDx>hQ+KH1YNlj6eSq1r941McEcN=9W_C6U%XJWKHc^ece$NjT(Ru+0qn8 zl@!*Ofeuu8apD)#52Fy4&4V+Gz@v{Mt#&y_fp7@t%5GeMQktMef^{WKvTxY{RD1G}Xl+zYfOq=3DAl4Lsi%!_pn6 z?^q^vdnVKSs-(Nj0-P9@V{6KCVf7?}0Z4i;oMPH;ZdjIXFvU_0)}w~a`nKD1 zbxY9|t*;wPvj9rkXC5dq;iRHiKCm>AHCK?NPeTyPzmAbr3_)W;sb08Zs#W+?Ka2sA zJt*;B90vd`Pxt_h$O5&;a$U`mbjRqqmaFTo1T00f45_DBiy%MAndDI0IGq{mUU4Jr zBvUqO>F_0Y$;R6GF3jN8G?z6x^Gw%TSHizQKd0o$AIQrya&fYbS_%fAmPmmTA76(_ zYRf~)3l5T!fRL7a$K7`$Kk>SjH;K`jJ<6MWx^frPds*JD)b~zI?N$kFw_@*TEvDDh z^Qnb9eSma;90kP25pz0$ZF!x*A9Vunsxu6)yK~8ZlbscvVza#Krzq&OKSB z?}&2mK(VZ#UCK*@vbY^4psP$++;O8_3HeT-Ye@NynD9e-p)jbbRc}tItG?u;5=Jsa6?UdJ6WrM1M@pog&-35*f z{L4A>^!<@R`&-gxsxF^ysxI%1_&$y?aYIV_Sc^LJsD@3bZpTYE%@DZ9N_Nc)Imi7X zF9ELCNygV`n-P4l$OBs&*mCCY&rCWqgU;W^>>qiw#c$=p3b4IYF(AthBQ^`x>gHk# z+c1247IoMj=P}Zy;^cR)!nf^Qyx}a2E^fnMQz*Y$|7RU(!yW_S8P0Tsb-Y{HbhL-Fm!k)6Z`Q&yS7^QsVzBxbdlPPX5E}q-4a_nJIdx=4N7<~~}CP{787g24? zN?Yn(%Gy9N2derTB>!C+Bk}=d^-}Kf0rd`Wil#B*aWzILcxCoze!8+6uB-YT=DOA0)2bS%|MSkw0PJQ(`Z5Agy*=HKqC>+8A{y46izOI$1~ z`oOMErEVI&9$LfK#{uZs=@bjUY}g65cytcGDeQ)cy}LDqEhn`FyZxcWtUv zpQu(gR;oW^J!;&;Wcw%MJz=1s|4RDbBZli8oV~UR+&n6&tZ&`=XsGJWS80e{*|6)w z^>SwOnSqwPrZ;qEr6tc3uV6jBJ1@(@`9n%KY@YX5=G$DKMOaA*g)reVa#$aCTsd}i za7-Bx>qBqybXV0T1Eqz)i zYo>IA*Cy^9OvBHQ%ULU2H2ux{w=cHZn}tgqp{9~XIX})tIDgj7zFg6Fb4!C%-WdNY zfAZ-YI*AA4UkxXp&OTY8@8B0tsQvvg3X{MY;BVc+kK{o))s}F>y=)~|Ie5R@qPwR2 zlB|{DCD?Fvo>v*CS)XjS4fDM_gU{VwQky!>8;)FNJ1Ps6-GlX@df4rBtHU`Icg;SU zWAQOfs&{Q++sqij1JujZvGtMed38+0HLVJ0I<}bB17bh>QSlot(r}R~E|M!7RKiQv z7a{5{F^CZzez#<~sg*pn1NV45&$Y^q`8^6GI2?}i5!}8N+z4y_G~B2bH(FPGYDY#? zZ}=^FP%2fd4Lm=bBeSC%)?Chw=37bCr_aB-czU*g(}kLx$9E5C2T1sfYZ%|E1%6dY z*PKnhK+kLI$n1DtYL36#y&+P5+Urt&iV`WTY+-j+E> z@a*jzu+VtxcHtHEYh~@qIEuGtWgV(}z-BfO0@i@B*WjYtPm(X4xT273*@0bOqjuRI zaCip8`4$VVy{)8jule#_26#V6Mx3joUdCy6sV=?)OWl3ityo|#(IV>wUV3is`qbPR zr(O<_y8SsfHANgcpI>wg%}M~yGk)$si?}s-E9IpUxbK{6-s8eE1)f&LWYM<&V9N^f7=JphcG1_wtU6X7>mTggMZC*D?)O^FLO^!HG9Xs~y zv}ThocNIx9#5>F7nbyuE%H&n|)WiyJOIece7$#7bl{qbI2=-ArWzCzH{e9ZIbY&wm znoO?3tb{2NZ;;LT1swM2kAcJ9lj#rsHI2}?@cHzE@4e%1&K~@n{qjThWm+I*HURzQ z2R_2dFX<4X4jv7@heu?-*fa?`UBbNV;1lZe073vgnEpKS?2x~^wkQ1)FC5XB)_eHE zgZmR6Cv+HvZWv*yB3@0V&dbkOIE<(ZI}s&OS|WA*^5$Uq)1QYcz(V9;w#2-Z>uKZx z3--bo!%!)KLBohfg!#l;LKeBnO_2s0v3FxXoKV}F{v6;}mOvFoEF}Sn1uhDxNC|%y zKq=1PDQ5G|)&c{Gm56cKJ6^lvTKqC%kxiYjh12stb~qZd391mDL!ESC*+O7a9w&-V zVM&mY1(%a&VNg|^zMjCkFypSeg%k@h)xxf8F3ChW4Mu+u}L=aR`Gvx&ZO_%Evfg z*);l}bjV2(q{|rtPw_$m7cgL$1Dp*YR)>bkzf3$Ii?5{h1RdO1ZnBKA`U}9x178U$ z+%tTp*ziLe$+bns3Bxe~H6t3uEW-xh7lPGaUX;yV%xu-m6TYW;hS@>gRSx4bXXwgtzW@pfU^wy6)P{lq{*|Xo_V^hRqC@_7zD{ zWVL52`GM*;K)Y_In+<{$i z523pH8bilc)zuYVU0u19jDai<6t!dawLk7sT^$CL+lq22n*&7}$o)=VHJa+`$5&l_ zg`IAb+Ujd*x-irs*85(z!+(6C^#BTX&n%6(Ld+Fb#>Xqnmn+LK&u^5qWtbZ?SYrk| zL^Ig_D;TF4Ei+Sob<)fHk$rN@WnaO5i_P9Se1b6?7f=jk zf<@rIh8r3#(2*CEMg?qJ)OdSST4}lCtT;;xU95&o$*Vym)ubV|%4zcYR}PRbv+|V-E_( z=DNi$YQai)^xf4k7Dj4J)|2Be-ee#fB&9pc5{rDuFuA z6h*T8mgy?4?5KTLv9zATFdwrd;iAMJTW1vBT-!!1YSMxK1`52=RZT?C(#?wm%g+ygECHfO+r|@mDu79Dbjm<>DHHlJE(>(^+};I7D}E23OXPF~C=ud*3V@-uMslEOxkb6Da#^61sc#mV_6m@lL$B$`D|&VRaMo$m_69i;TQ?Fhsr%J+QACzJ&~c@v$uu zgwZJG>*FT4fQnATuxJ403~o75^gXoDR*lO-nTled$29}EoKb4t!Y0NcN8SuUWE_qY zv4UAkVY$cWQhMZu(I_UN@5~w)=h_G7kj$&`Qy`GHgi8~A3$xbDd~Dc;6?@tt&(PK= z9>s6PMM~&+3@gAzTcWElx0_whh?lQp3~+h&B%`g|textl6-83yT<~LQp2_R<4jzy* zZ#?}u_Oim$=@0yd1MK@r#v*18Dt^A=|Cv0(FoqDACO-SbgLQ-x?1vT4`!h^`j66~0 zdNx)|lnB8_0ptQsCIi6mDxFM!ay=WUiI@1C{@VyY8Fo)L3`=n&S?MuJH!Zu@>&y9p>Zh^X z$u(HMAs7x^k;9U2!E=&KrvE(%g)`&{h+WK|30%8pkmSo->r~eImSUO;>pPS=s^0J0 z%wVR=dXl@8V*s4+zauBiDl@?H&O$rk6|b^ZLMKlu5=ibOb4A3OfA^i3*kVjmDOF_$ zD~4K{YS^-5E0(UgR5O&`62M=beA!MpRlIrX%09W^N7=OnCdex*I8G;_7OJAD_6%CS zW=h-SwA|{->m0c?JaCh|m6osuiI1tqtGrd0V}${SCIY5n>*K5c6PndOw`{j(2(!R{cPd(pDRc_mGsP${kJLH7QL} z+md-HsRIoLG&-7THc4s6m6W!zFQYn5ZIy%LC^OPNvi^Ah>1YLSOfDs@;E8!$<=H8z z<(F$47D~hud&uQ*801#)*oMP9Vg-A%3GSdXO9am;YL0#m~!s!;{q&o%O4j|g?sF{ zK+AtEu7_x~SeXZQ zX9kS&G}oe!NSL=abvso?q1SdfmbZ&u=ilr$`u2m@w-+_t%yP5Vu+t@^ zJy&s58K{DqeMMnt7Oqwk--mTw=@tf&S zHWs~$xymzPQ3Qk$kHXB*P2hrtM-nuNkFn62GyRE|t4mi_E6Mvg3?EJ+`1^V`U6+GL z7NPFU)6SL>VkLcDLhK7R3ei-Y*)YDLSR3!R#Ec{l$3!2*&aZKt1hvg>9{>zQv0Tf-a}W@6}Lwn*P;xW_cj@>8}ooV*sR|^Dn~W7 zx}xQuKcuc`8)koAcmd%O6%5!Rr>EG~V0;vAwk_22qfBwzL;#(KcO3`prwJbaAghv&b?3e5kA67JgL-dM@r2PMmvB~Q z@OWHdKUw4d$?m~Y+vnkjfA3)4xfem>n;U1A1ID+t4lVcaDR$;*TK6>3;fE0&zTTp9 zgSl*x#+0APd`c zd$Tl$v^k^)cStjxsD{#X;H~y@=n}2=nxM%hFzY}9vsUusJLP=)Xv4CCksWgIm2DYR zc1^iwG0U)emf_kuQyo)wrJhW+n!#7KMxowd2rM&ol=?Dms&RcDQOzBg&exh(*d4NXV2GLmO0UD23&Bh|B!K&ud zJFb82_^)}oOw2Q?qtLG(9rh%QFugiTCF?JfK;+|$eac&MvUrj>*x}O#TD2nV?aF+j z;ZJl3jGDLdlnq5IZ|?De1_^#O2|J|SCI-3ce9Ci}{ZEM-`kwgu^DuDcU@zk6QxY>< z^a#yR6ytZ1$4It(p&HoP&wSJ;2^ro#*odK$^W1@B>STIk^au$N-Hhz1vz`C7=pB)2nMh+^lVc|Nt zN0ALXTnKw9?t2d+^<36xlCIlU&v7(UwY0v(l%8eVnnpEhY!|7wXMS9UYW%e=8Er}1 zG>k95leOabnisa0nC^eLV_6T6d0M(dsi`}rZpc)Tdp)h+S9%((#&KN9rnbB|vA8og zkD^@Y`nwzJ-^2138GF1Z&c2_e?VE2eFq7S46`Om$x#u5X@Z%bD!&bqMDi40_UCP=( zF$b#p8wvjG68y*ql+{bQH&CU4W_C11Z-O6>J@|1gX|4)>Twxwtc=6EP967jzpX2C) zIQFKO)CqaqSPm#wHQ4y=+liojyuHzHm%17G+Hr?se%pU-_jH8s#0YNYz+U6BtMk?^ z9k9&$yx$7Q2Zx5ttp32!unX&W(1=@bgxrfb>b4qW8cCSbKRCbtJFXq19D7-VcgSNR2==nCUNzfUF7I!bovaWPN z2+)*=8hoF}!2V+IPyF}5UdSKV82c$*0hK+Q*_w)^`_ktaAwnRtRd~n`}tlrC`5ND~~&!TZy9*O_d$L7j0fXU|BwJkCU zLELb{`P(3*j>4>i)`Ye2tqCBN5j;;r7PZp(K=7ab3=@c+k?_e0Z9e)0Q+a`msXt7x z>J5#@Nb8~roJ|3M35W#mK~;d4t^pHS((epb@hDzr4hsn2C&2$o5C<>fZ*39KdrjK} zn`#l;OYkuc$qN|6_oqLBybUOz7HkYq2dXI39|X=Xfc|yci2#GmI!wjs>R0 zI{Dhh6tKZi@FzFT01^^@$~fU zllxdxdDg_@-o|^refw7H2$Ja1A@Ig-TEL(yE8i*NEBqJxj`G`5fKo0+E($q^7%I9X zrc_0WN{5vZ!ZcTR`^?oC(>0UXrfQp#+|!J{ZELzjx624wAu*6PDUlgE*mbj1XaMfM zQf8>J4YLkN{6}DlKpPo*$++8#qFhuzgt%vFT}74+rC#PlfsX2ATI@3Zo)R;1225LL ziqfNo)9=|#QY}}7dqz_novA7_mk#6elhMEKh!E~!(;1Wih|tD{@fM<|lmMlsj_^9E<`I+9(WS9;ci>7I&?ic3^jvj@GrquF$bQ z&OLW1G*%~3CU;~v&y%_5&~L{W_jF;l7ARUY`FVLtUY@O^)jAnh+feIAJKdWS(#Krb zp0>6TsgFUVZit}{F#sJnp`DR~*jlu=E4{9w>d5DtZ9TwkeIRH2j@9M|kYKsaw{#y9 zEM=wBZ}KaTJ->22WvrzOinkl4DFJ3JMB<4RIqF-(mJdK;Yag!gWn|u{j zuXRE<3D}R9FN+7BN}CI&df=F-h!T6?Seb{vt;w<$k?dHNwt$M<*ZWPgt``flBAI&g zz?j@%_b6+kNsw+5q*o_MuMb%%hN;iOKOV0O{QwPp_Us`gcy8;vE6Y(^?>DT<>{z9p zRVIX@$+wu6liB!>Fe5beT|dtGtv}0RFIHtHH@aS<>w&JfsPG({VYJlT{V@YvN^^GC z2IO>)(JJ{-A~BnYbdkvblmcC{VJguRbl*uqt0N&(n4(akAagR#tl0 zy1dFFOna)XvW8kKT%R9X&TY6&!)^A%ZPo>R?Zzc z7@~i}8Fw*-Y@C#mGnlNdgO#KC{TA3-7ka9JvPV!-23d{DFL#tM^yOwm& zC2XjzTRlxRWXfdMGXxvnGYZ9cUMC4UNkFeLbaNm}a$oMM20uGT zRk&s^Q>BojrVR$LYKVa@YDq7 zwMcQP^XTVvK@$97^xUc?=1 z)zkB7P}{qKW1R%VDnH&Wazn*P?=UhZK<^-iK9CI8P6fYeWnZN8#i?_Ul9V)gDCL~i zVLA9?p3+Ioi03CGxO~Zk!gM?UI0F!s%oIj^F99(~?8hX8A1I&+oh7(jpZfqaobjcm zYrcW!08>L2Fp&3B@8M|rBW$DZPk&6`rX#ra0Irb49xN2KXA&e>tCPoWdO@}c9r85A z&WJ{U#F)ovqIOK|gkmAj5;}o7{L~>V9I+sQ;tlE0YQh`fPlckgh+WSW@ui$(o?g>n z$b2s&lx*8sklO={ZRH@262RuCJUA4@hcKhOBE10ZdFgvjIGTRIl~MjuN*uUOr$BNo z;=;%ru?q|NR8XVv=i+R$LzQll>oG)+O=F79#K`3pn;#7xy#u$2(B~dauIPlkz4GiU z;>8?ipAgvm3A_@D@H}`{pc0+U*9=qu!e1Jt3%>^T1clvW3Ev zA+CSECUGi~7_c!6@kNW>gz#IKaCG{hJ++7hA8=~?67U+48`w4{ohCaO4E>rT7!=m+ zMnDa~hre%eba6zxpgO<_i7@1V*jI4&7|?DE%N){(50;?-r}Bav%Fx_#$UaYiN_$Q;GQJiuUt;-L*t z;259s2eQ+_w_{Flf<6IuEclhu^L)Hd`TTSGL!%Cy6ody(mYA831*)b?0WBaqcUG1X z(<-a^q;@~pu@^iXh9CWt_hZku6PDF>EGs^C#uXY2iJbn7Ix;x{zBTsgw?BtuawD!Gy_X}Y6o%rYFN z$xJca?Fxe3nUGq&DSBH{N*KzD(_WJGo^gt=#8sWg>_R!5j;v{h+?8a#qiG!=<9^Ko zU^$8}KL3&DKAtY`mS);|-*RPBr@cO4#*|r4ar?U7muzOy#fkYnfGETLJb>Y23%r#h!Kez*p z@BnJ3q`u%>;go_O`0*OY^11>lTFn9~jVq^d zZHI*P$7KO++ZGnNyP~NJA&4R*VAH6^sR1N5^DuAqtnYiCa`D0f(TODz^?=n0K18-T|EDe|0Ag-n~^lbNd;IaZ# z>!XtIv_{W8(ici`FpuCbgK>R6z`+!Nyi4ksHWInp>|&3lz1@@vX2vQ0xRM2 z$poJ%JdVN~6|bNuhTz%L5Agns23jPTexSi91ZD=KjDgJoPs1>GY@OrNi1)qmL3$X% z=ih&Ge%=zH{(vu#4n!s9F>DB&3=scu73>}M6bevs6jNi=`%skj=U;%R5l<4Js}yDz z-e){9u7qIc+wbhtQ#y^g#9q&sYF;st3k-u0}H5ktwx z6Jh^n3}&B1d<{5nejNxxA~dWCmviz+LVQxZL3ZQyMo7afzB3!O7k0TjbcA4V`AJP;=^ z4TicG_O_aAI^<=T#GKl1FjkQGfBnhY~<=UzxD-mRkXR-VKYf=Lv_N%#sGtakcupgRzF3DyT#M?zpTT6Xyoa`3uw3f1%j z(ldt&)hLClh;O2Y0+!6@Nc(o(0Eb6rGN)xkj_E0IH!vMJqb_}q~y%fRd8AseXNSO#Wr3@%0oa7K0b!UqN^R}=366Z6@HkrJkIfo zKh8KFZ`B18Af-^)NT6Ik5sVjv$u%w;={FXV6nqck2m+2%bR=jDABRiu1R-cp?GaDn z2uI@oXGBSf1&Ek_aPbIz5Cj|NW}K5a6Wd3w69NR7q9?zbusXyuptO=SRu;8Vkw$I&*?A;7HVBpJw+bD zB%j<>C9Je6yeQo<@2@Asrz0+N01671y*v0Hi@Wbee&S^vKulTdDypjLhf?mKa47C) zwXD$lPDCpplC+*H8Ft^%nX9{+=^Bn~=$c9uyQgScPtOljzgpHu7CUQfMR|>^$qk0{ z7um#ag>n>4?rGgO9UH_&pw$j8qa&LgENN1#SoOj~cW3BMxUrjo5J#RxbbM7kyejL8 z-0zv@(pa~$zy|80hWdEQp~;%sv--NDDjGF_iOH6xII5(uzHG>_gvE(pOh15d9Eqgy z3mYdE4R`!spU)-FxnO!Kaz^K@?$*k5tsGc$H5Cx-+C5ih6p-wyeW@=2z_d?+WiNqQ zL3C>;Qcb3+j%J#&VJo)PlN?#1OzFYAE|X+)aiTA<68-8i4Qd;zAt`-D(M=$1+mcLb zDu!*?l$xfxIONyCnBII#{i=cIo9`NZO;P%WK^2QBj@*}ZU5Bq+v+p>}vX-fCTM=>{ z6&jYM8%(iOgY~Fkv%c;2T-{Q1MeFN^U4pgt89I-`$>wG73Q}K7*uBNa*I|-M=)=%= zb`l8zwl`Ef84m`4-hSWi>5{HcyD!NUZKS4Sn5JYXroIH<^nXWX4;aXRycS(e5TC*n z0_u6H6R>!F@w`=R?4YXXt>woX3za41^p^C9jWmdcmg>{ePE76AX{ko_ye(DE{ng;S z<9ocU;2!lmu^0;f-Ii3?T$sD-dZ3P;ekYsXJwjA}@|~o2xG|tr<7Fr9^;+xcEvi4> z0p_SbVNn45s-RvLOF^DICtX6mfX@+02?8g4?hOM*yei@@;^Ov%-#uvXI~=0hDQ)^O zrJ>uk!v@_@2b)_GXURwKSu6rcG z+mkTA4AJ@T-)jleqkh099k?Wc?Y!X~9nl+3qap34?#Zy*Hx*OqsY`N0Exwt19!rp> zPp1td|JMkHSVUg(@l}l#a-$uJwx4L3@3aW_#t)pI9*xSjs zcqBsmWf-7%&mtQRF?>2>k#|jn1G(3cK!oBKoDiWJ7vo{M81IRK((L+2b}T=LI{K@F zZTC<|UzIwtqV8TtUr{a0+3*D9-BqnI7^h3Uhs0Cr6}|t*%W| z$$3E^r!Gx`4$iz^^C0BLh9}-vo>)`OIs%qitnxpi!|%LBgKe#d=UrND%mbvit<}F_ z_;LZfP2asGo60U$tmcsKT%LNv=<)PS*Bc`XBH!etJs4&-hi4@V*~8>TMBMUhjoE1r zW~bxgEO*)V=WtC?&0$z)gT+j>0=%&H9Y!&}9VWx8Fq!b#W7scPFZ_2!U|%LlbVIygmX*P9!hGbu z2o#Syd$*2|hUwn|)5qpB zcuF_1XwBYf42z9nvFZ+Pdb=G)Z?}~!^ggq$Ow;uzbDUhyn!Aows%nf7r2(I*y0#@4Br2P%HUJk=Q02}}&+ zpMa9!_(qsNj%m|0N~KLycbG$+4!HoKFJjsAB9A;>8`qw2bWAA4ZnDfy#D>ElnDfM| z7^WFkA@-PMQ50rVfG!Ep`0PE!RDR5EpZ)~kpdB85#pQVO{4N704XMqaEwo~l1qUIV-*L!= z$SYwj^?=Q@>3dcIYlpk0s(6-KRXl5aFIqD3JiE8m|}$ip{0?P|8WJ(dV}+ zC#mvslD$h=8z|;LReys5?Jngc`GB%|Dfb4dG|<$Jq-jk#$s;T$xxyZ`N!`e`L|quF z=rp`a*~>yLi}cIworK#Ltr^TY#@{d1uPLHPcp49}5avoCS4r^)P{y)grpuMGOigr; zmrMZpysw`tyORH>cZD3rkyS1yeF*o8pDtPdK%V|9h65mg!SomebI$Oo7^6*P;ZD66AwO$-fc=*1gt4+^dBUBu)P|?H*A4r_= zDzX0P1jD;K%3FjrZ*(vZI-J>+ncO$sp4~TWQ)RMk*^aHaif;8>qdc6s#0gYq6zUDo zg*I{Cb%5CqWXX`!uBs`0xqoD#%zKGP{(?op^hXfc-;;BTM(?LTTGY>EO6r4>FbPXT zluvTM<)f5wK>EuM^AX{Ll{URD#4XRdKk!A(9sd__CYf0F44zrH2_DTcfA%#SCO!(h zH)IIxBnpHNx6Z52#Pb#?N0Xa+ubcx=C>3EtoHq$}OFjtU>hpmqjj-iaqNL+8D&eY`!3q9qnPVWu7^9|G0}|+PEbTyR=JTY zn?vk{CA8R@K*LS$*MwPB>OJ?lSS9sgg=k&n9gcXiE?*d5Bwzci=w!WdU%^_Vj52U= zK`v;@uT7LzMwAY9>b-N3FuU$ny&-^vD>Qi!Y@1ca;#x(ZmKX~Eu7DAe#E4JF zbNQ_{!CY&GW$_8z_E({kYU~|ydBxxuNAVVq&q$TL$Nw=bwv-=UL#j zmR1`b@T<5F#KK~@A$cNwab=!M1ZDsAPyZ5&wT!$N1G~uB8#7om&I`j?SUM5$kl_Y9 zlBL=5F$kV~Pm6fNZ1*sLmB!?e2T$FAP(Nx+EsTDM!UWjd2FG=o%st^nO=}h4E{isO zt?~n~e6W`Yjd#7IWRgF#ylEb9+c|3pDonggG{_8yJv+7(!C)Mu^mbc&rQtN#CeYy}36T)(wd8 zu{y*E3ua_sXAooK$!qK;%D)Q$H;ZjHa|>+oKBTo5@6*k&hqVd#WyV(=!(a&L@{(gu zn3{pMLkEW9=IPn7&n8R7CA8kdiy-5Y>(B?S4adyTiah&<4>s|}Vwl-^=bafB@Xt8_ z($NSF(J>%2eZUD@vmfTK;l8u%?5Cdn>?hc`PelJwSoj;>19Vel!PbeR8Rj_wpbqA< z7dD$a7X%}i+qk)bJx~8RmT;5q#ll1xPr%NNHo8#KpE)xSp&$`l_Ea=r-RbZkt#L+> zV$RD4S}@@lC_iAF_6mf$LW1e6N>`Kh%B+!D&kNwFuK=+uA)iw#qDx6?muJ5xk1jbI zItx*|qdtf4_cVG(KS`sO(=e0Ev)91O3(tz89j}>F%vlqw77?;yo`v>n{Bc><$+%OI zsP-D9tm)4!a!MnPPubMfHh--OHf-a|onD;9b0u~9Zvns7Q*jXp0CtL8VDTEXjNHE* zw+yeTICn^4{fnlIdlc4R5u0B)KZV%*rP>4cB4^FO#wf0BDza|$EmODqT90Z>)A0+J zn%m{9-Mv0jV|O;-ZuN6G@)TPuPp$C6p7PY@ZY&22rOJ}*P-+-d1M0V_VY{|W?Y?Pg z(&8DY;@TB?X&=q>w?WR@$=NqgUtgZTc>40PL!SQ{p2UXq2n^l_K$8h*HqNgpGVWAd zCQim8Js#M=uBnhbC-1rJfDsf9210NeqY(%IFhs;AnX??pFT`L5ylddncoj~#u#^qr z@JCQR0kIr*my%4l%U<FP+~=soa7s z#iJ2kp#U6+d#K`pWb$jc!#YG&c%Z~3dpCjm)t&yFt^c{tJpAXYC>(nXj{w%9^Z(=j z!!5!_^^abkJtuM(q%Qpv01sjDxmqzt6E@Q34Y&cZ5%b>Rq==3g635Z>->zYu9%1js z*r%ITVDG6o+c8GpqR7@tgD(nO;vlRsOA6a+Q{RcE|B99)wi(4l7!V2KIbU9wAqeNk(qq2{p#5+n77zCV z;uYgY=*Z$$L|<$+PMFxBlmv591d`xo$pjXsf8h*)_gi;i4=5`q3?FcZpwbm`~uoZcT zc$+G1*{mb$?3|R^NU`@;ihU4CrmJ1CTP2w)JjqnKl#GF_4m7i)^vpl*l4J@4$}r}o zEDtnkAZr~{Hk%~V<4Q8!I7*;8>vUDER@K&H^2Bd7hO)4v*`^ zQ&c5`mfGVMQ$O>c_GPB4kzSfzfos!C8&7HDDQ!HZji?@a}PeRvKh{Tp+Fv zhCVJ>)4TP!fSTQX#|2h-5d3k$xMi^ZaUnq@6&x2#s&ZWLY@VcXT=1_-COIxhi%BoX z1)ILoiJ7tH(?%!5=?pf3p);rSXE5yn(`L22srw)Od{Tp6pY@~sM& zUQvN>`KQ$}+PUY2xA^fwto3Kz^!wQZBJeu*{XMG3@_=jhbvG@|_xFp%5DvZ`-}(^e zV%dRB%h(hHPDOvwim21{$HNRg6^q|4{xk*r4uVkAnU6OTbpgPd;i%%RTZu^(L)QvS zEm`{^MW+@kHZ^RZVFL{tc!+G^HcHf02XmK|t6Lu`UQsM7i{aj|T%Eqa{Fb7#kFY=a zsybHrz{$Nk~SRn zNI2}K8(*4Aa4UCyE`X_tdTgQ|!^N1kJ4pexaOr|J+@20Hoa#`II9|Ve!ocM6#13k3J1&A;=v{*?u_g%h%?!vqE zE9b^x1cXq#aOm&oSX@Lf+ac{R1HXVo1aeWx!}8Hl*B1O_!bP}{ ze>(&*llPK24RPEF7hf#;CPYi%glA^Jk-4VagiS5pDTacM3Rf~pvk@1-LT)K^yx2}6 zASnI^CD6I@PKnFVaDaj^xA4V8?3l=T2-184_)tU^c_f><(f*``4^4P~9sVax0^>S1 zB|y-CP8y&k6i1zJ zBW|~lo+$yrH(;1l!E?!iYh*Djy)FcejF|}HY@-E2JJcq19Ug29KFCCnVzMMK>FC#Qd@P0W6GUY&qAcBkCOcO?UZ=%A63&7#$ zC*JokKlz>K`v437^PG`OZwS)=DB&h%VQ}UNJ4lhUOc{T63T{MGq?9F*n^uZ99|#}K zN}#9UI^iMy2Q&eo-As`wZc{ZHunW^@?0bADoIJq8k~8~i?EpatTTaN1xOvXe%Zf$xy>ptx;gdV{&BxYr>k3w$G^-n1YW z4)pfQ1NvB6y;nR~xYF6FV(+ei_K^y(Sr(@WSER{_>x%#3yE?nfVDFs}pN!!^ox=|L z{)|rN_?#cB3FcN8c6Y={pJw}PMIp8VOH+LL3BccG$A-U(U}o#%$l-&mtf8sMYpcwq zM;@-&=M=Phiape#YfNY3u1_(nxQv#{YtDkfUY~#c{Q2tMta$FL<}8-ita^$(f=NE9 zQGD}O3SYt$EQrPHr>Eq}AIQs7^72dwqq~$7n)O$@w`GL5PSU~uSloR#@)NHs)W5DN zYh6WAb>mRV2^9{+{Va+Zdf$ntF+`GN>ZT-Hw%+S8W*Ukrxkk@Yd$OZCwks*y6~(+= z>AeQ#D+*rTAZsgfg_2rfX+LQ4m52h)<8PsSSVz(1p4NTSf%7$vZ=8sBa3SzlN75Fn zV!rUuTNSz!IIghIBjC@^B09dRo%5GeMQktMef^{WKvTxY{RD1G}XmkVA!Vjs#XYR^9L)zkQ-lrT{%3bfXhOx^LIhlykjld_`FG1B0Mha$`5U{*f(!522v=n)sBh3X1pIlHQi( zOSwN#q=9O5R888sp17i1D#}3B2buw_z2DRmKlXa!YbkS8J@GBVsgAPL_-z^c=gB8W zh9`_|DF(sJ5jM!hSj+<~(y)$sI7bm_2T*W5i-*e#j0*nknkSml{SS3A6VhR4Z}!xD?E57RWx=BSP>@6O+#k7 zD=THl3pkiLc8<70#s+adJCw`Vc$U4@&RA)_I*4uETrJqaLDdfoTl~!+a0uGy(&1_i#3L-gFJSqU-ft`>k!4KimMyZ;dkDm zm|QEOWtXMS17Kz)vC4gPAOCP41lPMO>EO6H-{Km^jVMu*!(~3P8cwrK&r`Y5wh*p) zD~GBq__ECRzdopQr68>ui|nF2%=8FFfC80n9>JQPi^4El2eOt8R$uVgX${L^t-GH1 zZE?Bs^w*z%$4%JND)uAQ-5%6c+rFicxm-)Fm|WwG$#ul&iU zZ|Ec*jDIzpe0rISXn|6KIqZQS5v`*`!BD7nKpM^q5^fCckA zD1>|HHixFXXax@DskUV>u20;p9Jo49s8nfKn}u{R#)pQV-B*70a7?Us7nObcA?Hom zypkc^XGX9^RphOX^M0!%<5%q6B%*XPyg*XKNm2GqLN37B=V7u#7AOWi@z8 z1$dJQ=;$r-CAG01ROe;r#cV(>flQIZdz|8Q@&-^l|u>MVE+70O46eBW&2vA}?rU zUjbev$s|{gayB8j5{Bth9nhiS+_|g`n+uBb4ICJJp*x~cSL$_#I6<3F&>kh9bN~~( z0M@GjY#ukYT>Vr2q}IY+TW9l35__8O|?T#&vwGIR%1D@8TWyn!2FrfrF=R1 z&dbrumvc(73tS87435cj_STEZr5E$_cyUGF!ud>M8ju$q^303;m@n)gzzzYH-3l#6efXB2jQ#C-2Xz(Pbzz`Wr0ki^N`VXC2AF{LW#W5ufwsk6^PCN7+IQ?U3?>Z2AZc_1 z+#NWvO~>5cdUpPY`Szb?sR}2&C1*SmmqsVWW`2Psy3;X*HIiH9xVljT_0@;H%`CHSMn2o1}0cz=PfTN0wA&b(gAYODgy2@8y*M5)@n@K;xh) z)!G+-KgNI8zH49jqT5%~Yp#a3{S@;g6Ojp!00@xaK@!QL_qJOkfjmS;L`FtNd~x)B z>>a_jXz)=q-@_=H0gSRKHb~eWS<`?qI8oTdtT8Wq&ekJVh4JzN)FN!^*Ur%Quy7_< z=WqQH9QT4G+3Om1DL)QfQ0v!B9DvAq7}EQ&MPAUryQRZEISFqEa45jl2KH--rCx8u z6;{hjin`B-6+R7si*XaX2^^j_`8tL_Py$}YEI=FZErofsUcy22n%*(4{QfJ3vnbe; zZ$*k_$dmBU)@;F|gV_$YDp+K?9*l;BVH%{*R9~G@cs8(q9?aIC&}$jgZ!R>bi*bdvX6c8d&gz> zLLNUb*X2{u5+Q8sdb7QzaVb|=JzV}8ljFazFR}6%j4Zfw?);!nUf@AGBb^fy2(Mfw zdg>&&gucQc;=4B-6o4vMz5!Oh&`lYC!$n9??St*n;!2TQDX9>jc+o9CB;m~uoE=`X z4`2krncj2aVT*tKqaVBnl2NX&U{%QV4_@;E*5t`_NOin6ziK0cy9_@KRHD+GXSILYZkl*eTCNOX=VY}vpwVr zi|iZU9rFE|dW_#EUb1`SN!vz48`RQpMsi=J2|PH5m;8dVMc#Y_-n9Btc`fdr>UzVk zu${m*w#P96v-P7#wN9zuJ)R`F0FUg zudu^@2j}v@puA=&8{LuNP0y-(EwS~yanPu@y}?K16nLbU4-jE#U(-m|c5IEYlV{%|U6k4y8+2f0P@Ilp7>y~lakfw#fr zbkV1E4yuP5>4_?{Q^Rh9cHoONAOaPFu%ck6V(E?xY8os>)XcyDuW*~@wUuASj>TRh zd)!A3SDTmgzRZC3#P57-;@}BQY7z`)5Jt459Z4#N8-T0z@oqjw5 z>Vowjw&1T8v*NxY^?FA=3HORaLAqi=x>w_0M(h)uaV+jWJENY%-2eXX|CZpNCf;~7 zrMA>$tgcZU#VHNwC!f3=|1xBs;NUa<7nm6N z{$9ub{%iVWwx9G>=nme^_M^=9bLCrj*lOVZaN5duF#RgqeDX%NU3G4;y;pej1OvQY zZM(Owz^!RE$t4`mB0T58G5*;l4W8F)p?~gy9)hMa1hu3RldcRg(O$gbwqJ+me?9?!?>v)7m ze-N%;XUKvBo}84ciwFTx614*6P7oz1MJAl4osay+m|(@_XiCEJvO%I=uIohOzndPS z*`T0Zj2D+KUFRfBk{_Xqns*iQO=OII|B znY7bs+g8Wcm}cl5+3GP})@?)UtX4H>!)mAoo%1^8B>3pWLWBN#NqtiSN&O9H#sX$^ zhAs1;(+i*8P_>#&DxT6a71?TcdmX#e?X|l`Pqrmh?tqyl8=VsPdW^3J`g2`J5r>Qp z4OAQBpF;8mHRr-=EInN}bjvn$L+Z*s)08E>-8L9iWmz+AMVi54UtVJ`sA%)R_S~xV zUp7~0;bB&FR^Q@TD4?d_0tU>aTnHIDzdrdopJ^iNs#ISWKf-*LR>ND1+5O1kRZLT~_U64MEORi@9^6R#9Dv;y_;XeQc2}CR(v*}NsoR&^eWlaV^gIyz zO33cAaw98!soj^g77Y4;?7r{F?h9FS9wF5xpGd|A@B6-o@dAx*Y zHH)`U=v|riKJkENdM$cCk4rtNbJ|y@?I#Yi^3XOU1>@t7Y_PYA#!u|aG*12rB-3j-Pg6vm6+rG;6 zxL%6={8kSzBCFa}d4Sa0#5G^AACfOu$3T6c9V){G39xFS#aDDE?1+MVP%~z~bmL=T z5Mbb3gdwI&U)oN9^7zFq{8z^Ms+0C|N^q5{;Puq*>OHQel6OJ^AJu5vFL~T1TV>vb z`kEM}4`8hvfO2*NC}*y4Zmm$PfRa<6$2SX&^T_!adRviIr6(zETVqB?p|(cbswOEt zMVF+~dGvr1a7?#(^hVOGkEuqx2Z-W2wUp>dqmSe)EORzd2 zA6$!^U<>FZ7M@Tcp}Y_@wjB(70<-?)r!|YQ2*Aj};>F{ipfLpXG9wt*kkc9j7&PlA z7;xAiw+Jve!zVw)PB8vgFu+B{ih=$}EFf=q{8P+{icDd+EUiWDgM9K@(iVb;!Qr1f zsRJCj>w5zyiYGOdvpkqokrqXnLq<47Jw&|5KPyC=*ME*Up!B5lxhQ~*FPUsoe!2K! z=nR2!=u$^+h4rR5C0+^!o){W|9}1s}~Dj$6_)vYwDP7?0t=Ft{M{W>V4+6gL*&OVEZUArlS8{}KhpVTzpJ@OowZ zGqOvF z1=)^I)AR?>*&T%ifoX;r!A9Xv83>js5Z_pd00tRdm9h>r98e&PJN_kkpBI@=3aMkB zbHw=+)nbf3Gn97JWBC!@r^*$J3{9y6?b}XTl?gs$ZFJ-urD{PYo5LNf2$q8n@v0tv zNERVTn22AdC7s~(syR`NOIBcrvQ8-QLpWIqsZTr~r<<~l=>+irYa$gE@5{8WYg~zI zg=<)E0xpSB9DIJ05xFVz0ioc0;Ugum@5IfK_LE=H`zhI{)XdIwy5!3(a-+D$1{pkpQl;S+c$ybrNLc9ww+r)BUfJ@&xX!Fe6#wy@FMa#TDeK zxS+0vA2}a!+9c92c>?9)Bx0WBpu-4OqMA%Nnq=QkHkiq#=BN5sTus6UbBJqwc1@m* zzQVFd&tdQ(#V%mvO|8`>&!krGIds@;w{-lV<5=MrjXaZ+?=}3s^PIPX9ZZ=dWu<^6 zdWK&Bvocl3t%C`x@k`?j#{&x&nI0MN^z5|3w>hws*!RAe6HKsL6~O;vb{zkSZ)c!_ z$3NpvnBG?5Y3;s6(il)khvW+Zhd%aeNJX4?J z0#J}#6+R%H;iFJ{2sFi267Yr8f}i!4@|-`a;HORth!0N=m6|EDB86>6 zzHq7%RmxhIuVy(7XP&l^s3j}_&JEc_x+YU7(eWAkkrP$U^oQk8%C|e|d2THA_sby~ z5X@scCr&JhImS^)?&#!{Qm_P}VKBSIYE65_*2^c*m>h5{J~m6hUrjYV zj&$gYzp;!AE=1?bOI>yS%xaQ9<{~!nMd542xzrS#B64Gn+2ZaAL`pa!>w9m~c^R9=?F0 zRObG^yF4N17O!)eO~Gw%U=qMjPg7wN0xlcRGms zy&5*b-uMJ-Eb=1FxEhNshC8V8tFHxku$!s>2i^FHS-s0>xW-{+w{c~kc-Q+ZlD+5O z1+SFst@32=_KmFdm0n*}{}Ur2*Gcx~4a({brjGaeTCb&a4$0nomh3$*j-fiQd+FeY zLQ^{(k({HDh)LeNyr4a>(03qxJ|BOQ&7Ggdo*Y+1vg8t;rXG&t8qN)PHSTCmfU%>|eNb>4a)A5-mYnazTz z?HrvJr2&2BI8YOM>R*#bnR$^A?9{Lp)IHgi>n`39?)kOg7HRhh zrq$xRzQe`a;o@zDi?`x=b0TRly#hKOP& z3qWxrwqjqAI?9m>0?_VOKtwIfp5yw0=n=&<}+Y*3<8DC9=gk3uvRiP2l!h)jQr_{Y;QRF0ndtkQ}rj`FX zU;JCVnlH~i)5A;Vh2+`?`+joJC`HDaX+M3D{rj)kF4}Wpv0xQ2JQ}TDIM%`5=GKIq zx(OV-x3TcPv?kaUsMuS_8;~Id{oygxyLH}sn?Rg4Ll`3?2D_zq4k)VSOOBh)o#XcV zFPSe2bkKhPl9}VbrKLIopO-&bt=f3L+e=PLm}=69S(8ZVWQmzj>xlnB@%$FmB-oh) zrwPzAP0%(R4wbK~3ysx>20H;$X#^W7EEVB#7q`gPgxR)(M>q4H_%=B^znr#>UxdB` zlm6gDwM{*rb7h z&P4w`oOWOM9=h-t^h0BUfe8fv4IR`_$!9q2@}NY_gCyv{xrNn*lPHh|n-57G;_zYn zqwo2EN68w%0q*9Op|j)GhjuJFittp5bqfMN4wE;~M;~AX1{|vGV?+iGBfn|Dm*`gt zN4FTuAP;duUyRUT(FcYu6#NK3K#NFF7&tuX!88KksxVAP zS274V#`_;aJ{AyeTr<2?fv(bt5hHfrb3$R@Fc?YvXbKyFBgSbkjCA6o2^q$*M9jU% zqq>!F7O?|5;D=!D*u{uXXTU(D`Dct#NqGm>me;p|rtqu|)Sw?cB)7Ox1{#QIc=6C+ zL3WTlhQWOA-isqQ4N(f?0N=$~B})q3HDEdscJv;MPmDwWYVdUg`xxxLNT@fAnm|Wb za&gXQACFL_K#MK%BL6F}lf_Aaq{jFP>^)tX=}be>cXUf5{t%z8Y5K8;w8I7P;|Od; z#)mv)CP>j#(_)N)$M2~z-3G(W?(uRgyv5byZ905j%=6;`7sL0AjfkfH3h9I`av_7B zCJW1lo3Kk+m)HncC}5^U?mg#6^f}?oG4fTE3qk!1MzZOTfNB$M+ahlRG^eqt3EfB# z^ZWiFac+dkte?^66V?hN$F@BXo+2Fb9lVd>HC)Zbd@fVX z3Y5;4TFKx%Oj4C^Zhcs3a>HDkEAUDbht$L+XU=H+3-DU9ktIKHQ!uShl}Bx;z-Opc z=?c0-wGr{~3u65`vjv~x7EBy+GBliN3HzF@US3}jEaW!bvR!gL{wHiX4xbmW;uYV* zw-E(m_lg$Fa_6Y4ce?GZ%YvO>k@!iV8GPqEap%XOg2)4V0Ctx-#nyS1f~3J=&p~N{bCw@#CV7y438dT}kOE zir!P?j%i3e+Ecn_*QB)9Q)kxB8`1mU!!f<0;VD;in5i17Y3rS~4U@1GmGvxDGP}LD zqMIh0S(EeD+jn2Tx;Xv6-oANtad`oTe#mG52E*GacfwgGF|!$3**+Y%$fe^_&tEtm zqe~^#)J$2kO`CQt$|M80MU$B!ciM_w9M83hIfdxXT^2gX^CLp3nh zWohGi9f5cdGi(liOwLZ6XO|;mCG}3kpr?gU#5F?A!&O6EUy6@cew=_F4xnFe%lRZ8 znTgJZzglowI5M0FtGLo(=CpMrbNcH*Sg9wsMwY!i3$3M*9*l)*Ku(o%$o0um9J1!| zdn4MeZ21wHCQKe_0VjxPPV*?a^7@WN-%*2bC$bDi+fh%};GY#1@=2yd-G~JqFhS0*@tB*)GQf13*hezZ#UtH*_`-l@eA&&ZKAKXxx%qwlgHgs3EU=sO-*}p zj*KOnHxJ);?pVrys4SEHB|I|jo1WPW>5Rour`mDTWeMUTeAPV?#a{Br(VZqKi;k35U(BlujwiEpA?77#S{1Bw9&MEtUg ze7R+;wz6INz|6Df+G)$UV1E3;=KzwuAR2xpkZgs6WGgpPw=Zjb8EB=+e_9758yb|G zigKf<{dTY4?zFUa_W+W;halM%nHTeBs+^qddYD~lFFQx+cTioxps4DCC4S% zDOG6KtAvVvL1B~oa$daXJb>&?hmp3+?xDob|1X1Vcf-u)uaY}WC9&2N5o z^nT5?I*WS&PH5~EC;85^prK$0YOidjtHOll^MFcmLRZT~oP`gaq$M5@K(`D5^k`Y2 z&lwD$T}1a>GTSeYwNrwml*SFsg*B^-2UpH^g^;2L0fj4s3ke7BvJprXU{ke7G>iNu z7sxQV275%rx8fK^4-NM=H|TFo#J3u4uLSzG8?c8fgfmXBIhKi{JV?PxQjkPS9A^T1 zzDmO<=3^pGubUGAc{GVyFp#;@v;}fodJFida#B8mX-Z4UMdRWGnWN8h=dIKsZM871 zN&NAiW2def-(03fYJ@qLy|!w!b6Hz8gPu#3;3LI8uc=ShTd)nO+uAF=q5jrmkqr%E zDc;)DzV@h0x!M-FMXrj{y>gmBTo#kFI30PFz4A%S_lMsmx5CtZtZi}$YkSRIbPM-_ zjU8lH3;fP4@QO|BQDJ~Ikb1d;vDz)Fpz_H1eLE(jJrJ>v(QzhbwcL8ZUSJuU+bx=pdcU$`+uClh@Y9L-#!Shr zkc7wfkIo0b*{C=y=|HL!hobKnFkN$fw!s%>kArB+wQjcE)1pM`x{lG@sl z)>c%xqAc#hKK80CZe94V9NjioQ1_Y{U`9COADbsGm^}0ojF8%tx#XGY1BF}9^QB_N zk295GlV<3@|C+Y5pxo|_?^(2Y>3d*A1PyXN1XBt=(_s8Ft>}9CjPvim(wQOTT3k?b z_rGEXbM;@1e;F~D@wY4(|H6|z%X^03{eS-zADHPiZ3%vdrt*b&=T*sl$@~+Sny@X^ ztH62@VTLhGJp}8>8vo6#UjX(Dp8t`f6^PjAFl9=#THt@&y5)9(v*RH2mmk47oYlR1 zDgaq{c=mw#U}oA$RpwgtW2Y1&c%mw={+fA+gK%+Mlvw}Y|N7sthh_<`1(@37pTJ%m zg1rdC@%$lZtBAZjfAjL{vB-S&0H$v5PBDuU@k8KkoQEUo-TQEwdUo>i{CNZ0 z1!H;;;0#c`jQ<%}{vAylf`^eh?G`zQcYMx!4xAy70u&RKqbU_a;CWa|8^#Ss1fY-S z#|bh9v<74OJJ(!%sm6b^ar{7)fKLI33!6*SAfR4xMB~3Z9~n+Ngg0Z{cEBjc%wdEL z1E%!}ki|s&4)Q}h{yX6*(?Dg!jX*QWfu6$%7#s#d5x4}H9S4WXTPz?GOcdnUrNKzz3J%fapFF1F4eokKD(aiKmbBbNQJ86-t)Jk=Hasm6Z6UC-DeI%I%2}F=Z zE&S#L2jWF6U;DD+A@c0q$@2wuF$=Hc;Env0zY_U=$qz>^xzBvbnTVO=pK%3+z#?bz zS1&xwlynB1^E~(nM>{-xIUpJykQx@|%s0gjHibe$Zh!Xj;_RJJljpd_4jI?n_v94z zp*Ro>398~W#tFb5&(8mNdiuOh8O{IqCo&meQFo=X1AW0^Ya%pWA~nC@Z@v(l@jv`j zvrX%9Zn*8`DtbO(=MxS3weP>*9YD^UQL|-;oU0pVRluxzeN^cuDmls%kU!YpNs#v&! zPQEcW-n>qh@$Su_o0r7i>T_b*vVj{1PyT1-$=^J-bJD_hq6Iq*TwDNiQ0J`&-^Mlz zJi2p)D823c%xO~N8uv4&P|5xDXHJnh7w-!oe>rmQ_{SoSOy=uNVg6=a`K5@zwJuR- z0e&}fi+cbE-ZF6DV=s2yq>I8YOQHtn1!wmGgqOU#%ZDT`o8|Mlou0QO6KL+~sX7y= zn1v%Q9`-m5XzuOBW^;&(?XCz@k+>9Ab$Bf2wpf<4J|BPK!2@ydKukOk`{wq4vfGZ@_pI$jvN$hfdrEc4_*EMxR(|eZ0db-(`3`;dlMKO&e{IV>+a>OfQ zv@D0ZKpzZ@v3&DLvkkk$bVcd)U{}<;iYBS1u1OV%mkT@JtHjHVq)&e9z)>WEqnZ-{ zcS0{`^bS`xn>{%FDw{Z1cr%|aIh*}<9{vzJchj8!k7JR~&p>zVcKov$<9$2~6n^*N z_(O&WZ-CzMX#5i|S1k;FO!icYIx{Cd>f+ti(w_BKlw3Lf_jGQ4 zRQeEk_UwO% z4NaD9h3WcgsW%U)L+TNJA2M51yJL1FTcJu51J*jJTtgh07rRnvonT|nAlIDCh z`D%`u+uD#w0X?=L!R4@p?eiA)Ib@_PIIgUekS->}BJ=fnp@0j*{Bpj4d0~F*cq20H3T1p;@G*RK zlzhKccaDcZz(XM5{6N6Po`k&bd|7=_z5%a!0xG-YU3?1Lq08@Gf)Vc&=u3un@A0?u`&zCmkZU;Nx5BQX{}r zV3*!8(pvNXx}YAzX-utv$8Dlww&*&O5REF?Mn?ey5n7KjvebcQ)%*lDLFOJb z+n(|uCU?Mi*Xco)m37vxUA}fYOx<#ii37!n$0sfy=YR z6T@WrMmx}`mLDtZX`o_P9;j%y3+?LpcDeb3HHbKI?+F8oYE)ZhWJQ7|JNYIVW*z-9 zedp)rdo1kQ_>_SgvSoLV3!3x#hO(>C+st# z??D}MLza4QZfJrf&+A{|5uN$O1knoUI#cPCPX(02H4Sg6_n!O~^|FPtxTeuP4Ci+s z9pno-9DNDjc;7pBgDi4FpI{I~Epo6hf*?+u(Nwzv#(SSsJe;$#%W@*^za2V~B7C*QWnAO7%%i$9Rx z`hiC+ak}E{IQ|iHGS-=EuhO*mlVdaJ{`WC3-EHjM#*|#&GB2qgc1Lk@^~g8y4Ccnc zjCwGm--E~nSf}lbu@MXU1TBB;45jRr;l2!_#~Ik=RoT9E!WbrxC!B%D2h}3yuddI( zy*R%D(-Gy&%l_JUi??#1bbC!%Rcpy!=9}>NtD}jTjpq5I2F`bMRqwQWwxw8B*D!Ql zQrZ&hbre(XNm92maNdHijM&FBi+|?`L`hwfkJ_qK6(+y1tGx=7uMe#|CEMoW>wg== zoUdSE{{=?JVMi(2(d)x4r4 ziX-MukeWbTftTSAP(-}CVcI6_7VsSO41(GsH#A@rO&Z)+fdPW?&!K<>Od3cFT2$0K zQ5Ht*zk9Rv75N)btCl}ETwgpwp1pg0@#fo$s)8mjK;4Xg^epy?O#j6FY8Ycg{^A(^ z`UQbKix&ij&xM%()tXo?Ibi)hb}Iz4EIn(g3$`q}a%FW^1r)CAR%U@MGZ{gTdx|Mp zT~)@AGL?4PO1op&W>Sx1>CA!$S9o=(ztYmysnAinv2A}ELs{NHqJVf)r>ErWKa$s{ zVmukUy~C9Jh59EmWy5w_Ccd9TEukFM(j5 zA7YtwfrU)HPlYX}1lrTO77uAx*GYj`%|ghLdc`IlHn*bSEXw3q6dgs!^E^;hr6A^d zI_eZjQ04@Oe1myPeszF0rHZ>GwCiN)=z;`|!9cdwlR2FL&5+lwardvQ_kEiA}4{s;9w zO-?SD%8*r<4W)O)Inr`8*-656uTWd&eaqE(@;8MT1u;o$G76$Q$4xvI_!J}}##E}IJOh5@K^b03Uk@gW{8W$Wp zyutVv>;}sQvBKbBbYOe<_$+ik{t;{Jh+JUj&WG0>5Vn!P9j2gS}WnuEiR$8TRD=)Lu=@NQ9FS zb6q|)3VPJ}L?-zuA+n)3wfH|Lz!f29AD198`C=o>U08SE#1(OND82kW?6xg+W;7fdQ!s{;0jv!e>FhxOM5*7!rFHlZ#!Q*)QHE@u{ z#q6$oLqQ3&A^16AL!!S{8I>q2aE^)y(M5j0|o9I;s+cHEHXR?W5KI= zp6%m!@nVJ!9#NPp z84$xmLt#9y2p=5RB?gWcmg5$-(iXY7#f5Sb6J!(vHPd-PS0joAyWo$=$3!lRe7I%M zK0kP-BIXT9IC4PfMDmaL1sDMv$;cX?n@7N=#3Wc>kdglrF`maiC{l?={FzCu_IrF_ zB4aQ7AQ__=b0ae*kxv!gyDl|=C8Aqs>H|zSD_U4Q$r9h9gx-H*RzhM%4Bvl0vuIqj zP`1+7ggi21yR_T%!eDs32!(-B3qG=tUDNQB2~=2&xll%B(6E|V9LkXn2;fI zE=0(iD96c=IJt4TL1SQGE}v8~&TG>VS0iR3VPX&aOLaKEo(*nIi(VBC7e>xuQ7ugL zc}ru=RHxPP|6e->Fl7_;bpU*K=hwKpJh{w={?|m~pNam5hFau|p{Q~dR;Yc$3)(PK zsG4va5&eO$FsW~sf*Z3(lr0Ny1Zy3)qlo3A)pOyFXK)aUfDzw_c!%tRg$T%&to5WL zx!0AN?Y6AlsgkE@RMHkiMCw}~y6-#Ri91wY=Bz{Yj>4p_O>Mnv*j=?Jn_bH^vD#Ui zwpR?@=e4A!M)Kt{mix^d%l&Q9a<5*6Zn>6wK}YXiU%WWKeoLN#_B?;}=6O}1|3-MF zPA))Jq#pZ)O9%Z?z8!(00`42aPVzFKquc7~Ds8#f>&R_wCe>l{1UoFjQ!K%T5>lqX zc9y9#MQPKn)oGhdQVm;`WLZ-zovAA8&8YoXCs#=ZFlc)O*|W$C5c_~y{799tFT%Wm zAr63V#b6U0hRkszB(Km8^D%cGi$Bt^qB6%8lmwAw2|8AEOS3y>k4bIYRLw5Jbz7|4 zHG9R5U&J1(n$EP79X7}W-QEWA1V4mZ&;PecCzJk*- z4erSs&qctnCiyy`pB#4~F{Q3-bfG_8l_W!_u=0VRNoi$Ho%Q)=E&cGY{8J;XZ$0){^MLz^d(-kLOsIVmsTVX zPzGuhw%BeXa+aq+EpCFRR$!uSr_`=;;L0ZXEOXfS6DH=tVG;gdR@^fEBc-FMy`vRY ze2tN9!mfDqmDqr_TxL?vb#^u$#Pr<- zG2Fvl%KffsQd=`Dh3c9l_iU@n^iEr*9SuZR13Pcl#Cz}gAH1m+y?-EEwNcU3_i01) z=!YL(eXxzupMLjhVBc9kT>UAGg4_4+k8;e7Z;wD>`<7$+VJk3O25_3=N6e%x)Egsg z6))mG;&wJ^U5N1!t`D!JF3+?YFpIkPCKnx?pwg2S{-3ti>GUK?pFfc`w$Q>rHfHLD zOPhKRwP3da~xZlrEs*87Uu>b2W{S_fkU z8kC{U8(HaVQeW$}^j`abF|vOcBMZWqmx38t#Me7tm9!-3@DXBMr^$!=L9^>=6yRme z__j2ziv_e0M~5+87?L%iS1k;aCobcf2CQc=M)JYx$7Mn9RmdVsMN!p38qwz@c6#m2V7LaID>* zV=={Uzh*Z-6>!hyI!RgFuX#;(pZ%KGGzD}4++OSzC-n}tqlg83-zu5ea%g*#c<|%- z882TQO`n{_=vyi4Wez-F<|pA4J1(Tae07w3zg5^bpD{w;(Xv2%GuRWmh`zUE_DCM) zA&KEc4n~58`QY z%e#gzc>EZdY*UAGsU})m_DWlwjUFoNU~x>+(G1WMY!QWJnR7+FjwY|ZQBF1cvbwe0 zY$-$%J&1xWixMikqgxz(bTm(Uw$;MiCJ|732zi&$Olt;LC2Fy{#TqeI#Wpq?YE{*E zt%$3Ivzs3zh2>!91WHv6Z9~oT-;-D@{4sA~}Z9dyT=MTW38ywXV zoc$0Bu>q0TATb_8mR2*}cH(riNn2>y`|#~P=sfmTy&wp`Y%f_Hi@(NRP-{FJ66CrQ zSxpd24*{9)Nx8Vvdb5kH%ZMxV#)C9{3%U2YTg*XHe6yIg_Ik`1L}D5~8y70FiQu7-y; znQ1-CpzUsD1va0y;t@wI&^t<7s;IcOu=PX5wPMA!6W5^w=wuK(E)5#wn|R>oE3Qqx z$<|xTzm~7K)+qLI8XGhur_}S8s_8a;E!(~_x1!q#dWgn9#^{r(U%l+z$+xJlpb^ZS zOz{VXPKfD|JRuSm)$jw4Vd3b34+@HUA*!t`-xLayVczS&^$k$1LmIqiJasnqhSc*g z5!4CHfZCYG7!m`sWkYm(4BUG%2>cJ28EN|>3wA`qh}<(8h@?}VVjD0XKMUF%lVkmK z%yt~bm~(p~5?ezcC5fD2lDZ0g;>d0Ev+$j>Y%+X6C?j{k{nj+&7KrCs1itrtygnr` z+0Y3I9}?!K67~_JkC10!NHCIsr}6UHM7J$MUOJ%siAc&F0<$_J7yhEw9rYqi`5_{$ zlfYtxNg`{+X;3!G2BE}@hX(WjR5kq}COh-|<~z)ZMu)0}sio*gyibx+9VRx6_6dmJQpKVoL_Aa;|-d3G^qa=s!w9Xy836uQD6 z#3~b>`x=OJUBOfETb9An#TGf(%*NFGaFTf)L3{96fb>F^iaoH-Q3de6ef|s26CO@- zoA}EC8@ZG(vgdz*?cqKw=OdnIjEmXg8ST&`u+V$KiBo|{M#~eQ15kHvv6+OkC&|F? zJpS#3MKC-11{uc@xzWH!Vv!nzZ`W4r^1l;lCMajIi3DsU%*(tx2Sg7O{-OIMe;gS0 z8reuNGYWy=+;knLDe^o8n!d*eVXR<^`LP#eq)`w$)gppcjg+#Osm|G+b3GSr&MDZE zQ_lK0cabg4lk)Fq;P^3o1)PX~0&`w)O5lJj$L5W%0NvQ|K16IX24&plR8Fav7lF3~N+<;WY4`S4o6AJ7Rc&|0L&Q-6n&eeJArgh!I z-S7gW4C34E)6hi)cy_e-?n|vR;Vj%xbE~ynbR)Y_I{12L-rX@*>Nqieg z&2yMoz7SXTs^zWkI??zi(?g2z)CX+2W~r`?m)V-ZCKol#*GDMN(A!V;|E+}%oLHkp)IJ`!{mn?wg+3^sFPbB^|-%4XVOTPrnS7 zdjOpDa$u< zr>{tTwcAp)_IilQ2l(W@gHJBpK`aB8ypU)25H5Mmlg$o5`-+>946twUR^+(sb|nYx zn{QkBpYvdYv-cs#vv(@lcFXRJdo_`M5BC;_dyB)p#o^xKaBtyzt^EUz%{uwE1>A8p zcVsE(<8EB_%mJ5Ne62J4^1Mr<16bt&ta2B?Dwn)#$w$QP;)!W)D%ZviPs^SJ(lf-O zlgnn6_f7=YKUJ{ryydD6zi$`lQ{olg!vvFDs+w@9mnE!~U1+Qsm|x!9`uWWJeH>3t z!Xx`Er-m)y< z#aG93lx*jqWLH2tPW}AiQVYHONX-Oi@HQH{rP#(>u7FcJXTF#9V6_~qmdCVOHY+om z7MZ8IrQ0H>MJWw7yLlZ_GN(|_T#T6 z4`rNw(@sRuKM{SD^VoeA&<{6g5dK;H3;DAS|90TtF8tere`N{(BjbNG{15zobAIxQ zyu5nz?rn?T2wjSk@Y}$*9XtoZN$Z6VCn`ANw$SCSwLm*2ePo@JsUvp&V3qR0W(RS@ zKpbDfk#2<5AuIWc5B1?c z?dE#ZQrDfsjs17m?y&9 z;<&qOjax4tcP8{Kf?*H%3`~ERL6sk&OXLWBFkv%bE*2dPkF<6ZpKtOPnxkZz%@64` znX_6CW1yhPrGc+(C#^3s83Imq3J1tOpUS6f<7?Z_3|?YnQUxeAg2r3KjeK3p^nN1bF?7 zyUy_*F5rR~Qk!vKFAoiZOSAkbN0{`(_Tz+Al6+a2YKZT*cwhm?HX!i_u> z=N%a9B`^|}kB^6aoB(hYy8qA98xYj-EaJtnLA}D>e~b1J4jutqc{J49z`3L5eIv03 zuJAmc)ZP}~ymAZ_Fvg2@!xA&ts!UKaoRO>Pv_6+r*a_z+*RXDHMh=g8d+ifRCP&;V z>7)Lc+A(;Kod$FHpaty!&N3KoH@UU)Q@vPz?Ul-6B2Z!j#h1MFZ=_Q4XE_3KYfn0QDhw>_`8W^Fbz4%yqT6+KlNh} zRCfs;&U_Ode|0oDHR5UWF_C{%iD?~Mrc9RPj%pgFDj8i{Zg=dqp{O0J68VQ(meXRl zF~;442_SDV)#XkbcpE z9o}og`Q3t}418nw;ecYWFkx^qj_D`IC(%c=nLJ9Kom^?p`vlbHdnft?s`tdylihIQPtSc&>2j35WWS7ebP5U$qEzQ0t4N1!du=iv`OyK zFoRv;KTal#hw%w`Rs}oHH*q-lFup7zFTe;CVdr?OA8-@J1k2Xul1yEEm+zfZ!m)7b1}gW!$<5mNx)_@im$y+1-)#-1Z=0ek~A82pxO zRCgSB9%&F^gafEnJf~w9eB{%ifsW&&`wv{E&_q)mXEJ4F*gJt!7y{ON+_3i|APkNS zw`b#?u;B;@<4hV)&CCcFb`oFB7m|I1(P6+jPT?+gJzy&X&qeoWOXHkE5s|@3%A+qZ zyyO#DN3bNfz^Jx}>rW>TIN^I-yi1C{RbZ$gr($L)J!sA6(Iu9Tg2BfjR3=FxF2fiO zBhY^shZyvT`17!m6EhNn%SY&zz)=FZ64M>VgXUY}?7rt=HE^;H1`apLT>k?|-5|gy zo@Cuz*f~SK8nFC#97Ug}M})o`i-GVIJ77f&a$!iO0R6W^ClaqsrzJvv7)HHE&Om$z z90ha_`AY%&A$9^m%Ta)xiTEDD <60sa((4$e50x6}aPR{%)>Oq9gJjC`PHNakbw zv@5YFzNR6<{GX!S2sS%S*720f$kFfcBRO#``X2X}i}yfI>db)rOoO1&Vj#t9xAZJ? z26E4L)qDJO;bEvO5Q|r4$OG&=5IW9Ux!porZgE2Q43cHWV%xIR?y8LI_HNhzn7Z{Lu!iV%xVIX!=UdT~Pjh@UQAe|vs?bMf-z^y1Cy3ecQKJIySCq_aUq`RF8m5S{r1 zvzQ~c?WUsDg+V!Nhx@Y~9@cY<)^oLuZXOhkc7UQ;%Gb zl>7CM=U*7qgSVM@_%q#2y!*$++krW4s?#=wYd_Oq#;t!`xD;3fmeZ~YmUHlBAAH#d zU-tg_viA@71r<~l?`8LAH0754npge;c)#YB$l$gB53T4(E=3;N!2Q!4SfI>LTa@s2 zK{^2l7C82z}CHgdAudYiby3pzRZ@+p#P=i-6o_z(I! zB;0dFu!}Cpx$FGc-pC@-)WMDQdAYIHMVk7oAZ_ODa0k0=m+i6x+s|Qp?ZEc?$@T{< zp&h{znz!*hDeJF*m9q>dZx%J@aq}Hz)sWjdHCdNgO2^VPqiwb|+R+S3ZKhP_JI13F zX%qfxF)QvXQm=O;t7=cJNP1k@)gkGzm?QXGXE^@JW1q;WOTqkZkZa$APS0Hj4oNJ! z%_j|({g}-kgx1)<|2l0hpZqv4jqnVyYVUkE9*IQXI55V4Bh4oH+IPo4-$AeN_o)x| zzd?rvt(OgW?VApVd}@$)fdQui(VnwnbA8^36AoFwf5J~HaFPjqEP4`>Kac+xXrGWo zY&iZ~$-B3wf2r5OsW=?*ds+Vj9Gt4gi=SR{IN;?70qw*%2f93EXHzF% z$LtQymv%t0ZU_b5ft|`3#Bflj)`T+f2Tx}o@OmWtc8B26-6i86;XIv=2N^yyy>!~; ziIcA=xpPA32>4L6h_LFHaJV#`k>m4Erz|k3<#Wh&62>Ip)m9=n8pGGTKR8)44kmx+Y;`aZnRrE1+#-bTmXfm;p;un7)UR0u;{j0W^#UT7-^(vn*`i zKIee}-fX3#j6*TLFlH`}4&LVlZPIxV(}|&7>LGYM%shdjU;;BfyOdE;Mz(l~a<8~-?TOkWTdc0YLedmP6t zP{T!L8vh)E;tG*P%$QLyGI+@nvMgk;7qVh%x>bI>Lc= z@M}2!#q|Z_j#zMqkQg|u(1G6~oEPY9&ROU^LKlZv3AAj;)U=tCg{V}f6eW&$X#8zL zRW2;oUHh50DYPusZJ$6kf7(LmL6!`Qqe(e9ZaCM4S!6bWGJ_KHYEvIYpzS z@XOyPgK&eV67;Qz8rSE(vuDfv!bf7qudXl2p9oquJYfQIn zS#5WEZC&fI)sie9&y=WK0$D28hnX@|jJ#k#Z`5Z#x#D&P`7H&l7Wh>em1}U!;!Mk| z#69NYmk!?H_;v)k45)(OO)mpFx~-m`uBo!r?&vd3ZHhhzIv?n~<8lOjDNhcH3Z7m1WJe6{)ygu3)x3U7qy(j$y(^SFwA#DoKV;b*5TMS23ui%BG?DlGm)6O#-^r8ZOyP0s%w(mv#l=EJ8hYEG)?VkX4jlyFHgWi zUS)3mXUxhCTa4@Q!KU}~`4-^R{Fr9}Hn#2T@yObygj*iz6!Pd3t(&!++t&W_#OIj$ zTs((UJwGD!jyf);=Tz|O8=mLmqVI%?8qNHdZBSo$I}~Dpv*g+$g9(0m^X(B8s`dz} zaU_VE;nSH*k0Zg`Ilf+64Q_DPD(B2?vY-|MGi;Rt-E!y6$JW7rvoZh8)-m5<_ui`A z`(cl~4{sZ6o5x`jw#w&$-?AWZVgvXsir#|Feog!qS%x-mV9Yl$szZhq#qP+{z(tWkKA^ z{vnPm8QZvDKo_-RF!yWPYs8A~7a>-NY27aZED?LV01XB<#=@+xN62HhUt)2u2sU%P=EKi7}0UDD&qmj;E+> zxcW}9tfR#@mQRa%@yL#rok+_ryCE-K+l4ppRq3CJ(|)-GY;^MHC@~3kdc+Lu;uuB` zk$#=rEgom?6~8)4zntct&bv`tBRP9#U zwF1@FIP4rx;@X}Zoi0xeknjR2<^)!(Lx5QoBIvLa+xS!An#{XEJnY`f@7_yq-qmrA z8-*)+sO@{v@E^zWE_YHE!nt?g=vgUoYPYD7%EQq27}{%<2=_9Hs}bnFrscfaC^saL zwWoK$E8AzhvU&UZBo=piAMgbB+9q#0XZN0xIc~H4us(|0Jn`OT5I05b7^Z5G1qIyPzG zG{|qM=^K6?ur2dl7NXDF~mTJXK#Vy;cz~> zy?v zO2l<`CT)oep4q0j0|Z?~yb_q7I8kn)C6F2_oW+jEFkJ7J67=eDjPXGX%fmkmee5xD z3*qZD_#pnqY&!9&TjDSAJ@W7a`p%d}} zgC6h$%74G}KRxHp7~lIo_QZVPZycF?YacmuEE9B3-Gs93U)yUG}Mm5Hl3+Ydg`A%|ae9f-JKVukOs zw80Qa9~K#QO5yRk$F;Q_?wTAR$wpvzvcyN34x3`gC+{FMoskO@16sLM3?Tn zN7o0&#l^mZ*#y`p3=GD=qfEsz71)|PW6_;?H~!f(WBe<0*l-l!hy#p!Gns|GCsanq zq^K6v;6bRq86$i6oEmn@kD`+y8zy~VK&$Y^0`E-Xl%{S?U=buwgfbJ2f5veLIWyru z=pPyKUvP;d$nyh+rE!>Nz&C@NK zLVKHlD|fW6dIajqY$1oYG*xfU&*{sF!NRiaji-Q8%T*0CdY0bSRKx0at+w3LdrD90 zNgV?=3$@p?1{ZdFtsP?v`r;L zgZ)YG7CSx@r<0npQIocqJ5w8x#mHYhfwrea%wt#a5A4?AMj zj`%qwx-alRZ6MKIlUkCjteNO8sZFVOBkBEir!TczJ*9g{blWqod0pxTd7`WK%Iaw$ZVBy4;aV(_{FxZAxmGUuc4i_4~HeX=|+8 zHT9mZTWt^^rEA)4&1~D$Fm;=^RdRd0XatA6nPT3|X#iv&|6+iGc!2>GPJ^6VF-7NY zHiahhT{eY=zwQ6~uhZA_=`@XEcb5nSJ%}T6O114@rhBSE3BF7uld)M@bEZ_(z$Up&1*fIUOuL9R16!l%r#HQ&BtlkT( ziFv?`+je!HpUj;ANqp)GpH1M@oDemcBW`CAL9XVc@KJcdaqA~foftmwVsxAM<6xah z9O4HuX^bQjcM{kh-)oU;F5&17qM!(2e+=;z4%~{UF6H@Fr&F4Tgu>~^4+hkOHo1S6 zd~l*$GC*%9hgSCy(gneW6Z~^=hidY%U5{WA4gTvby+bFVcn{Pa(1OEeczjX;M`9Pv z-M7dMe8ta;0#$=g(1r(L_&}le1pXJ_n#_s+$OzbrNR^O)QI%ux3>SqBV*4ikFi1O} zQV@&^D;8dG?k`KOX82QlJYXFGJ+b%PEZ`R>f^Hei(gBT{H2iyJw8(|{JGojWBMKwI zZ=8U@_6z5)gvfj{pTN>UYE5s2$Or7|y#rsuEYTyL1fg*Y$U)>o!^Z_-o0aBb;54$Y(~Iyb3(ZlO-23VD_>zKk<&S{I#0QQaPM8jPI2me-yrNl_ zK@YUgeWi{-nna&5Mi7YT48iW8()B!yvw7%eglKyiz7$*W9^K{3%^!Ye3*EDexIq{Vx_g}7oo4kbRXW-<9`I^O)=p5Bs7dENJ1P)-dpWR^oceq%%vk>=y%?8C!hTI4Lb zb{JA55pWb7B8EN}Bi{t+4jnG3L%vS@E@lKKSv0*TDdWpD_9n}XBV0MoSs@Jqu%3aL za9rbGxTt4oFcHTJQwU0lIh+FAD*3n75(Pz^O-cjtMhNquTS5! zcnOY>->^PqL~tbzSZzvFFMCe$qq8 zP5fy5v(4|WPC;(mI6^;DSeX!Q~&Sj69v1h-L=)O)U^!O zm5nwMN@qQ*ZE32lb-LMx>L&_5qzzv|bge1%Dij!3O`vtmn7P#oXs5FdMMBJX--(V5 z#x#{uRklpUGz?AcnzCdnhOXIE>niP;#I-jkzilQeBdka&tgDV1!d)vqUI813X9m8z zURs#t)9(|V5F&=VrF9g|05Yn~v^uh6*i_ZJjH#x?W=pLu+I`B?!5&6MZFa3To(iTZ zQV(bT9sJ8$sLonFyiiCJO9&Z{|FH{2gVT$?OeSX8CEs!7y0KqV|^z$y!D2Z@2n+0$`2DR6vid ze-5{LpZBd^(Yd{gaS?M7R~|Z$B;9NmU-2KxyzT~O#a!F`>1^U<@ftQ{%I-9dgU#BS zZR#D{YTF&#mL#;1+HI8@-L@=M;yA!&W%)p~kh(y%zAUM_ek5x${C^c*!=j${3a??r z$(@UK|AxbY%!IaQ3Ae!#sU{uzpFY8n=UWHJ;TOM=SypivI>yh~I)^j7R?PibOO97Q;<6;Hy!F z4e%DW^%gOyYqq8+o&1iI@&E&{Z8(-XKk>r8W%}e#4hT);w2;skl;gN99*ZLyu^lj) ztRzU~rQ_dG&vVGv6yCto)>{Y0>Nzbwnrmm^wff}Ge^DCbq}w20>kT6R`7ceq17C^u zTVP39EpkuHI0*SyAUH0}J`_!ygDDHYyAI3(mg=Pw`1mV~1`e*N-D zRg{FwCN}~vug=fjT%&V8v9=r^jNB0TCg$NbMeMgEUIgqRI)onS+~Sjyr^vH+*C(%@ zr)e2pgywuC^Z;;(Sh2?JV)E9L2{op3IbEZj8mc7K$pr(`;Er&l=lhRV6TGnMzdESADX)*VPn9i#tabS!sOu`;6}h8vc;O*l znA)hjmyw(YRkbMX=~hqg%2bitZLQN$+8T_;vTVturd;e;Z0~jJo08Y3ufX|m{?i9=qC1&%?xeRyF@`jt(O~our3bPMTK?oS!5f}ckir}ZLIQa zJp1M5k3@A+f7dV<@&N{GwDYjp#Zwn^ zw$kgwy?i|DqRz+VO#E8M)h;b~BlBGTtyxdkyr&Beu;Qa-!0k%SRNzv7W${)y`{}8i zyz|Dip6_>F%WJxXH=lc!wD6s1!A@gu76ez-dFz4n8tSag<|OPS36{B9`pn67ph{^V zOqO{TWW!uU)OD!m7pRf>dQ<5zH9e*C%Z#pjGgZGDY4MtB_RaL}Ec4421;c*E=_tou z?7B%8gT{Z~2oFNPDa( zG$un`BK71g3v7qGYfYYNIY^L_uFg&b3Mh5S z*Y8fJA-BtJYd*cqCp_$AQL8H^@`bl3R&R?! z@Af1*pUI}#?wM^>(tB!8*EObFl3{jC#pr?xXRAf$FC)rz2)T+Z*Nu%-q?RU0Ex8Rl z;Za+a^;)6hhqoT)tqsNCPd%!@*OJyVI?V32C0jB(ZQ7OF%HT)659{4gI)_x3 z{YrIN+D%!V{Ia6&axV+()jXE(K!LZcFEo>cvB(d4yp;QOLB+JW*Fd|Eo?qq*-mm%q zl|JijsTZ)2TT4TuCtl|{2>&bjv=1=+yB7C+fZ?y<;LcwN?*wFYiO-`fnsl)%r7FT9 z@p3E~-q$M-qWW~a$02`pj@X-xR<3A%Jy80UEv20hplysCs+23aU7*j@FSvhlRE<`? zA~_FySgPMw3Ew{G_ubd;I~|``LWw;^^jO+eQ|h)Wzu8>fRNM)jy6H(*Y&>`YeAvgU z^(BTqM(ksBT=Sw|KYDw9_2S~yuN3L*9R729a&z+P&C8S1i(j3pzv`*4;k$GE4~Rhe z%q#p0#NZPC=T}PCMhzJF@Bz2DUn#woa7v^j$!pmK<5^RO*J^!(S%sflddZ4Tva&Re zEsgK)*b3EHuWe~P%g`lDk<a zH-{kZVi5P4W4@=bk`3}Z8r=r>UOrGe`(-vtJAJKwJoei-7;rQozh)f*W-;#*On}vl z+P=j~at`%?lk2^azg(^N&Y8F#t@WrAn$E~|YEQrih6}^mL%C#@5io zJ-uZ>Bp`0hiV)L*)olMJSb|El_Bc`ws=2(Ejs>Pa!P|%oiRv@oa8#cyV8()zmSETk zf7D%ugufYB-!)|@S?KDuC@-2JxRw|4ni7KadG#qS09eXv(l+Ldo*_!nuR~H0@|nsQ zNii{mZ380)o0*6$8Kn(2CX0nt$0+QhLD;}|kPChM8%H*R{ml>k5&jA%2xoxB#w-{e zRz~AB-ds3^G@uE`V&avM!f0GJV7x*b?5iFd@ccr8Dg9B5XjGB^fd-->9H;BUacDXo zi;W!N`{yJ`8B)|dqeG8tk{0>K{{ZG=&?un`yl3+fW6;e#u^gN4G7-8PM2R&~9Sxcv zr7uCph7ThMaoi4;TjTYPNOF4% z%oXev@(OsaOH;=BM6nIdYZTdH0#7$tUMIt44q?RR9A-O&+O~seNuJMv?NDVAERTjP z2D_pI?ZL-i`@z5;xIlJ>N7|Iz9KS{WD~9nn$YyLXa41sVNx9GA;a8cKVc+v3-q)|4 z;gF(H#MR1F&Ev$f%gmCQ!u!dynAbgwJs@U-IJsvnM-UnrF<>P2C@lXd3qEWVBwHPY(E6?OW86ZCUPb3 z9~1z5a^fiRdFy;?6KyHjlnIi9VdEo%X0$J-KD-f342UVW0$tt$?+GRan-#lqh;N!P zu+sRSNF3^#u_p-12=I&V!6Xw&msc*vL0=S{8aBNunOn!ve~l&M%rHXX5@!Grq183%XCvoYDA~(20^u z@Z<+>X`)*u_U;R%ZNA0Qz2Se5XJ7z;M3@dfn_iRvXN40WvS|GOv-fU2awAE);QRR% zL;=$);Y2Wa4>DVojZ|hz%IZuhSE(xX^mMP_Bt0T2G012#n5op92BvXd_B@}(0`oQn z>;hX4^)St2B2;N7*FO(D2j(s8ImEsGe@6JB{NmFaN+6rU{-1~XKR;wPFC2S6BV?Q0 zuO$=eo`uG?g|^D~J_{}MKJbfrBocK^h1z;wA1KCvP}{L3%^En4s>rG=8_w!AVj9)5 z9ae01hF12U|MIWXdRa56<$hN(x{5j@PYtvaVGm?(z(7Vo91svWewga84bja9VAR9L z7T)j(t1{^-0nf4|>bjw0smGoRQqPtC?1R&^Im3v@Llcsd8UYHQDal({cmLeZqOd&IGi01TwNw z6kq=o!<;By%grepBbx@1^}elGmO}e3p{{E5`wlg!HK3X_m`c5Y)A2v>bj(Nx%zcms z3&vU8*xvKIOrNWF%;WAqeBd41lT|;*#MEvU6IYcT+n`Fl^?ke<0%<+BVSS*hK(K+M4P;7yXalt` z^`(Jskv@S5)f;ZxCG$EmiDRm!r1TZVuoSruV%Z{=VmhWnh-Inue!dUJrl56HI zcDQ?*e|#Gzsp>WieRnOT0yzC+Y6r$ZMOTv)MJI+qDFNOyh_305Yul2f5J$467N1!F zz|{W>^hR5fYv5dCNdiqmVrkuB9R}L`!7?M2rg8{Y^o{X!!qs{@7CSLTq%- zBWd$G%x-uY_#tt77h2a%9mPFU?vC9-mj=r-&pLbehL~rA$#@Tx3(_Q=h&Ul*fTAXM znH5+`Za4sSyf9umfA)qHhnahSnS9GTH7RdJ*J(NyTsPnK{4T90o2ss>D=hGhj-2=` zq-#Pe_S&r-A{$EFU8MHl8|J?C!Saj9B#EfNe_6sSq3e>S+paVBw@G>6CDR^hl1M44yckEbw zP|D8Me||gu^REFKVosFUTF?-Zxud8%vMQ>5c}-{tN$pD3I|+d%b|gi#Ol=Dd@wlKN zW<@wIQv!7^Lq9_Nhc)7qIB0(PQtSbsMC5^(C5VV32y=HTjB_5~gS=o}%&#(HYnns0 zVOx(jY>O}U#$&g5v)cf!*#TT>0h7lc#kKhgxOWXXQ$9W)6O>Moc<{L9RZft2Tr;bh zLXT^1OpkqJQ_3}!$2G4aPv&vWt0~dwam_GKhQRhLQ{n}%k+^9-`m zknR!A#}>TxD+6y`GB2zO-r9IUxp4er6GYVfV9V)Z$0IWQ;MHZ%LV;{+Ag=8eac>zY z+OlX}&6}zgOL-%ZT}_dZO}EAKBQDD*Wm5oLs#%*`OAP2EMB^$$Q>_?YngU`~npz>s zbWL2XwFW;EZ)@=}tUT-0@V@RY_@F&vVA&MfB3hs-D?#4ZCNU#Ia~<(G9nhJH$?VQQ z4FV9duz+$jK;ywuN||X8R`U~Z?ir+7;~a~WvL<;vKS8av;r2-IM!`Qf&iui^Q|2+& z{2G@BS_|oBST1Wy*$6*DWa!w{q3e(Wz(g>@VhJ!{#h^kSktZAQUuo28|SpUQ6$y{5-OW` zZ+_-x>uFj`hDnc;CG%^xM`&d?V?z4_+|BhdjBnbUY==BpcgxOc_z|CBcT^W*dx(97 zhuC&Q%qWyHE98H>Ar{&YGcy9$vwY^Y`-nHhZRtQ;-rty4eQkwk@h-NEzbNBx8F^u? zW)1wRn`Hb&dR}cmENd9vS$^zvHI!D&(Q3)ya(>Mnrjz~Wzx?aQbIG91Z6ew1L^2=| z%Ob1gSvA$jX*wJ3`eeAX?e-9illys|R77o?_+MSeZx!riv3^g>JrUWm2ChOR&9E$4 zapk^d5qY4=hGb}t-dOHw8rEQ;R*1KguG|>pg&lbx3A&#L#O_N2i>RtUDxCuV=&`5;yD!#}x53s@0&_cT=^XDlb+GLUrn$9NE`DUjIA& zB4mGbgi|t!JeXdsuk7oo?v8Bo%U^8(PFPX@A{NPffo!Hd!CPwp&Xz_$MnFZfH#OFCtfpdus2lL`|JzSDNH+ER4K(1Z3ggqtYDmin9)b-GQz^h!0655Fv(-e@dWJ_P-H{QE!HJ;G{(> z0U83i+qg)=7?cp;qBs#ybH+gv0d(IBydvdfb`Ws`)La@5Kvtp&0tc?(;znb`8PSih zN|7j>C7HRwHermG2&@i_Cqh#Km~a#TB~qe07#`2;8)OF1DcK?u7;fNtU&u$wEExgX z6C%z=afL(J@UH*nPdpg@kOt5VCnXycjN2>8N;hxQ-Pajpz^aeLZl6YD;dE)gdX4{tPr!F*a`|050Ml)FbKqLn1sj- ze4dogFA4Z;@b3_a5(`i8vJl1D)l*S8jD<0rrC=I`tpD=Fu2PzC!3SLr7=IBYGH}iH z-*C$b43x^lP30%ZF<0rBfr)AQWQ=wX1@^&_HnIu~gs~KR(e+7q;#Mw*nZn=}QQ(;RE9#yTfm2aZ&sU{x0u0&a$+0l;Cy( zRfLh4i%J;h6?>|-?4z-d8ucXO8JbI}xIXNZpE^Q}Ya7vQ$RjVp%`w1PQ~nLx9gNNM z4RFPFcmiL>B!XoI%8$`{#tQCN4B>;aMf5rM;hGZ^2;&5%%-9ozRK!cGqTxFt$o(LW zCteK32ZQfPci9LtUR<=xajL#3ybEy$VwcdnOSvre;!#Gr%4XBZdrODse+UMPFjf7-jus$9l0bQa~#F5$vUNU;z{@ z0!TE5K?Prf6gdz0xA-LS8Rk(+dq!CN+Nr1ylQ8rV7%EJVaNei~f>jg_68wctqa$E8 zflu8b#j&#}UKl!_a48T~#8Cx13^6JU{3|9Vvag8`;|9)|2nmvhtx`u|PKF5w>ca2X zbQxwB!b%Z8#>Xa?lrq|kgeMvKP*kUX{{8>_`~S>E?BD;7?o&3s{I#@k20F#xfD`qQ zjTyMzE=ZlLxG3XB38Itm@c>;W9C@m)b~D5(w#Z{LM$oR7bD4Xx`04TO{KgN31t?2aD8v z%X4}<<+5)x@X`M;w^Yj_;OeNiBK7Pr`~>SSIOk-XiVFPdzJk}{T|?+h2z8@Sgss)P z9Cw*ZJec&7Fz8C-r->A(?%VEkg-*-A)rV9U`Wl6-*9shF;h9f?iZ`Lhm>4 zj^4fgwr|M~2>sZ&_hsc+>fq(zgCr?CI|?1Bn4CZi-Bne$Z|j<=E3%|pmP2I6U9GI# zLor4dTEn#~mff1aZi!-2n47vo69%E&i$SRa7IETU0k|)2#nx7A-Be+bEfan1O!Rd? z!q2&CS_u+f!{`cGekb>L6lq5_MO8P~Lxh**uA;t^r5!|>*F>|w1%!WGfbdJ-fwrT> zuXr)~D6!)2zm~fQ0R8PjTC}6krx(B_4EoyT?{4EQawYw|+PG3QUflSW+Hbk--JNxj zJO}$eJO1Wk-*2zHwpU);E3fU9*Y?V5d*!vg^4eZ`ZLho@=9SmutG&?}#Q3B5fn%!C))U#gt{k?+I+#;8MWys}Ah8|TD7t6-h)?fWq`>X4g zcXlr`_m*f`_>M)wq$>Mzp%)$z`qRuyuU6m?S*?+6Za~l@1QaQwcPt%M)Pyv@QgG3l zup=uDd_ySnYs`3aa3`~NKqh$>y;gpRfe|}Vxg^B{j*+;tP@Q|El0Te+DBA;Xd*E#k zyc<67Hh^<3_m`LAnd4DE6!-mJ`q%0C_%~;4K;ObQ zANHaNRx@FgI)$BaCiP{rEA)q`~5f=etQ_OKi~NR`ydLkBx;h;@3p)L z7D2XDWv5%_V4S_?O3RuL6Ab(j-ig9@u*z@A==$df>=Dibe9@1u|2l@RUIK0Snrlym zV^;Xh;oj-=>YQMC%%4UcFW8AbCW^rp9)kX%yDfuPgkvb)0i`?rnbFxHpxyKG4@5XD z-E-oPM*;dYy+NS-V&Laov_@$R!;zKwDF z-ztQeK~pB_kr@e`yS)GgPT(!D;fy zV9Lgf)HsAZ!R!@7!(;6hnQ3JLY!2US&-Xrt-B++f{2@rp7g$5yi@_kpJ@(h2iau8P zHT>l-!4wQxrMW8V*(~@mmZjsdeX}4MSnQJpfyVX%R#i75LRWYN8<$0Nc7-!h_}vfe z-{1Xkv^Y>z(He_;1qNasW9F|0LC?mOhO17BN%272>z~G4!~Oj{ zh;!^y58Vc=tKZXLy1{Pq@8R!j6w2dS0g2Svb$Hf4C{!@Z} z8#pq3=2b_D#UA7bAv&lrLYjjuIYl1}}5*$$QR; z)WOKu2}s<4Mar@;Jzw~kpo5zSV#h!_FG2atiROZ!3*sVz76t8y_+*%1cu9JH08;nI z08ND^y3W9IkS(iUO7hEbayfGohou)_*Ds2@j4R=oPS~|d?W*(S`bQWRnSly%^!>@6 zD3mWeQ&qy@@vYNtom#VPx2*Xy^Wh;lPHP1lqJq=995O`ftNGr^F|_Yz4(v4CX#HHd zM~SK{pA3x|t~KU=0O$^m?V5c_Cz@e91iIHGOSLqjy0-3uvf!+S?l4P&!{?BXK=D7Ci!qN1s?)UyeN-A6=RYVktxWJ}ZyJv|ZgW)~_{ zUFpE+8-3NlRPJg^H#ETcOnw`qcm!lBw8Z<50MeP+@%%cp)sBm*wsRVuUw5|JaC=qT z#*T{Gf2yT|u>)ePubR4Mx`V!^64^9dSs$o29ZZw1O>JmX29^a;sT}4V?#JUF--b!5 zF%3iCUF)Rx;y8f$gB}`8c6Joab!0`dOxZE|x<-wG-B%RbG~~W1DelxVJ}1{d;Z6_^ zxed;gSky^{A|d}$@-3B?WpR1 z#Y?4i5TaOvvieTeut8lCEyLJCL_KzhsJW~;3nHrJ;;Hr$$d^($!g(rMYIzRnms8`UCZsxx%Ha~6;&LB zbD^TPmsx9HW^Ep&A#LFUZ(#s;K#0GP#ahmZxLxF}8`{UE%i+!DcRn~uFp>3@(_oS7 z{5dJIhoBZ+|9j}yh*bJ|y&T!1{_2W%%GDCCb5d^eciK~K*SUx;wR?{O{lDVe+ub1X z{j`hPFH0unR`=N>%g>#Ed0Y*)Z82k(^~@IL%rgGqUQKcOl|T9QM>2_b#=ja)etmmt z)ZhQ@mi4H=|J!XTQb^KXLDRZk)fAAPUX#qsm02{?=vK^w=}7)tWZxz2gw|vubg)(E zl>0;Kl7ViEs?b9`fx+9My?r7_cdH0yK{W>nJ1$n z3^lusXdcspcNP`;=~OgRRe60Na`Fx`p0@Jw@yf@ClZV`No8H9a^Ka>UvzS&j>zlec zZr~BK@cJPMO-aN{MkX{hv3!8Sq63*3r_eZ*1;Q29%z)m+5^N|PdZo47P~WRA`VCHf z0dCpUueTytob~rZDpa|z|n&bT7(9)IOGFNgBLZ@CjbeEHG z*?8eLFo8vUH}_Muu8(1S(@M#L;n8cJyG!-5n%HgJqwS%GjcV@Pz{cK2U`1{xTHvcj z6q#dRKQf2Rg91od&g|YS4$AEUCb*K-CpH+$rekVqzuz|%tpWjj6e%P zJspoQ{7lz0g$Ke;2f58{{ZRoWYy|J%)I-=-#AMkIqX?_w9F9gLNH90L40TZXwv2GV z8#u@tSRCZwFrX2u9B}Ag|KuQiB3T@FVE_6@AC02)3Jm6t$mdy|E-WoIVeq{k0^BmF z5{9uGr}e(l0M`teBm)F5%5(YGKL?b=Y1uN~GFJDZ@t8W-KVyhTc9DlyZ79=%uo=SO zFyM%X?C!8q;VcDO;a8MQ-jNhXmtSn=XztT9x?*dFPG%OafHiK|&^h{Vjx3xGjA8Wx zu_8DPfs>qs#caihVPvB>D5H!hydxvqyZ$LCMjEpUT)3840Wy4!E4_!iLQSHk;=I+?`{S@1A=0;;xinI@5D3l zNE8k{EH#2Rnl2oQU_1E2lkZPYp0*<~>_u=WT)?hPuchGWc6^1?(Dx>=K{HsN6XHfF z+;Px6N}RV|!>G^FA7Iycb~5482T>(_>P4aOeN2K7xaQDJNQ5;(=R-GSnlm%}V0>5> zD-2?$MMKY6;OaHXm@>s_A4+FwdEJZSO({>bM5LKp{>1wWGIfg+&=UEX0;vS!- z3#{2Ha`7R<02HApRPM{BW;=FYa-D%{X^Nw3RI+uoKNvWMTkp70@GqMuf4Aao)05w* zB)fsZtt+(Vq+o8(s>ga(eGLhha}54_mvE^{JBlfqs<}hvggEk{e*R#RCA3mmX@q#IzR7 zv)p>zwjQ@x9=B~0>28Db{oZ5=&b8nd&j)OFUe3r3Y_P+&@Bd}PzaF~Mb}yny<}7A=NLfAbeadO{m_y6zZl`stS2Z@b!Cztd zpsz``EGx!9wz}atz51fDNwZjRGgHHLMg?gS?^=Z>o82P@v4Ez-v`-ecLNpfWL4IrUNjUHQ5I!g(nQrjzs*85(aM~# z*7FZxz#FgnW>M9qx@8)=tvIrys=BKxsO}DE-&S<3zMl^W9)ft-&I#w1WGYDmWuVyu z$F}>rq&v2u$&z6V6jQhCS%s1|ax%^-4tK*^^b#@XiVMm&So(}vhbM1L6m95` zL8eSK4Zg7Z!t;L=UhfO954d5pPKBXWw32(ZV;gvE1KP_yo8sz94)G_5-ZP;U9pbiG zUF#~^owIr#c>OUIiOVJ8qpcD*!|%0dM%0pL|J3#VvhUw)CRF9dtegzha9g%J^Cqq>BWmi} zV1ZmHY0;WBED<{N$RY=@v2HTv% zbeZUFuS<6$_ubPS-twjXHh*rLKlfPj=a#&9t_Bo7oGK^{ms#8#UG|QwC6IKc_S&A7 zpw&px;2?bD1ydHeJzN)DrCwURu<7`7E>`Ly{jn=TMwY!<* zyZi4Xdk}?jHh!Ppl;Jl2l7Gh5#%uBI)?Q$1r?P&x6t(g!I{<2m?2T;wMH=2HI>No- z+{bpUwv}9WCAp-9JzHOmyn1Mw%7!wvAEH6!u!vp?TXQ8iBm&6+*7KJ_ zj|x0kvkcmG7$xISI668?k+04?;fXBEPr*Wu1UmL6w2qRpD#3vDL$?itC zXiEoAUXs@xVb2dbpxnKF3N0d}jvFGp4E)P*a^RgLVgN6r#DTLxz#HZmt#!o58F}!i zn;;v&Sl|_Qu{wr1vKqz2jxvj$f#!HI6N}yjv=ELbA&1F zf;wl6LOTYNyIc}nu5?m zi?9Hw8bdd7#}eod!jmZc7(QjB2P$`fIx#~R!u3=;8hhmXAD;6k*ovGJ^vGZnMn_41 z-{-&Y;l27Hq9u7is4?-G6+Yo^8ScrMAg~p*!Iu)DlS&NO&TT0(9K8l6-Sa-bBy#uj z>jk(@cyjXm@Tou$9LP(jHHK4*;{;)6*MA){(4u4l%k+gKDyn(6YeYeOCBKBREFfSL zJaPT=1Q?TX2Mc2hea6#^onN*c60eKvg+ngq3tbY5@1(_65P7i~=NYM_>;{B+iJC_w0K^kPjnIB2qt9ppEyANnGQKH693l~p@gw^ZMg+XbdK=M?(2PU) z_D+Wgr!C0FAR~BOW-JEOg{y&LAY$ND!V^0g!10J1DCO@O31B!V=s>JM_dEEj2NKC+ za*(wM%2MdCA&inM^ohbYlRA=_O1gv&*|Q*N5+L#PTa)Xb$0=^J)X_|~gWYh9Fl2Z? z)lIGG8UbJ*`uJ}_w(?BRybee!hXnj1Lpnp*i~AKs1K)nKgj?eRJ0gN?3C>9Ds#D#M zu?qq^3^dI~|D3|nI>ay+GczRk`W!<9o7=e{`zl(ONazHFF=_j3TK7m9fb_R09xJ!?lO+< z);xF|d5JSZG$Z7+EJ3Kp72Nu_IvJAon8d2CNKF9|X}#go$uT!s#Z z$08eNn2SW%4k`WkD3Pd~L`?d7lLQZ1;$x&$BwYW^pLioUCD^-D`+e5_$mKw$FmZhe z=zjrv76$QEQ0knDJ~pMJaHXNpQ*1a3n0mrRdKfSP0kRq?J@R5MszFuq(nhLk%JJdB zQ!a4^;Uq*1)J$i#3D}mf#Al#ZO(vhNQZ{4^gGa!$(!)0{@0kM(f}jRb6(jB{Y*`yL z3yy4v+k=7k!GVpit6>tbu>rYuPKG^3w~oMNN}RBHZ~}xFNr!}DP_B4pLTjp_3&DsB zyD22ZF`SAOZC2KW(OxH&g2iF>;{B@={MUr0*hDJyNS6UdKSCuf?Fd+MjH>8G#!1+8 z?|mx4Lv}L3D8B!pcL5WCJ)WLBb;+lR!hXmOSy;!E(<%(l5D&ASrv*T|x!4x0fbc&* zz*D8F6otCfi?r{m%eF7HpM$g^7MlGr;x+2M{L(RTx>!xm5mQLVH4Tz^nE7otl zU1v%2Iw(q4mf-&zp^s|9h}LA0(9Q3}CRf6-XGa8`z-vR3D3WQIJ!ERW`$XnhTp7ct zrMV3u3>BAtu3{OEEXgh*rb%?zK@KsUfg=;AZ`rz3-$(D5^CFWpO#l2Hi9VCNMD&^6 zv1H1&sSGSbl_c9B231|fRBYm^vSS-msdv1Omjtl6?`KKeMc|0mIR|uI(sbKZh@nfe zHE>O8^fj6EbzQY|$8_qG`abv+gqJ~)N^;ePMpQ)lHuLt8VZElf%6X}Vi#uB$;WC5V@% zfS29b;WTX+FU?NVMSxzU1%Y0+5i8q>m2Jez*Br6(D}VCqZP3a#Xk{C;vJG0<2CZy^ zR<=Pa+n|+MK`UEK#oaV5;f{gh3b>Pb0YO!0lv#jt%`rbS(B~T5`CH_+EppqVL~ffO zg;Pz&Yw*itH^&>n-?@|Y_4eD;(5_YDx~hlR)`z+EVNR8^Pkrc%``RCb#BGe2XcD&bLowR&|I^TiB|$&7!nqx( zs2UEjCOT15V;e*yy2&H`{-f45MLjm%Vb71IY}s84>3rMWRfUtC9bvjISnVRMueQPN z5R>%|?)|qYt1UDr;**=^I6vBX6%qQ-LoMzL7}BiJiLJP96xR<)+u5)>eK$DcyRAnT z1G(A|mfJ&?CSN_K+bl75cVc5%0JKE}!h=RVb0yif*y_sqK-DZ*Q-`RwMI9m$cLCHh z&9d5Y&*rth#XYOzp8Y_4JApw9rwO%v>?N^jP{lhdzA6Kq)!wY4oOSB`oDhFPFr49p zOeS6pPc8qd+K=nzdt?C*j^9`F3vxe&e;BvEIt9e3kT^fG`GRyfAL@TU&wwJ^H#(05DyW#kAW zMsm%jWqJoXu~|Sf-s_3NdkkoU8D2oMlXEiZ2$vwV zDI3==8`lVyFQQ`@0uFwVF6yN>IS1NALyF;O=qmq$t3dyh5Fg}tCmlaDf#FAjjX`Z` zgc6G}!3AN|_JLoKumqMR#K~oGFZf zfUV+O32*k`XG;;af5@qwEjGs9MRdsK7y8~X9G{1?^sa1D~6ot^lw;*>D3%P~-!E#rvsSc30(0gz_|oU1EETQS_8E& z^`(Jskv`E?Qg678bLMqqf|yoKN$D$!VJULou_cRGis_gRA#gTNLqd(o)qAi7?hrw% ztU2Wp_}#(DJk@ERTkacBp1e2?V3HV-Hrd&6siX`v$2BBbrUT#xVw$e38MdxlSjTvp zsb{Fb^AWb%cI$(}*jiO3G8P5o*@SxC;g>J`<6AH{`7=Z4H+Vq&)1Yu0y`$GZ9DX=C zJo$ih{h;(^P1nQ{TI=F`w3Z0DQ9i2h$W#HJdALoBpz$qkYi;5vnqJphmF@K;!)Atf zOQusr*WtUC9Uz$H9(WUv-7nYL!+WwXGzi@gDYhd^lBCPZszPUJhT0?$*eZ;78=!GZ z+VeLsxJ?`$Yf6RhLcHnn@B1OZ$CoZ);)%qKi5-jQB)TAxD}r_3gU>yr-P<#0U6d_N z6g9Q~1AafjaKcIxABTQu6JPuoPsCv;ejN9}AO}kQ=DU~2Z=UZRe>gh$78{ckMb>rG zPz*&*kB2f~RgCvf4&^qy2%Dsre$z{@(RyRg>z$#K5PgI)s4uEzNR@V+Frp%H#N<;C zGcbH|)sr<1H0Zw3&mUa3zoiwfO)B+`2j~5?ALq)gop>`E4#A^FW34M`=$d%g9!{c_Vp>;FbQ;YV}xVUnA<(RnLsj#{k z&1yxnc|2UH;UX4pcPg5$+-gHmuRNy+)L< zb>sJ@p~JSgV_V#@bR+eag%^8#m}2uo>8h!24+mExyCiH%t(|}2*pk(lUF$5FoL$ds z%pJwIn3SzdtwGH60>A#L>H}}O4k`uJHQe{i57FzD z&(>M4?Yo0FX;UT`z{!0w;BE{6sMZqDnMP+vO|=glLKZL_kwuhgB78L-qZv2Bdg)FPIS9|$Sa5eA>_QtOTQTAh zqY>dp7T_FXjBFlWd+Z(Ty+y-4@*Ghp`OL)3T_y^$CUp#1>l%tCf@Dp{R}TEif}Cbg zRB4{*a}b9ZL5hX6-iRkK!`B2vCI#d85N9j(VJv*`Y%(;$Lc3lbJnVX~$gq#9wl>C8 z^0@0R+RCo*I*cyCw2sG7;?OK;mHR`QJP(vRLPX&VKDi=L9t9PR^3FBx*ChI>uefn9DE~ zLsv|8WwrbosLgAzAD*MK{#{3q!5$KYJ)!14A&B`gN<9)nz+47?9%y{%CXR3#UpeO( zhJ>p_v{L`nA{hhC@?cw>UU-6 zW}Ug|&2u{-{y1a>s&PL&kCHDKzuys#VR9Y`C@2se;eZJP$pp?!;?%8n4(!86k1j&y z;yS_|UX7bBLX)Q*b4U^&h*fsbiNew0;i16N7)9YF4Z*CgNej&op6EyvO*Xo+rp(_O zq06_0s;LTUcoH#XM>R}MwY9!ORo65W$1ojpU@Mxn(ALl;(^SmWc81Zm5*MD^5nhRV zD3^vHg+CJf_?ppP6@@pE3$o&D0tCdG_2}dS*>nYDiO?MgV(2sqZGz4~BIL ztaS?GpM>4#unC;7Bf$J$;a|=t;S7HrP;Y>n?SySM39}7GiO*#;`bXh>0X+#hiun;i zLU-YRk|;>s%sjrGyKY&Igy5WK6`B}vSPqPE6(-Q269(8Dd)kgd;`)?iKIFLd+>*79 ztZTrII>@wj=xDakF(RKR9?H*wAP0#}uE;s4&2LZ`1>tA-^CkSb?~~{g?$+n9lYQXa z>WB+HoK~&tVJNJnLsZh@`Wog?_s|06W?VV!}gL#M{vvnrNlR!7J9)# zY0}Q062i3NLO3B~4ZMJM*~tTkIbn%2@DD1|?4@(m1wzd0Nhz8uMpgRlz#-=oC&8+q zldDvhNV8+WR`Z5In({&5q`Sg;fJ`{TfPy4U;;cRH=Z{nEfS(m7@X8QQKv{z$kkQ!C zyViOvC`*fw_=F6D1av2$>sg85P4t%KoZgsPlke7;FvXDRjO>;HiRy%~d55T9@udLU z2q=Sx=6NZ-izoRwXgvb!GU7A+vJ0c|@XDjmL5p=^npnoj?OR#z7^=}VG;2L7wkX?J z7u6&Dw0#Q)Y{*a7{qqEDB$R%jiJlQ3Ra8{L58xaOftD|ry2@#J0v3kXaPsn_mLJf_ zP6MpF${cBQg`&)D#t&D|@h;MyZ}yN_%9VK$~C+qJtv@9a@z;vZ8fmrAZ}h)(=44wRUz?Yrtx3 zI@Un9G{qnW(FVF>DvquWUbKH-jg2b(L5&asj))6}YF zjI3hmVz4i|sBxhq2DGshQ-W=WJaH^e#SYWGQo)pN7B9HKYKGAj?p^U9qeqD#RK!%vvthM9OUuK_3% zXh!Xmpk_;{8fwxS@Pwq#XNsyl1)Oe6hU-$>H8exhXx~&+T~ZXKuNktVEyNTG=%1#p zugDcG3j8bL`yetv@IFr>R1%Iv*fP&iDL4?{pw^WPxI8`)-@=2F@DpWUn3g{2w`7;;d%)#h~mqMDi3R+x*$)LcZFbS}UqA(`H80Ac^OEsKhl9`=j zvT1b{X-&b>qG`V(m#~w#HHSj>w=`ka+-ni_&0`u9(4zK$qLm%D4<|B^#qscD1K~|v z>W6~?D6>d#pri%H3(rb!K7TIgkWToi)~s8xt9O!~WZ@WEVEzPlPO@cS$^zqRu#7Of zH05m6KZTc)bSSOoH<4S^_*c{)!C8yP&|6%Zz99++9u=NL6V7>%@UaP(fK5sCcl0WCD`~bCSPM419>7%>f$~ejfT18UgK!B+TOy z@zbi*Bn49J0VPr3zlQICn@kggKG@3YdP=6;qK3Z;<4-J~_B-+I37$(Ptoa#cpo4Y- zi=1XN^W1Y(;Xp$PD!3wS!6@he=d9>4oWgJSWlaO6SU7rqg0He4JIo-PU~S}AsoJ;J zd|I^T%6vqe5a^CY&<^hKR3l{>{;8PYr%2TY#^nTXDGVPCd)o`rTxEDPWQJ>8GF*X! z-!khIsOO+%0&DjiPIw;GBMQ3G0U-^C_nLmd7FG5;68i*~hJ51c*D*n#$Eo<5$%D5c z44p&h}VD*AvQjQgaIfA^8 zx!|k0j2VGkmU+zQn?yFcy%8P3%d|IG{bM?r2(r>K&3@Okn*2LU%&$ec{4IzV8>h6N zor37XI}TC$2oAal(9{Ku96gXux&hcH=FdY%6|4p7R;OglNCVnxH+_E}5fFj^4Mw0+ zIx0AOJw)7jax{TW^>n>iwWx@hGXSF@P#=8zTQn`u&4KqS0UmM3ncO-J$C(N2UIm2B z5~l2jwjUzq13Rp__j(5!As}$s19==9kA>V(XXAv<4c&hm>frH^_ z&QTmnp?Azut-DEzlOFy$2$LaLaovH(eMy&?EDfsDIUV`v*=pX_Ks7;Ct*&aw>k+L* zSu7K+OocK5b5a2T&yyj+bN>}8$CdN)USm*UzHA;)VR$1o6C7*=7cdfgGOTSa@*h+m*EQS?3v+3lL(xB=>Y2<#7 zyJ${{gMKu;1e=^A)3+_9rFV3>-__MMIlV86W*M#^O%w<53cF5^zM7%y@%G+TVag!9nuWURggvFf0KnL zY2rgNjEE~TO+K+5(ILIJdwBOBUG-Skz4OV)?@7`TUV@7FK@@)QVx}emslP*)3=VZY zW-s7ePm|o5F<-^=}U1By6Aa14Q_%2En4k~mtgS5Ymi-t!Uh6fJjia%_H*g< zB|V~2{3pDXbC`dvYMFp~2zojTR7&qVm?g?C>695W;7dhHilVhejV#YH}HNp`ju3a5>b?964OXa1q{e z83qcTX?Cq9$JQ({4yGQZgT6*(gAzkm`UBUZt}-C9YRNJS*_tcH`?9J^R)58Cjuth% zMsp433gXz|Y~tpkdEoQI6YOD7hR`|;D!MgamXd~@InZ40TCCiGrB@XkAaBI41>DSg z7LSPUm##HiR<%_vk$K34qQQpOn{4Y@%j67P-658&YK9|ISJ#L}EtOiPZBkuE+TZjt zwJ83tkI#fI9IGjG;4zC1;FsYAT44F^pdc4JD09FS1Y%^jMH68}NC0Qmge0*~yh#{~ z15Xs5l%(Am9W!aCt|{$W^z}D>IEa~_29$vpVd!3=fp>yVZhAf2AD61-_LgU?8xI< zR{8})>x&Sk;U!*vBQRt41%}kZG5>|#ftSrI9xTNph0}Yr7zo`>^Ar&>H3K3Y@SM4- zYp^(wvCuAHq zsc4cYb+ueo-G96HT6_x{T8L}FQuopqzU94^wX(f&#N?E*o}AKJl>5`<6BsRo88FAh zI|nX$gMXd7G`q4jm_Id-_+SA^BU~+h-yKn&|9a`+$wzSCG%c*!S3q1yU3r0^xaz^x zr-tr2CI~duu{77|Q=%$XUvU&;U>dHpkeR6U4ar1BVU?3hYtic8hBkWR(Fb*iJ}3=T zj5plZKtGRp)-jtCH*_w;%0)t2m zyFW$;5K1n*{spGP>SF^7EUI-El5N3_D}%9uOgVGXU@3?C#=U@DU~Mv@eqSeK|ql(y{FDk>31UfW4vJk zFEdkyjTxbzL%gDS&c1l#!x7}6t@9QqeumM(p9qfpEqc5Oiy~k<3#@>>5&04Z)wuP- z=RM{&07 zZL?PY!F1G)rgmh>P|<6k36;r~XVA!E6>5}~yw@Oyk|;*m#)X7x&-H+}DY@x=AdkZ< zUmw)GksGbC48+EoM{+6ih`oqxJlcekV}gTChK zs!0cqsSa$}avVYwT_U>NcNdEJ(=AgowUyPM78|#+RJBWsnvkt5j{*CcC8U(km2?j> zuW%C}OFegk!~Styi+Oc$ol9z|hu7RpVuwM40Oqpy2-X!o7*a>?Yiq|VXf2j(t7NF( z7#OhC_os4ESfoa`7OLUm+J<8`)wGaC9_W@AXvoVkG=SS6j7<3o0k*YjDR2TeZR=U; zxr>oGD)5(qmp6_j%Zg*VI_*m;XeSszyMXp7X*#UHitO29I$sd5p5SDefQZ-wUG4o< zfRQwBUxA4ONA@K_ZS*Zmlah>#6v9R5C!=cYTPe6G^^?^6cxilhDN2Tg<3~Itk2w?x zrtEcNI%F#bLd46=-!sf2(iI-HyovB`t;OVhPw;a1fBjC(bUj{RDvNbbXPF)e$HFL! z3VTn`5A~Mire@))N~%n%hKxqKyxxfQ7UL8zl~|ll7XG!@oU{gJ@|b0*jq|t>EUyCn zfNU>kuIV^p2?o4g?3UhQ555N)4uCbp?^sv|p4r?5h_=ptnE3GoKYk~^&n^g0!Nln{ zy4`k!_uk|j^al1TSMVnxCV8jp#m?)90RX`~vq?87DRO$hA**10%j%j-)ne+sKldgq zI(omFaq$C*&O`Jsu)o2ci_q7?q~CrRj$;h_*=K))q4*qcgy3MGu$wZpy5d|1#B@w7 zYnEMgfV^X|*6i%g3fL59cYaMOz%sC1N#`JJG1pakpHhiB&)i`c$;oxiw`rbsjN~RR zOb7F<)53;X@Z^i5?(e$%b_*W@jV%)~XITCPgRXw?@PZCgE}riTzo!9X_CxWv+1>s@ zNyt~!w5cTDa!5~ph0hE#e$SB=t!asB+XIV5L2P>>Euem*1tgzpEBb%jc0xukb zbdJ%y#+^YTpY4;E$O@Tl{}w%-5pl!Rf$$PmIC60_9b{RDHKX7H?`KL`;Pp$x*6SA@ zDcN!>*6+xDxhtDZmT9xzrf8-PR+(m51U9)X+YV86nHY*L%Nl5Gt}8iYq5Bp?(KMq! zCzrGIO}zW8m*@Q>UVy;^MCp0Z$84pe6#b- zKmXwyUZ;~+q4jv-)mnK+>QLLp;0?#rEHGtNTa#RuYObwPP3DDLiwcjt04`RPbX`|W zh%JK3v!nDSMdy{}r|U<~?SNMr#?imas?t^6h+H zbPw#)Y^eDY#aETl!+YeHzoajgJ5i@Qd68hGU-umdagaAJ;6cZWov=QW{D-OD$_?iKjbV^^AyU5=wx34!W2OSO(HYnnm$t3t#>!A3mvK*z?$~_oO z3y$TXydcCRX_C~b1Yb2QcJ@lS?f!Jz&j}n+7$Y3fsdi~Zry6F@m)kTOScowW3#bU{ z#4dkZVV^3`SktXyHDnCPkEa+Oznf|hAFpgAsc|RKfR!_eu74)o@51={r~2AO-Yz@PI8T&{*ka^5qeLw<_`t`~1Z>M`guhuVrsWnpRxoI=A z8%sFx!9>M~x+56w)H<^-rx&F1YNexw4gG_60qWUlh#tnXBnAzv-V6VBs+;1)N*4|L z_$>zMV_YU2H(tGU_T^OL{FO?>4I8+Sc>AF0<0flhgcb%UNj?5@rL%^;d;pC0rTd&- zUH{1V&YPwb|8}aY17t+@DoF>Db%TFt^Bx{v=k*bnzO0By-{0w)fu18y2GY zIQi1FEl?E0Pt(kVFIReKI;g>TBmMo#Jm1?mf*%VQ`oe_ zriBDM9dI?^_28HPp6bO}cTF4FOIU?T?+c0W*7v)F)OCa6tIBjLFIM!*hAo|wi;3{* zQ=(CbH&9G%*xRpY6u@49Ju1BQE*LBWjj#U}r#w5=8Q2i8>O#u5)A}V|P{HQN+pF~7 zbeKn(_ai3uwBo?Oon9&aR;BTV{a(_NLC|56She-?mvgmbwPAn1i3WJVzvsh|JefL; z(yynQXKz)SZrIpop@02TYDRzs)nG$p-@Qx%MP?ArsBUWpns@+ zW^3j(ZkYwi92%SxuO=*t3nNz8Yq<%USJq`iEZ%ti@@%mO_gq9HeAnGOK3?nvta41o zSBpJ-8ihkJHoVqRB3A95Xs|L82eMJ=TbyaVi*2JX>f;)WY)UoeZ#BOPI%bpI1Hm~iM*I6 zp0Rkx#Jv)Q-)7yXPqUsXlf|&zFbUFpA3-*Re-NH1N>5g;r|cpd7Lnq*vv}uxIzS{R zjCg*t%VKvzHZ6vug+BA@u*#Ezoj;7iWQ?uz8I`6Ohr)nFEdGE`66?XU;NeeAQ)wd_ zmUu@;x_}rFY~cc|!@)do9xM0G6SLD{<1X*;xMe=_Fqt5P4kIV%r93wiIlen!F7YI~ zfb+^1o*ce7d)i@;X-KHz`!{D#d1he3p|w~BwSyDoOO8d*$I$Q2d7Q%r&8vG_INRr2{Pb>i3(g+P5Qdi_P&&wpiCU${2; z-kJfv-YM_WISkDd*u2rx#6AJBd2~=~MAbkQFtn-XUxEBh_jkER!!UcYDM{u7eO@u- zo?=KAq6t)ASg5W`m2|RUNQR{vYQ3!+S;58`DVnAu0o4AYE7(qfE#GbVZp(M~n~%19 zw-(W*nyGE83|7*q2;IC^>XZtqy! zKReo=OVs%##;c7N1((%#cf~2XT@XWrbLiiyHPF z9x2E_z74_TVb7dj*^%e2W%pMLK*O^2b+<3;v=8hj6Zl&i7@8x?j^Pf7U8lowW@%;c zp>quUIj73fCi^-peWM259~ZPZD_((WMJvX#Tjsej=2=Uc+&J^haTj%Fdj*4VQedNO z&@ZU#lBR+Z1YV89WLeeaUf<{&D=^rd z&FW!t-Mx}-_rMZgfIG|1tfkrWUQDs!2w%W&cfXEsqbr&BRm;5s=D4OgvRCNrI-TU^YA0l2}iy3n;zkNq&N1w-kBsLCX!+(vLu0S zHr>vYrOGYXlEKFyZMrCUB{=V>){4@Azu#?P}D&7 zi(cUP-USh9oXr(gJot1Ph3-BxPX@iy;+GfErAXrO=ihklw{TWliys)bHQx;!^nZl^ z`=;~F=aK)-&Nt6~1OH|yqF5|Tl>YF|Z!k$e&CzGM`}~GffJcLP$Bn#iD`LdOs-sai z%5`CpCOwRGmoW=8Ae6Up`8VED57-kQ>%za{=h)3rK-i3PfQcMBqq1=Y z?DgAaHk<%rpa9QUbbI%9VF7Xwirr@XiD$jiBbDcu2BoMq#X=&}i@McV*pK$yQ7x<;wt z0$F2s8k=KAt7_3|im6c^Z4FOjw+WbIM&nx1cnL)s5a^350%xb>0 z4-0UMir~nAEq0sWF=jPwRYzWcT9lg404;W#fH7vBEu~GG16q{#X$EGoTL;6K)o69Q zRvOGCei{B^R#T;|nZs95tzx$qUU`Yj)pDb7TBK{AYq-6@QUS&f$V zXAUnhUwup;oz-w@Kjy#^rG`^biQPIR#jGYv8?iu?D7BfPN$l3ZC}y?UuQEsmA~E|Y zFJLIjO_xv%cDDcqTL6PCfWa2PU<+W-u(hA{%A!(umLl<#d{;SUO~rjxd|P*2r7e7@ z3a}5r>#Uq=i$80Tw{FOVSOyt-)=LNSb&}%s%d@YX0wq|eubdJo-qlx5g&fGMOb4F? z!vqT~7KAtohCV|&tPrC3woZtwiRKBHe?`|m2bkGi7F!0!%wNn~SORFK^fJF=+^qr4 zJnN<3XIH>XF)T<@NtLaTrTR;qG`z8SJkj)2j}L_nyEd5bU;p;?oA>ognBv>kE3T}) z*|%J!r*IJXy@we1&oZvuWjy(;mw$dXjRe1oIPiENPyO(Wapf-X<0b zsg6OJT8~=jOM@K3pFS`q;18YM@ZEoSCQDIBu`bJaHZZ+7|ZlF)ef)bu>_>k0J>lgg7#p&3QuY!gqzic}bT<=t_f1{~J}efs6p7WLc@%yH4YFH67PkV;k#g?`q94%--}J;t+9D9()_ z?X`7O4eKr%UnWqz6v45}E>nffg`eKBmiNc9N7AjtpVUvIhA?VQnq{N2+YciadHw(o zqxu-eH*pfBIcB@1{IX}g?91v&vQT>any1wgeXMt6r5CM2e)rmR2BnuA0=x4lKk%SM zV=2x)myGGmSbUUF37Tr63%^Ge*yXa%FPEDhW|Tp3W+dyx(J-eIY>($hQhjTD(%A9yyI`bWzVKb72*ORbw8(vQX&Vp?O7JSpF z{_{;`>j^wKr|EYLs}?ZAx_L}2vQEG_yQbfHAfJ#$_O(xo z2x(;KT$j?aFgHzmaXz$8yWe2ZEdIuR=RyXQT?!a!FR-Zlc4KEe^6EEvC5Wlpw08Jc zF`Z1CIK?T&?VDxA`idE^6{@V-#!Va+w~?^cauXnL50XW10ydj?yrw7SCI)jmI@z80 zF*gEL)g-~^V`go&xJ_)nR-Cgn_tBhoH=x5|#-UKoM4YS9Y#g|%(fpW;rd3uwvXmkr z%QAX!HyQVcs4Yh^!g6h;!6vRk+F|E5EG_B{Z{(hyU#;_G(?%PLB~uD4zjCfDDWMHF z=#KNTi(BoUJ3djXaR_O;QJEU|D4gv<$j93=qqoY?4T`^+pu-z=crAB{&?@}4Szg;L zuX+N@eZRkNif1Zm0^F%porD{DwHeF+kXvI?yK_P_R&%)B)roY|p4F{WY+bv&6+n00 zNTb>9uBm7<93vqEHfmC`Sw4T0URhE+>6oW2&$ui}ae;(9*Za|gDK$vE674)NP;#)7 zM#^LuRYC?^)v^l0w9h140ha3DKbJs@X&Q6 zcY!sU5p-Ei`OeVra&}Mgn}m8~TGPN<3Z*nj^Z7McDmEdaSO^^3qM56LkFfhoAX^X4 zn{_*JZ(r4^_O*rD=0&P^O9iT*^@^{j98(|6(lbxmua5qTg{S+X^U z$W*38HHfRoifQord^t`m$INbknq8z71hnOghUuIiK4dJV8oFlaQlAb?_*;=JSstjW zMX4nllFgyC`Sjkk)!Lm-jD^QAXZFtxFx*Xrcy9%VyC@pWS7c5BVUkSF!>n+_Ao2Ym z>+=N-hQcfHA4QJaO@dGGwagIRE6*J#q41*ekTU(y32MqT?!FCOkjKM>ei05T@2J!7 z>_sCAM2&@)l_$07C-NS0aT6Qj<6Sx(T%~p zM;ucn^wGzJJ}1zpft|uJ$dpkSfa08k-FP6qliq&V+e<+1p|>weoJj(|3!t7vNM!E1 zI)|m}jtKOKOcHu7@de)lvJX)QzhVO}Eek5gDRR23mypvleFHZ~o{2x}YS7QVfr*|F zH}ZUX{qG?%&>_0Iyf7B_E=cf&xbPR8MXzBJ4reXaVVJl6WiXm$~SvFRw1A8aX{uKtD3+<2G;RTGE zHX(gkqq1fi=73t1IEFrOY*$6_>8fPUPJONi6mdT&e~wNL4)>}j{hZ47H+inyXmPU} ztD1gmg`jS#boU=iN>g?i-%548s)x^^hS4AB169&&u)@^7A=@hLJElfe-IAuJu!s0h zaQ>a1U&RR9cuXe~8pVs0Os{UyrFE-yxMkyK=!{}`m9xE~j*V9@kJ;h=8UFEYn55Rp z5ZVqN4mWUuiw-=CvF1ezVEIVtX+6Rew+~El`;aPbAJ7zDtuWj=+w^8w=c}%?nQ0F9 zzS3r{;@m`~&E*72vwgRQK-pZq-J4d7ZqRVQ)mhv}2D&%pS#S8@`7F-RL^`Dsrqu%~ zg*&K9gHF+I;)%qKi5&wwMi(SPUb1(^h=@l}xOY16_ugI9-Q(iDyLtS_>_WV| zJpN;LUESTr_>bA;a(7GdAG3?z?v~>}DrLB)gOIaBuQ3R@p|C;{)Pt$N4_u06j4(5) zf%+>|R>(swrb{N=bf`zE$9t4_`W2T^=q+0{dZf^Y%0~*~CWY`1V1Gj?LJ`W#~`^#32M6C^|FAMGmE4l_*cWpuTR2ge2(JbN|+=8LTO={aUrIW$7`>*`RFgtJ?C5)Mg;4+ z`&RlBx4C>bd9L_4v?gf1OYI07!NC;s(4)GZ{R(eg)&Awv`67kCM?7AR@n4-dk z7ki*Iu(twABGi2%0=f@6vxhVmg?9)<>4MT*%wD8H?P4`+)FUJ@VLiwg_wY5MgaunRcIL$+CcLD+z@M z4be7mh%@MeoG?_=uuR>RCF)RHaj0Wj(xMP;i?l+#on-8UBha5>5&lAhWk-?ae$VPl zef^PPu`QbSTPDEfK-}JXHi?B75gB=LN5~3uvX=^=xAM0!S}6@LS3%(Z$ev_pL)vRG zku#d6GjI>+1T^0?h_F+l!6$lk2@9NN6BTbCQxY*(?%CS~-*`tphe0^HTI}ilup2r_ zl(NhsZ|v~8FxUTv-rb#@(+KtSv`oxDUH=P6yjYk}_B$3~$k5$S_y4$9`?H$ZnbCP$ z>(h{TIZa1lLLn;Je0>7uL z%^br-h|%JXx!-+632<*(YXep}TV^!uuu4QkeD6!@k_vmyc z{uMkLha)P`&oEmLk^4E3U?aj-LN7KbM(l|si1Gfw#kd9Ar=yq+E=(qD%3w5ezd3$l z*Ah_*e9ZcSe!WrZQ-?X?`3aANY;SoN&^>Yruc47nFwUPw^5(RjfJOQ1Y)RPV1q=cV zTPK13;UgO=w_-=4aE6~Q5CWc;%t&i~z=j7} zz{YjqCGPQ0#{@Gg4T-wxApU>$-le&5Buf|E&#!Q+MNNcVk;EIIGD~fGMT8=y2(_e) ztgKsQVGh74Ap!)L03oH~-Ztx;`xC}&%x2d)v$$Q0T6EW}W^>J2HoLU>5%Wvt9CrW& zK!9&Os9<(iCPf1NJbwH2TuG5V0u?`G|)3yJ9-!$706%xU*iLhmnZNAuQqv zQ9un?XC!d^L<|pzI8OR~c>9ZZvX`LBKE7A@BoKc9N;~BQ?qVMq;irA^@(yirfz=^8 z%H6Iu4bc%!eR?&LQdRl0OMDIIy{5?ND zK&Du`2M?b3WwX0Wk9=Xda|LAaNKl_Vf8?h35tPpDZOgQrE|q0Ta=NmkcQw#WOzPN7 z@5xlxoW-C%7Zs&uSkr|zY&vA(=J7c-t`(s+b`8Te+@5XNk|lQ?N0tmtvl&%oS$BGh zG=}^&|Gocs$c=L%2(Iv2f8YhKLG)Sv3V852+{ayDJ(P&Bk@z+e-)488yu|la;KNKd zZ!d%o1Bw>4v1G21E@3S1H%MpiClb6b+O6{{_o8+4dwj}9}9FejnIOmfCl z-dyXHu;Q$Q7yNmk+RU$$K4I9I>`0BAQY3Xr{h{iR+NpPLP0WWJ-D|%`h#@n&+ zc5J*IQ@tJ4#EjG}T;L?RYqG>lM&DK$N!r54a&h`;34^n|bx#*dlgInpPCBTsb6JY6 z{I7f|vKWQ0d>M+t7pq;0G+j84Nomd~9zC*38Eo+&k*}O27L9%>E?W(3AH(`(r8PSY zE-?+FX3Z8t>|5MlKMwJ?i1WVCicscvD$e_FJL&gNDt&*0>-Ux6`pxN+EyvS7Gjqcx zTc#*#lz&D6Po3oi3k zP4{Ni6kn=sm)Wi>Z{7BUWc3YnOR*P^@d(S-uZ+Zca0OyHAzub z2?*9u-L9f4hRqaRqdi6Knp8o%PEEpjUdsyU)unFeBz?BMZJVxQE3)JCEXOpumTAhi zs+p3cxkk^{MFrilpsDF2STqlvlk?gpqRW!5tDUZ*@cPr!(H}3p6h4yO z%Y^^;ghnV?k-Psj?gQJ2$x(n#8d&M*rvy~c%jE8FVhnXDpNgc&(^5WnIW=#Q15VFD zHXOv&8s7b_mpXt@dOJRSy^r;qDBt9;KJ^n020nib(&Q=i`;>dsUc(BIeNeB&VGzvkw2u0@8-uJDecdz+J7=jz(=!?%E`gebIP%5UgWSiz5p;E$WO=W1Y z75?q3Byf4qGCz}Z<)Y3~Z+efyCEyF|L7ajltVI}m>Cmoe9Q!!qX()7BI)T#y+2N5( z_&q4m%z;Ni#d34N)6q}y>*~YtQN!c^^!y{_r`@TGM#vi&(JpD*nz}6;ZAmhgiI7*Z zBCD)k$lA6lVZ^;==$jDv=M^HqprqIwC_lIO_?#l=3!JtdKyLl#lQ}yCY4_;CNcGhk0K`m`e4^!p=T-qH)Eiz{E8SESZj3ziVy;`{ z;fCp-_=6X?LTgBY!bi`H{gR@6gL>QJ>TULD<=ce=9n)tPj&$4!65mhzsQfZ#2xOU4 zuTM1OeEyvWya~Q0U(3Ag`?-mr)JrpH=4TXD6(PqoUU#* zdp^n~FSgS{7qTUrW|NC!T325s)NXmM`)h%+7_spaD@`m@Xu>M*->g^9$ zvJ4noI3SeZWy))TIE66x1qc^b27>AknGT?ETltfss4HV$gYVml*c$m0Zjkh!~ z6PkCu|M>5C*oLK!ykU$5L?~VucmLav^y$xD-}{1rM;9NUc^9sP=r*sJCI}gS%^S(q zu1(%2Q97FeTt=P)O%JbVp+ELByeecg&~kZTlka_I{w>Y}siZ$(u*4ouItI;xe+y!2 z2!b~cAuW!nO?C^viK)U|!r@guuVjc7`CywwU-@c)KFABNfCQ1w8wDbGHcSR>(Bvv( zDJ}$Dr&q**!39i|<>PD5cywkMFwu2dbg$2avMklPZWVBZVF2^TaM3vVkN-pHLhyx8 zqf0Ce^9k7Y5Ni*{p;%1n`&s{SVE|clXbfj{Gzi?VpArwt0j5pek{F+M8HQ@*F{NCC z;X%1#)?pNHi@a|htLmh|z5{yOg2fm;1g6EKIJdAS zpOI!T;XQ^PDZGaj3|)R|QG_8;F2ghd@Qx}Nj2yP{Rf-?EM?seGLHR{Z*-bEd@7%cx zMeUzM2g9T@u2bvAO>1WG>Vxf3wInYjX&og;gaP70iVB5A@9jPo5&9I8^8~}A#f^ui zC3y3a;+wEo`y<^uCNHp!mroO;SHxqllRLC15v;90bWE&iaEO);LCGOHcoIOaq4!ZF z4)Q9ovZ|Z9rRch3u%4qEimOPl%BH4tT~l8W$aOA}pkWocBQXVOEVGjLKf@j2l02XH zmYl-kz?psj{prglwATqWcPjhbmf@cc%=J@S2KC6A4enGjm5!_{ypHEgaXm;*s;+C& z9M5y|n{7Z9dM&*$Q%R%>s+}gurfR85wXJ(z*OktDLjxNc*wDaNNdqfKpmr@s(Hv^^ z6cukxsvDAInTA8*qb{Z8ZwgY$E{JU2wTJFE-v^ZycU-O}K4n&tCEJ$L>l&&g*#ME9EQ&*K8 z+h9ty@qM)Fg6K7M)H;J*I{-Pl>VHY~>~2?MMNJ7wAQo~{Bl^c<}xGYS;Z zQ%%W~dU}_dR99)W!zO+(ts{#%mTF0gsVGKQkxj>zy0oiUj^$9=?T)SD_W`KSKY;Sv z0Ct`hmgCR<;Q8#9TpXMT^I)^)wC{&8sI(Jxr@Y+fMAfNW*IjX3TXP-Tm37n5B^_v5 zVUB6pPFFKkvs;}Uni?{iAbF5tE!cUnWOkJ&&aEabb4FG#F;CKuijw^#{b(LE>@DHN z5`UhBNotZvgH4}0F#s(LKxG<+?S$G`aQkL}1;*u$WmvM@&Z`B@LlDax%6lS+<$SaM=Z90+UB!~-+SU6UY|}B7TN1)&uK>4xjdyNU z$MTUjP;IF-eD+@INX%@NykCK6hMFQp6>aYAyPq-7TwS1~UQW7XWsK{Wx+~cH)d^Nl zMT44dv6gkq>gcemU1I?ndCZwL(ZlMhkj%o(7h(s?s*p1}B2r_^+`GE`@aFS1WJaMW zr+h^$ewJN*S`u@!muYtO>G5_qH)Y{SsXt^e(IG4OYSu~{TX(K7r+e~|31^xMQo&7q zFm9h8)5qrvF_+mU(J8|f%odf3MU3%Mgdi~Df{`ftgLIe*>|-g8-xCjPL~^L%V2j6Qq8 zLJe~WA)_~du4WktJ1 zDZ$gt?m(6_sU=H>-o6_4{jo`f0fNK%o*hwG&l@z*@e#S9*MU*2#+VJ~ppB3ge%WXy9BgFGikP00M`<1W7=qCEg7os6u zZA!(equ|oN`)gcTmeIG%0Y&8D={vJ!xujqM{EcyFh4(FAd-lSvb<1gz z@~=@`Z=Ef4dX#WyYCj2i2^!XwO)?i*dBtv)t^|tUKG&U8apFRjZ=fwJ5k%A4-rybIuv0 z(3y1fFaV=8w>*IAd7r!~Q4-4?znL2Ux}}Qef+JjDtabD0)60)u)zmdpSjRF#$|wpX z8&&4UYI2*c^)3`5&hz=!yXfyn6`b5zV^Dovhtos%*Gv1pPf;@sAl!(Wi;J2WD_*&* zS!{SgVKe1TQ@rx?9)ozyzV)f&T;^ayT$X6R)u!4Nej;9$i6Lg$0CI_*~uYIzU^dRl}Wo97i7sZGp*ruD>y!XH17IncMoQz z#8mRq>_(kE@2hE5d1ACb_$JL2OV8*0#?aEhmd_lkYjuGHvbts#&LeA@>HO(rOLD<` z+xLm=^8pRqvoFP}C~ZZWHJMysJL& zXk8tKkCv>sJ_mNAcRsq_Ig>wO8KG*Hh8Hx)6cU1Xau*i{vNXPUi99~gyNCER&*$TL zVC96qRI?=U!1s7ElNXU5Y#`{beMW3HykRULn1@BvoF{xD3tirVjy<@gV8jj_2>*7A ztED$|2v3M4Kk>luDoie~2y}(q!3*$5k-(OXhl!NJwBam??j{Pc(^1Bm7+XkylN6_$ zXV9e?h79KJf}A__Vx*iQCQh;-VIqU^D!dWNTR6UKtRCL7=oE_iU%_alvpl6OK1tk! zbUi{?fQt|R-#`EF@n~keHx7lsm%?SmCV3(o16lwX+$|(IdT}KwtXPXIO&*`OAmGEK z-G1e@04HYeu5~~}U7XG1G51we0-WSiUguJPimtj81bxNnb}g%`sV>O=CbgNRsixAc zEd@9Z;4o7w)Z30c7BX&L-KIj4=qPfx`^>5UuB4RTVGwyg!9B1~BR`DEUf8#p?|V~< z^}S7KKwc*vtVYxi$sUbt7I|^G0N>bKui^00>yW&Lm+pQI$XVzO`!p!NrPdU+dldpR zz!iZj$Vp;RoEf+Ybwj+;q5muVtB=WZ6fRsR2>rXiU&8#i;39MBrTBx~!owh5hY_sH zHN`s4J{7m5DEJ(wnR)fq@!o{D_=w@f5f?~)fmPba{;3DQk} zMNzB~rNehGBh-j+QPL{k< zhfZ<6*SIO4D2xmUHv5Cz3i|SVoCNGxFToY%i-Ku|>4wnw?r&m)hO}>ccYnnrkYX4B zX(mGtCztP4IK(DK8x*%Em7kN$#`1ITqe~j(EyI`&V=>nwU?L$f3oKEOYyRT>;nB;n zGIXV@*J79 zKzUn_UQ*9AWXq;Z?=nM^B*q*`QdCp2CD-U_YF8>YGYn$vxTX z8KCkxiraNew<{}Mm+7jmxZ@qqa>v3Ym0d@1Y+F|?N0uDLHuN6V zEkzpx5PrY=_Mt0gt$$6aWeW*~zPrLZ{k{loa*?U=v%6MRrW#n!ESJX}!zR}U&V?H49y&MMp9icXcV#WJ#*G1x zyPyOHY}if$4+O+d6us#&i>`e(6cm#9;Xi$q0xqa`=br>c%d zy8;CdZ780l(!s-ppN^M1*lCD1m|WOkry<&4!dsxe?leRjOeuAc-APkR9cjM&vom06d8^@}RkXQ#}8x z!;z_Pc|rW$8bNwu7}*SP_`+r{XmDQZe`6xoDsbK!EN=IEqMndkd2huF9ubb{@oaAo z1|R=;r6I;S&0@77Hdp17zbYR{oSce}JG<$47Rzt;X$I5syxaK{fYI|Xne{8$QWr2) z%Oa-+VogL^Q>`lBsSgc%ekZLYJx+@{s{;yKAoy%a8F#i(t_bpJ;pshpyr)NjFPP|& z+uWU#?#`k+=J)>an@@BYZx4QdIs9hhz5E)zmsJPd+F)iqOFd!3yNXE6Jk_HKg?XkV zh}8)h)B0HGrmurbda7tG)C93w(=3;DHEKz+(vzuW*_LEdOOGbA^hK46c~f7ko%7knB2GX>b? z8g7+$9!qXmaFuvp5Iya*UG69SyVyhNjWP(#U-+Zk0F};b6j%M~?r$+npTCf=$!B(Z z0~?<%UKt67*G`Xb`#|x)jLDBQxNMz(?jE=HL3s;W=UzVyVni_GYU5T7jLQ#91ojDR zwc!;ozLx6&n~Fsx_jiAgg-Ng64QcEKG1+=v0D2>beDi&fKf*zbNO16r&*(J>8Axyf zSS^f2AkOesjKKAJoPH7g?K1Icl%n2&m;tol#^e(2@4{1QSF9&Q9!7-AJFbAnA&TA* z2Ou!{iZiVAg8*=quiy$SzW~=A|KZkSzRTwk!`6=tVgeut@XuyIkV8Mliy|Kwce&n$@hpkh@sEo;e*dO@G5O0fZBB~CBAXCLaE{P*Xo{B*GCuMYtqD)k11s^}5 zZsfTufwd4aO9|7WF-d67#Mm{-Kp0CT5$L;ddPj6TH~pr=Z;OuH0Uq_`)tRif^EG}# zSq!VqRgxvy=)_W2ky^T;DlfX`O9@7;w~R?yOOoYcip~XgVyk> z(={bo(v@nXH2R}=U1?zQ8yV(f=@mt4>k>>x10ANLcO^`nd(7A7PajnzNwZW*?wU)i z`|O)J{{U4Pnldca8rzY38Q_8ozoGmM<$pDlzj8>OE>mr$cP&G=6-Rbpc&@6rlqo%C z+KN7&>I%BO;|*B@Ov|Xef@E|hy=$ASXK7MTa!gRJWsNyqMKx`MjbRhiiPBR+XC4g@ z=&Od&nY{5vPwOg5Pt$e7k`0+!j@#3yt0>)W&v2EomN{2MOBZ2+1XBHpH30S$Az5<4 z=;GjH8Zt|znBEd!9%w{PuWn<{ ziOC*mk<%!2Kyvni%b0*bIS^iscsg>=Bp$O?7(UBrAm%I@$SOb+pzUC-Dd(Pb-bcmp znMy%3;4Tz-3+GGZT^O_ue|G#NM$^)P7lW2(CnF>2!vi(k8|aVwbGyJATcr;m53zzg&%us63zEdWjrJ+o!(?^UC_c3mGb$jqA(Gif zTf<0Z(#ikwbrRU@x6|s?zUjh?5Lp%=s?`ng>CnW=Mc_vxh=b4>(ix~>tjznkwdCF@ zqhYKyY-TF*!VC3WF32SFQ^KbB3aTtoSAmtQmM-=Nyk7o;+au zcw{@f41`X=0lhw6-9(bAe zZ}$D_fmUAfx*?&OcQI)=h-ObB>VsDljD{M_Vq0DXFz2qb181vbn8D+p5)8 z4Sg9%No4~c`K;h0>wh;Tsf}x%3FB$dxOunZuoqid13he{OqvzrUyGOhrPGKWcWHT8K&3(s*nuJ8A|H_9#+`g!sgc{ zlR0Nv!xaEVtYWoRgj?|x#kzjflT4TfYK>&55pYB8+hs#T&P)g*LeYZIB);$#xsS=9- z^}07a%*O*3Y_|EY6~y3NVZ&mZ=8aR{Ac!m=JD2@u_bX%8^|c@t?-M{29C?5jJ&60q z@6QhpD&wwXh^+HrY>>mu^@J{a5QOXu?5CJh)C2)EQwGlJu(YT3vDisofe)eCC$#Y) zH2V;qkdgS_z_uR@cX};EuvKdh8}>dodq-rcA^UmQBr(;?N%>n7rLB@JbVU-}%(lU) zVAYE{pBt&^5rOm<<)Pssr=^NEijgyBTrMj~&ehE` zu1oZ^P2RFmS&&hJ-!nWrYi;Fz9Gm3a_S{qnV#9GG?i5BKtSaScUZpuql~9TF%O_{1 z9?!0tXsUjSNYl?ny4ZO6CL4+;XYDL``_Iz&Grsd5rBK9LOV1BpkMMc=NyZ~w_db*( zTc3g}Wt5Y-RzA&|%5A$b#`@POXTlQ#Lle5^gWMKB71haj}co~Dr(aQ!PUiEMS- zzys5EiYW|9$>V(shnxEw8r=31*6@r`jADV!wA7cST`w($tQ+~Z ztNKzaq1Ug3-biv?L2{P#lBniVT^sdX8~pga_UJVQUgm9){WcKX35L9jsDdIrRu)FJ zWE@-<&Z43y-=tpyt8SA6&E!9+FNvD^AO!`OO6O-W7um^zwL2fe zS25P7^T`QwKpe#QDIEJx2@QrxpY*ag4E_ppVTvwBZS2GHu^VqF_#h4)WEV$4F#5K86q%@F-rNS?A_ z!o^+`DgRy3tdE9X{w{Bj@E0e!p2%uWafgDr*ixS89iMc)sQ5sp(NYXzMfS{wpr2QO6#_| zVs6LbEhWbxEYX(`xOo`g{S{c-Fbup(NjLb3{q{@gw_@3~Vmnjnx}Ah*+20R+US)O8 z0&@Aw;cvfG%DdHc1j7l*`Q5JpFcY|K(|YZP?j?=N{f+%n?o+fwp-XJ2*sVI=?OEtX zcYg!M_=RIEK<`^9061Y>?z8Y+sYD*U?)~;l{ywctRI8mR_3=uIncwGCamzhrKOCe# zR3=CLV*C1d`yW`u_v*0k`Ccffq};>k7a)?cKH&Z8b{RFZfXn3WuRcusHH{+A4f(QR z`B=DMs~ZNtz#e zJb%udB*KVSwqS>QAhIx@hqS^?8PE38QE!N0|NP#m!sy=tQW@#EFNf& zFwZ`Z#I}9bkGHsB0jmaT<#32KuJP3nRIy8V3xA74xLV+2Jb(^RKDrM5Yc%S>vWb15 zNcl~L5JB$J8VeyF1;WB4qMb98446MG(lLx2MR6heaEvUpXW>%)5QWbF^S{s=5F|sN z24XC=#S>X81?^~$+{|7Bf=3afLvZRNP#_9561SR4D%LzrUVOi=p1fR1VZ}7ighvoU zuRsN85#a9DH0CSr*te~~X$5XaZA%^an~!!>h;gwdP>#u>Q~z9q;Er!WU+K5~Y>Qg3 z39VtaMF{2P1upx!J-8Zl@MxEd3u}_xQS}r&clL^qi_c-*fDy;N5hcOwiGd(4E?m~) z8CDp^6pYMAD*DJVP;Po3B}v)drkbj&noe!Uk!4k~scFL+Gs96FO}1>S*wE}nA0Oco zoMAiekAE*lM$>DYJ?!%!?(=RK>OpS8*J}c1>#0nvR#f zU#FJh+TSO8KazL*VJ~ppJl+4d4sZgA#2;Jw5=*_RoDKs0aSi=fXWl? z`tzVGPo?ToQ)``(P+B1oY+1@mU6t})N?Tn=JnD(7_@KJiEYQ}Ai6o`9U$BY$#mufc zd)ogR*<4)JfH~voq0aG@(0f)LrJ;DT0mH zdUbZ1eMJ*`(nmd&JUiuip;dop+%q(xU1QX4#h+7XIrru4)Om#_v~HHxr|7&Xj=RQj zvopmpG%=TbBuP#MQqL{sRlttoUmNGc6z9Y05!2HaK8hB4Sv33<$dg&#dVniRIP{8W z>tEx7e~l}vGWvE^B^y@cS(serVTIn| zS&Kh3yNWj@Al9t}#7+PzaN0%XmpN-gl+WqKLe(@<5jRt-8Eek5?ulYB{#7*(|lmz0vB)UvM zmkP;#f?+wJP%$89msJyNi{BP|sWvbrk9aqN-#~p!?*1+Ol9@Y&0kbwoIt_0a>o#vE z`0c-il`P^(8L1!G74i%$NP?*1G&L2^lJ5ED)A$sw-|6BC3TNfEq2R%-PU=4V?&61ciQGj8PR`HT(8{(B{FXi`dS3>s** z+DsV6L@gRrQ7J(*ac(o?eG)&sX_baY$cy8{KfOewSL*zNAL1*oH$=txx{e0PFub9W z8y6=-2%pW!&XodWcrfj}G!BRJK-HU9(J8h+n(}DQh+H+^JHgup>kQ@gw;OMG1&@Ga z?t48Z#(~DbIO+4XB78sj>cg$K-eREgzk}5 z^?AqFz(#36*oYQ$TOd|ELPH|T#loUkTCC#E$K=KCs}C>BlZ>w@>>6M35Ikm$y#fD&q+^CDax@A6; z*^$a8p*<;kTagPn67@OOANN3jA$vlCfeuNAd|1?4K^WNe9qQlxT27>%F+&>mK$N-= z5&)uy*R_o(Oal;qDFWZ24oeD5b$X_?@R*e{ryCQ=o=$$+6G|O2JDS;5x=${3s38HU zMBGm^(>U}#3U@j#5;Gmu=~DUr%guux~{}@rbE{~ zLzSyN9VQVB6%=UDjX-YWDh3iTISpa^r_W4d5^QeI)_XS4nkze9g}Ds=XPLmeEUVgI zjm9_btt{$&vjKd}r8;d12bcKsEKE|pRoEGp(yYoQ(PiKo4r4vnQgtSEdsOLC$?8%W zj=QU}vGbjWiweCP6aj|G%;Xf5Q*uy?+T6XIjrSmBaWmfCY6v&`iNT8wa;8cRRC(&cvo>#8ow*V>J+eq_DG$rfU zLtj2>OVL+AE7MKCqoTYU;+Q_t8pbhI3;)Yn5>@ej%I%~V2Pco?Y`B-}Y3z1dnYg>K zhW?aKg#AwXO&(=F@Vw5s ztQvAxk-7@*w53`(d6!g&f!iyn4s9OYGkPxZ+SH9fy?`SVT~pXg<|abjfX(7EJ-*D} zJ;m>b5p{;8gEWjh7pTDIz47pO+0qqLvr11j>oQyOP97U*Sb(`D)K&UMYLDjAbnk6o zccu1 zVR#qJZ>co6j)~&qDQ5Q*VK)nK?8B6eZ^n0JK^dYn%e~nt=IqP{;5y2*DYQ49WwXC#rCFPixLVB4$DU$S=H&C5Wn+ft^O|NQ6ZLsbF3*^K zj?)sEz0YfAl?>$PHMxRpJQ1xp3957YJ{XB(H4~c;u6PVjc+O173829nG~jhZ1MbWV z`a1@YvyMo=bEYSjabo60ud64ciu2`Okme?YAk_hHO^QOMDfxVI5{xL&R;4Oj>8m9tHf? z8LOv<4>#hljZzcp8lIl$(AaRZi>Sf9qgt|f2QrFUl{6Inp5Y1-D+^s@{- ztY8E)5YmczSwc@-HStU#-BswUX@XUWw`JcwIe_PJ?R2Ap%_n>s>sjrfI)=eUUwa4v z^ifXB-02H-cH8MW6B7;z<`S&oAz6+HkSW041{Q-8ORxkAuq4I*C)F>JN$d&7>4?6?T|QOA#;g|&NL4Yc-f7f{7& zh;KSZS?6g*RjtLzf7&53vQb%(5ysLpg*P3J*;6wgURSV1M?he?Y6$_oZhJd;mD zp{}=YuBJL}ql;jztK(XZY4=RYbPQ86RM&M}qh}eiEg4kn3V`QobzH;&iM#~XrJW3} z+brG&7NE-=S?QXxs9;fpIx?-JO%dE`0olXYp%Jy|Cy=Dh?|u#7Bz%FcXpe$u!)=P? zI~29CwKoc_B=j*7ywb=+2T~;Rj#TbSwgpwXd2$;4Py#+06cv}gG6gfNL$&! zX>j)|=nAUzL!DDz-1tY0*wWU4LW8h7W8FXTqB18DXsLIOH+2;jt%gcDc=x4z_hgZPlKf{(xDg-Qd3(&vinw_J|a!3#bV|e$koRYBh z57fW=SEM2#V$>B7KJ$`yARUf;hOh=lAPRd!PBvpE-%&SzEp_wPyqkHd3&Zc0-qlRY zQ0}Qj@KRXECzQ)0!z-3mw_lTDz(pBRvkl9di)?(jy%6L9KH2`d z6d<`|Z7bbv$!IIe$^gkVWm~n-*R%m7e_nv(lf4Jc=;L#{6rLsS_|ui;*8sb`;vgx& zfHyctcBZ>vih&?CKGWEfANEvib{Qp~^JKAn01>+(RSdlT0t+xn4`Iwqd-=0q#AvOX2x=JU?Lp5P!|F41ag z!wcqNrxxn@tl5v|EP6sh)V!Qy|M>m+;X!49l~|Aa;T}sf$$H+vqa!xFVaD{c+2ey| z2FhCrYniF##dym)x>YA45v+rUbE(9yS@k>yZIGQCNIb7QBiS-$@K{opXFm z({W8l?i!L|*$&)7hHg8$tH_R{%POtS@g0x0nW+`(ZSx&UK+1%L^230_yQa!|irS-= zMG*o*@2aM2F-?{-<38I-zYU(bNVnGmZO2PLAlYQ(!AD z9K^SkWq4b&4CQx9UDTY(*KlmVz}D?oTHYUTSv*>5u;whD)70`X!&uX2OHJ0Czt?wv z?_-OvD=ogR8Dsd{Fa1i3HK*){+c<9RhY`KqA{UibFUDIPJzi?K=78)~IPZGwr|S^*8wWEErI3Zurwb7F z$Pva_`7D7+;rw3kX};`;*@A=tf;7k%=ZOt{4au%QxS|B~K#Dik5Zdv{5srhA-Jd-H z?{*ZB59n^e)!r|x@EA_Q5jt4}Vi&MK>Jm@11Oi$HVF%6il0f*X_=E9)IXM0zoLjah z80&Y$OGfWQc|z55eesfy~BQ92 zSjHICW5)-rw@rRvun5?}%@yOt?hvRA-5m^?=MZ|#iBK*q42};!+vALI5g%^AU|FuX zYs&V+5Z<8y%z`Abm;sF50rfT-O@4sUXY@8EPPPfS@TiTEYY*E=2i_*T!~rcAw8BeN zc3Pn6Lz{g>8K)6vB8b_Bo%CUWuvFiUAxUTgh|#)$6T*3W!F33?s*9t=-DwTO7T=}P zVca4&KpEMnvj)p;;eUG=3W!a04P50GO3o-GZ-GhzW*{^495SaL8^RpHV{aXTh!Zjwy;>7Fc zmG%K6W_=LkU!0O}iPT2uwwHXk*k&42$)G_{OVXTM(FWQ;_+)WS55y{fW}ftI)<9=4 z$1q4ZN%L)uq0haO{g>p$*;PpX0G(@+EPq2z`Daj?P4u*d5kqR+%9lX!it-BKSOa!R z)3E~Wg^o8GD;^inTd515acX@EOEpYfmfbovVo=PpT@J2^3tQZ>^04i2Sls`VVn_R+>gs486xn*2!&Tvgcyf*gPRNg&A+o5ur+a1X>Z8q1sjuJU2v{o9>Z(_ za^{g^c|t$pzPn9#Cc`AnH!R;Oex>EyG`)yucBOG>RcJ_bl>VBLp%`CGB6M*NM|;et zTi8r|EZ?os3di0TmJtY$-^uCwvy0t#`-dDx)CFAwxTQ-F>>mDm$u}zK`SGMv`YLXs>N9?@k(f_7h^d(kq`O`Gg< zrZKt%n4o3`Pm++|i}r1L72=&Yy8W`7Wdyq_2$T;L1Q;B-n}EQ<$C-h9@-`A^JH0&d zv|>bzhkOQp!**)LL+q~~@@GcVtZbv;CLlrZ@xkx{JP^pc@{KQuB9|x+4Ea(dF}^`b z09+9-Vu#;=AzZR?4hl7m4W(?e#E9p{LWH*;Q;oC$RgDmWT# zSyrshr)v*HQB9LuGa}0mX&Qx{!Tm^O-2|@PRiG8hT-7j5hgq!KW17^fwmP$%G^O7zM!35@ z+pr}|?mCVv8JcD@s>1p?Jw>Ya`~4|eczG!dQFM>Tt%K;xyq|ge770P{9A+ap-G5Ak zs>vN=Q9w)Sbq!ULY=at1brnmosjJG4Z7`+U@jeFZfG{w%VZEoTFr}WO^<+k2T0Pa2 zOsS`LsY&(J`(J3d>2yu&XnG;9%HEC2f{isiZ_UJV%To(8P6Z2_UpU?pfjZ*PvoJ}O zjH_Ea^4z6l=WKmAWc*bzSPhVU!GAc3e|)OdXDEbuHe8=!Ei7T>9Wesc2Cz zCKt*?jB~oVCFN!4`936{Nl0QvIjtmL55{4{zIHOfWd)6&lM31xU>ESaPFL>8Mu%l9 z-bTl1J6C1rA)Tcd$$gQ|3;=VFrd>b8){mExSpkn&Sa^YDP3uLmk|z2HyFOumD>a|i za);l3IGS&V@B8Y0u)Uw#7*lguEJ+;ixqHC(hG?cnqo(*}Ln`Y|Dl=Xnq8ide8fhO` z6`KxOTT-9rMy!3HNz4jvJ8lbjbo#Vbz{>820PoQTaW77S2O9Z`c+HYGQ+iK4fx8Gd zFBv%Lv4u}<<|ZA%LCV}?YxDv_vIdUrwFAH34!o=OWq6I*?0x2T6ie?ax@whPsMC5X z8c*qY(bjk$S>v73(~MtbgT5n*IE%y5IXOIN7>rn$I3E$*#OO6#{3gCXi%~6ed_o4? zr>@&QAuw@8GL?>|n7S^ni{~^ed$|Ixxb_>0oTHZnE=Ble?{YZccipM*J_wAG&S@07 z*eIN!;xDhF8{jpApZ~#g|6u5$wB5;5b0@T-jatdmYYR=KskJ+myxa*5 z>Z1lnNwtL*M+vn%V>!7Knl(nvW@)sAMr%@NcN$W1C-h+!)}p+kl?2+Iu@u}1ty+!y zl;zLvjB;)#w9_rDK)Ibt!tBmi%I$!vne9inQzpczA?#w!lZk(EDB z8KEbHbH6DT^pzD0nlsnFjD?&TV7bkmp#6OYqbw#z)f#2<)BWvt!h3cCnL8sToWsU=(%h)=JpDl$DR zzTiwpuaz57QgTh!_=t8+Jm(6l(c})tsnZ$deAb<(=^3QOQSL=pm!ag{T|53$8}Tfb zX5LWh&W3eNV;z{GI%Gc&cNU7P24OIJzr-!3WJxZ^ie_qjIev5z2DfD75UxAuxkHzv zKHjMZy*Wa>?%+?4KVO^5a5JZN)UoO5c_dA>J|;QRH(a4u5^iGd&yr=ErnI}*IW3xL z(YgDH4&!YeeusAR-M#aRv)%m*{H0U=seoUVzc;A-K*%Pv;wgq!(Ei|?iQyGvr99WU z%2Xk1D=jjWUp>1)5`U(Ihey_3O=*-P;X#kZ2g(c5r3A&m|tX}oCNGrKIQnV0i&1hDZGwh!V9IV2aXP=h>YO_}geveTdkswYYX4GzO^A z;JVQeX1c)zf@{+~-i&p+wdd@yPuZhg+Usz|^1#QGg2SK3=@N?WT_TkNLB1?yIjx zr|f*2KU7HH7!4ZCTP=vLdxft;<%|5f8E3Q4L8olr}J-bRLDNc`o5VsVGwQJ@;e!53H=sEUIV> z|4+AQ*Or~0X7+lPOEt;pI%2C>;S^@;eT)Rm7v1dY-ELQ$ykh!HmMS!P6O{T*P}*cK zajtBSFZB887Ws}2zmStRTjULJHdwZIuo3TnPob%APAIUpTXK2=KZZW0XkU`^gEo<2 zSZ#6~2A3^NeB+QQ@@S!X7JQhb8TDcVk@_;km^RrxIyfiRHmGE33-z-4Es(lqM?!WX zsZtC3lq~qGg|dM*=YGhr@I6guumdVHXjeC73ByZN$B=AluvzxQ=vtDZ8H;U)TE87C z5GD2dG{JLl77i~lnP`g~MxVVez`T#(ONx*UuqVLYfw*DAPCYm+_zPBp;(L4P06h^8

q{2-=N>_oYbl&bCX}Y57OOcgcPgZ(+L^*Hq z%q2dsR+#-hl0yXhfZKe`f-mIa80jDx#z=6STH-6%=xfTa&jMqIX5fYe5*)(PoV>$U zVaFry%NPFWhXE-{Ss9E(H?t$N^9er7=190k4S2X3l5w_5#5R zuYwpFKYR~&h!=e(ui?pK+P6a>U;0#p}Zn>F-u!fs*c#|7Fw zNqisK$Xl5Idm9KFCUP43oZ#>vT_xlM8a{Wf5^u=*6iF0rvhyoO$tfJRiz^noAuujn zz#piOzwbfkKQIqv1e@%_693=@pYcTQ?;`zUf1hv3`3auuTjz=<&a zWeSp2lz7IX!v!!~>R1#fg=2qlxE1I%3LWx}4t*Zq3#;Xk@5wpYC2;j_%bKBEqy_&m z)CLY~s+|Gwp|-5cnq`>q)>5ays6PcpyTHlLuRP#XlzjIAmYpHJ?1LNuw1xX{jAVL= zq!=Nm`+-rPA*KE43Qj)x4vDvK^V73WLEOF!ICYl~z<&7Ui`P14gNW~7;S?OgJJ5C% zhM$q$y#hW3nk~@WX_1!e$Ty?fj8f10hz9f)-uvM;@Gb>^FLCkR0FEbc#Lpm@;ze=H zhSxv<@9+#bSy~~U^EdPsqxSb+mFTWWkX&RoF~rWbaPnTx-tpPNHnF}vp_Tn$Tq-I_lpcTqofIh zRN_kFdE~rrIXMjxJ?e~qe+kbacZQk1rA}z$L&v96Xl@S}Iu{Gxp67!9UIcqc&fm^P zk;R(^c^)k0c_W=}k#|QR+L3*R0T7(-fX+Hx+WSBq2bMxaHgG+byx4!U`{C$irVt7b zK+Xt9j4uO1tCBB@1zdtaufcJ64&(x_5`CM=ASaq#xt@?!Q_cdm+5_mVojtt_HCy%6 z4B8LeG`M82?0BfIa%9lUG>fxmis~m(kNWQON z*IX2qj$u2+~D(?ZN$lcl}vt3p)G(uUu-|^zw>}8s{PT5f0cn$a>+y|wB&+j8DlsSZ)01PyJokEz3Y01Kh;T}T4utC;$3Sd0x|r@5a__A zD9gP`IG@nyjITLRRZc4s*=5$tZ>w>rV?!jD{vt)J2R^Ue+0C{6WDeXkY!_Av2;)itp>4JRl8D$c%S;Q zY)A?|JJlrRhV)Y!W*l#CD_yGF8dG#8xvnnDJxy^n#gSx3mwPfZW-<|4ri_3bi`k^) z2BQ>`*fG0-MLIu2HlKX)!sO?a%K*KOAQTZgGNNwEkGDaBydgJmi;x=zcl3~Omk5%m zPi^4jnDvPI$_RKf=hvXmP(S{M)as*eTbAA;R*?~rTC0x@Vrv?VF7!R#Y* zyHBcTYDiYQD@n*BOYBpbQ52&LeAXK3vlJx zXmsRuBVL|BCem^sa7hE2aSq0CkuKWyk7}-QE2* zH#mZXupIF!tjwU$n)*+Sx zELK*5#Wwn2vWP83>sl>oNqM6|nUrCB-!PP;xVrRqZ26Gooq=-@KB6iY7PmFKxwFfU#3?zVkNh*i4=C7M#bBkFWo!LT1h=&r=Vi? zLC^CETvnRdQuJ;UyOhjM3?hz&j#|0ZRScuGaLf%ZNn$f?hhfcy!KxxSB#9hJ!h6v%@&S^BEBeYtP97I zXFH^@))Bi*1UCG|mE7St$7THSI2rIF1Ndh$%m^XiL=p6S+V7{F|81@q$X#t)Rl0gh zRVA&KJ~T<|cF3nxDQL@BM{P+T%455=HG?FkamGq18;~PtBX-*Z>tV`xVg?UKod9QmPUb8{+fcWt!Sgt6nIMy z!=JH81@UkDgR2m}!NVPo)xMZ`8|6qN`5hyHLaQh(_*a9FmEAEGoWJ(ySw8%wwzxsz zQ!os|yG$4s)i~!y4&XoP=dM!Wsan4vq`3k3_6gqV30&plmuK^-+64n(;wi-y}Lw!dieYua=Xx`pD)qp zfVu$fy9sQ-0q3Ob7StRxU~&n~#o&oU^xV5Hx9`FEJM|DZhC3%t0w=K<=arv9*b3+u z56|hI7hi<~>!AMtTcrv2s)XS+D1F>K zQ}ociVGeK#t{Iis#_KTgvFX#?jE6K>twzJKg?fCGL}iE^>4yU;aonNEMX>*ZPa~_i zkQWyyneH@$wDcxQRzZ3X`mzo@cN*1(sWye7;@jJXrJFJXO4UuKQD&N|t2mZUnIv^J z#=!`tEABwvx~VK0+tZ-9D=^nRh6=&S8IlVMv?u7{M8y<+2M0hRVNA(6l3$St?ZO>! zMf*G&3tc8$>*5gz0Vo9A8Sy$~gB}{BPj?HP3Mm6$5HS(og~@=N?Plx?cM&f4EUZJ6 z5)l(htAk*sB?1PE!WOqVzE)a?DZd~}Tx*YO7P4e+YbMBS1pZws*%9FGmeS#hNDD@U z(OxZ(@~q_;dxMog(U7sH7O1@_>G$DmALGZ6J6OmG2Jyh^!+PQ^j_l-Z{^UNMQ|Rm# z-hYMG`9#EO9032gfEKQVhu?>`xPktFf_Qd#E_MQ)KJeUIlzaj_mEXZaG1Ovr`O%G2 zd%Fll`5eNT`NBmtXclIOJmmvLdn@#ObR$e>JSt^~3A5*g!_0*?5K@l%10-y>T%mvt zsFV7vsgn#@N0ck#cQeb-#bSPHuh3kA@MlG6GoM98{ZQLBb=Y51Z}QTNGI|utXvuX4 zd5(Q;uu`DlP}t66?!L=6I@`u0v@#*HLEtk}bK%x|+V^9yb_S z6g0BX;D&N9@hf^A@k7(e2GCI`U(@fQ~Ry^hT69LINCLy0nHJ5?HPx*wp7AGjl=1TNiKg znn#&eBd?4&H%Vedu*@lPskLpej9`@OTg;u{7cQ1q=iXMqJ6H?DsZym_=1#8MX;Ua8 zmen20&E_`p5-JM+9UZrf&gDQ>rD8?Cx_~!THBsNSnp|Nb-VLN2eaWS0ut_LzU7sY# zT|V=$nll_)FC_0ya}O8ZWGExVrOH;k?vE#`t8vqs4>GOiaQ{sz6<|CcXy`3;+%-(E zAQONJ$$^I6wG_=-(1D{i zm=RQ9Rewz2YJJ0lTS9Vhk{RO8#5qdc<$aN9och8>vbZhEf0YWpWHw>$TqW-C7rTz0#B&If~*(w~OKY5Nr@y7i}IPsoH zF@S5B#*P=F%JVL9DB`;#$G1I7KMICrRE;AgmjK?fOfwd@>D;g>H7M$Dqpj`j4o zuH<2mM##nNS0M%iWg*f9mm|RfpQX@0WLEp4;A&Qr6xSCMs_-V^HSM^A4R001r455o z#YAp7hQ?tz!XUK}-~I!^5}8Mw_whsrS$lDw33RjoaW40^Jq!7UcXle3-9wxOxj|WOC@LcrU z!R-|DDBB4PSVD&}FjFA_$_xlh}x7u zprU)oA5e0@6KL2KC~@5E7b2I0(VlUIeITNR(oaCFyH|K4;_UaY!am%QoXFW{3Ih9C z&WA_A7MX1+z76OgW@0fe;bKBJk*K;?_Ib8*6DtoQ(6wz9G%&T%4hI|px{BJeB3aW_ z@`{VolpIs0%%GCpb#?RxGqc-cYFA>a(VHo~1ISO37WSp94O*e9p=3GqX`k}2KgK=- zF}~iyQcP%p->VHJMa$1|5`c6i$aCa-PGhuA&{K-MNJKyPQK5T#SlrdzNqtbWt$P@; zhYw<1QsK)_1M#+b=n#i<#!bcWt6*}Eu3MI*ORk~yfFaFv`Zh*e7fIo>l5u?#J9;VjZ z{va80VeSm>7);C+RSyv#s_@)?_8ia#TqFTSBE$3yx*X?e(3w=wnO?(qeTHOq#-pu4 zVI?0#5(}`t12iCoKFq~MIzM2;Ar~aj5S0K?43T;deIG8XJ#=uu;6?oZYla>vY&;Tq zEnyURpV3V5CIoFN3Q!|?lb|o+hb?kMog~Jljw0_15caJwgL;@F@hVr?DRZvUlLPFY zhv8xv>p4YZ$Y+sJz=M7a2ZteV{gk5Pjvs$!6L=DR95 zo_1Xd@)KH0&cBEZD;@?NQBbj?A#VMy>k5D6D^MI^^G<*-{+PsG9|`CjHjjsk?M2}Y zh_N4#I|i?l!H}ZHf9e65pi0#m*=Ei?xE_HLeZZzZxG+a}@Lg;ppm2(aKd{?ll7mVj`E0&+4 zr%l7i=pY78fTWgt(^?m;kvA=6=6C?JBG_T-mBU2hOK!HvqNST0Y?K4plYZh4y@AgN z8(syde|fB6xHB!d1}w9sC{iuml9A=kwAy-m^e~^weAakZX2AMQxCS)K>DrcN8X!OP zG>`{On_>{*EY^6{>gtQ$0}cAV3i^J2weN)(b&f{81hetuR9}dONpOoC=Y4_!o(IBh zjzYi*zxO|JQ}Mg-65gV%w~3FnXLh+qJ^a#omAiaT>hd@<2m2u~V2%=>tIcQ|WI^^# znqgF}te!^m1a!zP^}@C?Dm|ZNt>;zcfv&C@FkfNwtyY&8N=iF3M&w$GA#cmFfdSUC z*=Pw=JC5hV|E@K?&M7-F@z?BW7ISP}b3hr<3`wVs)@8b*1IyB9WN#)`FEFO)CWUX} zoo_pN%08z0rzjOXvmd4M;q8w2^4m_I!Sx7h%)iKyF8&;Ro1zTt{O+Irr~mT1PjMLh zZu@ut<-cc`FA<8g8*_cyM|?2RCWx9c5O+wBx)GC)$N+5ZlU*L z3nm+Rm$0?|5w`H3|NJ{$`mc+$LcMJ)3m$=7h`5;4 zpw_|9;{`C`F|3XGPw-<2SObR%Gj`)6F=tu&uk+$pV1MyD+80Ba9WCRZ#6)mk#Ilc| z`f4Wyatz2Q>3|U$Fv$8m0X2)>{oCEIzy`ca%*&wSmHdhPn;3IXimwH~{WtpCFZ@OR z0Pz3M@T>e!Th(4pu@Ef+ce2Y%?!txpDk*nU{Ib|n@zr84%6KmyQs*?NAua<37CRWC z7zJVCmirp}rPv{VyV#RD-qU&L-~Ed~^@_=>1oMVsxEKAqe+SXF+}CSA{OzB!Z;Smv z8}zq-21O-rq1dT5-svvfmHoTFff@yZ2hanruUDB^{#CIj{-OsvCEhG{qmOsc<%G=M1FDC(&fa8oU!;c|nN1Abw=EoBX@-suf=?_G9tmmsfq}hl;a${@(T>NPTD; z-%dYK!u-2`b%t0t1-`=*9-)9jhvYxs{d2@H&;T~}UxwaK35!Qqaev``=4U#uRLy5O z>hBn1H-kJ^$jJ}=^f!IlAH+@1;vBgLL;wcEZ+kTM9OCEw^?>@XuvfU0QQX4A=Uu}^ z%zbd9=lXOo;6apFQHkKql5*O-N|NCCXxWLqFGBrvlPI?dSGvSdI#6ZU&y*LqZj0(M zd0_t`P$Bssa6*npLH?t4I!VW~%f|yZJ})Fi;}LdE8zpuxRf^nF>mQC^b&g+=O)SQx@Y+AKx9<@C9%4UAKBLymtzx$QO`0Z3Vw z!z)l^_+Qy7ixm~hATO#FiuSWfVC$x7Pb<9|qPm84V2n#5>HWQXn+W-oXM7=kuZ3X=zAI!iE$VNd%6%UWH7=(2_1-4^PC z0TbZSJD@BC_;3_{X53I>e_d!RX@~kw>XZ;(z0rf9MvK#yL?Cj* zMhLMYgKk4Y&P%1AWs(TNB1C`3B1GbNr$6v}9;I23xy7N#Yov=-^R#+#dn2OtsB}8Q z4h(^_px>?35_HD~2$ju+Y4?9V3_!BF#Tk>soE7ozW^C^G>@{f>PS6=_5siSVIRc9q zDyEtgFwcZs;)HH`GjZHAc+CI**?W^7$&o8z@ZP_|P=LBKXME%4q!&CX{HW^fxc?Yp)Ui-=QjEk`X%$8 z8CQ4G_t;(CQigLenmv2=_Jwwk*tW;&p8YXeI~4*KlGKU8Qp#2OYC^|A_ly-IFQU`% zSX`e>jCUuJ8@w3$j@ZnSrp8zy(g2mQ72`l&=)A+3$Fq|SZXJJSz*Q-PVuI9h zo4`u;EM(DzGx&S+c&en7Dkk$NREyNf%_H{`<$3uU?@XnO$yj`TZk+_)GOWjFVOhD{ z#fagpuVJZ^`ujP_zzmXG*)Yoc$SCi_&X!c~-|X(nGr!9N|9kkquPR@C?mJ(#zuNjW z{7bk5FEZ|GK+}GWrrO9-%PqK3Wmb694ceA(f1Q9F+qYRLxcJqy32=>tbyPuu0X112 zZvXD9=-to3*t?$;ZW5Op4XOV9@eu~F^l3Gf<71PukOQ|9t1}GR25OTEa@&iUdbjG6 zBjg50K%aE?1j=9=X1O8D7;E2Fk(7DvbVMP%;1j0Bf0kiyUi;a z(<4vFYjX^c_)2h&*?VH7I-n;$R)v3Sp93p#Qf*C?7v^xdvohAHD`Iif3iY7u56%sTU+OdXWcY zd#Yl7W<=S1DAM>BjBa4B;3 zKC%^Zr*w8rNmO#WsL&l>&^tG-9bDL~B_5+#(97OMDKBc1vYZTXKev*bDQecYx5n`_1A$_E%|hcf^)ZLB5arT=clf}?n^N=?%&MNigJeQI zogf$Nm;E``u}6e`3cHWj`CunzlCKx{mt}?oRsA8VrtQ?Ybvs&WEGaLhH_an>$BnRO zXp$SD_NfcC<1%548}sYeg$>njv*GE3=wNy0Enu5VwX#&l09D`yeB@0nOWjR3(C;-f z?%1$%wtHQ6)XwWZM9krSvO3TY2B`DV5zQ--lA5=Y*AS5oLRAR zjWQ8>qN-Y5l9~$EZQW$WPGpjfe*E%>5Ib7=*vbrh*In^+gIr>4dM%efO8@?s?92Jo z*lC%Ar=BzVdC2|Q|Kgb;7RVg^GG8j+LErBA{CQ$yp@jI?d~Y9f#6a*4eb%9lGv{m6 z9eAHW2^wINzw1S2nslOw=DI-ABAOmWfA!(~sf%yC4vB&Q;2d9KM`3QA?u&-Lrf>63 z7_O*mo|zbyn@fuwV*jB53@j2fLSDik%0f8=YGV&WfZ{*?{J$NTp6m5(D%!AONe<7A zGJtO+O>FJCSFoSV4WbC~J+k{7@^+8B-51AZm7wG}7UKcMA&-;FdlZQL0P?XkLuXw> z)2Y<8I%-#uo2IUI_%tDWnrYgS#W zLY|q=$(Srcu=t%*tMKBfs}tGVg!+w;+S^^gaz}Yu>W7Xs(i*KbWCXb$V8#`-9qg%E zf*Geic8YcvO9fo>fhfQO%ong(rxtcg5`Kgjk5QdN zY|OK`qj{^Iyqv@Eg@#z=C@uVEK-RJtMaY(~w6o1x441IN1tEE7s`kk^WX8u<>p z&|)u7(bby77EHWbFn)F}(IwV})eWrft#L6`2EzBpH17 z)9@1aMpzYU%}B0f8#o#Rd)tsRwPVOGqKnZ1~6eb^Tkl62ta)WpSoJ1{9mo>w43ushXNyszdzIPdyv>A?aa5^TZ)Cxqg^YXyZ#T zWuI|yz$_3T71E&roOZa$!3?h-hiHkW*d=^uyCMJlw=i4~RhM&;GOptvs*wX20X_rs z9R1bfkMU3Ufjojtie=g&pQuTF5MK3~uC|Ct19%ohSlDC&qA>`BAi+is9`+ffL5wQG zihV??wG_S9lBFC5RPl2)m`_}t7YD~WYC{^(AlkinFR%F>pMLRNu@do+>4&(+YeGMO z$Vu^UQNoRHM-RTj=Wy!T0kG5>5&3wOZ`;v)$bxr}e*&B%tm%fZqz1WSCZb8m#eH93 zar$h8IpmqrtSBJx^PwnFDmnqt8Q!k}!<>Z2^V;ORXukuX_WaS$h%y2Het+~MViTx) zOzhsJ*GCDaLTHF}(s;=}oDsv-fM#e1pdW?TEGlwmQ35J!@-TA9yzQH}{xYfRp?tbK&MQ ztqFiipe8F71t4=e<&()lPQDo+JAzZ|+Grpy`f!DC9QWa21jbu>h2_uU>56pJ&;+DtiWmm zPK+iAZH13*u?};iiBUgb9J~1$uo$+l2l(>)0CISX(~4+zND>C6cpSAG%}8@-jEVDA z8l}0aNILn+ItT^`Z*lYmA3h!ZCm#(~vSY!avym_hT6iAuT1uXsADo}=oIjsPV}C_W zSW)K5tD3A5x2ACePLw(m1zVk)_QBCfGzW^*?aJneiIOcyZ5*`e6oZV=n{Wf1a#0ez z$N~^^LbHASa@cq7V{g(OkS@(gzPg{eJVIxxfmmnTc8vw`g~8?E*FvcI8m_ztF&P2< z&jV1Ur|MVtDEFr<)2p#a3$8m!tM}~pBDxEK)RDS|uF74lMOCX~7?NbrmS%Mgxv83( zp)Z$Ke``|mKmPo0shnKCBduaucHjiK8$Q|nwk*Luj@>yCA-7pr<`Q*5s*xH}v;Lvh z<7BLcAFef;AnQyd;UC**J@mRmINq?~Xb64f)4^r==x`)d!)U1TMEBI2nBNmw^V$m? zG^n98Syxpov(wZ&9o_6$s%2@7R!5RL`b2*4-pSbkd3JVq^8NSKci)hm9!TvybelY% z*+16ms+!i2n+?4s>#8m>vun^MWiSCl>2|g8PwnoU!>2MVWudR#Sp^5xpdGo}>Zn>v z(`iT3RkI7sfOQP2H}&R3<5ds~B7KdB)*hYb``y$H;@MFREtWP zWCkufJF`AyP}5M1hOEnS6I4(@$3}~`WWzK}N?WZ7e0=8)==FEJY*S&iiFITpSA>OL z{5R7f zNl#&5ojjRyI$>E@P+0}hZjxzDD>sp4syQ)da5LHL8@yTT@xF1qxyvF4^Xo;o$7ya- zH~MGC?S0MMe;$f|m#A!V^Q$kXBV1#;rLr||JoHXD3u(+CXQ^7pl4(S{@nxgv_yRMPh_p3kHm3ub5z zIbBJ9)oF*DT!wW7t)ghnxN>LqW!LQ3HP`9aT>G-t^R6&Jyw5Cn$50h5qH@_6aHasp z=!>S;uSZwv5$UWCY`e~}6Oj{9Q!*4a83%@8AvVGp^i4kDWSYkZ`^BFHO_SBASqneb zEjt)E^tx`yx&frFxdew@r^HoAIq$7{bF;1Cm$5+I$az90`?+>g^4VJTIsmydM7Q^l z;>eb&gQZp*&8j3}*($lw zR*V{ujg`w_N%Do%YRg7j)@wkRHf69Ldl{^mm^rNs*7UH)M_CN(c4NFAK&`A?lxC)2 zRwhDoyD-?Zpz%vQyqpgI{bVVwBtmmL7u`8!JS81(EGRQGY&IZsyEwjc$|uzH6C#Q- zGjbA>xt)pSoHCX+Wh@bmxt#=XP8p_|4wD37W=4twFt_v3o3rwB8gu67YyuHCfry(x z#7!XLCJ=EGh`4+p;&{g2!!K`_&0~06h5K^(7LROc1$iO&r8eQ(f^G{-7)}qooh((j z_9)!>sv!!q#C8&ryH$^_+^Z50n?i|CtWe^D;k09g5+glKsJGY+b*x)zkwIxO?|V_L zpxOTZj6q-@>s{23Ikk!o z`dxDCN?c@5w}{TXfp#m@(^>2jkG(`)QD8=q2V-`?$5dQbm9e!YL0pdz*RzK->vqxGe))F~;!liwDRw?wLJc@vvOlAH0~BW`*~oR4e>5O1E0AKJP6~@pkMH zkG7=sqJdJI!&<+|(|Dhb-8xEXb0-Ui{AI?(X{N&(Ejl%)eUg$S?Gf2i5{3PbXlRV_Q5WA#4@WFddE$h*;c`E8|HW|HpW~ zBgUM`a3D$ELym7JK~|X(WBJ&t_~9rdpNI(k%pjXi0r#*|z@kdPogTp@S_j;;T)f_P z19%Ch;nPF6>&0jg#c2u1!*~Ra9L9=9X3*=e<9yZ3y)MHn}3-l2=B;%xX{s$XPGclA@BXPWVt^Hc2q;<-|zrD*WN zq$LpUG#jux7W-Vg9QJEKjiOm+qvzsC!RgDFiF^n_T9M;JMDc^ev7%HRHjxZLJwECsp#&~_1I2pG#2U#^p#x9@h&4o`5@ zI?&QJK|v{of`WM`9UtS|$gxNa074Ri*w*y{wLf|OM{L3q*ycwUU3Ih*@R6|{kI#mo z?_whze5@jx4Pg%-&m|oXTkPQFOg( z=mpD#iG$F|)A-R{(FeIO`cJ#9tBe|E2c#rLG1QK!G$o04Evu`_RBjp#aT-~M2FFM0 z2q(eMRTJ>hkc9@eIEd9^5UVxv#=E4TfVkwF3Lu@ueEj>(c?D4H<3}K!@C$y3?!Y{N z&j_Aj)g`0RqYU!}=uNie8_hNxy{craglk$cAha5J`JN16F)$a)k7S}FK>5XDc6eUVMCwh(SyG;h!*hp#Rz*+kXIvxKzU9 zR^cRzaLObbs0IKUbm3FS%(6`Y7~g;Og`~A5*{CYYN~_k3SFMk)`k1+1V8XuSR^)qD z2mT6X1m|3wU=jbRV5#d~O?Xyw$=~X9YiF2~NjpIA5vuLTVXp zRjO)@l~!sLuhc-Q?=v%E5#&`G=zHSBa$??58jw3jFz@^vhsU(!cr?j!oRt93Dh5Cc zWDAgsVU5Qi1bdc`thI+t`;Nu*s2}BIiO^=-jXJw=eB~|Qc-S+dw&*VxrMB^aV?*e6NK&%0CXND&vBgA+p;26)fE?6CM<%qc#7@&7|#5(ZH7KBg2pSn>n?%fML4i5U*?gxBB*7+ieNB)uxj z3ri1{)l}7%)a-V3Q#}Qdv|Ay+ zp+RVq-&BBo1|RJKIo(g>hFwtN1JJ}#qX3|Hk^6C7=F;=5=yID-f|*qQP&7qExnVDP zW>36|-7Vq#!5=7TpqT6uA=L{&3r?X*ZKAar)gn1&qg(;x46zVU4CtC@Ac@(p7+@lLPhv7h(gTv`X!v12 z?o0_X0pi{{$OL$r-jV~i=Kyxa>utVJG_BoiX;r;Z#2crfR@0Cqt=nj6rY0K=(CsB% z*DR__hM~2j&U~n)Gz@I1w~5^_qs-5BGu!{9^vPzWwOK@b&WL4~=t3=i6{?f5t7Fuf|~@*+zVT3l{l6Xg!}A4wG72>9=JLJD|8 z4hA;*V?75%$9_L_BP|cPV@rRbNo~+>t3}$~Gzw;zhSspUj5RD-lN&};lc}V2x+*nf zjdtccNwBe!p)a{9B^reUD#fc0EU-Y}00sIr4ST3!p8+WGQhJQe$CY9(W%R?q41wHz z#N<Sl;A7LA_;WF%HAk0|eI5o+86 zfh>w`6KXun%*40f0Nq2PmuweG@-GyAcvT0VL0kvBl%Uw@*2eQ1&R zj{mU_Fkh?S@{QY(jVyCAW4 znBOA@HF5#q@PT0BT6+v_Ja|Ua^IT{_%iz{9qXcJhtOKfJ*G57sYS;uqa6=u#8Nx~H zaRmX>L77#_;hC@#!6e>#K8or<0UA~KA7DNQK$VJ6PJSWDZOLdq#LG1S$%bKa%}DP%?b9NT6CEOk5%iG6N9X6-*CR1%We&+}BrJl0 z!?t;2`#9lPl#+p;qOuHJ$ybZPne;`G2ZyGGRIrQ+pQP8HCB5k{3<>Z_skTp{lKnw5!M&#iVT=K)#TlSo&c zYrbk&vKOKW0$Ho*P!H(HLvqP8I9j~89@YuUSs;Lt%Su}{TGdvO47&grN|YrTt@t6* zMW9I^7=u%R6zT$1_DPAbk|U{{4o&j3AdxzOFvWs?N=$0#&INg42cZN<_KL4ldZ8$7 zsZ~{0-0c!|MOIM>!7yl`y{|BX-Xoy?erscyA;!h$Xmj6q9fI}*m@map@j*;f#`MEz zv2D)GUfM1s z!L@m5)1lAlXe8IP#~m{nr_3Nnf{VCP>O63;zNr9#q_sgytx840$~2iDw6v~9EuD1@ zquT+&QrEgB)s1eq(bQYb`SfF1*0six(xqG?^2;FdpB}ueBHP+=hQfp%p$rSACov%L zTc1xbbqVTqvOb@nTq0J>F!gUimcFEJ^$lO^>k9f<;5Rvn_x_r*Um&EHSci){q7aX~ z0qQj|bp+xeH{ehJm3A#WXuCjMUvPYxYv!3N!S+@?@w=nHK8ixAr*(k0W}~QuU~xt;{R3qtFa!4wx}pQR?gPalvYzVI&dB2@qqh+))Y-R*CT!p>68s(-8FpS4FJZ@_lZw0WxK@l zw+{JR*;esVLHG67ULS>d;`|E$)CoUF(i3RxJj0D*MTN(HqeH_=odJn(RZR79H1zngl;tDq+Hdv7-}5+59HY%P#e!TZG3hYh>PtF*&jkU zhKb8_^dlE%vSWdOBz!RFDP{f@v$LhN5VRQMJ^DZU_#^w`^BZbUThtihHv2Q(RXzVp z%c%i{wH#J+8K%@{h{eOG&!5Bc``L$<7tQ+AeGj@WM%EhC=(H@!YA{SK)igzxHTbDF zbxUoT%k`mMj9wIiQfe7mX*2AX^osk~qS>w7OEgU&?8+@umMqXz^sa8SRDg}tZ77m1 zE2`AdJCjI*u1cUMoF-5d6llML=Mc-6o`AGEoY`$tGMXszb>vn{X1Yb0r8ITjWCm+> zncC=%e-o+We@E&VhnE9%lOVYW)bDn7@3OsUQ5(x!?=5#jdpSwAP-yk3>7j%Z$)$tK zK|RLOq-F-5{JPp|EzoY{7FILQ@msQ)(cSbEyutHS)MV1KY9a0Ly%%C>U1`dSzQh*a z=)@n%Hg80e+Yn8j>`t%cMBnK~FLtD#Mvtf&Hie~nM9Ec2t~TV0hS65gV^lM=)(@*S ziE1d-M(d){YD-F6YSaK#Hcg@)SCgn|_OSAPPzxK$9%Xx|M5z>EJ2N!P2hiRrGRtN9 z2IWk0+mmL!84ZMD4quoOcl&kp$II3)WZ#=ooBpnheVHlUqpD(?!uL1EL6D?dK!^=R+EAp2RBrG_<&@1~SmrQbBnpPjZ7=8yv%F8P<55dE z#Oj)+A()fVa^+DLcd)Hi(`ec6tybP>y1t(lF^Z=#R@msS%v*~dn|q`?KM<;SC-s%4 z8k07pZ(ccIs_SqyDm81|rKr)S{Mo}Oe>U}a-rHlbL}|1|uzbB`&Tconx~r+pmepvP zn#MY;LnVosn!#wNW65f_Sr}eD=@^=86l%9E7^oBWYGKc>v7jwAR70wRFsvAlEWmoe zp^Z~WLJ(eaY~tglkqbEpEE<}fbVlYMN(bGy12+1r$Dc`FPFgYZD2w!AMQLa?Nt^HC zonDBoAX&iESCA^s^VR|WDVh||_rej{U7|3}nGa*79*U{33w&VD!^Xw)J=>J^`5wfI zP9Ca;X!c=qf~Ic=y!SML&FsTpt~ckj&On~*l|MWgO@A0?I`VvXN_;!`NPsK@i{oWv z2EJ!9E95nSu=i**A4a@z*rg6;CwYi&BHM%x1~D#TuE{B;p7cV7jpAt_Qef=}KQjG7 z1^vKEVZvL+>xZ~N(lGrLD}VJVIqBNGrMxgZAtG2F&(-;#!qLx|6)8wXak#&vmPYT! z-npn8M)cwGyny|PR;6WvVRsLKp4-FFj54H!F|2&x#Ce$iRs05d_U_eL6x%Q2EyRp6 zpvKm7aXgSa^kIHOk(&W$#tU!^EcrD084EsD;+8z4zqman4%1s;m;q1J9Q{lQJPw~8 z{U2hk#gnR$eKr8allFN>S_=!8nWG`r_$00ZLL`9n#zJ2JTBdo?NE}e_Qa^%A1Il3sg!NxkF34`5GzQalpWf5*VMC3W& z*T}L-QD%ba-uKY5u~R)q4B?cG-9yU;Ch21?DQ4U548=Ku(^Y-zxpgz1iG^<3Fx&{DtkOjQho&I5;v6zb6if$dmcRp^$gcK6zlE z@A-)Xp(*Q690K`hfhP`(qA-Xj4w5LoBMXIsTx#a_#z2703GCpq00fJrQKv!P#fnls z417H4dd1+0DQVc#Vkf4&mboR2XWL_86Y|ASD?5pD z0AGEyN0i4f)euvf$ACVayv429;r0l#8p>m@bC%yVW-d&OyIdaC4O-Ri%vo`dIN+jZ zYvbhDI620h9E-`aoAkVOv$aNZ3_cSr9|cv3ll4}tDHHkN;6O3GR<;kG&n47{IlcW#=!wPs-4y<k`y=8Vqay`15}yZ&Bxj@m|v})VGA}bwGW1=A3u^Bj={g7hNHeR3Wm8P5o4S7d<^q{zci3dh zSS-&l+mk0w$Kl%@z-A6PDGhk?(3NM>PW{l`<7qihv=5i0s)qbv8gVU%3RAZ8=wGNn zKkK8A!? zp^5yPH&Ev>1>zSB8lvipzsO+ppPVHTj|`I>Ieus8@&58;KzZ~5LAlr^QCtsTD6Jw3TM# zuJxQf23_l}5$LXarERG0gQvQiqjO&hB5Ww|n)*#HD7oI&)SvifJwemtlc+{A-37IB zcTlA*Ni|hnTX#^o+18Yr(lj>ZNFG-?l4dxmj) ziq>rC`hq>frdnujq-)q5UwqU(`)*!@-W;oJn#)v^()wFZX2)G)H{2Vh+z&icAvUze zi3*F7f+Nu{_t7{k#By%eYmq~i4M5ib(1*%@He%q4X_O;81I zss(bz%+6^A!E!*)*8__~7UmCa*wpQNAY5W&rn|3fVxxZDk@~eM|MoB{qD|Gv9(V<@ zsfx{dD~R1{bt6rZHA|Ca)=?xyX;DL!T8h+Zb#+Ar&7!!v(L|u_e4}7vK-?hKx8BG9 zX}24a4DXm-lXY}W({uwA98;H7+Gy%6RcUMrA59iMx(H#xF!-_kkvX+|p(B26qQH@O zW3sGKWXz5}lBTG<44IFDWEgy;mQ7w$v|+=hHpSPHIVbHc3W_A{gGviybj46tG%(S8 z?+|RdgZ&olpzo170cl$-gosJjq8&}jP79$WbF1>0i~BAd<^GL&f#2NdT7T#rybiP{LUBKqvYAk^Yf_Uk;p?3 z6Bt+`Uw*93!+k7`^x(THE1C}ETGH-c>R3rGU=D2`K%pei?qyADNT$-X8XbkIP1dz! z!&GRqiOgWRLP@KUj5#1Xe+nmJP_xl!&0Cx(gK={|zwG#Hs(y59OmD5_c{a4^KGUZ0 z;yinMWcN4Z?H+l%zk2bVN3r$Mp?{oPKV^F(n3z#d9#gdB;l}j{VS;++y6T!$*Q%?$ zSkndx?rVXAEccpH;b4vEb8fqY;|VOFM)vmLzuw?l75&gq|`oq!BW{7`Y z*S#cUCUUh!fjN&E$?hkUSq#atBF|bx=O>WJKg%4mhD3f%t~FZnii!M^RF#zrS!yed zc0;YHO=*+J|F{zQ7q%3ZXZJ5@K75op{>!S#sk`}cz#(cdL5b9?J%F+Wk!dzlt{^tc zZs#q!Q;QO$46o49lI_=*wV950@>{)uapY%DOvvS~`1Fx=?tZzAdPa-l+bPnpxXF03 zK*O>&>&f22`F_HihQpg`3=g9k!)!bAgRI7ICqX__`NpOL@L~-U6XkTx4HL)4-Bd)_ zoPv82ADuDf7sWqMHFe&=A<2xpwk(6WaAV#>bz>IG+^*l;S2kz<>7Mxq%lYP2e`5u^ zT=8D?xG&+~%5CK`?{ALi=7>HxzWcyS4rO2tyBf#y51#{fq zq+UFP)QcI+^MR*cOce|tbn1oHXjvVDDynYEmfA9#hNPNZDl^tKTk5>27xRol?Y2j2 z;hs5(zc>1s z2IP#n-WBaQEGWydsN~*^fpU!v*Ind8FbqO#$KfS~ywY(db2ScdSQ_UolRmR-3#K%n zK1HJd>)K}YHOA-r(-12r0U5{CXX^Cn7km=lRDraRVI8xZ@h3GI$PM_H=h#=2T-kmP z^PjPsJH8I^BFEey!M}U#xkKu7MnAgb*#+#s>fBeJ zGo&_3atsF`W++ZB;mjLX5P>3w^b20!YUyF~gTX#j2+Y4f`Z1{BJ>1hE95A2nI^42_ zY?=KIZlVKEF~@^}0FiLR>GU}wZ2u720}fi#>)}p`yA|?>5bg)mAN|PX+a3*~-3&}R zh?fs&&;k06zu*D>4xn*3gbnx-p)pR1nbhLn@ci(D2W#SY#OM43KWc>Rd3}BmE-7#B zz+VHB#C+THktVPT-UZ}<7x~v5nqL%vq6d2qPvOL2ru2(AIUhXyR5>lk?*O1@0_5ns z@IdZFn25~LYv~FOoD3Sffz9!n?{xrEGc%B<(Q!_vM?@EbA!17ojR9T`LVzvmp(jGBj>oN4om~z#JQVzk~y>i zDFFt8mD+sb9#D+{w<^T?bQETar!9t}IDhF~L0mApMQBFwX@qOX#$ArLGc9}P7Sb!& zBpAnr8Cb9pCO_cB7wk593%3$qg!R_QYkJKGhZ8~ZogwXa?9opyzli>2a>+2RK`>rQ zTn1;H0?H+m<7?+Q>vaTN;N*yj4h2~PnGwt7nenPJ6A<9UW%~jZ0K_5=Zx=bb=2eiHit`e*M>kDr$(SMK;A zV_$hmpb9*lPoSL4i!98CyLAcbv>ZPkeWiY2}*?@=XT$ zJ(tLr~Ed)kANTNN$>?r1^1kSRa;2lv~1qIFT&3%osP3Fjwfi zE9#Ln+bLMqSt0WFE!dr9JU4E_jdP1xapRq01~-`!SL6s)7}aGUB3Yi}Yljp}bu zebJ%}_xvPm%*h*bG7dYDonIt@t(nhmsPKjg-(G2)H4@?V(%?;s??XuOopB`Z?Gaca z{dYEQKjfreRcf^~+3LuE0#>WrP-(ZNc3QgG)K#k~7bg9trH`uv-*$(fkM|LkER(8K zF(lUPFh$WUSv9m)r=iIzGh`-$#wSBsX3k)f7(AXByyJK-h;X|gzflmTD0gQ$HLPQRYt<-t$L4$G%?p3Y#xgI4P8jPBHYT`J*3QM(2AO1W_SM z&-eH)h{_;d?!e?0NI-rKQg0;Wqo-{Ug4mCNjYC!=JMQR5#|8m{m}2$3hcYb4je-NN zLI=I@i6_`4-;RFrJ=eye_B}g*wT1|>H*k6MAALr4qaJw$ZRUpPy`~-d!9(LKh}^=_ zP2tC9yB`U&{Idg_eW8V)dL}m->T@DGK>KX;OV6iWk91*+&CyRmZXOi^^xUv#-~boQBTpreLReadKWI_+3)((07=uS1n4_OTTZ%lT z6#c9sej+^ViM0|h5P=#l z@F@{G;h=loA4Hw8fpgFroD9;ZxT%Rpm`6=ebh*P<%+W7>uWX~aJDh!BzKMQmK$JcQ zD|b!BaSJ~4{2p~-tgOGiPccj*#hRPkTDPArpf1xY9=Db=S{t<-Zc!(bR(NDzg zb01j*P!^pguR8D?vAY3BcZQSjL2L5^MiQ7ZA9o9tIZ)V!Ax3lo4#bf%{5XGxiSUi3 z_@P)K&4wT#4thN}^IfhGhk({>oEA*YCzSMAO%)HSr<^sRL{7o;2OR_`Hv5?ZRMhLfBm4G z%!*?|ovei>*SA$elGSCh9oIL*DU_x)ZyMYK*`|FD-5S&PC@AVgKY4s_e8WoSx)RMV z?dUdbyBBU7_;3ZX8$^Wr*6MFj{E(HkwUvCh_;HW+d3AyPDde=yFuX36)2db*&8j3_ zNP1gtv=yVK=<-^cGRkeO+1B)$v1!TpxN=;VbS9K%x-RTtc$9gtx38H$fc)0T50c4m zo$3SGp6zET%X%%h0nU-}8uO_+`YCRl0~_bSQ*;hIzC^ciN9yBBaw~E8KCVS3j_Jp> ztgHk5es!{)(x7%J|KBE`aFb8?;PMG)d-gURY3B}VRF;Y}*2my8W}gvritrbFq@p+YRjjeE>Hy6**)7iu8@6tWm`3J zammQ}07P@*b&=YVb6}347X~%*He^?n{KoS?w#gaoTb>I?CJb1G08pDXfMGOZ)*u%f zAE!@EE}7b^o#Wkuw-xfr20`2Qu-E~-HjE2Jwa0B#<@-`8Fs>N&*qKLhex z(4(~9*T_o;mWWC}P}dja!0J)o5)NhGBhO$bt)TY2#>oNPK73&`Uo zHJR@K40eHhISjZy!xFEkyYUJTNTpsgP+eSd%$zG67c`+>Xp4fkyqGJCz!!KRh@}I; zZ7~n@VohJ`uacq{O`i8dHv7Gm|wst>~^{Pn~{sqciSZL zdYaTBXAG#OIYg=m8ofmhL7x8(h!a6U2ng4yutEIe&;PXs&kiqb_oMChq7kkuhB;n6nv?vf74VUS(WulcO^ zd>G8MgMKtwwV9i&DkiJQdR0+cD^1oY*`L?P2m2Lr4$@jgdZUm9Vgj2B{n*n7w4?7u zwjRVa6>x2jnINW7a)JeSki5Xb55nsd)%NUAgjtZ|(NCX2*lUw_uFa!D@%r1S{4yNK z;pi9F6IPe#QKWxf&!>H?lq235^1{LJ$bqPv!wp2QMn3_5dmMvTJ7TR82p#Q>{_?}l z87I!HYr6uAk9gYHqxI!$3Zw4#2Pm2$dF^;#>`shz@l#5k7su zFreSfCZa6$v4;Ga-N72uDW(D|G#Tg&u&^VP!uaW1jU7oP!q}e>HKKakXi>Zj`zU?D z3^fC02x1pr1%6E4dRG*gayrYL+;U27YxIj?F00W7sZ2O~cYbmB77_mvApN)hXOCM` zxgdg7=c7961M1L@PlI1`D9f;h11sc$*BR}A^lle|IhnoGGFBtQQi+(gyK{DS02qX( zdJt}2klm9b_z839plly^AIw}8T}~gc=2rmxyPYGj^wq^U=IyW+-sTcBOp;)K6SlW zJK0iKTVkn{10tN)xw278bMVh$7yySj?2mrL-VN~W9uNVPW3I5AJYR&o>>`ZS6dkEV z658E)cK}S}UBo}K_6yKKTylurC6M59rf{0!EoUI*pp+U5G)g2RNc`vQi2k7p^Rqj2ICKL4yk9vpsLyzn_ z6o8t{>X?EmYqdGe3B`ewB1u7#s*2%;Ub zk4Wst;{F>C)W-MuA{>fq+9qaHZY{3 z1ZMZgPE%}6kvbtG{d+L_87U`hsL{d=J*6F>P=|gw-xl@-U?;$O0PzI_U(vWyB5Ulv z-huzGpf+oB#^8iK>PPq$3HV&M1 zLw>jeu?wpm5V=ec0l8!6?fEx{6%Z~>q&kN?rw2zzClxaKU#^(2$f2BZKCl6oaB_Qe zfaNhk;7CjluYd&MVhLy?IByMRE4fV4IC*!re|Ye&LQZWo;-awnMYN2G zRhu9^IYZ|8Q3wJRQR=u*56iWGL&I(%7fz4Hj!U`n$B9^^v+HWrSWSkKO9YJFvxDPv z#5r^gT+$cheCO@{!Pz-%mk?G6yP3wQ@jEeD@3X&4OAB65rkb6i0DHlUFgx)zpg^+CWztUKV*BG>aw zy|J2&NG_8w&Q88VZpmEh8Z|o0!%5?z-v_!3nEjT)Q3jUBPejZ~&W8-x>m};Xo)h9H zvP=bz83m2mhMALR$2-s4qeAWSmG8PGQ+_FF=xzvx~*kuwD3JqJ8W@n__T$krQBE6l#1cOEOA{Bm)Y@4s0Uu5BLWpr@;2Y$XAiM;|5S3?X~9`y7ZDbugDqP&NKsc4JpNsY;p83h>R3JMdvH+~{Z02Vx9b z__wj10^&^E-cQsy4-!84%Fsg;NmiTe6hjAyP{;zT2={~V0|QO(x5%crdJ%$b^-b9AX4TOQTEAu% zk~?^OHI+gs5i|C7&MxqT?>Ou;P$^+Z!#QLeFoHBi(mAStXS}vRtq_ z@}+22F$JBNQo8AtvAtf+tfG{N2QeUVM-x@lfjJ8x3d$hJyHId^G@HQKs2)Tea}riA zEZajQD96m{ppMN7>Hey)E@9;R2juO}w+CmVzknQf7+@8~7%yIPbMa-A%ZPZ=1=JCq zBoP(Nb6I2Gh?NE+=kWTN`6g(BcpF}P>3BYn^C%^4$WO`zXP|3vWvXS|`~?~Pm%YO` zpiuAhvF&7{7aTF(I3m`5;xOtcE0=b8N7mRngE@gTRdv=twAB)B47+lA_ygY)5He8} z5VpsCpx+b;d3=>iPUx0M_$PY&EQ-q7ZzZ*%R^fk;{D6LBvufn{XMIFH#LWo{)Vb}4 z%n{mK6x?RRFBpo%eG)kH!@#y+$6ll2ift``etkgtC5+@} zqo1<@BI#Bk_dc>ae&obo5wS{{>w!Z+z3BxY(E?9R+*Lfb;w7K5Hwol}q-E>g=*J-% zx4G?XHP)n-^3naY;Vv?G)M12W3}lfs8*z?FPGff}^@TJlq zoSZ1InmneKa@X^L?RsB;nsjZDK0Q2_8PWsm&>NW;ii5!PF1;bPelD?xU4#Khw!-WmzXUjJZ-Cq{A#IH0cL#VeGw-5F2f#@aH?t~4$6sgvv~Ju&lYJueT=-hybl zfA9l2JvrKWyLSR&b?AE_o1LER?4#2;B4RJd*~!t{!*43&9E3`cDFD9jc>8%5^_vW> z=JzoW!nHpFpwZM?MdPB!V+6S6Ppd=$R6y7UW8lq&O^NJEe3Pe=1_OJ5F;c%$UPjST zl5CU!k2KUe{QnwYS0ao7>|VVSo=l1{1GUt`&vnIfQE$Y);wFDQFaSF*1YJY8m;mc8 zF%oJxWC4#9F7}dftr5tC-?Lqtu0a6SN+h$Fhv$2*56-GbhkLItK#V`|?T?gq&xa9P|SQVpkvxWw1NG2Lzj0S&uAt^q<*q>H!41p#unWY+DwH z@ZKkvJlnf?-X`C%z+u+_u|jiMw0KFD&;+vD;!6WUGc(5JAM^*Xhd{W$v4Q*Wd`Ju? z&WAYMxSzRMChVNN+uM1Ek(9?ac826foBE(Cq|!17(-h7J7nXV2MCdatT=$ClUqlu? zYQ7+?6<9USIW_qH`@FVBRoEi$lP-yN{>DndCvKG7S_FAbN_f%}5B3 zkWjBGD`H@oIB~Y~_L~zP{qV6yV!{%M6hWWVv*LB^Isp;g@jSf5&CZ9e&@d)X5ptl_ zrd?{!;ojo`a76BC8gk#d8S~B9zrM?^;jD0K5_8We2rW*!kW#NODy0J2TC=LJM$wxk zQozfz(SNzvIok(a8{vl=&7mH!w9+4iqu-8P7N8lqo?EpjFUNL@>H^w^m~UdcUg0wb zrDqL1s5vdA<%lzF04Yqy9Q_n`3QndbjK~b|#ZXY6fV~x$IV!?~;@4iBTm!V;@vbpYR@{i0-YX1tvB((> zUAy;!6D9Zo4E>$cCEta;9^)YeNf3MVt&DSsi~g_>6ZAnm8Cjj6!e?`aiS|4VJ$5?7 z+7j0i9Ow`ZQY>2}`oS^1e(79C;sJ7*NIa0L+KR%#v>=hgejFhH#1Z{lF`!Nxr27W(GfOAGI9yW7viDCOgTm3~@>>>^{hCAmkK9gbV$pF2cb-@^ElK(vEL)!xn}>&d78MC`d-N4y#+9iCt$~ zWj2j<=;eA@#D_RR*Vr)~u!o9Bu6J&qa=-XapP`-?$L@5l5gZmDl*1$m=_z1RXtK!Y zbPIw*Qg0L-)1<3&Sb8P0K?pxKCT$9uQydDF72pNi-aivk{yxtK6s0+0<{gK0soAUn zNp7tyuj{3rFkIEpcAPdj5p?rgyMvmP!-QBEnBIVsi~S`24oNzqGlOS#eSkwS1ng)2 z3sCoctWJYMRhp|2VZBt= ziro1p-AwAOh(_RnnyEH=jfR_mUISlrGvRF62t*OqP+L?PTTQspON}x+J)r1*Chj0R z*XvA9EKer{HB*!$;3lYM#@X^R%{4m8G13LOEMWGE{5P)j4Se4VV!l^8;syZ7b9RJw-=HTrQ zhX>*fgu?BRhE0HnCV7r3=)4;)bI(73yxDQ8X_XXZ+D0i5t%@m^AC{}|CB4SU-7YyeV|6IwNdmTlKS;fQPd$P$Pvk|rCX?P@{C znpqze{V~hiwDMNs<2AM6F)X)T)U2zhdFHM4wOq6-KkweX*j$Tjhf+E zEG+28lRjq1?W#wx^e-s*i}v!#?f62}J7(e)(Nv7OAK{L~yS(K_KOctXM+fcf+?SId zJH3=0J$@xSvOGERG4)Lhta;~vKz^AYICCvGZ~jVlTxD|H*S7f)&PfH+5XR=4T*(d^ zJ)a#_ogDS82Zsd{d#?MTpC2`IEjuoMBRi@#IV!TgA^nKp75Sv(ZGP8bi(|M1=k2Ls_A3t*~x1Hjh?8vRjk-udQPf6KvU7L*`nD|n5Sn)#curvxz z3)WrxLV4mO*~^d1FHP=a^h$2zqH}0o_HAnjGyV)*_s!Up`L%+v#Y?%dWlnjLjvJhw zq@+qHfc%Ao6eS%#_^i@r&o2(Ef$1}D2Al5{AUx#LCDyOF!pc~m$tQbwN%|4G&3qjz zLcv=gj?MQxhW@b6AVPqs%qv7+UOV)2)O~XHWY3T11jO~=(st$@_B`~j*!6tRf^u$o zeFsATu$~Ix4n)%n{b0^LI`t1u%br9$z=D=(Tl90!L^B+!iX8@fWP4HdA<+qiVlh{c zM1yF(ZSJ3N98@|x6#agnZTj39faN{0Y+yBDaiM{LuNwqw6vZ$=oPP%`AB6Tj7(&*K zimWve9g2|z%@vB28Yut}F1LJ$A{s=dgEqR&$C&IOgRw zz=D_Qg`X6F>;cr_4Gx2Nktum56mHXiLk)-EM=F6uig5r?7#23!{U|^W8;KJilu%Og z3@Hpx%XTgJ4q)nTREQ*Z#4C7ouo;XnS559h?EV;F%$FAiM3P}m?(h<32Q)vT8Z|}v z6?xXuezkYlhF@~CD%aG;a~x8nyJ5Q-To0ZTzU-1u@yVGUrjW$YP-~i+4zAX)noF(O zd>%cnqo4Q#oYDBaChZWpWzZ}75I+jhf#KhXZowyCAAU=oQR_pJDP*4W#%C3Fo&l0JT^uj zoN_edkHRwL?7|zc@KzJ^d*%RRieo-fix{AXX8=(cCWV7i({ae}Sk<5f6g2F)s3xOhQn+ zm?>c&W9-F!#U6Z zn?wl5OpY-JoJ?UC3AE8tv87n+i!si=2Hqfa1SF8L4}jVL0#p-ps3d^F>2XSk;)CaZ z{!mzO^Rxf&T(235RGc#@XR)4q!TaC&v{6Vo)Tg1X$zB6oC$y(8z;vi7t_b zB+&GNXT7^Y*ynJ3&1*b>=F#(QQLskf-Am5{k>N8lL(Ud=dK?1J_AY6cQ9LasU#a-; z!25)-LgP*^#=tnc-WH61TwrxAUQ_-EdFM0_7zYFb@P zy{;HFaxz53?1}(4>ST5UWk7yhaT8%B79So^^Antw;1V6qAf)(wbH#xCi}17`+M+UQ z#Jt{uV2x1+u`pE2IMhV#B7AWlmVi2(4$t@C-{)dtnGVBZ za6kc!Rg0jw6gZz9?CtM8=W|*Xr}@j{Qvt+{I{!qM_)`VA^6WfhfN{|h$9hZ)+n))7 zBnPx;SpknZn7%76Wm{ID2{i61A|MoJAy}TxJHG>sl^RmB{-Nej7gVttZY*4CsD>hD zkT!$oy8pyvQsr^t)7eH6!tfx%@cESEP8ed8J2A?_i=pqNfGO`rl{LlwzQC@0hbvgw zsyyBSr09GLCj}D=oaj3C$KT^bPzea4sbu{P7*>hLEV1AV)Ad|y5u{)inM zGte-sLpOZ@ekI)@`9pxi??sPDgmM!w)@n!{RqM7)!;+XTnY!6(c4fJ#u&$aN=xmBh z0hjVvOdO>Y3iMhZgwoyvb094BS{^MTFdjwy_2&8^6c#JOGQjbkC~FWGxj-*7+`W9CM#2wg4CDGppvB9lPc_{hDiE)_UGmJ>{x%V#@kYdNJDEs2G52 zjoa+9oECKZsM|2AvOJGcSXr3@u4qG86UQyU&h=Rd);7nL%>efZzZ$K1@$1$x>9%B+ z*+@4g!7q$&x3`jcQFB};S(9<<33E~WEEl@s?oeijHgr|fT2`m4GP6N7S(dvnXU*tZ z9cwQbG@2ES*pSML#~?}gp83dg*fv71)2cGIfY#lGju$oy+w{F+c>;~LqxMhaXbzyIS` zm9IYcov+$oZT%YlB|H;I^yjHHe)rX{F>fPE+u-ROQNLn%)D7A|Y`;#n0E=7+>RU~F zE7w?9N7rdEpeCzZ7(^bu`#BhU_mjf80cNMvzdt@Y$HvdJntJ%+{1V~5*-lbWV9>VL z`H+&|?Zpx~w(653HuuE0mw>K0ywJ|lGc`6ZAJMX8}J`7(!}EhgI)O4 z4WimpnE@r`LQ-)+&8W2u?T74uW3R-?HaTc)l{7zRS-7WJN@4&QF&!Y;+gf~f zaL3`1+Jrw`UL8`}_l0I88iqL%@2d-r=n}2MlbAjtJsN9rR5=|r%0SDEn`WPFkL9jS z8Bs|`j8o6DQ;pKjw(}`#Q^r%%@kGv9X0#;VY&(~!Hf1y|9WBl^%ZwN0nQiAX)TWHq zOh?OPmt_XbWR`6gB&bapT2F_La?3Kq#(8Dixs0?aqZt|GNpi_D1LpF`whJ@Tri|W7 zM~^efvYQlVk!_FVpiLPO+bE~l&XdS8BgP42+qu-UX~Sg>Pn<^7 zO+%ye(a+S>EqGXW+b%Ja%=6X_*|Wq(ZPlYG zJWU|vDB2T;MQH(+Ck~givdj|)L|Mt4Ck_^nU!FJwVoCR@gMnL~o;VPul%;y&kU<45 z3j&AtTk2(^wM@B~mB>%eZsh_}trZmXXB-kPj@x{Y;DQIDCpvBeQ@8_;g6F8GquV;tPeHw{TFpduQ`W#IQHZD3jvxAw- zPmZ)XIa5zg6s*6^1NFD+@y~)Ib!*}7h3E2?XR!1Ezk_)$M=1(JQAOs7ID?Dkuw;VO zwnfMk;Q>_HP?rZC_nAw+ibFmN!I)-H7m~A?`PlV`Ldi23mg!0tN!Xks@?etVhHx)V zxU-b#CPkSA#$S>`MPwrmJ;XMMMbU!WJnWH(Ajs_fy9=g%1Yyq!^f9QCh!D#n;P1GU zWJ8V1$vw#AV{o66ySM`R-<~e?3%dOv$m={=tr4?cFne|?-Ie7H-7TfNS@Jil1jmZh zb-v+OBd!=IPol8-qT`(ew38%oJ^R>$I7Z9m7Oo`kB5@i{bGt5EX9(U|yYUtdif+Q{lY&fYErNk&wh@!>t1R)Y93>ASv z&azB79I5TbS-DGz<%K|nduP)4Ly4(2JPM_Fo=2IP%};E7)MX=SOn!d5*k zv6WQi%GKdYsKxJG{FB#G`|6MX^siri2t42Php_{Mr#9Fe^(v7Co)h$`J z6nKHjn$Dat4TOqGPKOBC^ zeWeEnLw_CjC1nmy1U5YOL zUOE`JDD!A4!h&H)(d%|weTo(~TIN(5ub7sDxz5nP zAUi;XP+o)P|8MWxdK|aX#Qq9jX55WbNxX_=c~Xn+>b70p>Px!1J>CR?aY!DO*rrIC zm+G#5@+3fjyamWpfL&nxl<|WT?94-eMSulHuLklT=9lFA&LJg|q9jV9N|$11Gge9D zxqRolo$tz!hSf2_S~9MG8+hoX$jJBvcERhr@S+MXy-hKm1DR*zkCG4%Vv>o@PRC{E zLO&AR;!^sXI88=@o5swb;>XdM_WKw%lENihTLuHRGLlxDa6!q*N^Zb6L(HYm;Fuv1 zAM}Qx$0aQ%Fg-CMc+Afhil4t{3(W|_4G>tU z_fxn$Vf}`bteA_2at!NJ_6x8akAycVk}^ug)8`HyI14;?M!NhQQdqzrjJ>!F_xcGN z30A<0BGC^M$nf^6Q<5G4)GVxXGNK_?pH1thq!jSt^6lyM2+8j9<+IDT&%YI}Gj_xe z=^Clv3t+dOeI@_{#xNtmKwjPh*8?Ou6drvELNFHmjv$MMu9}ZiBn|@>oHSWNzJLb9 zD29BXPy*i$CunkoQBm)K4+lOC3C#QG8bE(dEu=HzKRywj-+IAE{NX7L;p9N~VmreU zBX7wq{g|@l&pvy6^7jFJ;G?`q{2P~~rDuMR($#^vgD}D$zaT?`2m_kPYq-xPH^hTo zV-o5^&`~0hSoo%+c#cEp};R(7JZv!EV z2?L@fAk2hkfz9>jJN)Z1E3IW?ZMkA!+-x5X!Kz!H@S2Q&iG*)Rgw+gj`Mhp1pFKxf z-ARe{ZMBlc;=z%P_Wq`fQ^McwH4d%6ZN)~G5wqSSXJQIgeOpJFup$r|lt*HO~< zqp9nAAT8jtskKQ_MOm!Zc(Q;X8+PYxW)TfFvCNseWlOTC=%|J%tF|KRfFwHnNA8KH zE@?BQ_~gySSC`+s5}sXtdHMFqmGJt>nu%XdL%w_Kp*Xm+LU*%{K5i8GuwRz$bavM3X;ROUsY(@7?)N2Gl1!0kRM!kc zv6Y!ke;uIJ!b57mK+PVA^Y!FTS-XK^CHfy?2gPbSzV8-_%{2+TANt>lLa~EgY^4uW zgDBN@`)PI4hu+_3vCnEB`}?R`$JC6rzt8dY_nGH!U8$qbGK4;!jy?ql&1vP+j1A|w zoBgbIS3iIjJ2|&^uB%7U<$XTKfnKlOPUiE@H^<5QyU%Bx=JmSnWKQn7qYYY#H1C@DEdjdvod54`oD_u> zRJ$vRk1Tdv64jje7Rm9s=iivV7px zeP;p$A2>Xw{O5rKqQH6&t^V_ZNR$OdpVyCr$qmYFcju+>mfv;L@AoPVwk;#wmXTgu zMtXTj$pMy;UdszunJ>81{n)0xZkqPG(9hRsA?TTL*Gpqup6#Z<#!a;D`eXPZ?-;8%>#WS`L?{ctrs^uoYwkl6sm1pln!qIb-%?tJ{WDtj3 zsYkP9+XQ?UWu5l>nqiBY>PSwnuZol?=G;m>$%s4AqYixPP$xNq>y6%3R5%N5PPv|? zb3Mp%Jq5TmSFY#EBk}w3Un1ctaWLNSnua%SvG~jMt9<1b_G0ksAJR9AwR#qneYqgO ztyq|VaC$0S0GfJN6qoLhgTSh8jOV~9ZmF&&Qo7`s0t6bneB zXY1YTi?g%mJ|N~DTF(UAy>-2_vrE4p5(;y{%{;-SxV-MP8-HRxFoD&ejxl^4nPzl$ z_BQ*Bk$Eg24mUySE$>Z0oE0j`VEbMdd(;qSD&#wvOes7)m1GqQWHWEz=zBqtF_d2* z{14;b=$hCOV8O$)`5>)MzQ;C6bCOyo0dOQ1UIzfF@thL{{%bvFLO*@Z{&-s8pD`JdcTmQRBf%dI5lgaLS-gf7780xh+k zcRT*&MEE+0=%`LI|C5-H1v8w!41C-w4O!o{m%%eYyxT}nI-7SN)q5drpXq0$r_?^ceM#>l@JhKUI;IrtyD;$Az^gSr%+KvpaHRT z@0-|)e9~9Q_1+^4YSo#gOK(6aS(4!2Dzf`|@;g>!A3`O7CFZcbtONkGXqas!fa9wK zu+W~nN;!a~SbaR@016P|u9Txo)!*{XyH0NZYAQm}6TjN4+bt51n%i?p=`*JG^Gd6I zvEkRmxOqO$t_jcQo~rb3X41IQzxe>uh*tDnJn*EeojGM5INX`HX-SeB!`<;uWlos; zx~Vj`TISEn1i1BmN3&^h>o}4Y=fQn)M{OQln+JCwd2lOwUk)%2Zbue$i$gRv%>3Mrn9R()%dEf+ zWDy;fs`}8=kroQ5V^a}!%3T`h?Y)Q>9CWIXBhwy{bB0QyjF#cb0G0g|RbbtL% z@l2)B8~^mt9TMg&dxZr(1cC)Z-GG(G5hOzt>*dd6W7@W9 zHu~G~&scmYXl{5oX5w!=_G~`Sa~~))bRD!hyGiIq!Y#vsU z6;q#LZevXGdiVugECb;Q%yE+hskZsl1vCg^$`yW8Kd+x$&F{1+X)0=xr*c)y`WGR^ zGTtY`*Ob8j_PvfEX-e-@(t5^Bjt|@|#y^2B9~9<2$tC}M`R4hvSJxfkO~&HgK)3`( z=Ee!T9edfEg-VCQHSq|Xh)s&QK=;;uvy^DFy`a<}`2KL>5euAo&*Bj)qY6uyegNq) zqAA*%BulbmDyl8@Em2fW+0adgz_qa_u9qIO3LB~c#}zq7{iab2y(GnuYPOclKi^RD z#u27Yv^-{icuX_V?(clOLD6|_H4pN{IT`QSiLb}H7Dc(6ZIt^;m$|i~++`N!F25Cx zGev_bI=Z3%`%R+US;0M7c`GYtDvT+0dYaNkxgTGY`+QTxN+IscP`WVGA^JXN?R522 z%mZlEHnoQ20vaku*R4-WC1Z=~be)FABQcvB)&@M*V2=Q>kOBR~88ZJ&-8h6v;wDqOe7PZ(( z_3gPTcU%1Yz+*ddRMZ^2qe7d=z2sMe^$({MmTIQVDFJTvj!Kb)*&Ugd|Gu4Uyovfb!w^ z7x?%_K*k>Q;gEpk0)7UVOEnB`!tqb=b`Vtu)m@SFDbj~w1Z+Vb_<@c883rszD+MMK z8}K>(C9vMp7~AP$?=IUN!OezxtkEF|9Q-eF#=k@{nrr!$W)xUnx`jc4-TzL6z)nU0 zItLq{lW6=4J{cu~;{vOR@W~TlQUe1Xr?-KZWI(XfHuHQE*#8YIEhHX|I>K`w*Fya9 zPjD}DS*eW-9x%o5?_Et#B|6W%Rz<|4ha6b6W}W6xu5Wk2`pnc{t1SH z*O7rTFy>%r0}H~l39%KtOF2C-Dw3Wr$}OAr5$gN6}LLo&woM2xW@vhf_m$`6mwUc}wAI#pzKY zL2Exw6_X*F7-0Js0h_+>vI8U*uxuC*7lbJIw;%tpDTea5AOGnDSD%s#4v_#f=*JlF z70whpo~B1LA}|&n{+km44+f9Z<4DO*4RJG0;97jbX65IK6toBsdFm+~j+c!JGT`+k z5EiZqX%G<*j?#}qzFnNlVUAbgpRtap@Hsva-nz~mQe6U1U!dy1Kk^p?mA%DqGIJPg7J10X^`;!haqU44kR9P4ro0eW!2}N* z-u19uD<_sDp7H@iLI4jZOMpWXF|bZh8);zVB}0pbfWifQV*(H0Hd)PNoa}IbUwE9w z`RqwxGSkz<_2noV|70)*c+JSG5$ns7cqGJ4z9mD;9slg-P$gvO z7yC;9{20M|03SF=7XW<+q=(EK5YHhUrgJ>w`o(B)Cjg3l?*|`z>}$%}>hc~%!iNFs z0PH#)Fn1+uy_-G*6zMXc0pLNsU|mx22c|~A@<$1eGd_W3!NmTJ@WG7-f*S)BxSuF& zJgcLX>JZE!gpGt#ok6e%>FKa}cn@Wqfwv`gSr=v}+l6Z*iGetbafNU_#p?kK0!TR^ zw@7xL%R46%)IEUbvg4W&L@8)1ahn$7l~o;jNh{syg!Yw@4MUv892 z{0PC|pa6&mVUXMmP}p;6&tx}2MgvMT{=$WL05Y8a#3r|Z1+doH!yx26pVB(hKV4+VNM**fg5YYsUilhX=l=WT2+mKFKw9phOs=}cRU^t zHtwy9EVhKoOpt>bgTFDU%{vjlf!*5QqdT7+6S*b8rvb6?JiU|%hk$;R&4x6DfZ}bx zGlp8d_xu8iIL1NGj~BPh#5TT#3<)L%jJQdsV6Bq@AA=A0JKpdeq>&n8oR1Q~03qPe z7XjiG44ASN0_$N3?8wud2@H5Ib;hEQ;qzYVcyJ)D;Y(ZtV++|H*iaD@(AaC7%WD85 z;*Q@Devd2)bIzxHVwf=;I>T_7P)0mwoGpiNaS(WXTx1fUZ!tEJt$}_dAnIvKD5E6A z4(WWKS=)5@Z`w_swL%YuD2&-*aB!f|NFh*aE$`+q)AhJ4PLCT95U$ETo*@z|BO?F+ z2VqhAauj8fA)}fS?lAvWx}bERh!Zi#gB&!b2gBgRKsuGPWj4cG|RDoLx^&jSLi70`j{plrE$SO_z#e+9grpzhR2-zyvmC|m7_Fb^Q1Pk3a~s!g!w zkJE5Rta=k2T2}##(MIEE2mX&nU2b0Pb|pzwrLLsPnxXGq*2SFMi2{WlKrUb3Bz@CT z4fv02sE(uCeO-4mSgyD#MjTA|fdqxR8SkZ!Oel5T2}N)5xXjJ`}@6L!zg6~hs2 z%Q8fTmJw$&p4-{C{z!YxpLTjmTVz{1-q!*elTXvK7+cO7Y(?8qiMEH37P!Q_yLZiR z6!A>eI*O*XX@SR;7PxZY!Te0YHKP%ZAYX8Ct<;7z+D3ZC%omlbMUE~QXS?}}X%I#( zd2uEHMXRE_SK6#*%^xArLW{Q(zzX`5ZvDhjha{-T&sxvE*Q@md83VtX!U z25;;ouWFRE+*LoPtU=~iZyP1mu5eJpk-lL&q*|k|*%N73KEl?k(0Ub$UWKy0xJTMj z=Gt58`D=yRPK)x?QaD=ab==ZxhYD32T(T^<U9th8T*tLn4FzbWrQn(hEJLnQb zRPXDWt%;Ij=zYahWJz;ur6-v(l@(`huR_kd0y`#?@I6f-rmd-tr5mE&@2i$>%RSRl zL`Sx%B3g5r7%pAGQI#O7+ek$gJDMnVBvsMmuBIE3+GGu*7t|mB>^t-mhO>F}JuLJE zVOI}~Xe(7Sf?wQU|B(MFU)2bPqQCwjKUT4@(Mfsk*CccT;R=*83%1h%qFKrxt@GNKLExaT<}~P^ z_T9MNOPXYMWJRs_@C)+M9VSB|1wm##NU3S*>yHL2bg;}o76}$XhZ5YyBjyDIjfud6 z7|D~MZ&i3SILRN3Fr)-9ht8?Q^#tzWxidjiHS^Ot=m(li#eIZxcFUYrFsp#mraz-| zmb=o?H|L~FE3cS2@FF@z2=4QvebDtV;D^TwM0dkc959!^2f7Bjjyxgp(MQMJqd*Em zNN>;=hN1d);B#Ld4F5!k(Y=TZyeXmst`45$78#1pb&4zG-a^oS(P-kN9#|f5P+o`w z-!i=qm5YIU8o{0qJ6XNK=S*^R{w99PO@ zrVg|x!IsNo?GqIDX)W!EuDP$q8}o|9PZYu#x?xvX^+=B@j%>&laTEy-k;*e?hQ?hGvr7lo`kDe;>f35xq6D_pS9B4CoMsZ8XbLG0 zJdd!JO=J+;Pz+JlWw~d{65yO@5>qy8!zRQuXOQllKO%R(2YtOpyXT2)dkOmV)QY>T z_49?tZa59-m~IsLut~gF#b;+amF2!>11|PdOH)*$^<>>pCC8C_M6`_ZR-d4X4hw;c zJSO?FP{1o1$DokUQ7b`JVd=|DIL^wd z|MN|dq&Gh+4E#^DMXAlme)}l3S=o>#Z>S9mv8`T38(-^o?wpHQ!-1al^V8SJ527W< z4W~^R} zSXm@Zb8~=8+i}MZhTZg=F5y_*C05iOxo-CwL5vVK4F+G4?3+8@_wA^86qN-VrOnRCqnSIBgJzbTEu4#&?fhI7k1n~5^H@Uc! zDkkTA?3C4(>!vi}M(@b?&i!g&LDQ=6f#|x`Xw%dMs&RYA{>#lh;W78@*jXa^q%QOG zvq%I*?&KVAd*kTYAH1q|3$3P)9R*Ug-ootZww~imJ;%l2Zu?lzQ4>3+(QE5D9#=ib zHKRD^S9M%Dw6iqSLAUUpWsF%T@1x5kXdn`wumbGl7Dfc6qvYY~% zhC}D*dlGD0Ao1z9W%5o(_|7iWC0dpTTIa3p@yR-o-oNXnYkZhc%FBlb4vJE2{sV`` zOwz#vhes)!;(lLvr(- zTV`{$9R3>^n9is10htMC)mkzaSFf(0KP#=f6y>^=VlN9_u-xjns*ks_WWdx9xng}6 zsa*LXa|YdGU6=A2M%yUrl6v*e=Xmsg*UcU%)-I^!58to~%nEa?R|{rUa=8~vVVP8_ z1LUd-&MXhTYiVRH8o>dhkIP%3hoz_Pp0Opq0f#9x-X`V-=eHCYN25 z3R@bau%Vc!IsOsTK~?x;6~-?QtL(NIT*BZLqDD7k;j7KkVDYSTf^g}| zr7aD|?ykYzs|`k_)Q&^CFwf=ns#4nIBsb-Vd3^)fE~yMC47nuqx9XJ*a$h#orW)D* zLg`e0EwupG&|Disyg`XPKewn-t6Q46v<|D=i6VR7T<$?>v=7HYTIB)QNGz^7 z-w_i@qb8+Rm^^6Hewzl=a2n8x2|Ja=N~Z}sZL-DwCtK_+y>J@Om@89T2y<+pYs~#5 zhP5qS=IftF%AyahQ_UK3_>gZ5&& zE$ZhqrOEx4@6J*fsMwHZQF9iQ0A2vn*%|+2W24J@eMDUBDFz}PH5x#^(4`J>>b>fL zrWO+HNLqI)Ke$bMxQ$?%oAF<6YBywYu>UFjL_iN5{}f^NVZHZ;etYbCeAbxi_qTET)OWVIUr`{o*4jEE#gZ zLT<5WN08!A3U3&PrI-`g{6(5dewQ`@#=`(M;Ku;6gxyap51Z$*bpWc6Peg#>L(2L~ z)XcA_vlwE&7S=0yO5KeUCyJi}DJMKe z2*S7DUY-fQJN{?J6afX{x@+S(MT|W->Ek&vjo_3LpG}K9bS(hL=>U%k&m7MUn_~OC z9W=ubH)b2h!=xSIxhKrXzVy0D5I@*$7tryJpBUGa8LJy1tQi211}a%&1sey%M=XL9 z91t8O!Q<@*j7_Fz63in%4%Zw`r)-Vy9q61gCQkjSu<9^jfURXnjT0N&mStd=DK!c%?F8Elphn0tA;C^)XP7dh8&*tXfT&#zxE>Lw+rxuRH=-J#xa z-4hfv>hP|&EPamy_IZZ1#gf29(=FDxrh9*iv4l6xQ^Pl zsDd)m&?TyY_BXF$xCu;cT~y$|tZ+f!O5YGhtS<537TrmFH*|yd1n=V!Tvt4}SuAK- zdfWSrt^CGPeE4E==Y!7<#V>*g{@3>K5hAhMAN+j>|36CLf3iE^bcETPAn#6avX-G@ zrS}{-8!sTvd?QEZI~`KnN!_O1k3%nUC$Qbj?6HqsfwpmjX_#vtxM*GppKIV$!*N zgw!j~Yd0K5EwFl#aaHxfmIvX@saCSkhSb&vE0-6UUv6th9n&VUQ<>#`cZFdKQ8(*0 zaNAu$`g`g*q8Meq0W%%U;!+(XdVGXbERei<{ApEL`MP1K_21o(Xw<6pt`I}9&g<2& zspXMY4N={+j`q4}(ohlo)$nVplkO8@Hnh=|G5Vh7T|^2EEMYwwI^rU@JF3)|1$?%; zZ=I%L^FxC-tCzlP-1YFj`l`e7`6`clYmLP6&WzuB@#pVJ9G#7RdlP^D99=0f^MNN0 z#zC}h_y^&JFFUvnJNPS4u30hAy)h(itp(f;Yf|q#67AkPJY~U8bwi}8do~u6jreL- z&0jj0ep6K$Vd<7v%ru*CFPtw#_giPW*Y5MV8UQu=P?bz)O|_n?81?~n7O09(Zi#tH z%nqeM`j0E0hlcop)v0Yu(ygl5F$+LJP8V z?@t_V!B)R?Uu){rlh!_Kl?iQo107&*poMJd=q_3N6Sh;cD^L@B=AhdIQQMM~Ue7i} zT{JACPfeXrO((VjpY+N$K`iCj4LZpvU`fuRswiF4P}8pK!+;dnpp= z#}N%PPZbUh3j{7o(gmg-wjUB?i+t>uqnAa50RDDA;X8CVqMz_9I{^U|2-t{`M3~J6 zpIL$?3=-Hb{FM446PB2Hhw}pwuxID5iy5K&7BG@qEGovE1|kuu~-nk5AlR>IAr%EN#@o-GwOOfT5wUlDlRJ&zkiR zdcIRgGR<~^unftkOlgY}A+g*^^gYW)y#Xx!G<_K!iQx@`+Y;z);3e5l!{7!E9+M+r zu?#cFWbfGelVH

R|eW8$|4VKv&8jMSVY*?412`f;h#ZG?kZn9v;MB|24(V3phAr z8bS2ef8`qqMy@~6KjDlie#8t@fN@L^-mKjx94G?+um6hY@6l`)b^!#oDRZ7$YuC3HvwS>}4^vxmGOn8b`;FD@S-s{##B;D@L6gfJi^%1ngQ91 zY>*8nofn5~Ea!Y7j?6K{EH#%`rCuzz${$Py#8x5D((#TGE4D$%HO762`Hndd@Ic(m zURK+x)>_G0D_LqKD;ryO<~`V>JJcb%5*D(InqGw^x9~O#8dd*<*OvK>2eZSKC6ds0_3 zdt#fvQIWrKK`gAqeH7;H7C<%Q4i~!6*7}6mI=m^hf zG;&#yFAE7i#_C?Q4;zlzE?|qNc(sF(;>lh?lY^bJ{Y3aKObF|OB_S9i&^aR-11z16 za2d13T9mip;p>S9BxWzTQ8pBOo1cMr-W@y^2FcA}F)+_d(CnNGzo#&#BSZr%ZOrDn zhKp$*$POJA7)l$$FpsV;9%n~^@Ho#+kqvfoE_@ic_5id;`+N3Ox?JJOi`P%s5sZMW z0L`)C6YG>kJydiDF6f|w1KNZOgQAPnkA`lHcVpDKNm};~2oN-^j_?-yZvn{ZN{AzT zaN_}kBWyMC9Dp(?udF~DuoC`j9z#JNnDn23NdZiL@PUDeLx~+Xuo=LRQ-jsQ>}|8~9v?PYfC#+a&CSuuf+& zuI_F4{nIzhDSdN;g#y`vQxN$y3Gv1^0v=PzB2p_zu_h?HnV zGQ_?knV|6X`hA5ETT@k|Z)?_ic^5m-8|I&daw;1Ye1Nflc}W}#Iwh1M(mN8|kh%kR z6m(=!*VJ-;$Lqv;0RXssR(ewwMMIWNqh>xxjtYv;*J?9Snpx_>QJX!bJXUkNZsFI$ z@1Eh;P@2^J2#RF(pp*V z+62d%7sm?{#R}@dB312VvEp6oQMAYB3@i~YtsURC+>Y-JJH9*U+yuAXsW@0UDX=Me ztUmf0@(Wi?yclb_$F!jB-VYLqsP9W=-E6~q!J}p+%2uvvO)jr~25DZA*)`(Sy(l(- zHD3p4{PlLseF7#-JuXW-A5g${;NFMQr$X{{i4_5Fu(_Dc#-NOSjD|LoS6O+k7tf#N zcv$Z7lD{I?ziEmHxyLchZiYE>rdd7yYFnDTx(P5=md8!1W|g$IXP1CYSa54YJ1`sC z!${p;=$vgPbz9mvcNPj z8R66Lhg>4~B=q5YCU~OI@uxVdJRLP3_K+Jl3wb!7iJDlFRW8-dDsN*I>cuKx2fT8) z0)Dq~C#NzbYRf8fHB--YF+M@{X zHcR$_Wy!7>5K#;qo{|b9qay7bNY34E`XQ{9dbbyCTJEKr;KDY^(_2!r?U#$=HjzL# zBsbr=WocH+UbsPa-TBnKnB~D0R|Pa*TG9fu<0cE4wNhpannM!kj;th!9k!j_(X}~H}O7G#at{`%as=FOg?LldrpiOuw$+#WK`uVR!lO#ar3KYvf+=!~i3XiI;R&Vavki$6^s9^qf;2l?BN z|7|}!A+V5r_J#cI$A2ulLjLySKdtErfx)UAAvj z$ox9CklpdurmR1$o8ZB?3EF~Dhfy$UA^q(R@0%zGwE&F|xgM0E8;(LvLUjY!t4(xW z6;)j|9a$opX4I?)RcjOs;lu-q9meLrviA0VBJ67^Ln5IfDW(zNj$882}I(sKZB2M+GM5PsFz_-^U1G39tiL!-hN%Ih{@ZL zd`wG1EMNcWSHde?-Q;|hzM`qUdJkUT`KOLMxN}0-zqi5+>5Fdq;tGJvPLZy5icj4D zSlWpBb&>PJT|c~A1pz*C1nBeg{|t_woqi0TCuhI*-s`={_Y0_YmqMW4YyU18|1!kJ zJXQt+lG0TjU}vp^0F73Br~WHnAW-X4!=h(rYr9M>2MiYeI}8#YyYPX49@gPwT+i-B*sIA&AH#9rT#0vwtL2-9cdZ2Oj9wH7fB(XNrJmqZ zCGe$>0&2kREAUMVjvKVr#*?G$2ww#sKt~KuQc&=neM)yb31Jq{J?t0P1wcuyw(MWx zfA0q${EjfgS=iHl;G>Spr4IE!-3@7aNZFJ;YB@!J+l|Ut!yz>c06PPCGMsL`BU}Pz zqaep6Ov9GHpx-=t_4v`NuO9QGg(*fJxdUVZD(B-dOF6+UqUoK-c(V<|ngP+uYnXxu zZW%Kh%|R2VPmz;Yu7{aezy;tYkPH(vMW?4XGc7H&?h0<dSa}qfLkiq{UR#*T8WKxWJVi(T> z`A9g~F`$e^NyuCYfY+=_IG#M{dCAbCC)#^GCU8K57~=ysLEzxp=tmn+4;Omp zQqN%mfphP`%La>tN6Bci5mWyS@U(zc2;(n3=FwnQ7!U{3b8Z+Jb6SXmD^?I5kcUyD z^S2-W$3#ouY@ElBGhp#YVMm{UUICTx6N-Wf^TuG~9}?h7<3Ax=ideOF_K$INe9I?5 z?|=Y5BTbxUD8pib_|22JVdN!PZ7R7!lU#!{_nO=o$hLu?Be)MnC=|9vjG< zACmE(1pW|7w+N%)ieMl7G=do2KbS-iA|+5KBmxHKhxbo_HBB8qPn&jnxY2v-QsskDH};5LuGr3oXRNR1R+ z5vx$2dtm%WfV@lC$}@D}nYC04tS=q=$A13N8}RFDup1H6g8I01VwrfA4m z@XJ65xId214f0g1{7txvc`Wzfx~u!S`aXc zo5W#Y;uB@!29TAr<`e!RNE>&UQ@FW63_t}K1`OA-mBB*1`FPE#5j#)%amem$VIFuL zVfblM@sZUgJNH@H*jU@Z#5tj_^D~P^8z$J%L;6P{2qvT;uqE@(Wai zi-~d65P>+|Ket2h<3>VHQuV=0MNUpmA>jeZWtf%jqJtccODn>kK#|8Xbh0r3qPMK= zg^hMEc5JC4G?85FJQ0)*@aGT(7?)qj?ErsBCb=>#2$=@VLT_0Y3ixh_pmefErik=R zh%p%E#2d1qOpw8Q9^=BE3xYW#mx&|-qF_$>kjV@m#hmZB!ztx)?6#QR$7S1J|BuIh z&X1B(2}rm{{J3%3DA$;zNJkXN_gsk|Y>At{p9>o5SxGh}^7o&{)6 z!UOV!?Ap1gfUP{gfNj>EFcoEAqGm&8irBt=_O0;fExSFWb93hOEeSu8$7x-KG^Ok9 zvp1j^6rN?AL3l%*s+zVuho8?=YJA8ZmqlF|b+TF$a0IS}jav8%7Xgq#BwQ2kCIR&? zI1yf;fZ(NRD))v_9w2Pv?D$In+^-6|830qg^pL8A7^2ec>56nXWkTe+WkP1_zU-*l zZlMLE?wE>fQ&FU*YEf!vW?$1pLQJD)T9RY0S8aDMEIBm}s4g3;U(73yv9-#WuKa?m z^}4dCD&n>#Xjio56*W{Bb{fjy$Y#%$M9Cq8jP~c;y2X>ig&gcd%p~2Rop#UP#jE6tj^@`dIIevb>MR`B)-<(uTpOyWTXB>d zEY#OHt{u+Q0XeSiRn{vxt}Ry8RrxIeW%cu1aTWf>MA{!01S+WXxHdRr?>w&URaVwH zt}PZT03GSxj&EzMqSA3~iC2Eg?`fAUB`dwRi(5)A%0I5xJ~I8VIO#d}Jdcz)FC{Pk zuA6>8d4Oj!=f2;gc2jMi$_Lh`a>Z2QVnNeY!y;BKK7ExcsMVpf>5og}XD5R!3Zmr? z-D@Om0eg;ot4Gvhv=)ZX77y>04tyqL`iQ$UERMUb=(IFH{JP<)GBpi?fs#>KQ!22Xsg6mEqhq)gI4ymvgcl9&w&eq91OS|SjjM4DR}3X1gfI= zLJwc1ms^$o=vivpQTZ@BDlfcW?-co)XsWyjl@GeBvSX`` zrJ0H(X(m-|$8q|aWUGDKR{Ci@wQ^TwEW?Cg#?0_eVs)sKoDtrLzAGt)!7J-l)O}hu zSKB|i+&}qCcR2ppr=NstI0TsFf`ncW6G+cvq_H}wjWyKiw% ztULe}zDjNarnQ3VBV5CwL|Ba^4oL*cNhAysP;UuK<@l0F2^Xf!7ga8PnD!;So$E^KL+2TwcBS z>hk*St2dWVW*;8^%dk^?9zLn3y`J$c`%99P-l?t_S+VFeviHr!yQdxD4Wj{o0pdbD z{-qDwWdH%?E!&JkoY~DR=sOhCk8yVp550YC0`!Cv`M%cH#JYJUz61=|p&vV=!KmBM zVNEwizZ;0*%N-D zePsc~_#dt$6G2_{z zTD5(PAHe1Qmbe~X);HpkE@-{5;ZX=e`0R}6j$;|7LUcuSsP0Ha6scj5K78Mod!)SU z3aPda4Do)?ZZVVWoyz9DlufTM6}L{(v&E*dj%-Nmpaa||clLP|ila{v17uAzC1Z`Y zAzRy69R|HWC*A5l`=S3lh)Um!uhk7y`j(WAD4UyC`j*wVqI4#UFlI+84cJ!tKKM%C zOYK7QD|@eOEIP`X-c8~! zYa4l-;Dhf4#7PZ3T}73Ro(`fk8@b>O-^hrxSxiX-Q?D1ucOcZclTeR^>UB$^`FN|1 zY;qfrP6P8cm}Gn(c)CZ*|Ef~08fF_gM=IgC)IGCKWN`JBrsOEw3iqpYg7J;7>RnsF z=+KYUGP-)Y$QoQ{qKPoh)0K3|>{aR^4~>(yaj%CF_qtFSI;gnUhB1A=!eakF6B=GE HVj3|3gLW9K literal 0 HcmV?d00001 diff --git a/pom.xml b/pom.xml index 9e554204d..42afbd372 100644 --- a/pom.xml +++ b/pom.xml @@ -960,7 +960,7 @@ 1.1.3 1.7 1.0.7 - [6.1.3-SNAPSHOT] + [6.1.3] cdh5.9.2 3.5 11.0.2 From 1ccf01cdb8f1334c04f0537a25386c15e2770d0c Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 28 Jun 2024 12:38:07 +0200 Subject: [PATCH 041/239] Using the updated Solr JSON payload model classes --- .../model/ProvisionModelSupport.java | 20 +++++++++++++++++-- pom.xml | 2 +- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index a8c9ceeb3..b450b95be 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,6 +5,7 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.solr.ExternalReference; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -375,7 +376,7 @@ public class ProvisionModelSupport { rs.setIsInDiamondJournal(r.getIsInDiamondJournal()); rs.setPubliclyFunded(r.getPubliclyFunded()); rs.setTransformativeAgreement(r.getTransformativeAgreement()); - + rs.setExternalReference(mapExternalReference(r.getExternalReference())); rs.setInstance(mapInstances(r.getInstance())); if (r instanceof Publication) { @@ -561,6 +562,21 @@ public class ProvisionModelSupport { .orElse(null); } + private static List mapExternalReference(List externalReference) { + return Optional.ofNullable(externalReference) + .map(ext -> ext.stream() + .map(e -> ExternalReference.newInstance( + e.getSitename(), + e.getLabel(), + e.getAlternateLabel(), + e.getUrl(), + mapCodeLabel(e.getQualifier()), + e.getRefidentifier(), + e.getQuery())) + .collect(Collectors.toList())) + .orElse(Lists.newArrayList()); + } + private static List asContext(List ctxList, ContextMapper contextMapper) { @@ -579,7 +595,7 @@ public class ProvisionModelSupport { } return Optional - .ofNullable(contexts) + .of(contexts) .map( ctx -> ctx .stream() diff --git a/pom.xml b/pom.xml index 42afbd372..a5bc6cedd 100644 --- a/pom.xml +++ b/pom.xml @@ -960,7 +960,7 @@ 1.1.3 1.7 1.0.7 - [6.1.3] + [6.1.4-SNAPSHOT] cdh5.9.2 3.5 11.0.2 From 1bc8c5d1735a43e392299089fa8b61de5204dc33 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 28 Jun 2024 14:21:12 +0200 Subject: [PATCH 042/239] [graph provision] fixed serialization of the instancetypes --- .../oa/provision/utils/XmlRecordFactory.java | 21 ++++++++++++------- .../provision/IndexRecordTransformerTest.java | 17 ++++++++++----- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index ec322dbd4..3d89b1e34 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -1315,7 +1315,7 @@ public class XmlRecordFactory implements Serializable { instance .getCollectedfrom() .stream() - .filter(cf -> kvNotBlank(cf)) + .filter(XmlRecordFactory::kvNotBlank) .map(cf -> XmlSerializationUtils.mapKeyValue("collectedfrom", cf)) .collect(Collectors.toList())); } @@ -1326,7 +1326,7 @@ public class XmlRecordFactory implements Serializable { instance .getHostedby() .stream() - .filter(hb -> kvNotBlank(hb)) + .filter(XmlRecordFactory::kvNotBlank) .map(hb -> XmlSerializationUtils.mapKeyValue("hostedby", hb)) .collect(Collectors.toList())); } @@ -1336,7 +1336,7 @@ public class XmlRecordFactory implements Serializable { instance .getDateofacceptance() .stream() - .filter(d -> isNotBlank(d)) + .filter(StringUtils::isNotBlank) .map(d -> XmlSerializationUtils.asXmlElement("dateofacceptance", d)) .collect(Collectors.toList())); } @@ -1346,7 +1346,7 @@ public class XmlRecordFactory implements Serializable { instance .getInstancetype() .stream() - .filter(t -> !StringUtils.isNotBlank(t.getClassid())) + .filter(t -> StringUtils.isNotBlank(t.getClassid())) .map(t -> XmlSerializationUtils.mapQualifier("instancetype", t)) .collect(Collectors.toList())); } @@ -1356,7 +1356,7 @@ public class XmlRecordFactory implements Serializable { instance .getDistributionlocation() .stream() - .filter(d -> isNotBlank(d)) + .filter(StringUtils::isNotBlank) .map(d -> XmlSerializationUtils.asXmlElement("distributionlocation", d)) .collect(Collectors.toList())); } @@ -1409,7 +1409,7 @@ public class XmlRecordFactory implements Serializable { instance .getLicense() .stream() - .filter(d -> isNotBlank(d)) + .filter(StringUtils::isNotBlank) .map(d -> XmlSerializationUtils.asXmlElement("license", d)) .collect(Collectors.toList())); } @@ -1540,11 +1540,16 @@ public class XmlRecordFactory implements Serializable { .min(new RefereedComparator()) .orElse(XmlInstance.UNKNOWN_REVIEW_LEVEL)); + Map instanceTypes = Maps.newHashMap(); + instances.forEach(p -> { final Instance i = p.getRight(); instance.getCollectedfrom().add(i.getCollectedfrom()); instance.getHostedby().add(i.getHostedby()); - instance.getInstancetype().add(i.getInstancetype()); + + if (Optional.ofNullable(i.getInstancetype()).map(Qualifier::getClassid).isPresent()) { + instanceTypes.putIfAbsent(i.getInstancetype().getClassid(), i.getInstancetype()); + } instance .setProcessingchargeamount( Optional.ofNullable(i.getProcessingchargeamount()).map(apc -> apc.getValue()).orElse(null)); @@ -1571,6 +1576,8 @@ public class XmlRecordFactory implements Serializable { .ifPresent(instance::setFulltext); }); + instance.getInstancetype().addAll(instanceTypes.values()); + if (instance.getHostedby().size() > 1 && instance.getHostedby().stream().anyMatch(hb -> ModelConstants.UNKNOWN_REPOSITORY.equals(hb))) { instance.getHostedby().remove(ModelConstants.UNKNOWN_REPOSITORY); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/IndexRecordTransformerTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/IndexRecordTransformerTest.java index 718b43f03..923a68352 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/IndexRecordTransformerTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/IndexRecordTransformerTest.java @@ -5,6 +5,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import java.io.IOException; +import java.io.StringReader; import java.net.MalformedURLException; import java.net.URL; import java.util.Arrays; @@ -16,6 +17,9 @@ import javax.xml.transform.TransformerException; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.util.ClientUtils; import org.apache.solr.common.SolrInputDocument; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.io.SAXReader; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -34,7 +38,6 @@ import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory; /** * This test can be used to produce a record that can be manually fed to Solr in XML format. - * * The input is a JoinedEntity, i.e. a json representation of an OpenAIRE entity that embeds all the linked entities. */ public class IndexRecordTransformerTest { @@ -54,7 +57,7 @@ public class IndexRecordTransformerTest { } @Test - public void testPublicationRecordTransformation() throws IOException, TransformerException { + public void testPublicationRecordTransformation() throws IOException, TransformerException, DocumentException { final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, PayloadConverterJob.schemaLocation); @@ -71,11 +74,15 @@ public class IndexRecordTransformerTest { new RelatedEntityWrapper(rel, CreateRelatedEntitiesJob_phase1.asRelatedEntity(pj, Project.class)))); - final String record = xmlRecordFactory.build(je); + final String xmlRecord = xmlRecordFactory.build(je); - assertNotNull(record); + assertNotNull(xmlRecord); - testRecordTransformation(record); + Document doc = new SAXReader().read(new StringReader(xmlRecord)); + + assertEquals("Article", doc.valueOf("//children/instance/instancetype/@classname")); + + testRecordTransformation(xmlRecord); } @Test From 14539f9c8bb554204e96246a94fa6f78aa721971 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 28 Jun 2024 14:52:43 +0200 Subject: [PATCH 043/239] [graph provision] publicFormat worfklow parameter defined as optional --- .../eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 6c58d2466..15d3b6300 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -38,10 +38,6 @@ shadowFormat metadata format name (DMF|TMF) - - publicFormat - the public metadata format - used to create the public collection alias - batchSize number of records to be included in each indexing request From a10e8d9f05b32d59ec5b9feda1ca0d7e5e12ba70 Mon Sep 17 00:00:00 2001 From: miconis Date: Fri, 28 Jun 2024 16:46:52 +0200 Subject: [PATCH 044/239] implementation of countryMatch and addition of workflow parameters --- .../eu/dnetlib/pace/tree/CountryMatch.java | 47 +++++++++++++++++++ .../pace/comparators/ComparatorTest.java | 19 ++++++++ .../openorgs/oozie_app/config-default.xml | 8 ++++ .../oa/dedup/openorgs/oozie_app/workflow.xml | 2 + .../dnetlib/dhp/dedup/conf/org.curr.conf.json | 2 +- 5 files changed, 77 insertions(+), 1 deletion(-) create mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java new file mode 100644 index 000000000..9cf616356 --- /dev/null +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java @@ -0,0 +1,47 @@ +package eu.dnetlib.pace.tree; + +import java.util.Map; + +import com.wcohen.ss.AbstractStringDistance; + +import eu.dnetlib.pace.config.Config; +import eu.dnetlib.pace.tree.support.AbstractStringComparator; +import eu.dnetlib.pace.tree.support.ComparatorClass; + +@ComparatorClass("countryMatch") +public class CountryMatch extends AbstractStringComparator { + + public CountryMatch(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + } + + public CountryMatch(final double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } + + protected CountryMatch(final double weight, final AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } + + @Override + public double distance(final String a, final String b, final Config conf) { + if (a.isEmpty() || b.isEmpty()) { + return -1.0; // return -1 if a field is missing + } + if (a.equalsIgnoreCase("unknown") || b.equalsIgnoreCase("unknown")) { + return -1.0; // return -1 if a country is UNKNOWN + } + + return a.equals(b) ? 1.0 : 0; + } + + @Override + public double getWeight() { + return super.weight; + } + + @Override + protected double normalize(final double d) { + return d; + } +} diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index b37e16cf5..8e72f4efc 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -336,4 +336,23 @@ public class ComparatorTest extends AbstractPaceTest { System.out.println("compare = " + compare); } + @Test + public void countryMatch() { + + CountryMatch countryMatch = new CountryMatch(params); + + double result = countryMatch.distance("UNKNOWN", "UNKNOWN", conf); + assertEquals(-1.0, result); + + result = countryMatch.distance("CHILE", "UNKNOWN", conf); + assertEquals(-1.0, result); + + result = countryMatch.distance("CHILE", "ITALY", conf); + assertEquals(0.0, result); + + result = countryMatch.distance("CHILE", "CHILE", conf); + assertEquals(1.0, result); + + } + } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml index 2e0ed9aee..6d375f03f 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml @@ -15,4 +15,12 @@ oozie.action.sharelib.for.spark spark2 + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + pivotHistoryDatabase + + \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index 6947019e8..7c633facc 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -198,6 +198,8 @@ --isLookUpUrl${isLookUpUrl} --actionSetId${actionSetId} --cutConnectedComponent${cutConnectedComponent} + --hiveMetastoreUris${hiveMetastoreUris} + --pivotHistoryDatabase${pivotHistoryDatabase} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json index 726f2b899..917c9426b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json @@ -49,7 +49,7 @@ }, { "field": "country", - "comparator": "exactMatch", + "comparator": "countryMatch", "weight": 1, "countIfUndefined": "true", "params": {} From 67ff783e650d3b42894dae334b6643778effc9b5 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Sat, 29 Jun 2024 17:13:01 +0200 Subject: [PATCH 045/239] [Person]First implementation to include Person entity in the graph --- .../actionmanager/personentity/Coauthors.java | 19 + .../actionmanager/personentity/Couples.java | 40 +++ .../personentity/ExtractPerson.java | 331 ++++++++++++++++++ .../actionmanager/personentity/WorkList.java | 23 ++ .../CreateActionSetFromWebEntries.java | 12 +- .../dhp/collection/orcid/model/Author.java | 3 + .../dhp/collection/orcid/model/ORCIDItem.java | 3 + .../dhp/collection/orcid/model/Work.java | 2 + .../personentity/as_parameters.json | 26 ++ .../actionmanager/person/CreatePersonAS.java | 114 ++++++ .../actionmanager/person/WorkJson/part-00000 | 10 + pom.xml | 19 + 12 files changed, 599 insertions(+), 3 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/person/WorkJson/part-00000 diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java new file mode 100644 index 000000000..a755c875f --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java @@ -0,0 +1,19 @@ +package eu.dnetlib.dhp.actionmanager.personentity; + +import eu.dnetlib.dhp.schema.oaf.Relation; + +import java.io.Serializable; +import java.util.ArrayList; + + +public class Coauthors implements Serializable { + private ArrayList coauthors; + + public ArrayList getCoauthors() { + return coauthors; + } + + public void setCoauthors(ArrayList coauthors) { + this.coauthors = coauthors; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java new file mode 100644 index 000000000..65670b22e --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java @@ -0,0 +1,40 @@ +package eu.dnetlib.dhp.actionmanager.personentity; + +import eu.dnetlib.dhp.schema.oaf.Person; +import eu.dnetlib.dhp.schema.oaf.Relation; +import scala.Tuple2; + + +import java.io.Serializable; + +public class Couples implements Serializable { + Person p ; + Relation r; + + public Couples() { + + } + + public Person getP() { + return p; + } + + public void setP(Person p) { + this.p = p; + } + + public Relation getR() { + return r; + } + + public void setR(Relation r) { + this.r = r; + } + + public static Couples newInstance(Tuple2 couple){ + Couples c = new Couples(); + c.p = couple._1(); + c.r = couple._2(); + return c; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java new file mode 100644 index 000000000..0bbda0343 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -0,0 +1,331 @@ +package eu.dnetlib.dhp.actionmanager.personentity; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.actionmanager.Constants; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.orcid.model.Author; +import eu.dnetlib.dhp.collection.orcid.model.Employment; +import eu.dnetlib.dhp.collection.orcid.model.Work; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Person; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.schema.oaf.Pid; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; +import eu.dnetlib.dhp.schema.oaf.utils.PidType; +import eu.dnetlib.dhp.utils.DHPUtils; +import org.apache.commons.cli.ParseException; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.*; +import org.jetbrains.annotations.NotNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.spark_project.jetty.util.StringUtil; +import scala.Tuple2; +import static org.apache.spark.sql.functions.*; +import java.io.IOException; +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +public class ExtractPerson implements Serializable { + private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String OPENAIRE_PREFIX = "openaire____"; + private static final String SEPARATOR = "::"; + private static final String orcidKey = "10|" + OPENAIRE_PREFIX + SEPARATOR + DHPUtils.md5(ModelConstants.ORCID.toLowerCase()); + + private static final String DOI_PREFIX = "50|doi_________::"; + + private static final String PMID_PREFIX = "50|pmid________::"; + private static final String ARXIV_PREFIX = "50|arXiv_______::"; + + private static final String PMCID_PREFIX = "50|pmcid_______::"; + private static final String ROR_PREFIX = "20|ror_________::"; + private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; + public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; + public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + + + public static void main(final String[] args) throws IOException, ParseException { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + ExtractPerson.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json")))); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("inputPath"); + log.info("inputPath {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}", outputPath); + + final String workingDir = parser.get("workingDir"); + log.info("workingDir {}", workingDir); + + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> createActionSet(spark, inputPath, outputPath, workingDir)); + + } + + private static void createActionSet(SparkSession spark, String inputPath, String outputPath, String workingDir) { + + Dataset authors = spark + .read() + .parquet(inputPath + "Authors").as(Encoders.bean(Author.class)); + + Dataset works = spark + .read() + .parquet(inputPath + "Works") + .as(Encoders.bean(Work.class)) + .filter((FilterFunction) w -> Optional.ofNullable(w.getPids()).isPresent() && + w.getPids().stream().anyMatch(p->p.getSchema().equalsIgnoreCase("doi") || + p.getSchema().equalsIgnoreCase("pmc") || + p.getSchema().equalsIgnoreCase("pmid") || + p.getSchema().equalsIgnoreCase("arxiv"))); + + Dataset employmentDataset = spark + .read() + .parquet(inputPath + "Employments") + .as(Encoders.bean(Employment.class)); + + + Dataset peopleToMap = authors.joinWith(works, authors.col("orcid").equalTo(works.col("orcid"))) + .map((MapFunction, Author>) t2 -> t2._1(), Encoders.bean(Author.class)) + .groupByKey((MapFunction) a -> a.getOrcid(), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Author.class)); + + + Dataset employment = employmentDataset.joinWith(peopleToMap, employmentDataset.col("orcid").equalTo(peopleToMap.col("orcid"))) + .map((MapFunction, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class)); + + peopleToMap.show(false); + + Dataset people; + people = peopleToMap.map((MapFunction) op -> { + Person person = new Person(); + person.setId(DHPUtils.generateIdentifier(op.getOrcid(), PERSON_PREFIX)); + person.setBiography(Optional.ofNullable(op.getBiography()) + + .orElse("")); + KeyValue kv = OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS); + kv.setDataInfo(null); + person.setCollectedfrom(Arrays.asList(kv)); + person.setAlternativeNames(Optional.ofNullable(op.getOtherNames()) + + .orElse(new ArrayList<>())); + person.setFamilyName(Optional.ofNullable(op.getFamilyName()) + + .orElse("")); + person.setGivenName(Optional.ofNullable(op.getGivenName()) + + .orElse("")); + person.setPid(Optional.ofNullable(op.getOtherPids()) + .map(v -> v.stream().map(p -> Pid.newInstance(p.getSchema(), p.getValue())).collect(Collectors.toList())) + .orElse(new ArrayList<>()) + ); + person.getPid().add(Pid.newInstance(ModelConstants.ORCID, op.getOrcid())); + person.setDateofcollection(op.getLastModifiedDate()); + person.setOriginalId(Arrays.asList(op.getOrcid())); + return person; + }, Encoders.bean(Person.class)); + + + people.show(false); + + + Dataset authorship; + authorship = works + .flatMap((FlatMapFunction) ExtractPerson::getAuthorshipRelationIterator + , Encoders.bean(Relation.class)); + + + authorship.show(false); + + + Dataset coauthorship = works + .flatMap((FlatMapFunction>) w -> { + List> lista = new ArrayList<>(); + w.getPids().stream().forEach(p -> { + if (p.getSchema().equalsIgnoreCase("doi") || p.getSchema().equalsIgnoreCase("pmc") || p.getSchema().equalsIgnoreCase("pmid") || p.getSchema().equalsIgnoreCase("arxiv")) + lista.add(new Tuple2<>(p.getValue(), w.getOrcid())); + }); + return lista.iterator(); + }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())) + .groupByKey((MapFunction, String>) Tuple2::_1, Encoders.STRING()) + .mapGroups((MapGroupsFunction, Coauthors>) (k, it) -> + extractCoAuthors(it), Encoders.bean(Coauthors.class)) + .flatMap((FlatMapFunction) c -> c.getCoauthors().iterator(), Encoders.bean(Relation.class)) + .groupByKey((MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Relation.class)); + + coauthorship.show(false); + Dataset affiliation = employment + .filter((FilterFunction) e -> Optional.ofNullable(e.getAffiliationId()).isPresent()) + .filter((FilterFunction) e -> e.getAffiliationId().getSchema().equalsIgnoreCase("ror")) + .map((MapFunction) ExtractPerson::getAffiliationRelation + , Encoders.bean(Relation.class)); + + affiliation.show(false); + + people.toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .union(authorship.toJavaRDD().map(r-> new AtomicAction(r.getClass(),r))) + .union(coauthorship.toJavaRDD().map(r-> new AtomicAction(r.getClass(),r))) + .union(affiliation.toJavaRDD().map(r->new AtomicAction(r.getClass(),r))) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);//, GzipCodec.class); + } + + private static Coauthors extractCoAuthors(Iterator> it) { + Coauthors coauth = new Coauthors(); + ArrayList ret = new ArrayList<>(); + List coauthors = new ArrayList<>(); + while(it.hasNext()) + coauthors.add(it.next()._2()); + for (int i = 0; i < coauthors.size() -1; i++ ) + for(int j = i + 1; j < coauthors.size(); j++) + ret.addAll(getCoAuthorshipRelations(coauthors.get(i), coauthors.get(j))); + + coauth.setCoauthors(ret); + + return coauth; + } + + private static Relation getAffiliationRelation(Employment row) { + String source = PERSON_PREFIX + IdentifierFactory.md5(row.getOrcid()); + String target = ROR_PREFIX + + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAffiliationId().getValue())); + List properties = new ArrayList<>() ; + + Relation relation = + OafMapperUtils.getRelation(source, target, ModelConstants.ORG_PERSON_RELTYPE, ModelConstants.ORG_PERSON_SUBRELTYPE, ModelConstants.ORG_PERSON_PARTICIPATES , + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils.dataInfo(false, null, false, false, + OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), + null); + + if(Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())){ + KeyValue kv = new KeyValue(); + kv.setKey("startDate"); + kv.setValue(row.getStartDate()); + properties.add(kv); + } + if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtil.isNotBlank(row.getEndDate())) { + KeyValue kv = new KeyValue(); + kv.setKey("endDate"); + kv.setValue(row.getEndDate()); + properties.add(kv); + } + + if (properties.size() > 0) + relation.setProperties(properties); + return relation; + + + } + + private static Collection getCoAuthorshipRelations(String orcid1, String orcid2) { + String source = PERSON_PREFIX + IdentifierFactory.md5(orcid1); + String target = PERSON_PREFIX + IdentifierFactory.md5(orcid2); + + return Arrays.asList(OafMapperUtils.getRelation(source, target,ModelConstants.PERSON_PERSON_RELTYPE, + ModelConstants.PERSON_PERSON_SUBRELTYPE, + ModelConstants.PERSON_PERSON_HASCOAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils.dataInfo(false, null, false, false, + OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), + null), + OafMapperUtils.getRelation(target, source,ModelConstants.PERSON_PERSON_RELTYPE, + ModelConstants.PERSON_PERSON_SUBRELTYPE, + ModelConstants.PERSON_PERSON_HASCOAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils.dataInfo(false, null, false, false, + OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), + null)); + + } + + private static @NotNull Iterator getAuthorshipRelationIterator(Work w) { + + if(Optional.ofNullable(w.getPids()).isPresent()) + return w.getPids() + .stream() + .map(pid -> getRelation(w.getOrcid(), pid)) + .filter(Objects::nonNull).collect(Collectors.toList()).iterator(); + List ret = new ArrayList<>(); + return ret.iterator(); + } + + + private static Relation getRelation(String orcid, eu.dnetlib.dhp.collection.orcid.model.Pid pid){ + String target ; + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); + switch (pid.getSchema()){ + case "doi": + target = DOI_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), pid.getValue())); + break; + case "pmid": + target = PMID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), pid.getValue())); + break; + case "arxiv": + target = ARXIV_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), pid.getValue())); + break; + case "pmcid": + target = PMCID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), pid.getValue())); + break; + + default: + return null; + } + + + return OafMapperUtils.getRelation(source, target,ModelConstants.RESULT_PERSON_RELTYPE, + ModelConstants.RESULT_PERSON_SUBRELTYPE, + ModelConstants.RESULT_PERSON_HASAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils.dataInfo(false, null, false, false, + OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), + null); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java new file mode 100644 index 000000000..71d2b491a --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java @@ -0,0 +1,23 @@ +package eu.dnetlib.dhp.actionmanager.personentity; + +import eu.dnetlib.dhp.collection.orcid.model.Work; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; + +import java.io.Serializable; +import java.util.ArrayList; + +public class WorkList implements Serializable { + private ArrayList workArrayList; + + public ArrayList getWorkArrayList() { + return workArrayList; + } + + public void setWorkArrayList(ArrayList workArrayList) { + this.workArrayList = workArrayList; + } + + public WorkList() { + workArrayList = new ArrayList<>(); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index 27970f2c3..263179d6f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -104,8 +104,8 @@ public class CreateActionSetFromWebEntries implements Serializable { final String ror = ROR_PREFIX + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror"))); ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror)); - ret.addAll(createAffiliationRelationPairPMID(row.getAs("pmid"), ror)); - ret.addAll(createAffiliationRelationPairPMCID(row.getAs("pmcid"), ror)); +// ret.addAll(createAffiliationRelationPairPMID(row.getAs("pmid"), ror)); +// ret.addAll(createAffiliationRelationPairPMCID(row.getAs("pmcid"), ror)); return ret .iterator(); @@ -139,11 +139,17 @@ public class CreateActionSetFromWebEntries implements Serializable { "institution", functions .explode( functions.col("institutions"))) + .selectExpr( - "id", "doi", "ids.pmcid as pmcid", "ids.pmid as pmid", "institution.ror as ror", + "id", "doi", "institution.ror as ror", "institution.country_code as country_code", "publication_year") .distinct(); +// .selectExpr( +// "id", "doi", "ids.pmcid as pmcid", "ids.pmid as pmid", "institution.ror as ror", +// "institution.country_code as country_code", "publication_year") +// .distinct(); + } private static Dataset readBlackList(SparkSession spark, String inputPath) { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java index df87e4333..a1545ebfe 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java @@ -20,6 +20,9 @@ public class Author extends ORCIDItem { private String lastModifiedDate; + public Author() { + } + public String getBiography() { return biography; } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java index 6bc47bc26..419823cb1 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java @@ -11,4 +11,7 @@ public class ORCIDItem { public void setOrcid(String orcid) { this.orcid = orcid; } + + public ORCIDItem() { + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java index 670170323..a8683aaaf 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java @@ -32,4 +32,6 @@ public class Work extends ORCIDItem { pids.add(pid); } + public Work() { + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json new file mode 100644 index 000000000..3310f16e4 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName": "ip", + "paramLongName": "inputPath", + "paramDescription": "the zipped opencitations file", + "paramRequired": true + }, + { + "paramName": "op", + "paramLongName": "outputPath", + "paramDescription": "the working path", + "paramRequired": true + }, + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the hdfs name node", + "paramRequired": false + }, + { + "paramName": "wd", + "paramLongName": "workingDir", + "paramDescription": "the hdfs name node", + "paramRequired": false + } +] diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java new file mode 100644 index 000000000..a4d56ba2e --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java @@ -0,0 +1,114 @@ + +package eu.dnetlib.dhp.actionmanager.person; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob; +import eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson; +import eu.dnetlib.dhp.collection.orcid.model.Author; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class CreatePersonAS { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + private static final Logger log = LoggerFactory + .getLogger(CreatePersonAS.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files + .createTempDirectory(CreatePersonAS.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(CreatePersonAS.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.codegen.wholeStage","false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(CreatePersonAS.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testAuthors() throws Exception { + + String inputPath = getClass() + .getResource( + "/eu/dnetlib/dhp/actionmanager/person/") + .getPath(); + +// spark +// .read() +// .parquet(inputPath + "Authors") +// .as(Encoders.bean(Author.class)) +// .filter((FilterFunction) a -> Optional.ofNullable(a.getOtherNames()).isPresent() && +// Optional.ofNullable(a.getBiography()).isPresent()) +// .write() +// .mode(SaveMode.Overwrite) +// .parquet(workingDir.toString() + "AuthorsSubset"); + + + ExtractPerson + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-inputPath", + inputPath, + "-outputPath", + workingDir.toString() + "/actionSet1", + "-workingDir", + workingDir.toString() + "/working" + }); + + + + } + + + } diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/person/WorkJson/part-00000 b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/person/WorkJson/part-00000 new file mode 100644 index 000000000..636595d49 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/person/WorkJson/part-00000 @@ -0,0 +1,10 @@ +{"orcid":"0000-0001-6291-9619","title":"A Visible Light Driven Photoelectrochemical Chloramphenicol Aptasensor Based on a Gold Nanoparticle-Functionalized 3D Flower-like MoS2/TiO2 Heterostructure","pids":[{"value":"10.1021/acs.langmuir.1c02956","schema":"doi"},{"value":"2-s2.0-85124885368","schema":"eid"},{"value":"15205827 07437463","schema":"issn"}]} +{"orcid":"0000-0002-3210-3034","title":"A Visible Light Driven Photoelectrochemical Chloramphenicol Aptasensor Based on a Gold Nanoparticle-Functionalized 3D Flower-like MoS2/TiO2 Heterostructure","pids":[{"value":"10.1021/acs.langmuir.1c02956","schema":"doi"},{"value":"2-s2.0-85124885368","schema":"eid"},{"value":"15205827 07437463","schema":"issn"}]} +{"orcid":"0000-0001-6291-9619","title":"Study of High-Transverse-Momentum Higgs Boson Production in Association with a Vector Boson in the qqbb Final State with the ATLAS Detector","pids":[{"value":"2736741","schema":"other-id"},{"value":"10.1103/PhysRevLett.132.131802","schema":"doi"},{"value":"2312.07605","schema":"arxiv"}]} +{"orcid":"0000-0002-3210-3034","title":"Study of High-Transverse-Momentum Higgs Boson Production in Association with a Vector Boson in the qqbb Final State with the ATLAS Detector","pids":[{"value":"2736741","schema":"other-id"},{"value":"10.1103/PhysRevLett.132.131802","schema":"doi"},{"value":"2312.07605","schema":"arxiv"}]} +{"orcid":"0000-0002-9030-7609","title":"Search for supersymmetry in a final state containing two photons and missing transverse momentum in √s = 13 TeV pp collisions at the LHC using the ATLAS detector","pids":[{"value":"10.1140/epjc/s10052-016-4344-x","schema":"doi"},{"value":"2-s2.0-84988710988","schema":"eid"},{"value":"14346052 14346044","schema":"issn"}]} +{"orcid":"0000-0003-2552-9691","title":"Search for supersymmetry in a final state containing two photons and missing transverse momentum in $\\sqrt{s}$ = 13 TeV $pp$ collisions at the LHC using the ATLAS detector","pids":[{"value":"1473744","schema":"other-id"},{"value":"10.1140/epjc/s10052-016-4344-x","schema":"doi"},{"value":"1606.09150","schema":"arxiv"}]} +{"orcid":"0000-0003-0305-8980","title":"Search for supersymmetry in a final state containing two photons and missing transverse momentum in √s = 13 TeV pp collisions at the LHC using the ATLAS detector","pids":[{"value":"10.1140/epjc/s10052-016-4344-x","schema":"doi"},{"value":"2-s2.0-84988710988","schema":"eid"}]} +{"orcid":"0000-0002-9030-7609","title":"Measurement of the energy response of the ATLAS calorimeter to charged pions from $W^{\\pm }\\rightarrow \\tau ^{\\pm }(\\rightarrow \\pi ^{\\pm }\\nu _{\\tau })\\nu _{\\tau }$ events in Run 2 data","pids":[{"value":"1909507","schema":"other-id"},{"value":"10.1140/epjc/s10052-022-10117-2","schema":"doi"},{"value":"2108.09043","schema":"arxiv"}]} +{"orcid":"0000-0003-2629-4046","title":"Measurement of the energy response of the ATLAS calorimeter to charged pions from $W^{\\pm }\\rightarrow \\tau ^{\\pm }(\\rightarrow \\pi ^{\\pm }\\nu _{\\tau })\\nu _{\\tau }$ events in Run 2 data","pids":[{"value":"1909507","schema":"other-id"},{"value":"10.1140/epjc/s10052-022-10117-2","schema":"doi"},{"value":"2108.09043","schema":"arxiv"}]} +{"orcid":"0000-0001-8582-8912","title":"Measurement of the energy response of the ATLAS calorimeter to charged pions from $W^{\\pm }\\rightarrow \\tau ^{\\pm }(\\rightarrow \\pi ^{\\pm }\\nu _{\\tau })\\nu _{\\tau }$ events in Run 2 data","pids":[{"value":"1909507","schema":"other-id"},{"value":"10.1140/epjc/s10052-022-10117-2","schema":"doi"},{"value":"2108.09043","schema":"arxiv"}]} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 9e554204d..c8d7a980a 100644 --- a/pom.xml +++ b/pom.xml @@ -994,6 +994,25 @@ + + + arm-silicon-mac + + + aarch64 + mac + + + + + + org.xerial.snappy + snappy-java + 1.1.8.4 + + + + spark-34 From 236b64d8305a93cea73630c89b14d3784b923ae7 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Sat, 29 Jun 2024 18:29:20 +0200 Subject: [PATCH 046/239] [AffiliationIngestion]Extended the ingestion of affiliation from open aire to include also links derived from Web Crawl. Extended the test. Inserted in Constatns the id and name of the webcrawl datasource to be used here and also in the ingestion of links from web crawl --- .../eu/dnetlib/dhp/actionmanager/Constants.java | 3 +++ .../PrepareAffiliationRelations.java | 15 ++++++++++++--- .../webcrawl/CreateActionSetFromWebEntries.java | 10 ++++++---- .../input_actionset_parameter.json | 8 +++++++- .../bipaffiliations/oozie_app/workflow.xml | 6 +++++- .../PrepareAffiliationRelationsTest.java | 7 ++++--- .../actionmanager/bipaffiliations/doi_to_ror.json | 4 +++- 7 files changed, 40 insertions(+), 13 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java index 006d3af76..73b4b77cb 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -42,6 +42,9 @@ public class Constants { public static final String NULL = "NULL"; public static final String NA = "N/A"; + public static final String WEB_CRAWL_ID = "10|openaire____::fb98a192f6a055ba495ef414c330834b"; + public static final String WEB_CRAWL_NAME = "Web Crawl"; + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private Constants() { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index c10eb5c8c..b0b757005 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -41,9 +41,9 @@ public class PrepareAffiliationRelations implements Serializable { private static final Logger log = LoggerFactory.getLogger(PrepareAffiliationRelations.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String ID_PREFIX = "50|doi_________::"; - public static final String BIP_AFFILIATIONS_CLASSID = "result:organization:bipinference"; - public static final String BIP_AFFILIATIONS_CLASSNAME = "Affiliation relation inferred by BIP!"; - public static final String BIP_INFERENCE_PROVENANCE = "bip:affiliation:crossref"; + public static final String BIP_AFFILIATIONS_CLASSID = "result:organization:openaireinference"; + public static final String BIP_AFFILIATIONS_CLASSNAME = "Affiliation relation inferred by OpenAIRE"; + public static final String BIP_INFERENCE_PROVENANCE = "openaire:affiliation"; public static void main(String[] args) throws Exception { @@ -71,6 +71,9 @@ public class PrepareAffiliationRelations implements Serializable { final String dataciteInputPath = parser.get("dataciteInputPath"); log.info("dataciteInputPath: {}", dataciteInputPath); + final String webcrawlInputPath = parser.get("webCrawlInputPath"); + log.info("webcrawlInputPath: {}", webcrawlInputPath); + final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); @@ -102,10 +105,16 @@ public class PrepareAffiliationRelations implements Serializable { JavaPairRDD dataciteRelations = prepareAffiliationRelations( spark, dataciteInputPath, collectedFromDatacite); + List collectedFromWebCrawl = OafMapperUtils + .listKeyValues(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME); + JavaPairRDD webCrawlRelations = prepareAffiliationRelations( + spark, webcrawlInputPath, collectedFromWebCrawl); + crossrefRelations .union(pubmedRelations) .union(openAPCRelations) .union(dataciteRelations) + .union(webCrawlRelations) .saveAsHadoopFile( outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index 27970f2c3..f4b0cbc6f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -1,12 +1,15 @@ package eu.dnetlib.dhp.actionmanager.webcrawl; + import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.actionmanager.Constants; +import io.netty.util.Constant; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.GzipCodec; @@ -44,8 +47,7 @@ public class CreateActionSetFromWebEntries implements Serializable { private static final String PMID_PREFIX = "50|pmid________::"; private static final String PMCID_PREFIX = "50|pmc_________::"; - private static final String WEB_CRAWL_ID = "10|openaire____::fb98a192f6a055ba495ef414c330834b"; - private static final String WEB_CRAWL_NAME = "Web Crawl"; + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { @@ -214,7 +216,7 @@ public class CreateActionSetFromWebEntries implements Serializable { ModelConstants.IS_AUTHOR_INSTITUTION_OF, Arrays .asList( - OafMapperUtils.keyValue(WEB_CRAWL_ID, WEB_CRAWL_NAME)), + OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), OafMapperUtils .dataInfo( false, null, false, false, @@ -233,7 +235,7 @@ public class CreateActionSetFromWebEntries implements Serializable { ModelConstants.HAS_AUTHOR_INSTITUTION, Arrays .asList( - OafMapperUtils.keyValue(WEB_CRAWL_ID, WEB_CRAWL_NAME)), + OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), OafMapperUtils .dataInfo( false, null, false, false, diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json index 9671129f7..4d85cf26b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json @@ -28,7 +28,13 @@ "paramLongName": "dataciteInputPath", "paramDescription": "the path to get the input data from Datacite", "paramRequired": true - }, + },{ + "paramName": "wip", + "paramLongName": "webCrawlInputPath", + "paramDescription": "the path to get the input data from Web Crawl", + "paramRequired": true +} +, { "paramName": "o", "paramLongName": "outputPath", diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml index e8e6a7c33..2e89c07fd 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml @@ -17,6 +17,10 @@ dataciteInputPath the path where to find the inferred affiliation relations from Datacite + + webCrawlInputPath + the path where to find the inferred affiliation relations from webCrawl + outputPath the path where to store the actionset @@ -112,7 +116,7 @@ --pubmedInputPath${pubmedInputPath} --openapcInputPath${openapcInputPath} --dataciteInputPath${dataciteInputPath} - + --webCrawlInputPath${webCrawlInputPath} --outputPath${outputPath} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index bceb9d754..bb0188e43 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -88,6 +88,7 @@ public class PrepareAffiliationRelationsTest { "-pubmedInputPath", crossrefAffiliationRelationPath, "-openapcInputPath", crossrefAffiliationRelationPath, "-dataciteInputPath", crossrefAffiliationRelationPath, + "-webCrawlInputPath", crossrefAffiliationRelationPath, "-outputPath", outputPath }); @@ -104,7 +105,7 @@ public class PrepareAffiliationRelationsTest { // ); // } // count the number of relations - assertEquals(80, tmp.count()); + assertEquals(120, tmp.count()); Dataset dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); dataset.createOrReplaceTempView("result"); @@ -115,7 +116,7 @@ public class PrepareAffiliationRelationsTest { // verify that we have equal number of bi-directional relations Assertions .assertEquals( - 40, execVerification + 60, execVerification .filter( "relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'") .collectAsList() @@ -123,7 +124,7 @@ public class PrepareAffiliationRelationsTest { Assertions .assertEquals( - 40, execVerification + 60, execVerification .filter( "relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'") .collectAsList() diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json index 985a8d14b..08dc3f7eb 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json @@ -4,4 +4,6 @@ {"DOI":"10.1142\/s021821650200186x","Matchings":[{"RORid":"https:\/\/ror.org\/035xkbk20","Confidence":1},{"RORid":"https:\/\/ror.org\/05apxxy63","Confidence":1}]} {"DOI":"10.1061\/(asce)0733-9372(2002)128:7(575)","Matchings":[{"RORid":"https:\/\/ror.org\/04j198w64","Confidence":0.82}]} {"DOI":"10.1061\/(asce)0733-9372(2002)128:7(588)","Matchings":[{"RORid":"https:\/\/ror.org\/03m8km719","Confidence":0.8660254038},{"RORid":"https:\/\/ror.org\/02aze4h65","Confidence":0.87}]} -{"DOI":"10.1161\/hy0202.103001","Matchings":[{"RORid":"https:\/\/ror.org\/057xtrt18","Confidence":0.7071067812}]} \ No newline at end of file +{"DOI":"10.1161\/hy0202.103001","Matchings":[{"RORid":"https:\/\/ror.org\/057xtrt18","Confidence":0.7071067812}]} +{"DOI": "10.1080/13669877.2015.1042504", "Matchings": [{"Confidence": 1.0, "RORid": "https://ror.org/03265fv13"}]} +{"DOI": "10.1007/3-540-47984-8_14", "Matchings": [{"Confidence": 1.0, "RORid": "https://ror.org/00a0n9e72"}]} \ No newline at end of file From 9cbe966b4adcc497798af69f1a01c47aab3b5926 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Sat, 29 Jun 2024 18:35:49 +0200 Subject: [PATCH 047/239] [AffiliationIngestion]refactoring --- .../bipaffiliations/PrepareAffiliationRelations.java | 6 +++--- .../webcrawl/CreateActionSetFromWebEntries.java | 5 ++--- .../bipaffiliations/PrepareAffiliationRelationsTest.java | 2 +- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index b0b757005..8f911e980 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -106,15 +106,15 @@ public class PrepareAffiliationRelations implements Serializable { spark, dataciteInputPath, collectedFromDatacite); List collectedFromWebCrawl = OafMapperUtils - .listKeyValues(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME); + .listKeyValues(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME); JavaPairRDD webCrawlRelations = prepareAffiliationRelations( - spark, webcrawlInputPath, collectedFromWebCrawl); + spark, webcrawlInputPath, collectedFromWebCrawl); crossrefRelations .union(pubmedRelations) .union(openAPCRelations) .union(dataciteRelations) - .union(webCrawlRelations) + .union(webCrawlRelations) .saveAsHadoopFile( outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index f4b0cbc6f..533b90cd8 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -1,15 +1,12 @@ package eu.dnetlib.dhp.actionmanager.webcrawl; - import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.actionmanager.Constants; -import io.netty.util.Constant; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.GzipCodec; @@ -24,6 +21,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.actionmanager.Constants; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; @@ -32,6 +30,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; import eu.dnetlib.dhp.schema.oaf.utils.PidType; +import io.netty.util.Constant; import scala.Tuple2; /** diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index bb0188e43..bffe41ac7 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -88,7 +88,7 @@ public class PrepareAffiliationRelationsTest { "-pubmedInputPath", crossrefAffiliationRelationPath, "-openapcInputPath", crossrefAffiliationRelationPath, "-dataciteInputPath", crossrefAffiliationRelationPath, - "-webCrawlInputPath", crossrefAffiliationRelationPath, + "-webCrawlInputPath", crossrefAffiliationRelationPath, "-outputPath", outputPath }); From a2b708bb71b1f8a81f3d2ef369eb80638ade7808 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Sat, 29 Jun 2024 18:36:47 +0200 Subject: [PATCH 048/239] [AffiliationIngestion]refactoring --- .../model/ProvisionModelSupport.java | 20 ++++--- .../oa/provision/SolrConfigExploreTest.java | 56 ++++++++++--------- .../dhp/oa/provision/SolrConfigTest.java | 3 +- 3 files changed, 44 insertions(+), 35 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index b450b95be..abcf4992f 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.solr.ExternalReference; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -31,6 +30,7 @@ import eu.dnetlib.dhp.schema.solr.Context; import eu.dnetlib.dhp.schema.solr.Country; import eu.dnetlib.dhp.schema.solr.Datasource; import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines; +import eu.dnetlib.dhp.schema.solr.ExternalReference; import eu.dnetlib.dhp.schema.solr.Instance; import eu.dnetlib.dhp.schema.solr.Journal; import eu.dnetlib.dhp.schema.solr.Measure; @@ -562,10 +562,16 @@ public class ProvisionModelSupport { .orElse(null); } - private static List mapExternalReference(List externalReference) { - return Optional.ofNullable(externalReference) - .map(ext -> ext.stream() - .map(e -> ExternalReference.newInstance( + private static List mapExternalReference( + List externalReference) { + return Optional + .ofNullable(externalReference) + .map( + ext -> ext + .stream() + .map( + e -> ExternalReference + .newInstance( e.getSitename(), e.getLabel(), e.getAlternateLabel(), @@ -573,8 +579,8 @@ public class ProvisionModelSupport { mapCodeLabel(e.getQualifier()), e.getRefidentifier(), e.getQuery())) - .collect(Collectors.toList())) - .orElse(Lists.newArrayList()); + .collect(Collectors.toList())) + .orElse(Lists.newArrayList()); } private static List asContext(List ctxList, diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java index 41eac2a30..90aef5adc 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java @@ -1,12 +1,13 @@ package eu.dnetlib.dhp.oa.provision; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.File; import java.io.IOException; import java.net.URI; import java.nio.file.Path; -import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrQuery; @@ -32,14 +33,13 @@ import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; - -import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.jupiter.api.Assertions.assertEquals; +import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; +import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) public class SolrConfigExploreTest { @@ -91,7 +91,7 @@ public class SolrConfigExploreTest { SparkConf conf = new SparkConf(); conf.setAppName(XmlIndexingJobTest.class.getSimpleName()); conf.registerKryoClasses(new Class[] { - SerializableSolrInputDocument.class + SerializableSolrInputDocument.class }); conf.setMaster("local[1]"); @@ -101,10 +101,10 @@ public class SolrConfigExploreTest { conf.set("spark.sql.warehouse.dir", workingDir.resolve("spark").toString()); spark = SparkSession - .builder() - .appName(SolrConfigExploreTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); + .builder() + .appName(SolrConfigExploreTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); // random unassigned HTTP port final int jettyPort = 0; @@ -134,35 +134,35 @@ public class SolrConfigExploreTest { log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); NamedList res = createCollection( - miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); // miniCluster.getSolrClient().setDefaultCollection(SHADOW_COLLECTION); res = createCollection( - miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); admin = new SolrAdminApplication(miniCluster.getZkClient().getZkServerAddress()); CollectionAdminResponse rsp = (CollectionAdminResponse) admin - .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); assertEquals(0, rsp.getStatus()); rsp = (CollectionAdminResponse) admin.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); } @@ -180,7 +180,8 @@ public class SolrConfigExploreTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions + .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", @@ -200,7 +201,8 @@ public class SolrConfigExploreTest { // System.out.println(rsp.getExplainMap()); for (SolrDocument doc : rsp.getResults()) { - log.info( + log + .info( doc.get("score") + "\t" + doc.get("__indexrecordidentifier") + "\t" + doc.get("resultidentifier") + "\t" + @@ -216,7 +218,7 @@ public class SolrConfigExploreTest { } protected static NamedList createCollection(CloudSolrClient client, String name, int numShards, - int replicationFactor, int maxShardsPerNode, String configName) throws Exception { + int replicationFactor, int maxShardsPerNode, String configName) throws Exception { ModifiableSolrParams modParams = new ModifiableSolrParams(); modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name()); modParams.set("name", name); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java index 2c62389c6..c04fa1cc6 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java @@ -85,7 +85,8 @@ public class SolrConfigTest extends SolrTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions + .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", From 7d3292551b976a5b07f7e283ec95d0bc4015e8e7 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 2 Jul 2024 15:43:11 +0200 Subject: [PATCH 049/239] ignore dates containing 'null's --- .../java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index f1221add3..7606a4d2f 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -655,6 +655,13 @@ public class MergeUtils { return d1; } + if (StringUtils.contains(d1.getValue(), "null")) { + return d2; + } + if (StringUtils.contains(d2.getValue(), "null")) { + return d1; + } + return Stream .of(d1, d2) .min( From 54e11b6a4352862b762b5127fe38a5cea28092c0 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 3 Jul 2024 13:03:15 +0300 Subject: [PATCH 050/239] Improve performance and efficiency by rewriting the creation process of "publication", "project", "dataset", "datasource", "software", "otherresearchproduct" and "result" tables, to be performed in a single query, for each one. --- .../graph/stats/oozie_app/scripts/step11.sql | 102 ---------------- .../graph/stats/oozie_app/scripts/step12.sql | 40 +----- .../stats/oozie_app/scripts/step16_5.sql | 74 ++++-------- .../graph/stats/oozie_app/scripts/step2.sql | 68 +++++------ .../graph/stats/oozie_app/scripts/step3.sql | 65 +++++----- .../graph/stats/oozie_app/scripts/step4.sql | 68 +++++------ .../graph/stats/oozie_app/scripts/step5.sql | 66 +++++----- .../graph/stats/oozie_app/scripts/step6.sql | 114 ++++++++++-------- .../graph/stats/oozie_app/scripts/step7.sql | 10 +- .../graph/stats/oozie_app/scripts/step8.sql | 99 +++++---------- 10 files changed, 248 insertions(+), 458 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 207c1b124..7597f1429 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -4,108 +4,6 @@ ---------------------------------------------------------------- ---------------------------------------------------------------- ---Datasource temporary table updates -UPDATE ${stats_db_name}.datasource_tmp -SET harvested='true' -WHERE datasource_tmp.id IN (SELECT DISTINCT d.id - FROM ${stats_db_name}.datasource_tmp d, - ${stats_db_name}.result_datasources rd - WHERE d.id = rd.datasource); -- /*EOS*/ - --- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables -UPDATE ${stats_db_name}.project_tmp -SET haspubs='yes' -WHERE project_tmp.id IN (SELECT pr.id - FROM ${stats_db_name}.project_results pr, - ${stats_db_name}.result r - WHERE pr.result = r.id - AND r.type = 'publication'); -- /*EOS*/ - -DROP TABLE IF EXISTS ${stats_db_name}.project purge; -- /*EOS*/ - -CREATE TABLE ${stats_db_name}.project stored as parquet as -SELECT p.id, - p.acronym, - p.title, - p.funder, - p.funding_lvl0, - p.funding_lvl1, - p.funding_lvl2, - p.ec39, - p.type, - p.startdate, - p.enddate, - p.start_year, - p.end_year, - p.duration, - CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END AS haspubs, - CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END AS numpubs, - CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END AS daysforlastpub, - CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END AS delayedpubs, - p.callidentifier, - p.code, - p.totalcost, - p.fundedamount, - p.currency -FROM ${stats_db_name}.project_tmp p - LEFT JOIN (SELECT pr.id, count(distinct pr.result) AS np - FROM ${stats_db_name}.project_results pr - INNER JOIN ${stats_db_name}.result r ON pr.result = r.id - WHERE r.type = 'publication' - GROUP BY pr.id) AS prr1 on prr1.id = p.id - LEFT JOIN (SELECT pp.id, - max(datediff(to_date(r.date), to_date(pp.enddate))) AS daysForlastPub, - count(distinct r.id) AS dp - FROM ${stats_db_name}.project_tmp pp, - ${stats_db_name}.project_results pr, - ${stats_db_name}.result r - WHERE pp.id = pr.id - AND pr.result = r.id - AND r.type = 'publication' - AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 - GROUP BY pp.id) AS prr2 - ON prr2.id = p.id; -- /*EOS*/ - -UPDATE ${stats_db_name}.publication_tmp -SET delayed = 'yes' -WHERE publication_tmp.id IN (SELECT distinct r.id - FROM ${stats_db_name}.result r, - ${stats_db_name}.project_results pr, - ${stats_db_name}.project_tmp p - WHERE r.id = pr.result - AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ - -UPDATE ${stats_db_name}.dataset_tmp -SET delayed = 'yes' -WHERE dataset_tmp.id IN (SELECT distinct r.id - FROM ${stats_db_name}.result r, - ${stats_db_name}.project_results pr, - ${stats_db_name}.project_tmp p - WHERE r.id = pr.result - AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ - -UPDATE ${stats_db_name}.software_tmp -SET delayed = 'yes' -WHERE software_tmp.id IN (SELECT distinct r.id - FROM ${stats_db_name}.result r, - ${stats_db_name}.project_results pr, - ${stats_db_name}.project_tmp p - WHERE r.id = pr.result - AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ - -UPDATE ${stats_db_name}.otherresearchproduct_tmp -SET delayed = 'yes' -WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id - FROM ${stats_db_name}.result r, - ${stats_db_name}.project_results pr, - ${stats_db_name}.project_tmp p - WHERE r.id = pr.result - AND pr.id = p.id - AND to_date(r.date) - to_date(p.enddate) > 0); -- /*EOS*/ - CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql index ff95524be..4f0b45fed 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql @@ -1,42 +1,4 @@ ------------------------------------------------------------------------------------------------------- --- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------- -DROP TABLE IF EXISTS ${stats_db_name}.datasource purge; /*EOS*/ - -CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS -SELECT * -FROM ${stats_db_name}.datasource_tmp; /*EOS*/ - -DROP TABLE IF EXISTS ${stats_db_name}.publication purge; /*EOS*/ - -CREATE TABLE ${stats_db_name}.publication stored AS parquet AS -SELECT * -FROM ${stats_db_name}.publication_tmp; /*EOS*/ - -DROP TABLE IF EXISTS ${stats_db_name}.dataset purge; /*EOS*/ - -CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS -SELECT * -FROM ${stats_db_name}.dataset_tmp; /*EOS*/ - -DROP TABLE IF EXISTS ${stats_db_name}.software purge; /*EOS*/ - -CREATE TABLE ${stats_db_name}.software stored AS parquet AS -SELECT * -FROM ${stats_db_name}.software_tmp; /*EOS*/ - -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct purge; /*EOS*/ - -CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS -SELECT * -FROM ${stats_db_name}.otherresearchproduct_tmp; /*EOS*/ - -DROP TABLE ${stats_db_name}.project_tmp; /*EOS*/ -DROP TABLE ${stats_db_name}.datasource_tmp; /*EOS*/ -DROP TABLE ${stats_db_name}.publication_tmp; /*EOS*/ -DROP TABLE ${stats_db_name}.dataset_tmp; /*EOS*/ -DROP TABLE ${stats_db_name}.software_tmp; /*EOS*/ -DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; /*EOS*/ +set mapred.job.queue.name=analytics; /*EOS*/ ---------------------------------------------- -- Re-creating views from final parquet tables diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql index 416298e4c..a2be22603 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql @@ -1,58 +1,26 @@ set mapred.job.queue.name=analytics; /*EOS*/ --- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, +-- replace the creation of the result view with a table, which will include the boolean fields from the previous tables (green, gold, -- peer reviewed) -drop table if exists ${stats_db_name}.result_tmp; /*EOS*/ -CREATE TABLE ${stats_db_name}.result_tmp ( - id STRING, - title STRING, - publisher STRING, - journal STRING, - `date` STRING, - `year` INT, - bestlicence STRING, - access_mode STRING, - embargo_end_date STRING, - delayed BOOLEAN, - authors INT, - source STRING, - abstract BOOLEAN, - type STRING , - peer_reviewed BOOLEAN, - green BOOLEAN, - gold BOOLEAN) -clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); /*EOS*/ - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.publication r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.dataset r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.software r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.otherresearchproduct r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ - -drop table if exists ${stats_db_name}.result; /*EOS*/ drop view if exists ${stats_db_name}.result; /*EOS*/ -create table ${stats_db_name}.result stored as parquet as select * from ${stats_db_name}.result_tmp; /*EOS*/ -drop table ${stats_db_name}.result_tmp; /*EOS*/ \ No newline at end of file +drop table if exists ${stats_db_name}.result; /*EOS*/ + +CREATE TABLE ${stats_db_name}.result stored as parquet as +SELECT /*+ COALESCE(100) */ r.id, r.title, r.publisher, r.journal, r.`date`, DATE_FORMAT(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ( + (SELECT id, title, p.publisher, journal, `date`, DATE_FORMAT(`date`, 'yyyy'), bestlicence, bestlicence, embargo_end_date, delayed, authors, source, abstract, type + FROM ${stats_db_name}.publication) + UNION ALL + (SELECT id, title, p.publisher, journal, `date`, DATE_FORMAT(`date`, 'yyyy'), bestlicence, bestlicence, embargo_end_date, delayed, authors, source, abstract, type + FROM ${stats_db_name}.dataset) + UNION ALL + (select id, title, p.publisher, journal, `date`, DATE_FORMAT(`date`, 'yyyy'), bestlicence, bestlicence, embargo_end_date, delayed, authors, source, abstract, type + FROM ${stats_db_name}.software) + UNION ALL + (select id, title, p.publisher, journal, `date`, DATE_FORMAT(`date`, 'yyyy'), bestlicence, bestlicence, embargo_end_date, delayed, authors, source, abstract, type + FROM ${stats_db_name}.otherresearchproduct) + ) r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index f0e5ce091..0abec2358 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -7,41 +7,41 @@ set mapred.job.queue.name=analytics; /*EOS*/ -------------------------------------------------------------- -- Publication temporary table -DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp purge; /*EOS*/ -CREATE TABLE ${stats_db_name}.publication_tmp -( - id STRING, - title STRING, - publisher STRING, - journal STRING, - date STRING, - year STRING, - bestlicence STRING, - embargo_end_date STRING, - delayed BOOLEAN, - authors INT, - source STRING, - abstract BOOLEAN, - type STRING -) - clustered by (id) into 100 buckets stored as orc tblproperties ('transactional' = 'true'); /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.publication purge; /*EOS*/ + +CREATE TABLE ${stats_db_name}.publication stored as parquet as +with pub_pr as ( + select pub.id as pub_id, case when (to_date(pub.dateofacceptance.value) > to_date( pj.enddate.value)) then true else false end as delayed + from ${openaire_db_name}.publication pub + join ${openaire_db_name}.relation rel + on reltype = 'resultProject' and relclass = 'isProducedBy' and rel.source=pub.id + and rel.datainfo.deletedbyinference = false and rel.datainfo.invisible = false + join ${openaire_db_name}.project pj on pj.id=rel.target and pj.datainfo.deletedbyinference = false and pj.datainfo.invisible = false + where pub.datainfo.deletedbyinference = false and pub.datainfo.invisible = false +), + pub_delayed as ( + select pub_id, max(delayed) as delayed + from pub_pr + group by pub_id + ) +select /*+ COALESCE(100) */ + substr(pub.id, 4) as id, + pub.title[0].value as title, + pub.publisher.value as publisher, + pub.journal.name as journal, + pub.dateofacceptance.value as date, + date_format(pub.dateofacceptance.value, 'yyyy') as year, + pub.bestaccessright.classname as bestlicence, + pub.embargoenddate.value as embargo_end_date, + coalesce(pub_delayed.delayed, false) as delayed, -- It's delayed, when the publication was published after the end of at least one of its projects. + size(pub.author) as authors, + concat_ws('\u003B', pub.source.value) as source, + case when size(pub.description) > 0 then true else false end as abstract, + 'publication' as type +from ${openaire_db_name}.publication pub + left outer join pub_delayed on pub.id=pub_delayed.pub_id +where pub.datainfo.deletedbyinference = false and pub.datainfo.invisible = false; /*EOS*/ -INSERT INTO ${stats_db_name}.publication_tmp -SELECT substr(p.id, 4) as id, - p.title[0].value as title, - p.publisher.value as publisher, - p.journal.name as journal, - p.dateofacceptance.value as date, - date_format(p.dateofacceptance.value, 'yyyy') as year, - p.bestaccessright.classname as bestlicence, - p.embargoenddate.value as embargo_end_date, - false as delayed, - size(p.author) as authors, - concat_ws('\u003B', p.source.value) as source, - case when size(p.description) > 0 then true else false end as abstract, - 'publication' as type -from ${openaire_db_name}.publication p -where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.publication_classifications purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index ebedb5dc5..8f203fc83 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -5,42 +5,41 @@ ------------------------------------------------------ -- Dataset temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.dataset purge; /*EOS*/ -CREATE TABLE ${stats_db_name}.dataset_tmp -( - id STRING, - title STRING, - publisher STRING, - journal STRING, - date STRING, - year STRING, - bestlicence STRING, - embargo_end_date STRING, - delayed BOOLEAN, - authors INT, - source STRING, - abstract BOOLEAN, - type STRING +CREATE TABLE ${stats_db_name}.dataset stored as parquet as +with datast_pr as ( + select datast.id as datast_id, case when (to_date(datast.dateofacceptance.value) > to_date( pj.enddate.value)) then true else false end as delayed + from ${openaire_db_name}.dataset datast + join ${openaire_db_name}.relation rel + on reltype = 'resultProject' and relclass = 'isProducedBy' and rel.source=datast.id + and rel.datainfo.deletedbyinference = false and rel.datainfo.invisible = false + join ${openaire_db_name}.project pj on pj.id=rel.target and pj.datainfo.deletedbyinference = false and pj.datainfo.invisible = false + where datast.datainfo.deletedbyinference = false and datast.datainfo.invisible = false +), +datast_delayed as ( + select datast_id, max(delayed) as delayed + from datast_pr + group by datast_id ) - clustered by (id) into 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ +select /*+ COALESCE(100) */ + substr(datast.id, 4) as id, + datast.title[0].value as title, + datast.publisher.value as publisher, + cast(null as string) as journal, + datast.dateofacceptance.value as date, + date_format(datast.dateofacceptance.value, 'yyyy') as year, + datast.bestaccessright.classname as bestlicence, + datast.embargoenddate.value as embargo_end_date, + coalesce(datast_delayed.delayed, false) as delayed, -- It's delayed, when the dataset was published after the end of the project. + size(datast.author) as authors, + concat_ws('\u003B', datast.source.value) as source, + case when size(datast.description) > 0 then true else false end as abstract, + 'dataset' as type +from ${openaire_db_name}.dataset datast + left outer join datast_delayed on datast.id=datast_delayed.datast_id +where datast.datainfo.deletedbyinference = false and datast.datainfo.invisible = false; /*EOS*/ -INSERT INTO ${stats_db_name}.dataset_tmp -SELECT substr(d.id, 4) AS id, - d.title[0].value AS title, - d.publisher.value AS publisher, - cast(null AS string) AS journal, - d.dateofacceptance.value as date, - date_format(d.dateofacceptance.value, 'yyyy') AS year, - d.bestaccessright.classname AS bestlicence, - d.embargoenddate.value AS embargo_end_date, - false AS delayed, - size(d.author) AS authors, - concat_ws('\u003B', d.source.value) AS source, - CASE WHEN SIZE(d.description) > 0 THEN TRUE ELSE FALSE end AS abstract, - 'dataset' AS type -FROM ${openaire_db_name}.dataset d -WHERE d.datainfo.deletedbyinference = FALSE and d.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.dataset_citations purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index 4957d8d2f..5e4c9f420 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -5,41 +5,41 @@ -------------------------------------------------------- -- Software temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.software_tmp purge; /*EOS*/ -CREATE TABLE ${stats_db_name}.software_tmp -( - id STRING, - title STRING, - publisher STRING, - journal STRING, - date STRING, - year STRING, - bestlicence STRING, - embargo_end_date STRING, - delayed BOOLEAN, - authors INT, - source STRING, - abstract BOOLEAN, - type STRING -) - clustered by (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.software purge; /*EOS*/ + +CREATE TABLE ${stats_db_name}.software stored as parquet as +with soft_pr as ( + select soft.id as soft_id, case when (to_date(soft.dateofacceptance.value) > to_date( pj.enddate.value)) then true else false end as delayed + from ${openaire_db_name}.software soft + join ${openaire_db_name}.relation rel + on reltype = 'resultProject' and relclass = 'isProducedBy' and rel.source=soft.id + and rel.datainfo.deletedbyinference = false and rel.datainfo.invisible = false + join ${openaire_db_name}.project pj on pj.id=rel.target and pj.datainfo.deletedbyinference = false and pj.datainfo.invisible = false + where soft.datainfo.deletedbyinference = false and soft.datainfo.invisible = false +), +soft_delayed as ( + select soft_id, max(delayed) as delayed + from soft_pr + group by soft_id +) +select /*+ COALESCE(100) */ + substr(soft.id, 4) as id, + soft.title[0].value as title, + soft.publisher.value as publisher, + cast(null as string) as journal, + soft.dateofacceptance.value as date, + date_format(soft.dateofacceptance.value, 'yyyy') as year, + soft.bestaccessright.classname as bestlicence, + soft.embargoenddate.value as embargo_end_date, + coalesce(soft_delayed.delayed, false) as delayed, -- It's delayed, when the software was published after the end of the project. + size(soft.author) as authors, + concat_ws('\u003B', soft.source.value) as source, + case when size(soft.description) > 0 then true else false end as abstract, + 'software' as type +from ${openaire_db_name}.software soft + left outer join soft_delayed on soft.id=soft_delayed.soft_id +where soft.datainfo.deletedbyinference = false and soft.datainfo.invisible = false; /*EOS*/ -INSERT INTO ${stats_db_name}.software_tmp -SELECT substr(s.id, 4) as id, - s.title[0].value AS title, - s.publisher.value AS publisher, - CAST(NULL AS string) AS journal, - s.dateofacceptance.value AS DATE, - date_format(s.dateofacceptance.value, 'yyyy') AS YEAR, - s.bestaccessright.classname AS bestlicence, - s.embargoenddate.value AS embargo_end_date, - FALSE AS delayed, - SIZE(s.author) AS authors, - concat_ws('\u003B', s.source.value) AS source, - CASE WHEN SIZE(s.description) > 0 THEN TRUE ELSE FALSE END AS abstract, - 'software' as type -from ${openaire_db_name}.software s -where s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.software_citations purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index 820ec4395..3134ba316 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -5,41 +5,41 @@ -------------------------------------------------------------------------------- -- Otherresearchproduct temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct purge; /*EOS*/ -CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp -( - id STRING, - title STRING, - publisher STRING, - journal STRING, - date STRING, - year STRING, - bestlicence STRING, - embargo_end_date STRING, - delayed BOOLEAN, - authors INT, - source STRING, - abstract BOOLEAN, - type STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ +CREATE TABLE ${stats_db_name}.otherresearchproduct stored as parquet as +with other_pr as ( + select other.id as other_id, case when (to_date(other.dateofacceptance.value) > to_date( pj.enddate.value)) then true else false end as delayed + from ${openaire_db_name}.otherresearchproduct other + join ${openaire_db_name}.relation rel + on reltype = 'resultProject' and relclass = 'isProducedBy' and rel.source=other.id + and rel.datainfo.deletedbyinference = false and rel.datainfo.invisible = false + join ${openaire_db_name}.project pj on pj.id=rel.target and pj.datainfo.deletedbyinference = false and pj.datainfo.invisible = false + where other.datainfo.deletedbyinference = false and other.datainfo.invisible = false +), +other_delayed as ( + select other_id, max(delayed) as delayed + from other_pr + group by other_id +) +select /*+ COALESCE(100) */ + substr(other.id, 4) as id, + other.title[0].value as title, + other.publisher.value as publisher, + cast(null as string) as journal, + other.dateofacceptance.value as date, + date_format(other.dateofacceptance.value, 'yyyy') as year, + other.bestaccessright.classname as bestlicence, + other.embargoenddate.value as embargo_end_date, + false as delayed, + size(other.author) as authors, + concat_ws('\u003B', other.source.value) as source, + case when size(other.description) > 0 then true else false end as abstract, + 'other' as type +from ${openaire_db_name}.otherresearchproduct other + left outer join other_delayed on other.id=other_delayed.other_id +where other.datainfo.deletedbyinference = false and other.datainfo.invisible = false; /*EOS*/ -INSERT INTO ${stats_db_name}.otherresearchproduct_tmp -SELECT substr(o.id, 4) AS id, - o.title[0].value AS title, - o.publisher.value AS publisher, - CAST(NULL AS string) AS journal, - o.dateofacceptance.value AS DATE, - date_format(o.dateofacceptance.value, 'yyyy') AS year, - o.bestaccessright.classname AS bestlicence, - o.embargoenddate.value as embargo_end_date, - FALSE AS delayed, - SIZE(o.author) AS authors, - concat_ws('\u003B', o.source.value) AS source, - CASE WHEN SIZE(o.description) > 0 THEN TRUE ELSE FALSE END AS abstract, - 'other' AS type -FROM ${openaire_db_name}.otherresearchproduct o -WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible=false; /*EOS*/ -- Otherresearchproduct_citations DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_citations purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index d2688ec07..de6b803ab 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -34,61 +34,69 @@ from ${openaire_db_name}.project p lateral view explode(p.h2020classification) classifs as class where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false and class.h2020programme is not null; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.project_tmp purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.project purge; /*EOS*/ -CREATE TABLE ${stats_db_name}.project_tmp -( - id STRING, - acronym STRING, - title STRING, - funder STRING, - funding_lvl0 STRING, - funding_lvl1 STRING, - funding_lvl2 STRING, - ec39 STRING, - type STRING, - startdate STRING, - enddate STRING, - start_year INT, - end_year INT, - duration INT, - haspubs STRING, - numpubs INT, - daysforlastpub INT, - delayedpubs INT, - callidentifier STRING, - code STRING, - totalcost FLOAT, - fundedamount FLOAT, - currency STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); /*EOS*/ +CREATE TABLE ${stats_db_name}.project stored as parquet as +with pr_pub as ( + select pr.id as pr_id, pub.id as pub_id, + (case when datediff(pub.dt_dateofacceptance, pr.dt_enddate) > 0 then true else false end) as delayed, + max(datediff(pub.dt_dateofacceptance, pr.dt_enddate)) as daysForlastPub + from (select id, to_date(dateofacceptance.value) as dt_dateofacceptance from ${openaire_db_name}.publication + where datainfo.deletedbyinference = false and datainfo.invisible = false) pub + join ${openaire_db_name}.relation rel + on rel.reltype = 'resultProject' and rel.relclass = 'isProducedBy' and rel.source=pub.id + and rel.datainfo.deletedbyinference = false and rel.datainfo.invisible = false + join (select id, to_date(enddate.value) as dt_enddate from ${openaire_db_name}.project + where datainfo.deletedbyinference = false and datainfo.invisible = false) pr + on pr.id=rel.target + group by pr.id, pub.id, pub.dt_dateofacceptance, pr.dt_enddate +), +num_pubs_pr as ( + select pr_id, count( distinct pub_id) as num_pubs + from pr_pub + group by pr_id +), +pub_delayed as ( + select pr_id, pub_id, max(delayed) as delayed + from pr_pub + group by pr_id, pub_id +), +num_pub_delayed as ( + select pr_id, count(distinct pub_id) as num_delayed + from pub_delayed + where delayed + group by pr_id +) +select /*+ COALESCE(100) */ + substr(p.id, 4) as id, + p.acronym.value as acronym, + p.title.value as title, + xpath_string(p.fundingtree[0].value, '//funder/name') as funder, + xpath_string(p.fundingtree[0].value, '//funding_level_0/name') as funding_lvl0, + xpath_string(p.fundingtree[0].value, '//funding_level_1/name') as funding_lvl1, + xpath_string(p.fundingtree[0].value, '//funding_level_2/name') as funding_lvl2, + p.ecsc39.value as ec39, + p.contracttype.classname as type, + p.startdate.value as startdate, + p.enddate.value as enddate, + year(p.startdate.value) as start_year, + year(p.enddate.value) as end_year, + cast(months_between(p.enddate.value, p.startdate.value) as int) as duration, + case when pr_pub.pub_id is null then 'no' else 'yes' end as haspubs, + num_pubs_pr.num_pubs as numpubs, + pr_pub.daysForlastPub as daysForlastPub, + npd.num_delayed as delayedpubs, + p.callidentifier.value as callidentifier, + p.code.value as code, + p.totalcost as totalcost, + p.fundedamount as fundedamount, + p.currency.value as currency +from ${openaire_db_name}.project p +left outer join pr_pub on pr_pub.pr_id = p.id +left outer join num_pubs_pr on num_pubs_pr.pr_id = p.id +left outer join num_pub_delayed npd on npd.pr_id=p.id +where p.datainfo.deletedbyinference = false and p.datainfo.invisible = false; /*EOS*/ -INSERT INTO ${stats_db_name}.project_tmp -SELECT substr(p.id, 4) AS id, - p.acronym.value AS acronym, - p.title.value AS title, - xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, - xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, - xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, - xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, - p.ecsc39.value AS ec39, - p.contracttype.classname AS type, - p.startdate.value AS startdate, - p.enddate.value AS enddate, - year(p.startdate.value) AS start_year, - year(p.enddate.value) AS end_year, - CAST(MONTHS_BETWEEN(p.enddate.value, p.startdate.value) AS INT) AS duration, - 'no' AS haspubs, - 0 AS numpubs, - 0 AS daysforlastpub, - 0 AS delayedpubs, - p.callidentifier.value AS callidentifier, - p.code.value AS code, - p.totalcost AS totalcost, - p.fundedamount AS fundedamount, - p.currency.value AS currency -FROM ${openaire_db_name}.project p -WHERE p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.funder purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 7a23991fe..2283e6291 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -7,16 +7,16 @@ -- Views on temporary tables that should be re-created in the end CREATE OR REPLACE VIEW ${stats_db_name}.result as SELECT *, bestlicence AS access_mode -FROM ${stats_db_name}.publication_tmp +FROM ${stats_db_name}.publication UNION ALL SELECT *, bestlicence AS access_mode -FROM ${stats_db_name}.software_tmp +FROM ${stats_db_name}.software UNION ALL SELECT *, bestlicence AS access_mode -FROM ${stats_db_name}.dataset_tmp +FROM ${stats_db_name}.dataset UNION ALL SELECT *, bestlicence AS access_mode -FROM ${stats_db_name}.otherresearchproduct_tmp; /*EOS*/ +FROM ${stats_db_name}.otherresearchproduct; /*EOS*/ -- Views on final tables CREATE OR REPLACE VIEW ${stats_db_name}.result_datasources AS @@ -153,4 +153,4 @@ CREATE TABLE ${stats_db_name}.result_projects STORED AS PARQUET AS select /*+ COALESCE(100) */ pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id = pr.result - JOIN ${stats_db_name}.project_tmp p ON p.id = pr.id; /*EOS*/ \ No newline at end of file + JOIN ${stats_db_name}.project p ON p.id = pr.id; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 90c3ebef6..b280dc946 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -5,81 +5,36 @@ -- Datasource table/view and Datasource related tables/views ------------------------------------------------------------ ------------------------------------------------------------ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge; -- /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource purge; /*EOS*/ -CREATE TABLE ${stats_db_name}.datasource_tmp -( - `id` string, - `name` STRING, - `type` STRING, - `dateofvalidation` STRING, - `yearofvalidation` string, - `harvested` BOOLEAN, - `piwik_id` INT, - `latitude` STRING, - `longitude` STRING, - `websiteurl` STRING, - `compatibility` STRING, - issn_printed STRING, - issn_online STRING -) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); -- /*EOS*/ +CREATE TABLE ${stats_db_name}.datasource stored as parquet as +with piwik_datasource as ( + select id, split(originalidd, '\\:')[1] as piwik_id + from ${openaire_db_name}.datasource + lateral view explode(originalid) temp as originalidd + where originalidd like "piwik:%" +) +select /*+ COALESCE(100) */ + substr(dtrce.id, 4) as id, + case when dtrce.officialname.value='Unknown Repository' then 'Other' else dtrce.officialname.value end as name, + dtrce.datasourcetype.classname as type, + dtrce.dateofvalidation.value as dateofvalidation, + case when dtrce.dateofvalidation.value='-1' then null else date_format(dtrce.dateofvalidation.value, 'yyyy') end as yearofvalidation, + case when res.d_id is null then false else true end as harvested, + case when piwik_d.piwik_id is null then 0 else piwik_d.piwik_id end as piwik_id, + dtrce.latitude.value as latitude, + dtrce.longitude.value as longitude, + dtrce.websiteurl.value as websiteurl, + dtrce.openairecompatibility.classid as compatibility, + dtrce.journal.issnprinted as issn_printed, + dtrce.journal.issnonline as issn_online +from ${openaire_db_name}.datasource dtrce + left outer join (select inst.hostedby.key as d_id from ${openaire_db_name}.result lateral view outer explode (instance) insts as inst) res on res.d_id=dtrce.id + left outer join piwik_datasource piwik_d on piwik_d.id=dtrce.id +where dtrce.datainfo.deletedbyinference = false and dtrce.datainfo.invisible = false; /*EOS*/ --- Insert statement that takes into account the piwik_id of the openAIRE graph -INSERT INTO ${stats_db_name}.datasource_tmp -SELECT substr(d1.id, 4) AS id, - officialname.value AS name, - datasourcetype.classname AS type, - dateofvalidation.value AS dateofvalidation, - date_format(d1.dateofvalidation.value, 'yyyy') AS yearofvalidation, - FALSE AS harvested, - CASE WHEN d2.piwik_id IS NULL THEN 0 ELSE d2.piwik_id END AS piwik_id, - d1.latitude.value AS latitude, - d1.longitude.value AS longitude, - d1.websiteurl.value AS websiteurl, - d1.openairecompatibility.classid AS compatibility, - d1.journal.issnprinted AS issn_printed, - d1.journal.issnonline AS issn_online -FROM ${openaire_db_name}.datasource d1 - LEFT OUTER JOIN - (SELECT id, split(originalidd, '\\:')[1] as piwik_id - FROM ${openaire_db_name}.datasource - LATERAL VIEW EXPLODE(originalid) temp AS originalidd - WHERE originalidd like "piwik:%") AS d2 - ON d1.id = d2.id -WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; -- /*EOS*/ --- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. --- Creating a temporary dual table that will be removed after the following insert - -DROP TABLE IF EXISTS ${stats_db_name}.dual purge; -- /*EOS*/ - -CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); -- /*EOS*/ - -INSERT INTO ${stats_db_name}.dual VALUES ('X'); -- /*EOS*/ - -INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, - `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`, `issn_printed`, `issn_online`) -SELECT 'other', - 'Other', - 'Repository', - NULL, - NULL, - false, - 0, - NULL, - NULL, - NULL, - 'unknown', - null, - null -FROM ${stats_db_name}.dual -WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name = 'Unknown Repository'); -- /*EOS*/ -DROP TABLE ${stats_db_name}.dual; -- /*EOS*/ - -UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository'; -- /*EOS*/ -UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1'; -- /*EOS*/ - -DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; -- /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, langs.languages AS language From bb12d0b4df50919fea19d5a4d25dcb39718fdfb1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 3 Jul 2024 16:26:39 +0200 Subject: [PATCH 051/239] removed legacy actionmanager dependencies --- dhp-workflows/dhp-actionmanager/pom.xml | 4 +- .../dnetlib/dhp/actionmanager/ISClient.java | 29 +- dhp-workflows/dhp-swh/pom.xml | 43 - pom.xml | 2029 ++++++++--------- 4 files changed, 937 insertions(+), 1168 deletions(-) diff --git a/dhp-workflows/dhp-actionmanager/pom.xml b/dhp-workflows/dhp-actionmanager/pom.xml index ce13502b6..1ef55a060 100644 --- a/dhp-workflows/dhp-actionmanager/pom.xml +++ b/dhp-workflows/dhp-actionmanager/pom.xml @@ -50,11 +50,12 @@ org.apache.hadoop hadoop-distcp - + 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 088e618c7..24d50029c 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 @@ -4,7 +4,6 @@ package eu.dnetlib.dhp.actionmanager; import java.io.Serializable; import java.io.StringReader; import java.util.List; -import java.util.NoSuchElementException; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -22,13 +21,14 @@ import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import eu.dnetlib.actionmanager.rmi.ActionManagerException; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; public class ISClient implements Serializable { + private static final long serialVersionUID = 4632443200867340872L; + private static final Logger log = LoggerFactory.getLogger(ISClient.class); private static final String INPUT_ACTION_SET_ID_SEPARATOR = ","; @@ -65,7 +65,7 @@ public class ISClient implements Serializable { .map(t -> buildDirectory(basePath, t)) .collect(Collectors.toList())) .orElseThrow(() -> new IllegalStateException("empty set list")); - } catch (ActionManagerException | ISLookUpException e) { + } catch (ISLookUpException e) { throw new IllegalStateException("unable to query ActionSets info from the IS"); } } @@ -89,31 +89,18 @@ public class ISClient implements Serializable { return Joiner.on("/").join(basePath, t.getMiddle(), t.getRight()); } - private String getBasePathHDFS(ISLookUpService isLookup) throws ActionManagerException { + private String getBasePathHDFS(ISLookUpService isLookup) throws ISLookUpException { return queryServiceProperty(isLookup, "basePath"); } private String queryServiceProperty(ISLookUpService isLookup, final String propertyName) - throws ActionManagerException { + throws ISLookUpException { 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); - } + + final List value = isLookup.quickSearchProfile(q); + return Iterables.getOnlyElement(value); } } diff --git a/dhp-workflows/dhp-swh/pom.xml b/dhp-workflows/dhp-swh/pom.xml index 80fff4587..ef9049b20 100644 --- a/dhp-workflows/dhp-swh/pom.xml +++ b/dhp-workflows/dhp-swh/pom.xml @@ -51,49 +51,6 @@ hadoop-distcp - - eu.dnetlib - dnet-actionmanager-api - - - eu.dnetlib - dnet-actionmanager-common - - - eu.dnetlib - dnet-openaireplus-mapping-utils - - - saxonica - saxon - - - saxonica - saxon-dom - - - jgrapht - jgrapht - - - net.sf.ehcache - ehcache - - - org.springframework - spring-test - - - org.apache.* - * - - - apache - * - - - - org.apache.httpcomponents httpclient diff --git a/pom.xml b/pom.xml index a5bc6cedd..47b7d5f38 100644 --- a/pom.xml +++ b/pom.xml @@ -1,1107 +1,930 @@ - 4.0.0 - eu.dnetlib.dhp - dhp - 1.2.5-SNAPSHOT - pom - - - - GNU Affero General Public License v3.0 or later - https://spdx.org/licenses/AGPL-3.0-or-later.html#licenseText - repo - This program is free software: you can redistribute it and/or modify it under the terms of the - GNU Affero General Public License as published by the Free Software Foundation, either version 3 of the - License, or (at your option) any later version. - - - - - - dhp-build - dhp-pace-core - dhp-common - dhp-workflows - dhp-shade-package - - - - Redmine - https://support.openaire.eu/projects/openaire - - - - jenkins - https://jenkins-dnet.d4science.org/ - - - - scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git - scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git - https://code-repo.d4science.org/D-Net/dnet-hadoop/ - HEAD - - - This module is the root descriptor for the dnet-hadoop project - - - - - - - - Openaire-third-parties-snaphot - Openaire third parties Snapshot - https://maven.d4science.org/nexus/content/repositories/Openaire-third-parties-snaphot/ - - false - - - true - - - - - dnet45-releases - D-Net 45 releases - https://maven.d4science.org/nexus/content/repositories/dnet45-releases - default - - false - - - true - - - - dnet45-snapshots - D-Net 45 snapshots - https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots - default - - true - - - false - - - - dnet45-bootstrap-snapshot - D-Net 45 Bootstrap Snapshot - https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-snapshot/ - - false - - - true - - default - - - dnet45-bootstrap-release - D-Net 45 Bootstrap Release - https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-release/ - - true - - - false - - default - - - cloudera - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos - - true - - - false - - - - dnet-deps - dnet-dependencies - https://maven.d4science.org/nexus/content/repositories/dnet-deps - default - - - maven-restlet - Restlet repository - https://maven.restlet.talend.com - - - conjars - conjars - https://conjars.wensel.net/repo/ - - - - - - - org.projectlombok - lombok - 1.18.28 - provided - - - org.junit.jupiter - junit-jupiter - ${junit-jupiter.version} - test - - - - org.mockito - mockito-core - ${mockito-core.version} - test - - - - org.mockito - mockito-junit-jupiter - ${mockito-core.version} - test - - - - - - - - eu.dnetlib.dhp - dhp-schemas - ${dhp-schemas.version} - - - org.apache.hadoop - hadoop-hdfs - ${dhp.hadoop.version} - provided - - - org.apache.hadoop - hadoop-common - ${dhp.hadoop.version} - provided - - - org.apache.hadoop - hadoop-client - ${dhp.hadoop.version} - provided - - - org.apache.hadoop - hadoop-distcp - ${dhp.hadoop.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${dhp.spark.version} - provided - - - org.apache.spark - spark-sql_${scala.binary.version} - ${dhp.spark.version} - provided - - - org.apache.spark - spark-graphx_${scala.binary.version} - ${dhp.spark.version} - provided - - - org.apache.spark - spark-hive_${scala.binary.version} - ${dhp.spark.version} - test - - - - org.slf4j - slf4j-api - ${org.slf4j.version} - provided - - - - org.slf4j - slf4j-log4j12 - ${org.slf4j.version} - provided - - - - org.slf4j - jcl-over-slf4j - ${org.slf4j.version} - provided - - - - org.apache.logging.log4j - log4j-slf4j2-impl - ${log4j.version} - - - org.apache.logging.log4j - log4j-api - ${log4j.version} - - - org.apache.logging.log4j - log4j-core - ${log4j.version} - - - - org.apache.logging.log4j - log4j-1.2-api - ${log4j.version} - - - - org.apache.commons - commons-lang3 - ${dhp.commons.lang.version} - - - - org.apache.commons - commons-beanutils - ${commons-beanutils.version} - - - - - commons-validator - commons-validator - ${commons-validator.version} - - - - com.github.sisyphsu - dateparser - ${dateparser.version} - - - - me.xuender - unidecode - ${unidecode.version} - - - - com.google.guava - guava - ${dhp.guava.version} - - - - - commons-codec - commons-codec - ${commons-codec.version} - - - - commons-io - commons-io - ${commons-io.version} - - - - commons-cli - commons-cli - 1.2 - provided - - - - net.sf.saxon - Saxon-HE - 9.9.1-6 - - - - dom4j - dom4j - 1.6.1 - - - - xml-apis - xml-apis - 1.4.01 - - - - jaxen - jaxen - 1.1.6 - - - - com.mycila.xmltool - xmltool - 3.3 - - - - org.apache.solr - solr-solrj - ${solr.version} - - - * - * - - - - - com.lucidworks.spark - spark-solr - ${sparksolr.version} - - - * - * - - - - - org.apache.solr - solr-test-framework - ${solr.version} - test - - - io.dropwizard.metrics - metrics-core - 3.2.6 - test - - - - - org.apache.httpcomponents - httpclient - ${org.apache.httpcomponents.version} - - - org.apache.httpcomponents - httpmime - ${org.apache.httpcomponents.version} - - - org.noggit - noggit - 0.8 - - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - - - - net.schmizz - sshj - 0.10.0 - test - - - - com.fasterxml.jackson.core - jackson-core - ${dhp.jackson.version} - provided - - - - com.fasterxml.jackson.core - jackson-annotations - ${dhp.jackson.version} - provided - - - com.fasterxml.jackson.core - jackson-databind - ${dhp.jackson.version} - provided - - - - eu.dnetlib - dnet-actionmanager-common - ${dnet-actionmanager-common.version} - - - org.apache.hadoop - hadoop-common - - - - - eu.dnetlib - dnet-actionmanager-api - ${dnet-actionmanager-api.version} - - - eu.dnetlib - cnr-misc-utils - - - - - - eu.dnetlib - cnr-rmi-api - ${cnr-rmi-api.version} - - - - eu.dnetlib.dhp - dnet-openaire-broker-common - ${dnet-openaire-broker-common.version} - - - - org.apache.cxf - cxf-rt-transports-http - 3.1.5 - - - - javax.persistence - javax.persistence-api - 2.2 - provided - - - - com.jayway.jsonpath - json-path - 2.4.0 - - - com.arakelian - java-jq - 0.10.1 - - - edu.cmu - secondstring - 1.0.0 - - - org.mongodb - mongo-java-driver - ${mongodb.driver.version} - - - io.fares.junit.mongodb - mongodb-junit-test - 1.1.0 - - - org.postgresql - postgresql - 42.2.10 - - - - org.antlr - stringtemplate - 3.2.1 - - - - org.antlr - ST4 - 4.3.4 - - - - com.ximpleware - vtd-xml - ${vtd.version} - - - - org.elasticsearch - elasticsearch-hadoop - 7.6.0 - - - - - org.apache.oozie - oozie-client - ${dhp.oozie.version} - provided - - - - slf4j-simple - org.slf4j - - - - - - - com.squareup.okhttp3 - okhttp - ${okhttp.version} - - - - org.apache.commons - commons-compress - ${common.compress.version} - - - org.apache.commons - commons-csv - ${common.csv.version} - - - org.apache.poi - poi-ooxml - ${apache.poi.version} - - - - org.json - json - 20180813 - - - - org.json4s - json4s-jackson_${scala.binary.version} - ${json4s.version} - - - - com.github.victools - jsonschema-generator - ${jsonschemagenerator.version} - - - - org.apache.commons - commons-text - ${common.text.version} - - - - com.opencsv - opencsv - 5.5 - - - io.github.classgraph - classgraph - 4.8.71 - - - - com.fasterxml.jackson.dataformat - jackson-dataformat-xml - ${jackson.version} - provided - - - com.fasterxml.jackson.module - jackson-module-jsonSchema - ${jackson.version} - provided - - - - org.apache.commons - commons-math3 - 3.6.1 - - - - com.google.code.gson - gson - ${google.gson.version} - - - - commons-collections - commons-collections - ${commons.collections.version} - - - commons-logging - commons-logging - ${commons.logging.version} - - - - org.reflections - reflections - ${reflections.version} - - - - org.scala-lang - scala-library - ${scala.version} - - - - com.ibm.icu - icu4j - 70.1 - - - - org.javassist - javassist - ${javassist.version} - - - - - - target - target/classes - ${project.artifactId}-${project.version} - target/test-classes - - - - org.apache.maven.plugins - maven-plugin-plugin - 3.3 - - - org.apache.maven.plugins - maven-project-info-reports-plugin - 3.0.0 - - - org.apache.maven.plugins - maven-site-plugin - 3.9.1 - - ${dhp.site.skip} - - - - - org.apache.maven.plugins - maven-compiler-plugin - ${maven.compiler.plugin.version} - - 1.8 - 1.8 - ${project.build.sourceEncoding} - - - - - org.apache.maven.plugins - maven-jar-plugin - 3.0.2 - - - - org.apache.maven.plugins - maven-source-plugin - 3.0.1 - - - attach-sources - verify - - jar-no-fork - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 3.0.0-M4 - - true - false - - - - org.apache.maven.plugins - maven-javadoc-plugin - 3.2.0 - - true - none - - - - org.apache.maven.plugins - maven-dependency-plugin - 3.6.0 - - - - net.revelc.code.formatter - formatter-maven-plugin - 2.11.0 - - - eu.dnetlib.dhp - dhp-code-style - ${project.version} - - - - - org.antipathy - mvn-scalafmt_${scala.binary.version} - 1.0.1640073709.733712b - - - eu.dnetlib.dhp - dhp-code-style - ${project.version} - - - - - - - - org.apache.maven.plugins - maven-site-plugin - - - org.apache.maven.plugins - maven-project-info-reports-plugin - - - net.revelc.code.formatter - formatter-maven-plugin - - - - format - - - eclipse/formatter_dnet.xml - - - - - - net.revelc.code - impsort-maven-plugin - 1.6.2 - - java.,javax.,org.,com. - java,* - - **/thrift/*.java - - - - - sort-imports - - sort - - - - - - org.antipathy - mvn-scalafmt_${scala.binary.version} - - - https://code-repo.d4science.org/D-Net/dnet-hadoop/raw/branch/beta/dhp-build/dhp-code-style/src/main/resources/scalafmt/scalafmt.conf - - false - false - - ${project.basedir}/src/main/scala - - - ${project.basedir}/src/test/scala - - false - false - : git rev-parse --abbrev-ref HEAD - false - - - - validate - - format - - - - - - org.apache.maven.plugins - maven-release-plugin - 2.5.3 - - - org.jacoco - jacoco-maven-plugin - 0.8.10 - - - **/schemas/* - **/com/cloudera/**/* - **/org/apache/avro/io/**/* - - - - - default-prepare-agent - - prepare-agent - - - - default-report - prepare-package - - report - - - - - - - - - - org.apache.maven.wagon - wagon-ssh - 2.10 - - - - - - dnet45-snapshots - DNet45 Snapshots - https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots - default - - - dnet45-releases - https://maven.d4science.org/nexus/content/repositories/dnet45-releases - - - DHPSite - ${dhp.site.stage.path}/ - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - true - none - - - - - - - sftp://dnet-hadoop@static-web.d4science.org/dnet-hadoop - UTF-8 - UTF-8 - 1.8 - 1.8 - - - 2.11.12 - 2.11 - - - 3.6.0 - 2.22.2 - 2.0.1 - 4.0.1 - - - 4.1.2 - [2.6.1] - 1.20 - 1.8 - 1.8 - 1.9.4 - 1.9 - 3.2.1 - 2.4 - 1.1.3 - 1.7 - 1.0.7 - [6.1.4-SNAPSHOT] - cdh5.9.2 - 3.5 - 11.0.2 - 2.6.0-${dhp.cdh.version} - 2.9.6 - 4.1.0-${dhp.cdh.version} - true - 2.4.0.cloudera2 - [4.0.3] - [6.0.5] - [3.1.6] - 2.2.2 - 1.2.17 - 3.19.0-GA - 3.5.3 - 4.13.0 - 5.6.1 - 3.3.3 - 3.4.2 - 4.7.2 - 4.5.3 - 1.7.25 - 0.9.10 - 1.3.0 - 7.5.0 - 3.6.0 - 0.0.7 - [2.12,3.0) - 3.4.6 - - - - - - spark-34 - - 2.12 - 2.12.18 - 1.3.0 - - - 4.8.1 - - - 1.22 - 1.8 - 1.10.0 - 1.9.4 - 1.15 - 3.2.2 - 2.11.0 - 1.1.3 - 1.7 - - 14.0.1 - 8.11.0 - 4.0.4 - 3.4.2.openaire - 2.14.2 - 3.12.0 - 2.19.0 - 3.7.0-M11 - 3.25.0-GA - 4.10.0 - 2.0.6 - 0.10.2 - 3.6.3 - - - - - spark-35 - - 2.12 - 2.12.18 - 1.3.0 - - - 4.8.1 - - - 1.23.0 - 1.8 - 1.10.0 - 1.9.4 - 1.16.0 - 3.2.2 - 2.13.0 - 1.1.3 - 1.7 - - 14.0.1 - 8.11.0 - 4.0.4 - 3.5.1.openaire-SNAPSHOT - 2.15.2 - 3.12.0 - 2.20.0 - 3.7.0-M11 - 3.25.0-GA - 4.10.0 - 2.0.7 - 0.10.2 - 3.6.3 - - - - - java11 - - [11 - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 3.0.0-M4 - - - --add-opens=java.base/java.lang=ALL-UNNAMED - --add-opens=java.base/java.lang.invoke=ALL-UNNAMED - --add-opens=java.base/java.lang.reflect=ALL-UNNAMED - --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED - --add-opens=java.base/java.nio=ALL-UNNAMED - --add-opens=java.base/java.util=ALL-UNNAMED - --add-opens=java.base/java.util.concurrent=ALL-UNNAMED - --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED - --add-opens=java.base/sun.nio.ch=ALL-UNNAMED - --add-opens=java.base/sun.nio.cs=ALL-UNNAMED - --add-opens=java.base/sun.security.action=ALL-UNNAMED - --add-opens=java.base/sun.util.calendar=ALL-UNNAMED - - true - false - - - - - - + 4.0.0 + eu.dnetlib.dhp + dhp + 1.2.5-SNAPSHOT + pom + + + + GNU Affero General Public License v3.0 or later + https://spdx.org/licenses/AGPL-3.0-or-later.html#licenseText + repo + This program is free software: you can redistribute it and/or modify it under the terms of the + GNU Affero General Public License as published by the Free Software Foundation, either version 3 of the + License, or (at your option) any later version. + + + + + dhp-build + dhp-pace-core + dhp-common + dhp-workflows + + + + Redmine + https://support.openaire.eu/projects/openaire + + + + jenkins + https://jenkins-dnet.d4science.org/ + + + + scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git + scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git + https://code-repo.d4science.org/D-Net/dnet-hadoop/ + HEAD + + + This module is the root descriptor for the dnet-hadoop project + + + + + + + dnet45-releases + D-Net 45 releases + https://maven.d4science.org/nexus/content/repositories/dnet45-releases + default + + false + + + true + + + + dnet45-snapshots + D-Net 45 snapshots + https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots + default + + true + + + false + + + + dnet45-bootstrap-snapshot + D-Net 45 Bootstrap Snapshot + https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-snapshot/ + + false + + + true + + default + + + dnet45-bootstrap-release + D-Net 45 Bootstrap Release + https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-release/ + + true + + + false + + default + + + cloudera + Cloudera Repository + https://repository.cloudera.com/artifactory/cloudera-repos + + true + + + false + + + + dnet-deps + dnet-dependencies + https://maven.d4science.org/nexus/content/repositories/dnet-deps + default + + + maven-restlet + Restlet repository + https://maven.restlet.talend.com + + + conjars + conjars + https://conjars.wensel.net/repo/ + + + + + + org.junit.jupiter + junit-jupiter + ${junit-jupiter.version} + test + + + + org.mockito + mockito-core + ${mockito-core.version} + test + + + + org.mockito + mockito-junit-jupiter + ${mockito-core.version} + test + + + + + + + + eu.dnetlib.dhp + ${dhp-schemas.artifact} + ${dhp-schemas.version} + + + org.apache.hadoop + hadoop-hdfs + ${dhp.hadoop.version} + provided + + + org.apache.hadoop + hadoop-common + ${dhp.hadoop.version} + provided + + + org.apache.hadoop + hadoop-client + ${dhp.hadoop.version} + provided + + + org.apache.hadoop + hadoop-distcp + ${dhp.hadoop.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${dhp.spark.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${dhp.spark.version} + provided + + + org.apache.spark + spark-graphx_${scala.binary.version} + ${dhp.spark.version} + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + ${dhp.spark.version} + test + + + + org.slf4j + jcl-over-slf4j + 1.7.25 + provided + + + + org.apache.commons + commons-lang3 + ${dhp.commons.lang.version} + + + + commons-validator + commons-validator + 1.7 + + + + com.github.sisyphsu + dateparser + 1.0.7 + + + + me.xuender + unidecode + 0.0.7 + + + + com.google.guava + guava + ${dhp.guava.version} + + + + + commons-codec + commons-codec + 1.9 + + + + commons-io + commons-io + 2.4 + + + + commons-cli + commons-cli + 1.2 + provided + + + + net.sf.saxon + Saxon-HE + 9.9.1-6 + + + + dom4j + dom4j + 1.6.1 + + + + xml-apis + xml-apis + 1.4.01 + + + + jaxen + jaxen + 1.1.6 + + + + com.mycila.xmltool + xmltool + 3.3 + + + + org.apache.solr + solr-solrj + ${solr.version} + + + * + * + + + + + com.lucidworks.spark + spark-solr + ${sparksolr.version} + + + * + * + + + + + org.apache.solr + solr-test-framework + ${solr.version} + test + + + io.dropwizard.metrics + metrics-core + 3.2.6 + test + + + + + org.apache.httpcomponents + httpclient + ${org.apache.httpcomponents.version} + + + org.apache.httpcomponents + httpmime + ${org.apache.httpcomponents.version} + + + org.noggit + noggit + 0.8 + + + org.apache.zookeeper + zookeeper + 3.4.11 + + + + net.schmizz + sshj + 0.10.0 + test + + + + com.fasterxml.jackson.core + jackson-core + ${dhp.jackson.version} + provided + + + + com.fasterxml.jackson.core + jackson-annotations + ${dhp.jackson.version} + provided + + + com.fasterxml.jackson.core + jackson-databind + ${dhp.jackson.version} + provided + + + + eu.dnetlib + cnr-rmi-api + ${cnr-rmi-api.version} + + + + eu.dnetlib.dhp + dnet-openaire-broker-common + ${dnet-openaire-broker-common.version} + + + + org.apache.cxf + cxf-rt-transports-http + 3.1.5 + + + javax.persistence + javax.persistence-api + 2.2 + provided + + + + com.jayway.jsonpath + json-path + 2.4.0 + + + com.arakelian + java-jq + 0.10.1 + + + edu.cmu + secondstring + 1.0.0 + + + org.mongodb + mongo-java-driver + ${mongodb.driver.version} + + + io.fares.junit.mongodb + mongodb-junit-test + 1.1.0 + + + org.postgresql + postgresql + 42.2.10 + + + + org.antlr + stringtemplate + 3.2.1 + + + + org.antlr + ST4 + 4.3.4 + + + + com.ximpleware + vtd-xml + ${vtd.version} + + + + org.elasticsearch + elasticsearch-hadoop + 7.6.0 + + + + + org.apache.oozie + oozie-client + ${dhp.oozie.version} + provided + + + + slf4j-simple + org.slf4j + + + + + + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + + + org.apache.commons + commons-compress + ${common.compress.version} + + + + + org.apache.commons + commons-csv + ${common.csv.version} + + + + + + org.apache.poi + poi-ooxml + ${apache.poi.version} + + + + org.json + json + 20180813 + + + + org.json4s + json4s-jackson_${scala.binary.version} + ${json4s.version} + + + + com.github.victools + jsonschema-generator + ${jsonschemagenerator.version} + + + + org.apache.commons + commons-text + ${common.text.version} + + + + com.opencsv + opencsv + 5.5 + + + io.github.classgraph + classgraph + 4.8.71 + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-xml + ${jackson.version} + provided + + + com.fasterxml.jackson.module + jackson-module-jsonSchema + ${jackson.version} + provided + + + + + org.apache.commons + commons-math3 + 3.6.1 + + + + + com.google.code.gson + gson + ${google.gson.version} + + + + commons-collections + commons-collections + ${commons.collections.version} + + + commons-logging + commons-logging + ${commons.logging.version} + + + + org.reflections + reflections + 0.9.10 + + + + org.scala-lang + scala-library + ${scala.version} + + + + com.ibm.icu + icu4j + 70.1 + + + + + + target + target/classes + ${project.artifactId}-${project.version} + target/test-classes + + + + org.apache.maven.plugins + maven-plugin-plugin + 3.3 + + + org.apache.maven.plugins + maven-project-info-reports-plugin + 3.0.0 + + + org.apache.maven.plugins + maven-site-plugin + 3.9.1 + + ${dhp.site.skip} + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven.compiler.plugin.version} + + 1.8 + 1.8 + ${project.build.sourceEncoding} + + + + + org.apache.maven.plugins + maven-jar-plugin + 3.0.2 + + + + org.apache.maven.plugins + maven-source-plugin + 3.0.1 + + + attach-sources + verify + + jar-no-fork + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M4 + + true + + + + org.apache.maven.plugins + maven-javadoc-plugin + 3.2.0 + + true + none + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.6.0 + + + + net.revelc.code.formatter + formatter-maven-plugin + 2.11.0 + + + eu.dnetlib.dhp + dhp-code-style + ${project.version} + + + + + org.antipathy + mvn-scalafmt_${scala.binary.version} + 1.0.1640073709.733712b + + + eu.dnetlib.dhp + dhp-code-style + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-site-plugin + + + org.apache.maven.plugins + maven-project-info-reports-plugin + + + net.revelc.code.formatter + formatter-maven-plugin + + + + format + + + eclipse/formatter_dnet.xml + + + + + + net.revelc.code + impsort-maven-plugin + 1.4.1 + + java.,javax.,org.,com. + java,* + + **/thrift/*.java + + + + + sort-imports + + sort + + + + + + org.antipathy + mvn-scalafmt_${scala.binary.version} + + https://code-repo.d4science.org/D-Net/dnet-hadoop/raw/branch/beta/dhp-build/dhp-code-style/src/main/resources/scalafmt/scalafmt.conf + false + false + + ${project.basedir}/src/main/scala + + + ${project.basedir}/src/test/scala + + false + false + : git rev-parse --abbrev-ref HEAD + false + + + + validate + + format + + + + + + org.apache.maven.plugins + maven-release-plugin + 2.5.3 + + + org.jacoco + jacoco-maven-plugin + 0.7.9 + + + **/schemas/* + **/com/cloudera/**/* + **/org/apache/avro/io/**/* + + + + + default-prepare-agent + + prepare-agent + + + + default-report + prepare-package + + report + + + + + + + + + + org.apache.maven.wagon + wagon-ssh + 2.10 + + + + + + dnet45-snapshots + DNet45 Snapshots + https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots + default + + + dnet45-releases + https://maven.d4science.org/nexus/content/repositories/dnet45-releases + + + DHPSite + ${dhp.site.stage.path}/ + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + none + + + + + + + sftp://dnet-hadoop@static-web.d4science.org/dnet-hadoop + UTF-8 + UTF-8 + 3.6.0 + 1.8 + 1.8 + 2.22.2 + 2.0.1 + cdh5.9.2 + 2.6.0-${dhp.cdh.version} + 4.1.0-${dhp.cdh.version} + dhp-schemas + 3.6.0 + 2.4.0.cloudera2 + 2.9.6 + 3.5 + true + 11.0.2 + 2.11.12 + 2.11 + 1.3.0 + 5.6.1 + 3.3.3 + 3.4.2 + [2.12,3.0) + [6.1.3] + [3.1.6] + [2.6.1] + 7.5.0 + 4.7.2 + 1.20 + 3.5.3 + 4.13.0 + 1.8 + 4.1.2 + 1.8 + 4.5.3 + 4.0.1 + 2.2.2 + 1.1.3 + 3.2.1 + + + + + + scala-2.12 + + 2.12 + 2.12.18 + + + + 4.0.2 + 3.4.1 + 2.14.2 + 3.12.0 + 3.7.0-M11 + 4.8.1 + + + + + + + + arm-silicon-mac + + + aarch64 + mac + + + + + + org.xerial.snappy + snappy-java + 1.1.8.4 + + + + \ No newline at end of file From aa4d7d5e20370435e7863a16526a113aa80bcb6f Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 3 Jul 2024 19:14:25 +0300 Subject: [PATCH 052/239] Prioritize the rest of the stats-queries over other tasks on the cluster, by putting them in the "analytics" queue. --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql | 2 ++ .../oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql | 2 ++ .../graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql | 2 ++ .../stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql | 2 ++ .../stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql | 2 ++ .../stats/oozie_app/scripts/step20-createMonitorDB_funded.sql | 2 ++ .../oozie_app/scripts/step20-createMonitorDB_institutions.sql | 2 ++ .../stats/oozie_app/scripts/step21-createObservatoryDB.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql | 2 ++ 18 files changed, 36 insertions(+) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql index 467a98872..4551d6282 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + -------------------------------------------------------------- -------------------------------------------------------------- -- Stats database creation diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index 9088ce205..48d8961ff 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -- Tables/views from external tables/views (Fundref, Country, CountyGDP, roarmap, rndexpediture) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 7597f1429..e98a778db 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ---------------------------------------------------------------- ---------------------------------------------------------------- -- Post processing - Updates on main tables diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index 68a46ded3..20784bce9 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ------------------------------------------------------ ------------------------------------------------------ -- Additional relations diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql index 4abb6bdbc..4940bb96d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + create view if not exists TARGET.category as select * from SOURCE.category; create view if not exists TARGET.concept as select * from SOURCE.concept; create view if not exists TARGET.context as select * from SOURCE.context; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index a8392b226..7e31408bc 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + drop database if exists TARGET cascade; create database if not exists TARGET; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql index 4469782f0..9dab79222 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql @@ -1,5 +1,7 @@ drop database if exists TARGET cascade; create database if not exists TARGET; +set mapred.job.queue.name=analytics; /*EOS*/ + create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql index a28206d56..c6ef15d45 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql @@ -1,5 +1,7 @@ drop database if exists TARGET cascade; create database if not exists TARGET; +set mapred.job.queue.name=analytics; /*EOS*/ + create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql index ce6475c22..8e3db49ff 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql @@ -1,5 +1,7 @@ drop database if exists TARGET cascade; create database if not exists TARGET; +set mapred.job.queue.name=analytics; /*EOS*/ + create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql index 62c68c625..286ffe427 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql @@ -1,5 +1,7 @@ drop database if exists TARGET cascade; create database if not exists TARGET; +set mapred.job.queue.name=analytics; /*EOS*/ + create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql index adcf23b7a..1a7f34e96 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + create table ${observatory_db_name}.result_cc_licence stored as parquet as select r.id, coalesce(rln.count, 0) > 0 as cc_licence from ${stats_db_name}.result r diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index 8f203fc83..0e1e02b12 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ------------------------------------------------------ ------------------------------------------------------ -- Dataset table/view and Dataset related tables/views diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index 5e4c9f420..0ccb17fcc 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + -------------------------------------------------------- -------------------------------------------------------- -- Software table/view and Software related tables/views diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index 3134ba316..cd7834d84 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + -------------------------------------------------------------------------------- -------------------------------------------------------------------------------- -- Otherresearchproduct table/view and Otherresearchproduct related tables/views diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index de6b803ab..d261c96e2 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ------------------------------------------------------ ------------------------------------------------------ -- Project table/view and Project related tables/views diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 2283e6291..6cab86a41 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ---------------------------------------------------- ---------------------------------------------------- -- Result table/view and Result related tables/views diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index b280dc946..d0b6abad9 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + -- noinspection SqlNoDataSourceInspectionForFile ------------------------------------------------------------ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 1d76b89a6..f504a5c12 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -1,3 +1,5 @@ +set mapred.job.queue.name=analytics; /*EOS*/ + ---------------------------------------------------------------- ---------------------------------------------------------------- -- Organization table/view and Organization related tables/views From 7ce051d7668c6d8081a512d4c623d701891ecc7b Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 3 Jul 2024 19:49:19 +0300 Subject: [PATCH 053/239] - Update the remaining hive-actions to spark-actions. - Update the version of shell-actions. - Fix missing "/*EOS*/" indicators. --- .../graph/stats/oozie_app/scripts/step11.sql | 2 +- .../graph/stats/oozie_app/scripts/step14.sql | 2 +- .../graph/stats/oozie_app/scripts/step7.sql | 6 +- .../graph/stats/oozie_app/scripts/step8.sql | 16 ++--- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 70 ++++++------------- 5 files changed, 36 insertions(+), 60 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index e98a778db..48373af9b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -17,4 +17,4 @@ FROM ${stats_db_name}.result_projects, ${stats_db_name}.project WHERE result_projects.id = result.id AND result.type = 'publication' - AND project.id = result_projects.project; -- /*EOS*/ \ No newline at end of file + AND project.id = result_projects.project; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index 163e92415..9bbf94480 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -66,4 +66,4 @@ DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute -WHERE datainfo.deletedbyinference=false and datainfo.invisible = FALSE; +WHERE datainfo.deletedbyinference=false and datainfo.invisible = FALSE; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 6cab86a41..1323adf80 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -135,9 +135,9 @@ with lvl4 as (select id, topic from ${stats_db_name}.result_topics where topic like '________ %' and type='Fields of Science and Technology classification') select /*+ COALESCE(100) */ lvl1.id, lvl1.topic as lvl1, lvl2.topic as lvl2, lvl3.topic as lvl3, lvl4.topic as lvl4 from lvl1 - join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2) - join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4) - join lvl4 on lvl4.id=lvl1.id and substr(lvl4.topic, 1, 6)=substr(lvl3.topic, 1, 6); + join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2) + join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4) + join lvl4 on lvl4.id=lvl1.id and substr(lvl4.topic, 1, 6)=substr(lvl3.topic, 1, 6); /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index d0b6abad9..b15efac4a 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -41,31 +41,31 @@ DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages -where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; -- /*EOS*/ +where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; -- /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_oids STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids -where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; -- /*EOS*/ +where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; -- /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_organizations STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r -WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; -- /*EOS*/ +WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; /*EOS*/ -- datasource sources: -- where the datasource info have been collected from. -DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; -- /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; /*EOS*/ create table if not exists ${stats_db_name}.datasource_sources STORED AS PARQUET AS select substr(d.id, 4) as id, substr(cf.key, 4) as datasource from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf -where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; -- /*EOS*/ +where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result -FROM ${stats_db_name}.result_datasources; -- /*EOS*/ +FROM ${stats_db_name}.result_datasources; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index d6fc864c3..8422af4c9 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -307,7 +307,7 @@ - - - - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - @@ -381,12 +370,11 @@ --openaire_db_name${openaire_db_name} --external_stats_db_name${external_stats_db_name} - - + - - - - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - external_stats_db_name=${external_stats_db_name} - - - @@ -533,7 +509,7 @@ - + ${jobTracker} ${nameNode} contexts.sh @@ -624,7 +600,7 @@ - + ${jobTracker} ${nameNode} finalizedb.sh @@ -637,7 +613,7 @@ - + ${jobTracker} ${nameNode} monitor.sh @@ -670,7 +646,7 @@ - + ${jobTracker} ${nameNode} observatory-pre.sh @@ -706,7 +682,7 @@ - + ${jobTracker} ${nameNode} observatory-post.sh @@ -719,7 +695,7 @@ - + ${jobTracker} ${nameNode} copyDataToImpalaCluster.sh @@ -738,7 +714,7 @@ - + ${jobTracker} ${nameNode} createPDFsAggregated.sh @@ -754,7 +730,7 @@ - + ${jobTracker} ${nameNode} finalizeImpalaCluster.sh @@ -773,7 +749,7 @@ - + ${jobTracker} ${nameNode} updateCache.sh From 7b7dd32ad5e4eb08d995ae04dc909b87eed01875 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 3 Jul 2024 19:53:24 +0300 Subject: [PATCH 054/239] - Fix placement of some "set mapred.job.queue.name=analytics" statements and remove their unused "/*EOS*/" indicator. - Add stacktrace-info to failed actions. --- .../stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql | 4 ++-- .../oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql | 4 ++-- .../stats/oozie_app/scripts/step20-createMonitorDB_funded.sql | 4 ++-- .../oozie_app/scripts/step20-createMonitorDB_institutions.sql | 4 ++-- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql | 2 ++ .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql | 2 +- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml | 3 ++- 7 files changed, 13 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql index 9dab79222..0f3dc1d4f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql @@ -1,7 +1,7 @@ +set mapred.job.queue.name=analytics; + drop database if exists TARGET cascade; create database if not exists TARGET; -set mapred.job.queue.name=analytics; /*EOS*/ - create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql index c6ef15d45..2a082c2cd 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql @@ -1,7 +1,7 @@ +set mapred.job.queue.name=analytics; + drop database if exists TARGET cascade; create database if not exists TARGET; -set mapred.job.queue.name=analytics; /*EOS*/ - create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql index 8e3db49ff..759843d68 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql @@ -1,7 +1,7 @@ +set mapred.job.queue.name=analytics; + drop database if exists TARGET cascade; create database if not exists TARGET; -set mapred.job.queue.name=analytics; /*EOS*/ - create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql index 286ffe427..118ccddac 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql @@ -1,7 +1,7 @@ +set mapred.job.queue.name=analytics; + drop database if exists TARGET cascade; create database if not exists TARGET; -set mapred.job.queue.name=analytics; /*EOS*/ - create table TARGET.result stored as parquet as select distinct * from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 1323adf80..0717d7897 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -139,6 +139,8 @@ from lvl1 join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4) join lvl4 on lvl4.id=lvl1.id and substr(lvl4.topic, 1, 6)=substr(lvl3.topic, 1, 6); /*EOS*/ +DROP TABLE ${stats_db_name}.result_fos_base_tmp purge; /*EOS*/ + DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index b15efac4a..9a3995a8f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -67,5 +67,5 @@ from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfro where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS -SELECT datasource AS id, id AS result +SELECT /*+ COALESCE(100) */ datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 8422af4c9..d08cf8f59 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -150,7 +150,8 @@ - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())} + ${wf:actionData(wf:lastErrorNode())['stackTrace']}] From ce0aee21cce147911d8e3cdc7c5d3c6606cfb012 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 3 Jul 2024 20:15:33 +0300 Subject: [PATCH 055/239] Improve performance of transferring the stats-DBs to another cluster and querying the DBs' tables, by ordering Spark to create up to 100 files per table, instead of thousands. --- .../graph/stats/oozie_app/scripts/step13.sql | 16 +- .../graph/stats/oozie_app/scripts/step14.sql | 17 +- .../graph/stats/oozie_app/scripts/step15.sql | 12 +- .../stats/oozie_app/scripts/step15_5.sql | 10 +- .../scripts/step16-createIndicatorsTables.sql | 182 +++++++++--------- .../scripts/step16_1-definitions.sql | 6 +- .../scripts/step21-createObservatoryDB.sql | 38 ++-- .../graph/stats/oozie_app/scripts/step8.sql | 10 +- 8 files changed, 145 insertions(+), 146 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index 20784bce9..a590c190e 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -10,7 +10,7 @@ set mapred.job.queue.name=analytics; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.publication_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_sources STORED AS PARQUET as -SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +SELECT /*+ COALESCE(100) */ p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource from ${openaire_db_name}.publication p lateral view explode(p.collectedfrom.key) c as datasource) p @@ -23,7 +23,7 @@ LEFT OUTER JOIN DROP TABLE IF EXISTS ${stats_db_name}.dataset_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_sources STORED AS PARQUET as -SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +SELECT /*+ COALESCE(100) */ p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource from ${openaire_db_name}.dataset p lateral view explode(p.collectedfrom.key) c as datasource) p @@ -36,7 +36,7 @@ LEFT OUTER JOIN DROP TABLE IF EXISTS ${stats_db_name}.software_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_sources STORED AS PARQUET as -SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +SELECT /*+ COALESCE(100) */ p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource from ${openaire_db_name}.software p lateral view explode(p.collectedfrom.key) c as datasource) p @@ -49,7 +49,7 @@ LEFT OUTER JOIN DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_sources STORED AS PARQUET as -SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +SELECT /*+ COALESCE(100) */ p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.collectedfrom.key) c as datasource) p @@ -71,7 +71,7 @@ SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_orcid purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_orcid STORED AS PARQUET as -select distinct res.id, upper(regexp_replace(res.orcid, 'http://orcid.org/' ,'')) as orcid +select /*+ COALESCE(100) */ distinct res.id, upper(regexp_replace(res.orcid, 'http://orcid.org/' ,'')) as orcid from ( SELECT substr(res.id, 4) as id, auth_pid.value as orcid FROM ${openaire_db_name}.result res @@ -83,7 +83,7 @@ from ( DROP TABLE IF EXISTS ${stats_db_name}.result_result purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_result stored as parquet as -select substr(rel.source, 4) as source, substr(rel.target, 4) as target, relclass, subreltype +select /*+ COALESCE(100) */ substr(rel.source, 4) as source, substr(rel.target, 4) as target, relclass, subreltype from ${openaire_db_name}.relation rel join ${openaire_db_name}.result r1 on rel.source=r1.id join ${openaire_db_name}.result r2 on r2.id=rel.target @@ -98,7 +98,7 @@ where reltype='resultResult' DROP TABLE IF EXISTS ${stats_db_name}.result_citations_oc purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_citations_oc stored as parquet as -select substr(target, 4) as id, count(distinct substr(source, 4)) as citations +select /*+ COALESCE(100) */ substr(target, 4) as id, count(distinct substr(source, 4)) as citations from ${openaire_db_name}.relation rel join ${openaire_db_name}.result r1 on rel.source=r1.id join ${openaire_db_name}.result r2 on r2.id=rel.target @@ -115,7 +115,7 @@ group by substr(target, 4); /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_references_oc purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_references_oc stored as parquet as -select substr(source, 4) as id, count(distinct substr(target, 4)) as references +select /*+ COALESCE(100) */ substr(source, 4) as id, count(distinct substr(target, 4)) as references from ${openaire_db_name}.relation rel join ${openaire_db_name}.result r1 on rel.source=r1.id join ${openaire_db_name}.result r2 on r2.id=rel.target diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index 9bbf94480..9e71b88f5 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -1,4 +1,5 @@ set mapred.job.queue.name=analytics; /*EOS*/ + ------------------------------------------------------ ------------------------------------------------------ -- Additional relations @@ -9,28 +10,28 @@ set mapred.job.queue.name=analytics; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.publication_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_licenses STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, licenses.value as type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.dataset_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_licenses STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, licenses.value as type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.software_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_licenses STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, licenses.value as type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_licenses purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_licenses STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, licenses.value as type +SELECT /*+ COALESCE(100) */ substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; /*EOS*/ @@ -46,15 +47,15 @@ SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.organization_pids purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_pids STORED AS PARQUET AS -select substr(o.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid +select /*+ COALESCE(100) */ substr(o.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${openaire_db_name}.organization o lateral view explode(o.pid) pids as ppid; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.organization_sources purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_sources STORED AS PARQUET as -SELECT o.id, case when d.id is null then 'other' else o.datasource end as datasource +SELECT /*+ COALESCE(100) */ o.id, case when d.id is null then 'other' else o.datasource end as datasource FROM ( - SELECT substr(o.id, 4) as id, substr(instances.instance.key, 4) as datasource + SELECT substr(o.id, 4) as id, substr(instances.instance.key, 4) as datasource from ${openaire_db_name}.organization o lateral view explode(o.collectedfrom) instances as instance) o LEFT OUTER JOIN ( SELECT substr(d.id, 4) id @@ -64,6 +65,6 @@ FROM ( DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge; /*EOS*/ CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as -select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result +select /*+ COALESCE(100) */ distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute WHERE datainfo.deletedbyinference=false and datainfo.invisible = FALSE; /*EOS*/ diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 1f3027b7d..08609afff 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -18,7 +18,7 @@ non_peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE and inst.refereed.classname='nonPeerReviewed') -select distinct * +select /*+ COALESCE(100) */ distinct * from ( select peer_reviewed.* from peer_reviewed union all @@ -36,7 +36,7 @@ non_peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE and inst.refereed.classname='nonPeerReviewed') -select distinct * +select /*+ COALESCE(100) */ distinct * from ( select peer_reviewed.* from peer_reviewed union all @@ -54,7 +54,7 @@ non_peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE and inst.refereed.classname='nonPeerReviewed') -select distinct * +select /*+ COALESCE(100) */ distinct * from ( select peer_reviewed.* from peer_reviewed union all @@ -72,7 +72,7 @@ non_peer_reviewed as ( select distinct substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE and inst.refereed.classname='nonPeerReviewed') -select distinct * +select /*+ COALESCE(100) */ distinct * from ( select peer_reviewed.* from peer_reviewed union all @@ -92,7 +92,7 @@ select * from ${stats_db_name}.otherresearchproduct_refereed; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.indi_impact_measures purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_impact_measures STORED AS PARQUET as -select substr(id, 4) as id, measures_ids.id impactmetric, cast(measures_ids.unit.value[0] as double) score, +select /*+ COALESCE(100) */ substr(id, 4) as id, measures_ids.id impactmetric, cast(measures_ids.unit.value[0] as double) score, cast(measures_ids.unit.value[0] as decimal(6,3)) score_dec, measures_ids.unit.value[1] impact_class from ${openaire_db_name}.result lateral view explode(measures) measures as measures_ids where measures_ids.id!='views' and measures_ids.id!='downloads'; /*EOS*/ @@ -100,7 +100,7 @@ where measures_ids.id!='views' and measures_ids.id!='downloads'; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_apc_affiliations purge; /*EOS*/ create table if not exists ${stats_db_name}.result_apc_affiliations STORED AS PARQUET as -select distinct substr(rel.target,4) id, substr(rel.source,4) organization, o.legalname.value name, +select /*+ COALESCE(100) */ distinct substr(rel.target,4) id, substr(rel.source,4) organization, o.legalname.value name, cast(rel.properties[0].value as double) apc_amount, rel.properties[1].value apc_currency from ${openaire_db_name}.relation rel diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql index d18cf569f..d61b4d2ef 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql @@ -6,7 +6,7 @@ set mapred.job.queue.name=analytics; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_projectcount purge; /*EOS*/ create table if not exists ${stats_db_name}.result_projectcount STORED AS PARQUET as -select r.id, count(distinct p.id) as count +select /*+ COALESCE(100) */ r.id, count(distinct p.id) as count from ${stats_db_name}.result r left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project @@ -15,7 +15,7 @@ group by r.id; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_fundercount purge; /*EOS*/ create table if not exists ${stats_db_name}.result_fundercount STORED AS PARQUET as -select r.id, count(distinct p.funder) as count +select /*+ COALESCE(100) */ r.id, count(distinct p.funder) as count from ${stats_db_name}.result r left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project @@ -30,7 +30,7 @@ with rcount as ( left outer join ${stats_db_name}.result_projects rp on rp.project=p.id left outer join ${stats_db_name}.result r on r.id=rp.id group by r.type, p.id ) -select rcount.pid, sum(case when rcount.type='publication' then rcount.count else 0 end) as publications, +select /*+ COALESCE(100) */ rcount.pid, sum(case when rcount.type='publication' then rcount.count else 0 end) as publications, sum(case when rcount.type='dataset' then rcount.count else 0 end) as datasets, sum(case when rcount.type='software' then rcount.count else 0 end) as software, sum(case when rcount.type='other' then rcount.count else 0 end) as other @@ -48,7 +48,7 @@ create or replace view ${stats_db_name}.graduatedoctorates as select * from stat DROP TABLE IF EXISTS ${stats_db_name}.result_instance purge; /*EOS*/ create table if not exists ${stats_db_name}.result_instance stored as parquet as -select distinct r.* +select /*+ COALESCE(100) */ distinct r.* from ( select substr(r.id, 4) as id, inst.accessright.classname as accessright, inst.accessright.openaccessroute as accessright_uw, substr(inst.collectedfrom.key, 4) as collectedfrom, substr(inst.hostedby.key, 4) as hostedby, inst.dateofacceptance.value as dateofacceptance, inst.license.value as license, p.qualifier.classname as pidtype, p.value as pid @@ -58,7 +58,7 @@ join ${stats_db_name}.result res on res.id=r.id; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.result_apc purge; /*EOS*/ create table if not exists ${stats_db_name}.result_apc STORED AS PARQUET as -select distinct r.id, r.amount, r.currency +select /*+ COALESCE(100) */ distinct r.id, r.amount, r.currency from ( select substr(r.id, 4) as id, cast(inst.processingchargeamount.value as float) as amount, inst.processingchargecurrency.value as currency from ${openaire_db_name}.result r lateral view explode(r.instance) instances as inst) r diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 562487474..6e7f00b53 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -1,7 +1,7 @@ -- Sprint 1 ---- drop table if exists ${stats_db_name}.indi_pub_green_oa purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_green_oa stored as parquet as -select distinct p.id, coalesce(green_oa, 0) as green_oa +select /*+ COALESCE(100) */ distinct p.id, coalesce(green_oa, 0) as green_oa from ${stats_db_name}.publication p left outer join ( select p.id, 1 as green_oa @@ -12,7 +12,7 @@ left outer join ( drop table if exists ${stats_db_name}.indi_pub_grey_lit purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_grey_lit stored as parquet as -select distinct p.id, coalesce(grey_lit, 0) as grey_lit +select /*+ COALESCE(100) */ distinct p.id, coalesce(grey_lit, 0) as grey_lit from ${stats_db_name}.publication p left outer join ( select p.id, 1 as grey_lit @@ -23,7 +23,7 @@ left outer join ( drop table if exists ${stats_db_name}.indi_pub_doi_from_crossref purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_doi_from_crossref stored as parquet as -select distinct p.id, coalesce(doi_from_crossref, 0) as doi_from_crossref +select /*+ COALESCE(100) */ distinct p.id, coalesce(doi_from_crossref, 0) as doi_from_crossref from ${stats_db_name}.publication p left outer join ( select ri.id, 1 as doi_from_crossref from ${stats_db_name}.result_instance ri @@ -33,7 +33,7 @@ left outer join ( -- Sprint 2 ---- drop table if exists ${stats_db_name}.indi_result_has_cc_licence purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_result_has_cc_licence stored as parquet as -select distinct r.id, (case when lic='' or lic is null then 0 else 1 end) as has_cc_license +select /*+ COALESCE(100) */ distinct r.id, (case when lic='' or lic is null then 0 else 1 end) as has_cc_license from ${stats_db_name}.result r left outer join ( select r.id, license.type as lic from ${stats_db_name}.result r @@ -42,7 +42,7 @@ left outer join ( drop table if exists ${stats_db_name}.indi_result_has_cc_licence_url purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_result_has_cc_licence_url stored as parquet as -select distinct r.id, case when lic_host='' or lic_host is null then 0 else 1 end as has_cc_license_url +select /*+ COALESCE(100) */ distinct r.id, case when lic_host='' or lic_host is null then 0 else 1 end as has_cc_license_url from ${stats_db_name}.result r left outer join ( select r.id, lower(parse_url(license.type, "HOST")) as lic_host @@ -52,12 +52,12 @@ left outer join ( drop table if exists ${stats_db_name}.indi_pub_has_abstract purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_has_abstract stored as parquet as -select distinct publication.id, cast(coalesce(abstract, true) as int) has_abstract +select /*+ COALESCE(100) */ distinct publication.id, cast(coalesce(abstract, true) as int) has_abstract from ${stats_db_name}.publication; /*EOS*/ drop table if exists ${stats_db_name}.indi_result_with_orcid purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_result_with_orcid stored as parquet as -select distinct r.id, coalesce(has_orcid, 0) as has_orcid +select /*+ COALESCE(100) */ distinct r.id, coalesce(has_orcid, 0) as has_orcid from ${stats_db_name}.result r left outer join ( select id, 1 as has_orcid from ${stats_db_name}.result_orcid) tmp on r.id= tmp.id; /*EOS*/ @@ -66,7 +66,7 @@ left outer join ( ---- Sprint 3 ---- drop table if exists ${stats_db_name}.indi_funded_result_with_fundref purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_funded_result_with_fundref stored as parquet as -select distinct r.result as id, coalesce(fundref, 0) as fundref +select /*+ COALESCE(100) */ distinct r.result as id, coalesce(fundref, 0) as fundref from ${stats_db_name}.project_results r left outer join ( select distinct result, 1 as fundref from ${stats_db_name}.project_results where provenance='Harvested') tmp on r.result= tmp.result; /*EOS*/ @@ -77,7 +77,7 @@ create table if not exists ${stats_db_name}.indi_result_org_collab stored as par SELECT ro.organization organization, ro.id, o.name from ${stats_db_name}.result_organization ro join ${stats_db_name}.organization o on o.id=ro.organization where o.name is not null) - select o1.organization org1, o1.name org1name1, o2.organization org2, o2.name org2name2, count(o1.id) as collaborations + select /*+ COALESCE(100) */ o1.organization org1, o1.name org1name1, o2.organization org2, o2.name org2name2, count(o1.id) as collaborations from tmp as o1 join tmp as o2 where o1.id=o2.id and o1.organization!=o2.organization and o1.name!=o2.name group by o1.organization, o2.organization, o1.name, o2.name; /*EOS*/ @@ -89,7 +89,7 @@ create table if not exists ${stats_db_name}.indi_result_org_country_collab store from ${stats_db_name}.result_organization ro join ${stats_db_name}.organization o on o.id=ro.organization where country <> 'UNKNOWN' and o.name is not null) - select o1.organization org1,o1.name org1name1, o2.country country2, count(o1.id) as collaborations + select /*+ COALESCE(100) */ o1.organization org1,o1.name org1name1, o2.country country2, count(o1.id) as collaborations from tmp as o1 join tmp as o2 on o1.id=o2.id where o1.id=o2.id and o1.country!=o2.country group by o1.organization, o1.id, o1.name, o2.country; /*EOS*/ @@ -100,7 +100,7 @@ create table if not exists ${stats_db_name}.indi_project_collab_org stored as pa select o.id organization, o.name, ro.project as project from ${stats_db_name}.organization o join ${stats_db_name}.organization_projects ro on o.id=ro.id where o.name is not null) - select o1.organization org1,o1.name orgname1, o2.organization org2, o2.name orgname2, count(distinct o1.project) as collaborations + select /*+ COALESCE(100) */ o1.organization org1,o1.name orgname1, o2.organization org2, o2.name orgname2, count(distinct o1.project) as collaborations from tmp as o1 join tmp as o2 on o1.project=o2.project where o1.organization<>o2.organization and o1.name<>o2.name @@ -112,7 +112,7 @@ create table if not exists ${stats_db_name}.indi_project_collab_org_country stor select o.id organization, o.name, o.country , ro.project as project from ${stats_db_name}.organization o join ${stats_db_name}.organization_projects ro on o.id=ro.id and o.country <> 'UNKNOWN' and o.name is not null) - select o1.organization org1,o1.name org1name, o2.country country2, count(distinct o1.project) as collaborations + select /*+ COALESCE(100) */ o1.organization org1,o1.name org1name, o2.country country2, count(distinct o1.project) as collaborations from tmp as o1 join tmp as o2 on o1.project=o2.project where o1.organization<>o2.organization and o1.country<>o2.country @@ -124,7 +124,7 @@ create table if not exists ${stats_db_name}.indi_funder_country_collab stored as join ${stats_db_name}.organization o on o.id=op.id join ${stats_db_name}.project p on p.id=op.project where country <> 'UNKNOWN') - select f1.funder, f1.country as country1, f2.country as country2, count(distinct f1.project) as collaborations + select /*+ COALESCE(100) */ f1.funder, f1.country as country1, f2.country as country2, count(distinct f1.project) as collaborations from tmp as f1 join tmp as f2 on f1.project=f2.project where f1.country<>f2.country @@ -136,7 +136,7 @@ create table if not exists ${stats_db_name}.indi_result_country_collab stored as select distinct country, ro.id as result from ${stats_db_name}.organization o join ${stats_db_name}.result_organization ro on o.id=ro.organization where country <> 'UNKNOWN' and o.name is not null) - select o1.country country1, o2.country country2, count(o1.result) as collaborations + select /*+ COALESCE(100) */ o1.country country1, o2.country country2, count(o1.result) as collaborations from tmp as o1 join tmp as o2 on o1.result=o2.result where o1.country<>o2.country @@ -146,7 +146,7 @@ create table if not exists ${stats_db_name}.indi_result_country_collab stored as ---- Sprint 4 ---- drop table if exists ${stats_db_name}.indi_pub_diamond purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_diamond stored as parquet as - select distinct pd.id, coalesce(in_diamond_journal, 0) as in_diamond_journal + select /*+ COALESCE(100) */ distinct pd.id, coalesce(in_diamond_journal, 0) as in_diamond_journal from ${stats_db_name}.publication_datasources pd left outer join ( select pd.id, 1 as in_diamond_journal @@ -157,7 +157,7 @@ create table if not exists ${stats_db_name}.indi_pub_diamond stored as parquet a drop table if exists ${stats_db_name}.indi_pub_in_transformative purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_in_transformative stored as parquet as - select distinct pd.id, coalesce(is_transformative, 0) as is_transformative + select /*+ COALESCE(100) */ distinct pd.id, coalesce(is_transformative, 0) as is_transformative from ${stats_db_name}.publication pd left outer join ( select pd.id, 1 as is_transformative @@ -168,7 +168,7 @@ create table if not exists ${stats_db_name}.indi_pub_in_transformative stored as drop table if exists ${stats_db_name}.indi_pub_closed_other_open purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_closed_other_open stored as parquet as - select distinct ri.id, coalesce(pub_closed_other_open, 0) as pub_closed_other_open + select /*+ COALESCE(100) */ distinct ri.id, coalesce(pub_closed_other_open, 0) as pub_closed_other_open from ${stats_db_name}.result_instance ri left outer join ( select ri.id, 1 as pub_closed_other_open @@ -182,14 +182,14 @@ create table if not exists ${stats_db_name}.indi_pub_closed_other_open stored as ---- Sprint 5 ---- drop table if exists ${stats_db_name}.indi_result_no_of_copies purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_result_no_of_copies stored as parquet as - select id, count(id) as number_of_copies + select /*+ COALESCE(100) */ id, count(id) as number_of_copies from ${stats_db_name}.result_instance group by id; /*EOS*/ ---- Sprint 6 ---- drop table if exists ${stats_db_name}.indi_pub_downloads purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_downloads stored as parquet as - SELECT result_id, sum(downloads) no_downloads + SELECT /*+ COALESCE(100) */ result_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats join ${stats_db_name}.publication on result_id=id where downloads>0 @@ -197,7 +197,7 @@ create table if not exists ${stats_db_name}.indi_pub_downloads stored as parquet drop table if exists ${stats_db_name}.indi_pub_downloads_datasource purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_downloads_datasource stored as parquet as - SELECT result_id, repository_id, sum(downloads) no_downloads + SELECT /*+ COALESCE(100) */ result_id, repository_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats join ${stats_db_name}.publication on result_id=id where downloads>0 @@ -205,14 +205,14 @@ create table if not exists ${stats_db_name}.indi_pub_downloads_datasource stored drop table if exists ${stats_db_name}.indi_pub_downloads_year purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_downloads_year stored as parquet as - SELECT result_id, cast(substring(us.`date`, 1,4) as int) as `year`, sum(downloads) no_downloads + SELECT /*+ COALESCE(100) */ result_id, cast(substring(us.`date`, 1,4) as int) as `year`, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats us join ${stats_db_name}.publication on result_id=id where downloads>0 GROUP BY result_id, substring(us.`date`, 1,4); /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_downloads_datasource_year purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_downloads_datasource_year stored as parquet as - SELECT result_id, cast(substring(us.`date`, 1,4) as int) as `year`, repository_id, sum(downloads) no_downloads + SELECT /*+ COALESCE(100) */ result_id, cast(substring(us.`date`, 1,4) as int) as `year`, repository_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats us join ${stats_db_name}.publication on result_id=id where downloads>0 @@ -241,7 +241,7 @@ create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet a UNION ALL select id, issn_online as issn from ${stats_db_name}.datasource d left semi join gold_oa on gold_oa.issn=d.issn_online) foo ) - SELECT DISTINCT pd.id, coalesce(is_gold, 0) as is_gold + SELECT /*+ COALESCE(100) */ DISTINCT pd.id, coalesce(is_gold, 0) as is_gold FROM ${stats_db_name}.publication pd left outer join ( select pd.id, 1 as is_gold @@ -272,7 +272,7 @@ create table if not exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc stored as FROM ${stats_db_name}.datasource WHERE issn_online IS NOT NULL ) as issn WHERE LENGTH(issn) > 7) - SELECT DISTINCT pd.id, coalesce(is_hybrid_oa, 0) as is_hybrid_oa + SELECT /*+ COALESCE(100) */ DISTINCT pd.id, coalesce(is_hybrid_oa, 0) as is_hybrid_oa FROM ${stats_db_name}.publication_datasources pd LEFT OUTER JOIN ( SELECT pd.id, 1 as is_hybrid_oa from ${stats_db_name}.publication_datasources pd @@ -284,7 +284,7 @@ create table if not exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc stored as drop table if exists ${stats_db_name}.indi_pub_hybrid purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_hybrid stored as parquet as -select distinct p.id, coalesce(is_hybrid, 0) is_hybrid +select /*+ COALESCE(100) */ distinct p.id, coalesce(is_hybrid, 0) is_hybrid from ${stats_db_name}.publication p left outer join ( select p.id, 1 as is_hybrid @@ -313,7 +313,7 @@ create table if not exists ${stats_db_name}.indi_org_fairness stored as parquet where cast(year as int)>2003 group by ro.organization) --return results_fair/all_results - select allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness + select /*+ COALESCE(100) */ allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness from allresults join result_fair on result_fair.organization=allresults.organization; /*EOS*/ @@ -336,7 +336,7 @@ select ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name drop table if exists ${stats_db_name}.indi_org_fairness_pub_pr purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_fairness_pub_pr stored as parquet as -select ar.organization, rf.no_result_fair/ar.no_allresults org_fairness +select /*+ COALESCE(100) */ ar.organization, rf.no_result_fair/ar.no_allresults org_fairness from allresults ar join result_fair rf on rf.organization=ar.organization; /*EOS*/ @@ -357,7 +357,7 @@ CREATE TEMPORARY VIEW allresults as select year, ro.organization, count(distinct drop table if exists ${stats_db_name}.indi_org_fairness_pub_year purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_fairness_pub_year stored as parquet as -select cast(allresults.year as int) year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness +select /*+ COALESCE(100) */ cast(allresults.year as int) year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness from allresults join result_fair on result_fair.organization=allresults.organization and result_fair.year=allresults.year; /*EOS*/ @@ -381,7 +381,7 @@ CREATE TEMPORARY VIEW allresults as drop table if exists ${stats_db_name}.indi_org_fairness_pub purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_fairness_pub stored as parquet as -select ar.organization, rf.no_result_fair/ar.no_allresults org_fairness +select /*+ COALESCE(100) */ ar.organization, rf.no_result_fair/ar.no_allresults org_fairness from allresults ar join result_fair rf on rf.organization=ar.organization; /*EOS*/ @@ -404,7 +404,7 @@ CREATE TEMPORARY VIEW allresults as drop table if exists ${stats_db_name}.indi_org_fairness_year purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_fairness_year stored as parquet as - select cast(allresults.year as int) year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness + select /*+ COALESCE(100) */ cast(allresults.year as int) year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness from allresults join result_fair on result_fair.organization=allresults.organization and cast(result_fair.year as int)=cast(allresults.year as int); /*EOS*/ @@ -427,7 +427,7 @@ CREATE TEMPORARY VIEW allresults as drop table if exists ${stats_db_name}.indi_org_findable_year purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_findable_year stored as parquet as -select cast(allresults.year as int) year, allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable +select /*+ COALESCE(100) */ cast(allresults.year as int) year, allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable from allresults join result_with_pid on result_with_pid.organization=allresults.organization and cast(result_with_pid.year as int)=cast(allresults.year as int); /*EOS*/ @@ -450,7 +450,7 @@ select ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name drop table if exists ${stats_db_name}.indi_org_findable purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_findable stored as parquet as -select allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable +select /*+ COALESCE(100) */ allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable from allresults join result_with_pid on result_with_pid.organization=allresults.organization; /*EOS*/ @@ -516,7 +516,7 @@ select software_oa.organization, software_oa.no_oasoftware/allsoftware.no_allsof drop table if exists ${stats_db_name}.indi_org_openess purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_openess stored as parquet as -select allpubsshare.organization, +select /*+ COALESCE(100) */ allpubsshare.organization, (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) +(case when d is null then 0 else 1 end)) org_openess FROM allpubsshare @@ -593,7 +593,7 @@ select allsoftware.year, software_oa.organization, software_oa.no_oasoftware/all drop table if exists ${stats_db_name}.indi_org_openess_year purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_openess_year stored as parquet as -select cast(allpubsshare.year as int) year, allpubsshare.organization, +select /*+ COALESCE(100) */ cast(allpubsshare.year as int) year, allpubsshare.organization, (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) +(case when d is null then 0 else 1 end)) org_openess FROM allpubsshare @@ -617,7 +617,7 @@ DROP VIEW allsoftwaresshare; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_has_preprint purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_has_preprint stored as parquet as -select distinct p.id, coalesce(has_preprint, 0) as has_preprint +select /*+ COALESCE(100) */ distinct p.id, coalesce(has_preprint, 0) as has_preprint from ${stats_db_name}.publication_classifications p left outer join ( select p.id, 1 as has_preprint @@ -627,7 +627,7 @@ from ${stats_db_name}.publication_classifications p drop table if exists ${stats_db_name}.indi_pub_in_subscribed purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_in_subscribed stored as parquet as -select distinct p.id, coalesce(is_subscription, 0) as is_subscription +select /*+ COALESCE(100) */ distinct p.id, coalesce(is_subscription, 0) as is_subscription from ${stats_db_name}.publication p left outer join( select p.id, 1 as is_subscription from ${stats_db_name}.publication p @@ -640,7 +640,7 @@ from ${stats_db_name}.publication p drop table if exists ${stats_db_name}.indi_result_with_pid purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_result_with_pid stored as parquet as -select distinct p.id, coalesce(result_with_pid, 0) as result_with_pid +select /*+ COALESCE(100) */ distinct p.id, coalesce(result_with_pid, 0) as result_with_pid from ${stats_db_name}.result p left outer join ( select p.id, 1 as result_with_pid @@ -654,7 +654,7 @@ group by rf.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_interdisciplinarity purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_interdisciplinarity stored as parquet as -select distinct p.id as id, coalesce(is_interdisciplinary, 0) +select /*+ COALESCE(100) */ distinct p.id as id, coalesce(is_interdisciplinary, 0) as is_interdisciplinary from pub_fos_totals p left outer join ( @@ -666,7 +666,7 @@ drop view pub_fos_totals; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_bronze_oa purge; /*EOS*/ create table ${stats_db_name}.indi_pub_bronze_oa stored as parquet as -select distinct p.id,coalesce(is_bronze_oa,0) is_bronze_oa +select /*+ COALESCE(100) */ distinct p.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication p left outer join ( select p.id, 1 as is_bronze_oa @@ -689,7 +689,7 @@ where p.end_year is NOT NULL and r.year is not null; /*EOS*/ drop table if exists ${stats_db_name}.indi_is_project_result_after purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_is_project_result_after stored as parquet as -select pry.project_id, pry.acronym, pry.result_id, +select /*+ COALESCE(100) */ pry.project_id, pry.acronym, pry.result_id, coalesce(is_project_result_after, 0) as is_project_result_after from project_year_result_year pry left outer join (select pry.project_id, pry.acronym, pry.result_id, 1 as is_project_result_after @@ -701,7 +701,7 @@ drop view project_year_result_year; /*EOS*/ drop table if exists ${stats_db_name}.indi_is_funder_plan_s purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_is_funder_plan_s stored as parquet as -select distinct f.id, f.name, coalesce(is_funder_plan_s, 0) as is_funder_plan_s +select /*+ COALESCE(100) */ distinct f.id, f.name, coalesce(is_funder_plan_s, 0) as is_funder_plan_s from ${stats_db_name}.funder f left outer join (select id, name, 1 as is_funder_plan_s from ${stats_db_name}.funder join stats_ext.plan_s_short on c_o_alition_s_organisation_funder=name) tmp @@ -722,7 +722,7 @@ create table if not exists ${stats_db_name}.indi_funder_fairness stored as parqu join ${stats_db_name}.project p on p.id=rp.project where cast(year as int)>2003 group by p.funder) -select allresults.funder, result_fair.no_result_fair/allresults.no_allresults funder_fairness +select /*+ COALESCE(100) */ allresults.funder, result_fair.no_result_fair/allresults.no_allresults funder_fairness from allresults join result_fair on result_fair.funder=allresults.funder; /*EOS*/ @@ -745,7 +745,7 @@ allresults as join ${stats_db_name}.result r on r.id=rc.id where cast(year as int)>2003 group by rc.ri_initiative) -select allresults.ri_initiative, result_fair.no_result_fair/allresults.no_allresults ris_fairness +select /*+ COALESCE(100) */ allresults.ri_initiative, result_fair.no_result_fair/allresults.no_allresults ris_fairness from allresults join result_fair on result_fair.ri_initiative=allresults.ri_initiative; /*EOS*/ @@ -817,16 +817,14 @@ select software_oa.funder, software_oa.no_oasoftware/allsoftware.no_allsoftware drop table if exists ${stats_db_name}.indi_funder_openess purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_funder_openess stored as parquet as -select allpubsshare.funder, - (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) - +(case when d is null then 0 else 1 end)) - funder_openess FROM allpubsshare - left outer join (select funder,d from - alldatasetssshare) tmp1 - on tmp1.funder=allpubsshare.funder - left outer join (select funder,s from - allsoftwaresshare) tmp2 - on tmp2.funder=allpubsshare.funder; /*EOS*/ +select /*+ COALESCE(100) */ allpubsshare.funder, + (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) + +(case when d is null then 0 else 1 end)) funder_openess +FROM allpubsshare + left outer join (select funder,d from alldatasetssshare) tmp1 + on tmp1.funder=allpubsshare.funder + left outer join (select funder,s from allsoftwaresshare) tmp2 + on tmp2.funder=allpubsshare.funder; /*EOS*/ DROP VIEW pubs_oa; /*EOS*/ DROP VIEW datasets_oa; /*EOS*/ @@ -905,7 +903,7 @@ select software_oa.ri_initiative, software_oa.no_oasoftware/allsoftware.no_allso drop table if exists ${stats_db_name}.indi_ris_openess purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_ris_openess stored as parquet as -select allpubsshare.ri_initiative, +select /*+ COALESCE(100) */ allpubsshare.ri_initiative, (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) +(case when d is null then 0 else 1 end)) ris_openess FROM allpubsshare @@ -943,7 +941,7 @@ with result_findable as join ${stats_db_name}.project p on p.id=rp.project where cast(year as int)>2003 group by p.funder) -select allresults.funder, result_findable.no_result_findable/allresults.no_allresults funder_findable +select /*+ COALESCE(100) */ allresults.funder, result_findable.no_result_findable/allresults.no_allresults funder_findable from allresults join result_findable on result_findable.funder=allresults.funder; /*EOS*/ @@ -952,22 +950,22 @@ drop table if exists ${stats_db_name}.indi_ris_findable purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_ris_findable stored as parquet as with result_contexts as -(select distinct rc.id, context.name ri_initiative from ${stats_db_name}.result_concepts rc -join ${stats_db_name}.concept on concept.id=rc.concept -join ${stats_db_name}.category on category.id=concept.category -join ${stats_db_name}.context on context.id=category.context), -result_findable as - (select rc.ri_initiative ri_initiative, count(distinct rc.id) no_result_findable from result_contexts rc - join ${stats_db_name}.result r on r.id=rc.id - join ${stats_db_name}.result_pids rp on rp.id=r.id - where cast(r.year as int)>2003 - group by rc.ri_initiative), -allresults as -(select rc.ri_initiative ri_initiative, count(distinct rc.id) no_allresults from result_contexts rc - join ${stats_db_name}.result r on r.id=rc.id - where cast(r.year as int)>2003 - group by rc.ri_initiative) -select allresults.ri_initiative, result_findable.no_result_findable/allresults.no_allresults ris_findable + (select distinct rc.id, context.name ri_initiative from ${stats_db_name}.result_concepts rc + join ${stats_db_name}.concept on concept.id=rc.concept + join ${stats_db_name}.category on category.id=concept.category + join ${stats_db_name}.context on context.id=category.context), + result_findable as + (select rc.ri_initiative ri_initiative, count(distinct rc.id) no_result_findable from result_contexts rc + join ${stats_db_name}.result r on r.id=rc.id + join ${stats_db_name}.result_pids rp on rp.id=r.id + where cast(r.year as int)>2003 + group by rc.ri_initiative), + allresults as + (select rc.ri_initiative ri_initiative, count(distinct rc.id) no_allresults from result_contexts rc + join ${stats_db_name}.result r on r.id=rc.id + where cast(r.year as int)>2003 + group by rc.ri_initiative) +select /*+ COALESCE(100) */ allresults.ri_initiative, result_findable.no_result_findable/allresults.no_allresults ris_findable from allresults join result_findable on result_findable.ri_initiative=allresults.ri_initiative; /*EOS*/ @@ -975,20 +973,20 @@ drop table if exists ${stats_db_name}.indi_pub_publicly_funded purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_publicly_funded stored as parquet as with org_names_pids as -(select org.id,name, pid from ${stats_db_name}.organization org -join ${stats_db_name}.organization_pids op on org.id=op.id), -publicly_funded_orgs as -(select distinct name from -(select pf.name from stats_ext.insitutions_for_publicly_funded pf -join ${stats_db_name}.fundref f on f.name=pf.name where f.type='government' -union all -select pf.name from stats_ext.insitutions_for_publicly_funded pf -join ${stats_db_name}.project p on p.funder=pf.name -union all -select op.name from stats_ext.insitutions_for_publicly_funded pf -join org_names_pids op on (op.name=pf.name or op.pid=pf.ror) -and pf.publicly_funded='yes') foo) -select distinct p.id, coalesce(publicly_funded, 0) as publicly_funded + (select org.id,name, pid from ${stats_db_name}.organization org + join ${stats_db_name}.organization_pids op on org.id=op.id), + publicly_funded_orgs as + (select distinct name from + (select pf.name from stats_ext.insitutions_for_publicly_funded pf + join ${stats_db_name}.fundref f on f.name=pf.name where f.type='government' + union all + select pf.name from stats_ext.insitutions_for_publicly_funded pf + join ${stats_db_name}.project p on p.funder=pf.name + union all + select op.name from stats_ext.insitutions_for_publicly_funded pf + join org_names_pids op on (op.name=pf.name or op.pid=pf.ror) + and pf.publicly_funded='yes') foo) +select /*+ COALESCE(100) */ distinct p.id, coalesce(publicly_funded, 0) as publicly_funded from ${stats_db_name}.publication p left outer join ( select distinct ro.id, 1 as publicly_funded from ${stats_db_name}.result_organization ro @@ -997,7 +995,7 @@ join publicly_funded_orgs pfo on o.name=pfo.name) tmp on p.id=tmp.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_green_with_license purge; /*EOS*/ create table ${stats_db_name}.indi_pub_green_with_license stored as parquet as -select distinct p.id, coalesce(green_with_license, 0) as green_with_license +select /*+ COALESCE(100) */ distinct p.id, coalesce(green_with_license, 0) as green_with_license from ${stats_db_name}.publication p left outer join ( select distinct p.id, 1 as green_with_license from ${stats_db_name}.publication p @@ -1008,7 +1006,7 @@ left outer join ( drop table if exists ${stats_db_name}.result_country purge; /*EOS*/ create table ${stats_db_name}.result_country stored as parquet as -select distinct id, country +select /*+ COALESCE(100) */ distinct id, country from ( select ro.id, o.country from ${stats_db_name}.result_organization ro @@ -1023,7 +1021,7 @@ where rc.country is not null; /*EOS*/ drop table if exists ${stats_db_name}.indi_result_oa_with_license purge; /*EOS*/ create table ${stats_db_name}.indi_result_oa_with_license stored as parquet as -select distinct r.id, coalesce(oa_with_license,0) as oa_with_license +select /*+ COALESCE(100) */ distinct r.id, coalesce(oa_with_license,0) as oa_with_license from ${stats_db_name}.result r left outer join (select distinct r.id, 1 as oa_with_license from ${stats_db_name}.result r join ${stats_db_name}.result_licenses rl on rl.id=r.id where r.bestlicence='Open Access') tmp on r.id=tmp.id; /*EOS*/ @@ -1031,9 +1029,9 @@ join ${stats_db_name}.result_licenses rl on rl.id=r.id where r.bestlicence='Open drop table if exists ${stats_db_name}.indi_result_oa_without_license purge; /*EOS*/ create table ${stats_db_name}.indi_result_oa_without_license stored as parquet as with without_license as -(select distinct id from ${stats_db_name}.indi_result_oa_with_license -where oa_with_license=0) -select distinct r.id, coalesce(oa_without_license,0) as oa_without_license + (select distinct id from ${stats_db_name}.indi_result_oa_with_license + where oa_with_license=0) +select /*+ COALESCE(100) */ distinct r.id, coalesce(oa_without_license,0) as oa_without_license from ${stats_db_name}.result r left outer join (select distinct r.id, 1 as oa_without_license from ${stats_db_name}.result r @@ -1044,7 +1042,7 @@ drop table if exists ${stats_db_name}.indi_result_under_transformative purge; /* create table ${stats_db_name}.indi_result_under_transformative stored as parquet as with transformative_dois as ( select distinct doi from stats_ext.transformative_facts) -select distinct r.id, coalesce(under_transformative,0) as under_transformative +select /*+ COALESCE(100) */ distinct r.id, coalesce(under_transformative,0) as under_transformative from ${stats_db_name}.result r left outer join ( select distinct rp.id, 1 as under_transformative diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql index 0da4394c8..80256e2df 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql @@ -8,7 +8,7 @@ set mapred.job.queue.name=analytics; /*EOS*/ drop table if exists ${stats_db_name}.result_peerreviewed purge; /*EOS*/ create table IF NOT EXISTS ${stats_db_name}.result_peerreviewed STORED AS PARQUET as -select r.id as id, case when doi.doi_from_crossref=1 and grey.grey_lit=0 then true else false end as peer_reviewed +select /*+ COALESCE(100) */ r.id as id, case when doi.doi_from_crossref=1 and grey.grey_lit=0 then true else false end as peer_reviewed from ${stats_db_name}.result r left outer join ${stats_db_name}.indi_pub_doi_from_crossref doi on doi.id=r.id left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id; /*EOS*/ @@ -17,7 +17,7 @@ left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id; /*EOS*/ drop table if exists ${stats_db_name}.result_greenoa purge; /*EOS*/ create table IF NOT EXISTS ${stats_db_name}.result_greenoa STORED AS PARQUET as -select r.id, case when green.green_oa=1 then true else false end as green +select /*+ COALESCE(100) */ r.id, case when green.green_oa=1 then true else false end as green from ${stats_db_name}.result r left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id; /*EOS*/ @@ -25,6 +25,6 @@ left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id; /*EOS drop table if exists ${stats_db_name}.result_gold purge; /*EOS*/ create table IF NOT EXISTS ${stats_db_name}.result_gold STORED AS PARQUET as -select r.id, case when gold.is_gold=1 then true else false end as gold +select /*+ COALESCE(100) */ r.id, case when gold.is_gold=1 then true else false end as gold from ${stats_db_name}.result r left outer join ${stats_db_name}.indi_pub_gold_oa gold on gold.id=r.id; /*EOS*/ \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql index 1a7f34e96..85d90eaf1 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql @@ -1,7 +1,7 @@ set mapred.job.queue.name=analytics; /*EOS*/ create table ${observatory_db_name}.result_cc_licence stored as parquet as -select r.id, coalesce(rln.count, 0) > 0 as cc_licence +select /*+ COALESCE(100) */ r.id, coalesce(rln.count, 0) > 0 as cc_licence from ${stats_db_name}.result r left outer join ( select rl.id, sum(case when rl.type like 'CC%' then 1 else 0 end) as count @@ -11,7 +11,7 @@ from ${stats_db_name}.result r create table ${observatory_db_name}.result_affiliated_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -41,7 +41,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end create table ${observatory_db_name}.result_affiliated_year stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -71,7 +71,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -101,7 +101,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -132,7 +132,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -163,7 +163,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_organization stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -192,7 +192,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -221,7 +221,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_affiliated_funder stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -252,7 +252,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; /*EOS*/ create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -283,7 +283,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -314,7 +314,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_year stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -346,7 +346,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end create table ${observatory_db_name}.result_deposited_year_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -377,7 +377,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_datasource stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -408,7 +408,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -439,7 +439,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_organization stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -470,7 +470,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -501,7 +501,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; /*EOS*/ create table ${observatory_db_name}.result_deposited_funder stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, @@ -534,7 +534,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; /*EOS*/ create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as -select +select /*+ COALESCE(100) */ count(distinct r.id) as total, r.green, r.gold, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 9a3995a8f..98225af14 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -39,21 +39,21 @@ where dtrce.datainfo.deletedbyinference = false and dtrce.datainfo.invisible = f DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS -SELECT substr(d.id, 4) AS id, langs.languages AS language +SELECT /*+ COALESCE(100) */ substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_oids STORED AS PARQUET AS -SELECT substr(d.id, 4) AS id, oids.ids AS oid +SELECT /*+ COALESCE(100) */ substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource_organizations STORED AS PARQUET AS -SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization +SELECT /*+ COALESCE(100) */ substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = false and r.source like '20|%' and r.datainfo.invisible=false; /*EOS*/ @@ -62,10 +62,10 @@ WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = f DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; /*EOS*/ create table if not exists ${stats_db_name}.datasource_sources STORED AS PARQUET AS -select substr(d.id, 4) as id, substr(cf.key, 4) as datasource +select /*+ COALESCE(100) */ substr(d.id, 4) as id, substr(cf.key, 4) as datasource from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf where d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; /*EOS*/ CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS -SELECT /*+ COALESCE(100) */ datasource AS id, id AS result +SELECT datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; /*EOS*/ From e9686365a21dbcf4fe14e9c8ea211d4db62fe39d Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Wed, 3 Jul 2024 20:24:36 +0300 Subject: [PATCH 056/239] Improve performance of creating the "result_fos" table, by using a temp-table to cache data, which is requested multiple times. --- .../dhp/oa/graph/stats/oozie_app/scripts/step7.sql | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 0717d7897..bffd59ef1 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -125,14 +125,20 @@ UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; /*EOS*/ + +DROP TABLE IF EXISTS ${stats_db_name}.result_fos_base_tmp purge; /*EOS*/ + +create table ${stats_db_name}.result_fos_base_tmp stored as parquet as +select /*+ COALESCE(100) */ id, topic from ${stats_db_name}.result_topics where type='Fields of Science and Technology classification'; /*EOS*/ + DROP TABLE IF EXISTS ${stats_db_name}.result_fos purge; /*EOS*/ create table ${stats_db_name}.result_fos stored as parquet as with - lvl1 as (select id, topic from ${stats_db_name}.result_topics where topic like '__ %' and type='Fields of Science and Technology classification'), - lvl2 as (select id, topic from ${stats_db_name}.result_topics where topic like '____ %' and type='Fields of Science and Technology classification'), - lvl3 as (select id, topic from ${stats_db_name}.result_topics where topic like '______ %' and type='Fields of Science and Technology classification'), - lvl4 as (select id, topic from ${stats_db_name}.result_topics where topic like '________ %' and type='Fields of Science and Technology classification') + lvl1 as (select * from ${stats_db_name}.result_fos_base_tmp where topic like '__ %'), + lvl2 as (select * from ${stats_db_name}.result_fos_base_tmp where topic like '____ %'), + lvl3 as (select * from ${stats_db_name}.result_fos_base_tmp where topic like '______ %'), + lvl4 as (select * from ${stats_db_name}.result_fos_base_tmp where topic like '________ %') select /*+ COALESCE(100) */ lvl1.id, lvl1.topic as lvl1, lvl2.topic as lvl2, lvl3.topic as lvl3, lvl4.topic as lvl4 from lvl1 join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2) From 1180d78b712c8dd308eb38d9a2fd1928239eb71d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 4 Jul 2024 09:41:12 +0200 Subject: [PATCH 057/239] make entity level pids unique by pidType:pidValue --- .../main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index 7606a4d2f..74fd13ae7 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -328,7 +328,7 @@ public class MergeUtils { final T merged = mergeOafFields(original, enrich, trust); merged.setOriginalId(unionDistinctListOfString(merged.getOriginalId(), enrich.getOriginalId())); - merged.setPid(unionDistinctLists(merged.getPid(), enrich.getPid(), trust)); + merged.setPid(mergeLists(merged.getPid(), enrich.getPid(), trust, MergeUtils::spKeyExtractor, (p1, p2) -> p1)); merged.setDateofcollection(LocalDateTime.now().toString()); merged .setDateoftransformation( From ddd20e7f8edb69e5493e8c5af80d9931da08c582 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 4 Jul 2024 12:08:46 +0200 Subject: [PATCH 058/239] [Person]first implementation of the action set to include Person entity in the graph starting from the orcid data --- .../actionmanager/personentity/Coauthors.java | 18 +- .../actionmanager/personentity/Couples.java | 50 +- .../personentity/ExtractPerson.java | 621 ++++++++++-------- .../actionmanager/personentity/WorkList.java | 28 +- .../personentity/as_parameters.json | 13 +- .../actionmanager/personentity/job.properties | 2 + .../personentity/oozie_app/config-default.xml | 30 + .../personentity/oozie_app/workflow.xml | 111 ++++ .../actionmanager/person/CreatePersonAS.java | 150 ++++- 9 files changed, 683 insertions(+), 340 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java index a755c875f..f550178d7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java @@ -1,19 +1,19 @@ -package eu.dnetlib.dhp.actionmanager.personentity; -import eu.dnetlib.dhp.schema.oaf.Relation; +package eu.dnetlib.dhp.actionmanager.personentity; import java.io.Serializable; import java.util.ArrayList; +import eu.dnetlib.dhp.schema.oaf.Relation; public class Coauthors implements Serializable { - private ArrayList coauthors; + private ArrayList coauthors; - public ArrayList getCoauthors() { - return coauthors; - } + public ArrayList getCoauthors() { + return coauthors; + } - public void setCoauthors(ArrayList coauthors) { - this.coauthors = coauthors; - } + public void setCoauthors(ArrayList coauthors) { + this.coauthors = coauthors; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java index 65670b22e..d052b52b6 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Couples.java @@ -1,40 +1,40 @@ + package eu.dnetlib.dhp.actionmanager.personentity; +import java.io.Serializable; + import eu.dnetlib.dhp.schema.oaf.Person; import eu.dnetlib.dhp.schema.oaf.Relation; import scala.Tuple2; - -import java.io.Serializable; - public class Couples implements Serializable { - Person p ; - Relation r; + Person p; + Relation r; - public Couples() { + public Couples() { - } + } - public Person getP() { - return p; - } + public Person getP() { + return p; + } - public void setP(Person p) { - this.p = p; - } + public void setP(Person p) { + this.p = p; + } - public Relation getR() { - return r; - } + public Relation getR() { + return r; + } - public void setR(Relation r) { - this.r = r; - } + public void setR(Relation r) { + this.r = r; + } - public static Couples newInstance(Tuple2 couple){ - Couples c = new Couples(); - c.p = couple._1(); - c.r = couple._2(); - return c; - } + public static Couples newInstance(Tuple2 couple) { + Couples c = new Couples(); + c.p = couple._1(); + c.r = couple._2(); + return c; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 0bbda0343..b7d5f4367 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -1,27 +1,18 @@ + package eu.dnetlib.dhp.actionmanager.personentity; -import com.fasterxml.jackson.databind.ObjectMapper; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static org.apache.spark.sql.functions.*; + +import java.io.IOException; +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; -import eu.dnetlib.dhp.actionmanager.Constants; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.orcid.model.Author; -import eu.dnetlib.dhp.collection.orcid.model.Employment; -import eu.dnetlib.dhp.collection.orcid.model.Work; -import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Person; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import eu.dnetlib.dhp.schema.oaf.Pid; -import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; -import eu.dnetlib.dhp.schema.oaf.utils.PidType; -import eu.dnetlib.dhp.utils.DHPUtils; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; @@ -31,301 +22,399 @@ import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.spark_project.jetty.util.StringUtil; -import scala.Tuple2; -import static org.apache.spark.sql.functions.*; -import java.io.IOException; -import java.io.Serializable; -import java.util.*; -import java.util.stream.Collectors; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.actionmanager.Constants; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.orcid.model.Author; +import eu.dnetlib.dhp.collection.orcid.model.Employment; +import eu.dnetlib.dhp.collection.orcid.model.Work; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Person; +import eu.dnetlib.dhp.schema.oaf.Pid; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; +import eu.dnetlib.dhp.schema.oaf.utils.PidType; +import eu.dnetlib.dhp.utils.DHPUtils; +import scala.Tuple2; public class ExtractPerson implements Serializable { - private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class); + private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final String OPENAIRE_PREFIX = "openaire____"; - private static final String SEPARATOR = "::"; - private static final String orcidKey = "10|" + OPENAIRE_PREFIX + SEPARATOR + DHPUtils.md5(ModelConstants.ORCID.toLowerCase()); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String OPENAIRE_PREFIX = "openaire____"; + private static final String SEPARATOR = "::"; + private static final String orcidKey = "10|" + OPENAIRE_PREFIX + SEPARATOR + + DHPUtils.md5(ModelConstants.ORCID.toLowerCase()); - private static final String DOI_PREFIX = "50|doi_________::"; + private static final String DOI_PREFIX = "50|doi_________::"; - private static final String PMID_PREFIX = "50|pmid________::"; - private static final String ARXIV_PREFIX = "50|arXiv_______::"; + private static final String PMID_PREFIX = "50|pmid________::"; + private static final String ARXIV_PREFIX = "50|arXiv_______::"; - private static final String PMCID_PREFIX = "50|pmcid_______::"; - private static final String ROR_PREFIX = "20|ror_________::"; - private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; - public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; - public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + private static final String PMCID_PREFIX = "50|pmcid_______::"; + private static final String ROR_PREFIX = "20|ror_________::"; + private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; + public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; + public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + public static void main(final String[] args) throws IOException, ParseException { - public static void main(final String[] args) throws IOException, ParseException { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + ExtractPerson.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json")))); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - Objects - .requireNonNull( - ExtractPerson.class - .getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json")))); + parser.parseArgument(args); - parser.parseArgument(args); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + final String inputPath = parser.get("inputPath"); + log.info("inputPath {}", inputPath); - final String inputPath = parser.get("inputPath"); - log.info("inputPath {}", inputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}", outputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}", outputPath); + final String workingDir = parser.get("workingDir"); + log.info("workingDir {}", workingDir); - final String workingDir = parser.get("workingDir"); - log.info("workingDir {}", workingDir); + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration()); + createActionSet(spark, inputPath, outputPath, workingDir); + }); - SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> createActionSet(spark, inputPath, outputPath, workingDir)); + } - } + private static void createActionSet(SparkSession spark, String inputPath, String outputPath, String workingDir) { - private static void createActionSet(SparkSession spark, String inputPath, String outputPath, String workingDir) { + Dataset authors = spark + .read() + .parquet(inputPath + "Authors") + .as(Encoders.bean(Author.class)); - Dataset authors = spark - .read() - .parquet(inputPath + "Authors").as(Encoders.bean(Author.class)); + Dataset works = spark + .read() + .parquet(inputPath + "Works") + .as(Encoders.bean(Work.class)) + .filter( + (FilterFunction) w -> Optional.ofNullable(w.getPids()).isPresent() && + w + .getPids() + .stream() + .anyMatch( + p -> p.getSchema().equalsIgnoreCase("doi") || + p.getSchema().equalsIgnoreCase("pmc") || + p.getSchema().equalsIgnoreCase("pmid") || + p.getSchema().equalsIgnoreCase("arxiv"))); - Dataset works = spark - .read() - .parquet(inputPath + "Works") - .as(Encoders.bean(Work.class)) - .filter((FilterFunction) w -> Optional.ofNullable(w.getPids()).isPresent() && - w.getPids().stream().anyMatch(p->p.getSchema().equalsIgnoreCase("doi") || - p.getSchema().equalsIgnoreCase("pmc") || - p.getSchema().equalsIgnoreCase("pmid") || - p.getSchema().equalsIgnoreCase("arxiv"))); + Dataset employmentDataset = spark + .read() + .parquet(inputPath + "Employments") + .as(Encoders.bean(Employment.class)); - Dataset employmentDataset = spark - .read() - .parquet(inputPath + "Employments") - .as(Encoders.bean(Employment.class)); + Dataset peopleToMap = authors + .joinWith(works, authors.col("orcid").equalTo(works.col("orcid"))) + .map((MapFunction, Author>) t2 -> t2._1(), Encoders.bean(Author.class)) + .groupByKey((MapFunction) a -> a.getOrcid(), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Author.class)); + Dataset employment = employmentDataset + .joinWith(peopleToMap, employmentDataset.col("orcid").equalTo(peopleToMap.col("orcid"))) + .map((MapFunction, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class)); - Dataset peopleToMap = authors.joinWith(works, authors.col("orcid").equalTo(works.col("orcid"))) - .map((MapFunction, Author>) t2 -> t2._1(), Encoders.bean(Author.class)) - .groupByKey((MapFunction) a -> a.getOrcid(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Author.class)); + Dataset people; + peopleToMap.map((MapFunction) op -> { + Person person = new Person(); + person.setId(DHPUtils.generateIdentifier(op.getOrcid(), PERSON_PREFIX)); + person + .setBiography( + Optional + .ofNullable(op.getBiography()) + .orElse("")); + KeyValue kv = OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS); + kv.setDataInfo(null); + person.setCollectedfrom(Arrays.asList(kv)); + person + .setAlternativeNames( + Optional + .ofNullable(op.getOtherNames()) - Dataset employment = employmentDataset.joinWith(peopleToMap, employmentDataset.col("orcid").equalTo(peopleToMap.col("orcid"))) - .map((MapFunction, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class)); + .orElse(new ArrayList<>())); + person + .setFamilyName( + Optional + .ofNullable(op.getFamilyName()) - peopleToMap.show(false); + .orElse("")); + person + .setGivenName( + Optional + .ofNullable(op.getGivenName()) - Dataset people; - people = peopleToMap.map((MapFunction) op -> { - Person person = new Person(); - person.setId(DHPUtils.generateIdentifier(op.getOrcid(), PERSON_PREFIX)); - person.setBiography(Optional.ofNullable(op.getBiography()) + .orElse("")); + person + .setPid( + Optional + .ofNullable(op.getOtherPids()) + .map( + v -> v + .stream() + .map(p -> Pid.newInstance(p.getSchema(), p.getValue())) + .collect(Collectors.toList())) + .orElse(new ArrayList<>())); + person.getPid().add(Pid.newInstance(ModelConstants.ORCID, op.getOrcid())); + person.setDateofcollection(op.getLastModifiedDate()); + person.setOriginalId(Arrays.asList(op.getOrcid())); + return person; + }, Encoders.bean(Person.class)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(workingDir + "/people"); - .orElse("")); - KeyValue kv = OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS); - kv.setDataInfo(null); - person.setCollectedfrom(Arrays.asList(kv)); - person.setAlternativeNames(Optional.ofNullable(op.getOtherNames()) + works + .flatMap( + (FlatMapFunction) ExtractPerson::getAuthorshipRelationIterator, + Encoders.bean(Relation.class)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(workingDir + "/authorship"); - .orElse(new ArrayList<>())); - person.setFamilyName(Optional.ofNullable(op.getFamilyName()) + works + .flatMap((FlatMapFunction>) w -> { + List> lista = new ArrayList<>(); + w.getPids().stream().forEach(p -> { + if (p.getSchema().equalsIgnoreCase("doi") || p.getSchema().equalsIgnoreCase("pmc") + || p.getSchema().equalsIgnoreCase("pmid") || p.getSchema().equalsIgnoreCase("arxiv")) + lista.add(new Tuple2<>(p.getValue(), w.getOrcid())); + }); + return lista.iterator(); + }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())) + .groupByKey((MapFunction, String>) Tuple2::_1, Encoders.STRING()) + .mapGroups( + (MapGroupsFunction, Coauthors>) (k, it) -> extractCoAuthors(it), + Encoders.bean(Coauthors.class)) + .flatMap( + (FlatMapFunction) c -> c.getCoauthors().iterator(), Encoders.bean(Relation.class)) + .groupByKey((MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Relation.class)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(workingDir + "/coauthorship"); - .orElse("")); - person.setGivenName(Optional.ofNullable(op.getGivenName()) + employment + .filter((FilterFunction) e -> Optional.ofNullable(e.getAffiliationId()).isPresent()) + .filter((FilterFunction) e -> e.getAffiliationId().getSchema().equalsIgnoreCase("ror")) + .map( + (MapFunction) ExtractPerson::getAffiliationRelation, + Encoders.bean(Relation.class)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(workingDir + "/affiliation"); - .orElse("")); - person.setPid(Optional.ofNullable(op.getOtherPids()) - .map(v -> v.stream().map(p -> Pid.newInstance(p.getSchema(), p.getValue())).collect(Collectors.toList())) - .orElse(new ArrayList<>()) - ); - person.getPid().add(Pid.newInstance(ModelConstants.ORCID, op.getOrcid())); - person.setDateofcollection(op.getLastModifiedDate()); - person.setOriginalId(Arrays.asList(op.getOrcid())); - return person; - }, Encoders.bean(Person.class)); + spark + .read() + .json(workingDir + "/people") + .as(Encoders.bean(Person.class)) + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .union( + getRelations(spark, workingDir + "/authorship").toJavaRDD().map(r -> new AtomicAction(r.getClass(), r))) + .union( + getRelations(spark, workingDir + "/coauthorship") + .toJavaRDD() + .map(r -> new AtomicAction(r.getClass(), r))) + .union( + getRelations(spark, workingDir + "/affiliation") + .toJavaRDD() + .map(r -> new AtomicAction(r.getClass(), r))) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); + } + private static Dataset getRelations(SparkSession spark, String path) { + return spark.read().json(path).as(Encoders.bean(Relation.class)); + } - people.show(false); + private static Coauthors extractCoAuthors(Iterator> it) { + Coauthors coauth = new Coauthors(); + ArrayList ret = new ArrayList<>(); + List coauthors = new ArrayList<>(); + while (it.hasNext()) + coauthors.add(it.next()._2()); + for (int i = 0; i < coauthors.size() - 1; i++) + for (int j = i + 1; j < coauthors.size(); j++) + ret.addAll(getCoAuthorshipRelations(coauthors.get(i), coauthors.get(j))); + coauth.setCoauthors(ret); - Dataset authorship; - authorship = works - .flatMap((FlatMapFunction) ExtractPerson::getAuthorshipRelationIterator - , Encoders.bean(Relation.class)); + return coauth; + } + private static Relation getAffiliationRelation(Employment row) { + String source = PERSON_PREFIX + IdentifierFactory.md5(row.getOrcid()); + String target = ROR_PREFIX + + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAffiliationId().getValue())); + List properties = new ArrayList<>(); - authorship.show(false); + Relation relation = OafMapperUtils + .getRelation( + source, target, ModelConstants.ORG_PERSON_RELTYPE, ModelConstants.ORG_PERSON_SUBRELTYPE, + ModelConstants.ORG_PERSON_PARTICIPATES, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"), + null); + if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())) { + KeyValue kv = new KeyValue(); + kv.setKey("startDate"); + kv.setValue(row.getStartDate()); + properties.add(kv); + } + if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtil.isNotBlank(row.getEndDate())) { + KeyValue kv = new KeyValue(); + kv.setKey("endDate"); + kv.setValue(row.getEndDate()); + properties.add(kv); + } - Dataset coauthorship = works - .flatMap((FlatMapFunction>) w -> { - List> lista = new ArrayList<>(); - w.getPids().stream().forEach(p -> { - if (p.getSchema().equalsIgnoreCase("doi") || p.getSchema().equalsIgnoreCase("pmc") || p.getSchema().equalsIgnoreCase("pmid") || p.getSchema().equalsIgnoreCase("arxiv")) - lista.add(new Tuple2<>(p.getValue(), w.getOrcid())); - }); - return lista.iterator(); - }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())) - .groupByKey((MapFunction, String>) Tuple2::_1, Encoders.STRING()) - .mapGroups((MapGroupsFunction, Coauthors>) (k, it) -> - extractCoAuthors(it), Encoders.bean(Coauthors.class)) - .flatMap((FlatMapFunction) c -> c.getCoauthors().iterator(), Encoders.bean(Relation.class)) - .groupByKey((MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Relation.class)); + if (properties.size() > 0) + relation.setProperties(properties); + return relation; - coauthorship.show(false); - Dataset affiliation = employment - .filter((FilterFunction) e -> Optional.ofNullable(e.getAffiliationId()).isPresent()) - .filter((FilterFunction) e -> e.getAffiliationId().getSchema().equalsIgnoreCase("ror")) - .map((MapFunction) ExtractPerson::getAffiliationRelation - , Encoders.bean(Relation.class)); + } - affiliation.show(false); + private static Collection getCoAuthorshipRelations(String orcid1, String orcid2) { + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid1); + String target = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid2); - people.toJavaRDD() - .map(p -> new AtomicAction(p.getClass(), p)) - .union(authorship.toJavaRDD().map(r-> new AtomicAction(r.getClass(),r))) - .union(coauthorship.toJavaRDD().map(r-> new AtomicAction(r.getClass(),r))) - .union(affiliation.toJavaRDD().map(r->new AtomicAction(r.getClass(),r))) - .mapToPair( - aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), - new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile( - outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);//, GzipCodec.class); - } + return Arrays + .asList( + OafMapperUtils + .getRelation( + source, target, ModelConstants.PERSON_PERSON_RELTYPE, + ModelConstants.PERSON_PERSON_SUBRELTYPE, + ModelConstants.PERSON_PERSON_HASCOAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"), + null), + OafMapperUtils + .getRelation( + target, source, ModelConstants.PERSON_PERSON_RELTYPE, + ModelConstants.PERSON_PERSON_SUBRELTYPE, + ModelConstants.PERSON_PERSON_HASCOAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"), + null)); - private static Coauthors extractCoAuthors(Iterator> it) { - Coauthors coauth = new Coauthors(); - ArrayList ret = new ArrayList<>(); - List coauthors = new ArrayList<>(); - while(it.hasNext()) - coauthors.add(it.next()._2()); - for (int i = 0; i < coauthors.size() -1; i++ ) - for(int j = i + 1; j < coauthors.size(); j++) - ret.addAll(getCoAuthorshipRelations(coauthors.get(i), coauthors.get(j))); + } - coauth.setCoauthors(ret); + private static @NotNull Iterator getAuthorshipRelationIterator(Work w) { - return coauth; - } + if (Optional.ofNullable(w.getPids()).isPresent()) + return w + .getPids() + .stream() + .map(pid -> getRelation(w.getOrcid(), pid)) + .filter(Objects::nonNull) + .collect(Collectors.toList()) + .iterator(); + List ret = new ArrayList<>(); + return ret.iterator(); + } - private static Relation getAffiliationRelation(Employment row) { - String source = PERSON_PREFIX + IdentifierFactory.md5(row.getOrcid()); - String target = ROR_PREFIX - + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAffiliationId().getValue())); - List properties = new ArrayList<>() ; + private static Relation getRelation(String orcid, eu.dnetlib.dhp.collection.orcid.model.Pid pid) { + String target; + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); + switch (pid.getSchema()) { + case "doi": + target = DOI_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), pid.getValue())); + break; + case "pmid": + target = PMID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), pid.getValue())); + break; + case "arxiv": + target = ARXIV_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), pid.getValue())); + break; + case "pmcid": + target = PMCID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), pid.getValue())); + break; - Relation relation = - OafMapperUtils.getRelation(source, target, ModelConstants.ORG_PERSON_RELTYPE, ModelConstants.ORG_PERSON_SUBRELTYPE, ModelConstants.ORG_PERSON_PARTICIPATES , - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils.dataInfo(false, null, false, false, - OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), - null); + default: + return null; + } - if(Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())){ - KeyValue kv = new KeyValue(); - kv.setKey("startDate"); - kv.setValue(row.getStartDate()); - properties.add(kv); - } - if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtil.isNotBlank(row.getEndDate())) { - KeyValue kv = new KeyValue(); - kv.setKey("endDate"); - kv.setValue(row.getEndDate()); - properties.add(kv); - } - - if (properties.size() > 0) - relation.setProperties(properties); - return relation; - - - } - - private static Collection getCoAuthorshipRelations(String orcid1, String orcid2) { - String source = PERSON_PREFIX + IdentifierFactory.md5(orcid1); - String target = PERSON_PREFIX + IdentifierFactory.md5(orcid2); - - return Arrays.asList(OafMapperUtils.getRelation(source, target,ModelConstants.PERSON_PERSON_RELTYPE, - ModelConstants.PERSON_PERSON_SUBRELTYPE, - ModelConstants.PERSON_PERSON_HASCOAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils.dataInfo(false, null, false, false, - OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), - null), - OafMapperUtils.getRelation(target, source,ModelConstants.PERSON_PERSON_RELTYPE, - ModelConstants.PERSON_PERSON_SUBRELTYPE, - ModelConstants.PERSON_PERSON_HASCOAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils.dataInfo(false, null, false, false, - OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), - null)); - - } - - private static @NotNull Iterator getAuthorshipRelationIterator(Work w) { - - if(Optional.ofNullable(w.getPids()).isPresent()) - return w.getPids() - .stream() - .map(pid -> getRelation(w.getOrcid(), pid)) - .filter(Objects::nonNull).collect(Collectors.toList()).iterator(); - List ret = new ArrayList<>(); - return ret.iterator(); - } - - - private static Relation getRelation(String orcid, eu.dnetlib.dhp.collection.orcid.model.Pid pid){ - String target ; - String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); - switch (pid.getSchema()){ - case "doi": - target = DOI_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), pid.getValue())); - break; - case "pmid": - target = PMID_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), pid.getValue())); - break; - case "arxiv": - target = ARXIV_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), pid.getValue())); - break; - case "pmcid": - target = PMCID_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), pid.getValue())); - break; - - default: - return null; - } - - - return OafMapperUtils.getRelation(source, target,ModelConstants.RESULT_PERSON_RELTYPE, - ModelConstants.RESULT_PERSON_SUBRELTYPE, - ModelConstants.RESULT_PERSON_HASAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils.dataInfo(false, null, false, false, - OafMapperUtils.qualifier(ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), - null); - } + return OafMapperUtils + .getRelation( + source, target, ModelConstants.RESULT_PERSON_RELTYPE, + ModelConstants.RESULT_PERSON_SUBRELTYPE, + ModelConstants.RESULT_PERSON_HASAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"), + null); + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java index 71d2b491a..92842bfcf 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/WorkList.java @@ -1,23 +1,25 @@ -package eu.dnetlib.dhp.actionmanager.personentity; -import eu.dnetlib.dhp.collection.orcid.model.Work; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +package eu.dnetlib.dhp.actionmanager.personentity; import java.io.Serializable; import java.util.ArrayList; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; + +import eu.dnetlib.dhp.collection.orcid.model.Work; + public class WorkList implements Serializable { - private ArrayList workArrayList; + private ArrayList workArrayList; - public ArrayList getWorkArrayList() { - return workArrayList; - } + public ArrayList getWorkArrayList() { + return workArrayList; + } - public void setWorkArrayList(ArrayList workArrayList) { - this.workArrayList = workArrayList; - } + public void setWorkArrayList(ArrayList workArrayList) { + this.workArrayList = workArrayList; + } - public WorkList() { - workArrayList = new ArrayList<>(); - } + public WorkList() { + workArrayList = new ArrayList<>(); + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json index 3310f16e4..5175552e7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json @@ -16,11 +16,10 @@ "paramLongName": "isSparkSessionManaged", "paramDescription": "the hdfs name node", "paramRequired": false - }, - { - "paramName": "wd", - "paramLongName": "workingDir", - "paramDescription": "the hdfs name node", - "paramRequired": false - } + }, { + "paramName": "wd", + "paramLongName": "workingDir", + "paramDescription": "the hdfs name node", + "paramRequired": false +} ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties new file mode 100644 index 000000000..d2269718c --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties @@ -0,0 +1,2 @@ +inputPath=/data/orcid_2023/tables/ +outputPath=/user/miriam.baglioni/peopleAS \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/config-default.xml new file mode 100644 index 000000000..d262cb6e0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/config-default.xml @@ -0,0 +1,30 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hiveJdbcUrl + jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 + + + hiveDbName + openaire + + + oozie.launcher.mapreduce.user.classpath.first + true + + diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml new file mode 100644 index 000000000..166e7bb9c --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml @@ -0,0 +1,111 @@ + + + + + inputPath + inputPath + + + outputPath + the path where to store the actionset + + + 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())}] + + + + + + + + + + + + + + + + yarn + cluster + Produces the ActionSet for Person entity and relevant relations + eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson + dhp-aggregation-${projectVersion}.jar + + --executor-cores=4 + --executor-memory=4G + --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=5G + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=15000 + + --inputPath${inputPath} + --outputPath${outputPath} + --workingDir${workingDir} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java index a4d56ba2e..2e7b21010 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java @@ -1,15 +1,13 @@ package eu.dnetlib.dhp.actionmanager.person; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob; -import eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson; -import eu.dnetlib.dhp.collection.orcid.model.Author; -import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Optional; + import org.apache.commons.io.FileUtils; import org.apache.hadoop.io.Text; import org.apache.spark.SparkConf; @@ -27,12 +25,18 @@ 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.util.Optional; +import com.fasterxml.jackson.databind.ObjectMapper; -import static org.junit.jupiter.api.Assertions.assertEquals; +import eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob; +import eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson; +import eu.dnetlib.dhp.collection.orcid.model.Author; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Person; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.utils.DHPUtils; public class CreatePersonAS { @@ -57,7 +61,7 @@ public class CreatePersonAS { conf.set("spark.driver.host", "localhost"); conf.set("hive.metastore.local", "true"); conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.codegen.wholeStage","false"); + conf.set("spark.sql.codegen.wholeStage", "false"); conf.set("spark.sql.warehouse.dir", workingDir.toString()); conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); @@ -92,7 +96,6 @@ public class CreatePersonAS { // .mode(SaveMode.Overwrite) // .parquet(workingDir.toString() + "AuthorsSubset"); - ExtractPerson .main( new String[] { @@ -102,13 +105,120 @@ public class CreatePersonAS { inputPath, "-outputPath", workingDir.toString() + "/actionSet1", - "-workingDir", - workingDir.toString() + "/working" + "-workingDir", + workingDir.toString() + "/working" }); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + JavaRDD relations = sc + .sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class) + .filter(v -> "eu.dnetlib.dhp.schema.oaf.Relation".equalsIgnoreCase(v._1().toString())) + .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) + .map(aa -> ((Relation) aa.getPayload())); +// + JavaRDD people = sc + .sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class) + .filter(v -> "eu.dnetlib.dhp.schema.oaf.Person".equalsIgnoreCase(v._1().toString())) + .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) + .map(aa -> ((Person) aa.getPayload())); +// + Assertions.assertEquals(7, people.count()); + Assertions + .assertEquals( + "Paulo", + people + .filter( + p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034"))) + .first() + .getGivenName()); + Assertions + .assertEquals( + "Tavares", + people + .filter( + p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034"))) + .first() + .getFamilyName()); + Assertions + .assertEquals( + 4, + people + .filter( + p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034"))) + .first() + .getAlternativeNames() + .size()); + Assertions + .assertEquals( + 4, + people + .filter( + p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034"))) + .first() + .getPid() + .size()); + Assertions + .assertTrue( + people + .filter( + p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034"))) + .first() + .getPid() + .stream() + .anyMatch( + p -> p.getSchema().equalsIgnoreCase("Scopus Author ID") + && p.getValue().equalsIgnoreCase("15119405200"))); + + Assertions + .assertEquals( + 16, + relations + .filter(r -> r.getRelClass().equalsIgnoreCase(ModelConstants.RESULT_PERSON_HASAUTHORED)) + .count()); + Assertions + .assertEquals( + 14, + relations + .filter(r -> r.getRelClass().equalsIgnoreCase(ModelConstants.PERSON_PERSON_HASCOAUTHORED)) + .count()); + Assertions + .assertEquals( + 3, + relations + .filter( + r -> r.getSource().equalsIgnoreCase("30|orcid_______::" + DHPUtils.md5("0000-0001-6291-9619")) + && r.getRelClass().equalsIgnoreCase(ModelConstants.RESULT_PERSON_HASAUTHORED)) + .count()); + Assertions + .assertEquals( + 2, + relations + .filter( + r -> r.getSource().equalsIgnoreCase("30|orcid_______::" + DHPUtils.md5("0000-0001-6291-9619")) + && r.getRelClass().equalsIgnoreCase(ModelConstants.RESULT_PERSON_HASAUTHORED) + && r.getTarget().startsWith("50|doi")) + .count()); + Assertions + .assertEquals( + 1, + relations + .filter( + r -> r.getSource().equalsIgnoreCase("30|orcid_______::" + DHPUtils.md5("0000-0001-6291-9619")) + && r.getRelClass().equalsIgnoreCase(ModelConstants.RESULT_PERSON_HASAUTHORED) + && r.getTarget().startsWith("50|arXiv")) + .count()); + + Assertions + .assertEquals( + 1, + relations + .filter( + r -> r.getSource().equalsIgnoreCase("30|orcid_______::" + DHPUtils.md5("0000-0001-6291-9619")) + && r.getRelClass().equalsIgnoreCase(ModelConstants.PERSON_PERSON_HASCOAUTHORED)) + .count()); + Assertions.assertEquals(33, relations.count()); } - - } +} From 814e650e12c15f3a74449b0c79f70193e12327f7 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 4 Jul 2024 12:24:28 +0200 Subject: [PATCH 059/239] [Irish Tender]changed the irish.json file according to comments #26, #29, and #34 for 9635 --- .../dhp/collection/crossref/irish_funder.json | 46 +++---------------- 1 file changed, 7 insertions(+), 39 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json index e50dc2dee..a49e4e300 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/crossref/irish_funder.json @@ -1,10 +1,5 @@ [ - { - "id": "100007630", - "uri": "http://dx.doi.org/10.13039/100007630", - "name": "College of Engineering and Informatics, National University of Ireland, Galway", - "synonym": [] - }, + { "id": "100007731", "uri": "http://dx.doi.org/10.13039/100007731", @@ -432,13 +427,13 @@ "id": "501100001634", "uri": "http://dx.doi.org/10.13039/501100001634", "name": "University of Galway", - "synonym": [] + "synonym": ["501100019905", "100007630", "501100020570", "501100023852"] }, { "id": "501100001635", "uri": "http://dx.doi.org/10.13039/501100001635", "name": "University of Limerick", - "synonym": [] + "synonym": ["501100014531"] }, { "id": "501100001636", @@ -468,7 +463,7 @@ "id": "501100002736", "uri": "http://dx.doi.org/10.13039/501100002736", "name": "Covidien", - "synonym": [] + "synonym": ["501100003956"] }, { "id": "501100002755", @@ -518,12 +513,6 @@ "name": "Irish Institute of Clinical Neuroscience", "synonym": [] }, - { - "id": "501100003956", - "uri": "http://dx.doi.org/10.13039/501100003956", - "name": "Aspect Medical Systems", - "synonym": [] - }, { "id": "501100004162", "uri": "http://dx.doi.org/10.13039/501100004162", @@ -644,12 +633,7 @@ "name": "Irish Centre for High-End Computing", "synonym": [] }, - { - "id": "501100019905", - "uri": "http://dx.doi.org/10.13039/501100019905", - "name": "Galway University Foundation", - "synonym": [] - }, + { "id": "501100020036", "uri": "http://dx.doi.org/10.13039/501100020036", @@ -824,12 +808,7 @@ "name": "Energy Policy Research Centre, Economic and Social Research Institute", "synonym": [] }, - { - "id": "501100014531", - "uri": "http://dx.doi.org/10.13039/501100014531", - "name": "Physical Education and Sport Sciences Department, University of Limerick", - "synonym": [] - }, + { "id": "501100014745", "uri": "http://dx.doi.org/10.13039/501100014745", @@ -842,22 +821,11 @@ "name": "ADAPT - Centre for Digital Content Technology", "synonym": [] }, - { - "id": "501100020570", - "uri": "http://dx.doi.org/10.13039/501100020570", - "name": "College of Medicine, Nursing and Health Sciences, National University of Ireland, Galway", - "synonym": [] - }, + { "id": "501100020871", "uri": "http://dx.doi.org/10.13039/501100020871", "name": "Bernal Institute, University of Limerick", "synonym": [] - }, - { - "id": "501100023852", - "uri": "http://dx.doi.org/10.13039/501100023852", - "name": "Moore Institute for Research in the Humanities and Social Studies, University of Galway", - "synonym": [] } ] \ No newline at end of file From c4658350613a3d124a3119a73efff7e33d8df443 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 9 Jul 2024 12:29:55 +0200 Subject: [PATCH 060/239] [Person]new implementation for the extraction of the coAuthorship relations --- .../personentity/CoAuthorshipIterator.java | 80 +++++++++++++++++++ .../actionmanager/personentity/Coauthors.java | 7 +- .../personentity/ExtractPerson.java | 37 ++++++--- 3 files changed, 108 insertions(+), 16 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java new file mode 100644 index 000000000..76e4c4851 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java @@ -0,0 +1,80 @@ + +package eu.dnetlib.dhp.actionmanager.personentity; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.Person; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.utils.DHPUtils; + +public class CoAuthorshipIterator implements Iterator { + private int firstIndex; + private int secondIndex; + private boolean firstRelation; + private List authors; + private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______::"; + private static final String OPENAIRE_PREFIX = "openaire____"; + private static final String SEPARATOR = "::"; + private static final String ORCID_KEY = "10|" + OPENAIRE_PREFIX + SEPARATOR + + DHPUtils.md5(ModelConstants.ORCID.toLowerCase()); + public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; + public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + + @Override + public boolean hasNext() { + return firstIndex < authors.size() - 1; + } + + @Override + public Relation next() { + Relation rel = null; + if (firstRelation) { + rel = getRelation(authors.get(firstIndex), authors.get(secondIndex)); + firstRelation = Boolean.FALSE; + } else { + rel = getRelation(authors.get(secondIndex), authors.get(firstIndex)); + firstRelation = Boolean.TRUE; + secondIndex += 1; + if (secondIndex >= authors.size()) { + firstIndex += 1; + secondIndex = firstIndex + 1; + } + } + + return rel; + } + + public CoAuthorshipIterator(List authors) { + this.authors = authors; + this.firstIndex = 0; + this.secondIndex = 1; + this.firstRelation = Boolean.TRUE; + + } + + private Relation getRelation(String orcid1, String orcid2) { + String source = PERSON_PREFIX + IdentifierFactory.md5(orcid1); + String target = PERSON_PREFIX + IdentifierFactory.md5(orcid2); + return OafMapperUtils + .getRelation( + source, target, ModelConstants.PERSON_PERSON_RELTYPE, + ModelConstants.PERSON_PERSON_SUBRELTYPE, + ModelConstants.PERSON_PERSON_HASCOAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(ORCID_KEY, ModelConstants.ORCID_DS)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"), + null); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java index f550178d7..17f46d5c7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java @@ -3,17 +3,18 @@ package eu.dnetlib.dhp.actionmanager.personentity; import java.io.Serializable; import java.util.ArrayList; +import java.util.List; import eu.dnetlib.dhp.schema.oaf.Relation; public class Coauthors implements Serializable { - private ArrayList coauthors; + private List coauthors; - public ArrayList getCoauthors() { + public List getCoauthors() { return coauthors; } - public void setCoauthors(ArrayList coauthors) { + public void setCoauthors(List coauthors) { this.coauthors = coauthors; } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index b7d5f4367..064fb41a1 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -26,6 +26,7 @@ import org.spark_project.jetty.util.StringUtil; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.actionmanager.Constants; +import eu.dnetlib.dhp.actionmanager.transformativeagreement.model.TransformativeAgreementModel; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.collection.orcid.model.Author; import eu.dnetlib.dhp.collection.orcid.model.Employment; @@ -202,7 +203,7 @@ public class ExtractPerson implements Serializable { .mode(SaveMode.Overwrite) .json(workingDir + "/authorship"); - works + Dataset coauthorship = works .flatMap((FlatMapFunction>) w -> { List> lista = new ArrayList<>(); w.getPids().stream().forEach(p -> { @@ -217,10 +218,13 @@ public class ExtractPerson implements Serializable { (MapGroupsFunction, Coauthors>) (k, it) -> extractCoAuthors(it), Encoders.bean(Coauthors.class)) .flatMap( - (FlatMapFunction) c -> c.getCoauthors().iterator(), Encoders.bean(Relation.class)) + (FlatMapFunction) c -> new CoAuthorshipIterator(c.getCoauthors()), + Encoders.bean(Relation.class)) .groupByKey((MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) .mapGroups( - (MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Relation.class)) + (MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Relation.class)); + + coauthorship .write() .option("compression", "gzip") .mode(SaveMode.Overwrite) @@ -237,10 +241,16 @@ public class ExtractPerson implements Serializable { .mode(SaveMode.Overwrite) .json(workingDir + "/affiliation"); - spark + people = spark .read() - .json(workingDir + "/people") - .as(Encoders.bean(Person.class)) + .textFile(workingDir + "/people") + .map( + (MapFunction) value -> OBJECT_MAPPER + .readValue(value, Person.class), + Encoders.bean(Person.class)); + + people.show(false); + people .toJavaRDD() .map(p -> new AtomicAction(p.getClass(), p)) .union( @@ -261,20 +271,21 @@ public class ExtractPerson implements Serializable { } private static Dataset getRelations(SparkSession spark, String path) { - return spark.read().json(path).as(Encoders.bean(Relation.class)); + return spark + .read() + .textFile(path) + .map( + (MapFunction) value -> OBJECT_MAPPER + .readValue(value, Relation.class), + Encoders.bean(Relation.class));// spark.read().json(path).as(Encoders.bean(Relation.class)); } private static Coauthors extractCoAuthors(Iterator> it) { Coauthors coauth = new Coauthors(); - ArrayList ret = new ArrayList<>(); List coauthors = new ArrayList<>(); while (it.hasNext()) coauthors.add(it.next()._2()); - for (int i = 0; i < coauthors.size() - 1; i++) - for (int j = i + 1; j < coauthors.size(); j++) - ret.addAll(getCoAuthorshipRelations(coauthors.get(i), coauthors.get(j))); - - coauth.setCoauthors(ret); + coauth.setCoauthors(coauthors); return coauth; } From 2a36ccb997c0ba5ebdca757c1d88ab3cfc7f7249 Mon Sep 17 00:00:00 2001 From: miconis Date: Tue, 9 Jul 2024 16:58:10 +0200 Subject: [PATCH 061/239] optimization of normalization stage in openorgs workflow, implementation of new comparators replacing older versions, openorgs configuration update, addition of inference flag in model definition, new test classes --- ...ustering.java => LegalnameClustering.java} | 31 ++- .../pace/common/AbstractPaceFunctions.java | 94 +++++++ .../java/eu/dnetlib/pace/model/FieldDef.java | 20 ++ .../eu/dnetlib/pace/model/SparkModel.scala | 22 ++ .../java/eu/dnetlib/pace/tree/CityMatch.java | 48 ---- .../java/eu/dnetlib/pace/tree/CodeMatch.java | 51 ++++ .../eu/dnetlib/pace/tree/CountryMatch.java | 61 +++-- .../pace/tree/JaroWinklerLegalname.java | 59 +++++ .../pace/tree/JaroWinklerNormalizedName.java | 74 ------ .../eu/dnetlib/pace/tree/KeywordMatch.java | 50 ---- .../pace/tree/support/TreeNodeDef.java | 2 +- .../pace/tree/support/TreeNodeStats.java | 36 ++- .../pace/tree/support/TreeProcessor.java | 11 +- .../eu/dnetlib/pace/config/country_map.csv | 239 ++++++++++++++++++ .../clustering/ClusteringFunctionTest.java | 38 +-- .../dnetlib/pace/common/PaceFunctionTest.java | 43 ++++ .../pace/comparators/ComparatorTest.java | 70 ++--- .../dhp/oa/dedup/DecisionTreeTest.java | 83 ++++++ .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 14 +- .../dhp/oa/dedup/SparkOpenorgsDedupTest.java | 9 +- .../dhp/oa/dedup/jpath/JsonPathTest.java | 25 ++ .../dnetlib/dhp/dedup/conf/org.curr.conf.json | 187 +++----------- .../oa/dedup/jpath/organization_example1.json | 1 + 23 files changed, 807 insertions(+), 461 deletions(-) rename dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/{KeywordsClustering.java => LegalnameClustering.java} (54%) delete mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CityMatch.java create mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java create mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerLegalname.java delete mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerNormalizedName.java delete mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/KeywordMatch.java create mode 100644 dhp-pace-core/src/main/resources/eu/dnetlib/pace/config/country_map.csv create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/oa/dedup/jpath/organization_example1.json diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/KeywordsClustering.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/LegalnameClustering.java similarity index 54% rename from dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/KeywordsClustering.java rename to dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/LegalnameClustering.java index fdd8d1fb1..8a76a4bc3 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/KeywordsClustering.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/LegalnameClustering.java @@ -2,31 +2,41 @@ package eu.dnetlib.pace.clustering; import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import eu.dnetlib.pace.config.Config; -@ClusteringClass("keywordsclustering") -public class KeywordsClustering extends AbstractClusteringFunction { +@ClusteringClass("legalnameclustering") +public class LegalnameClustering extends AbstractClusteringFunction { - public KeywordsClustering(Map params) { + private static final Pattern CITY_CODE_PATTERN = Pattern.compile("city::\\d+"); + private static final Pattern KEYWORD_CODE_PATTERN = Pattern.compile("key::\\d+"); + + public LegalnameClustering(Map params) { super(params); } + public Set getRegexList(String input, Pattern codeRegex) { + Matcher matcher = codeRegex.matcher(input); + Set cities = new HashSet<>(); + while (matcher.find()) { + cities.add(matcher.group()); + } + return cities; + } + @Override protected Collection doApply(final Config conf, String s) { - // takes city codes and keywords codes without duplicates - Set keywords = getKeywords(s, conf.translationMap(), paramOrDefault("windowSize", 4)); - Set cities = getCities(s, paramOrDefault("windowSize", 4)); - // list of combination to return as result final Collection combinations = new LinkedHashSet(); - for (String keyword : keywordsToCodes(keywords, conf.translationMap())) { - for (String city : citiesToCodes(cities)) { + for (String keyword : getRegexList(s, KEYWORD_CODE_PATTERN)) { + for (String city : getRegexList(s, CITY_CODE_PATTERN)) { combinations.add(keyword + "-" + city); if (combinations.size() >= paramOrDefault("max", 2)) { return combinations; @@ -42,9 +52,6 @@ public class KeywordsClustering extends AbstractClusteringFunction { return fields .stream() .filter(f -> !f.isEmpty()) - .map(KeywordsClustering::cleanup) - .map(KeywordsClustering::normalize) - .map(s -> filterAllStopWords(s)) .map(s -> doApply(conf, s)) .map(c -> filterBlacklisted(c, ngramBlacklist)) .flatMap(c -> c.stream()) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java index b055077d8..6ef550c50 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java @@ -27,6 +27,14 @@ public class AbstractPaceFunctions extends PaceCommonUtils { private static Map cityMap = AbstractPaceFunctions .loadMapFromClasspath("/eu/dnetlib/pace/config/city_map.csv"); + // keywords map to be used when translating the keyword names into codes + private static Map keywordMap = AbstractPaceFunctions + .loadMapFromClasspath("/eu/dnetlib/pace/config/translation_map.csv"); + + // country map to be used when inferring the country from the city name + private static Map countryMap = AbstractPaceFunctions + .loadCountryMapFromClasspath("/eu/dnetlib/pace/config/country_map.csv"); + // list of stopwords in different languages protected static Set stopwords_gr = loadFromClasspath("/eu/dnetlib/pace/config/stopwords_gr.txt"); protected static Set stopwords_en = loadFromClasspath("/eu/dnetlib/pace/config/stopwords_en.txt"); @@ -74,6 +82,64 @@ public class AbstractPaceFunctions extends PaceCommonUtils { return s12; } + public static String countryInference(final String original, String inferFrom) { + if (!original.equalsIgnoreCase("unknown")) + return original; + + inferFrom = cleanup(inferFrom); + inferFrom = normalize(inferFrom); + inferFrom = filterAllStopWords(inferFrom); + Set cities = getCities(inferFrom, 4); + return citiesToCountry(cities).stream().findFirst().orElse("UNKNOWN"); + } + + public static String cityInference(String original) { + original = cleanup(original); + original = normalize(original); + original = filterAllStopWords(original); + + Set cities = getCities(original, 4); + + for (String city : cities) { + original = original.replaceAll(city, cityMap.get(city)); + } + + return original; + } + + public static String keywordInference(String original) { + original = cleanup(original); + original = normalize(original); + original = filterAllStopWords(original); + + Set keywords = getKeywords(original, keywordMap, 4); + + for (String keyword : keywords) { + original = original.replaceAll(keyword, keywordMap.get(keyword)); + } + + return original; + } + + public static String cityKeywordInference(String original) { + original = cleanup(original); + original = normalize(original); + original = filterAllStopWords(original); + + Set keywords = getKeywords(original, keywordMap, 4); + Set cities = getCities(original, 4); + + for (String keyword : keywords) { + original = original.replaceAll(keyword, keywordMap.get(keyword)); + } + + for (String city : cities) { + original = original.replaceAll(city, cityMap.get(city)); + } + + return original; + } + protected static String fixXML(final String a) { return a @@ -208,6 +274,30 @@ public class AbstractPaceFunctions extends PaceCommonUtils { return m; } + public static Map loadCountryMapFromClasspath(final String classpath) { + + Transliterator transliterator = Transliterator.getInstance("Any-Eng"); + + final Map m = new HashMap<>(); + try { + for (final String s : IOUtils + .readLines(AbstractPaceFunctions.class.getResourceAsStream(classpath), StandardCharsets.UTF_8)) { + // string is like this: country_code;city1;city2;city3 + String[] line = s.split(";"); + String value = line[0]; + for (int i = 1; i < line.length; i++) { + String city = fixAliases(transliterator.transliterate(line[i].toLowerCase())); + String code = cityMap.get(city); + m.put(code, value); + } + } + } catch (final Throwable e) { + return new HashMap<>(); + } + return m; + + } + public static String removeKeywords(String s, Set keywords) { s = " " + s + " "; @@ -237,6 +327,10 @@ public class AbstractPaceFunctions extends PaceCommonUtils { return toCodes(keywords, cityMap); } + public static Set citiesToCountry(Set cities) { + return toCodes(toCodes(cities, cityMap), countryMap); + } + protected static String firstLC(final String s) { return StringUtils.substring(s, 0, 1).toLowerCase(); } diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java index 7ad9b7445..b0dc11656 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java @@ -47,9 +47,21 @@ public class FieldDef implements Serializable { private String clean; + private String infer; + + private String inferenceFrom; + public FieldDef() { } + public String getInferenceFrom() { + return inferenceFrom; + } + + public void setInferenceFrom(final String inferenceFrom) { + this.inferenceFrom = inferenceFrom; + } + public String getName() { return name; } @@ -126,6 +138,14 @@ public class FieldDef implements Serializable { this.clean = clean; } + public String getInfer() { + return infer; + } + + public void setInfer(String infer) { + this.infer = infer; + } + @Override public String toString() { try { diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala index e6a1c4ccc..c6db62339 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala @@ -123,9 +123,19 @@ case class SparkModel(conf: DedupConfig) { case _ => res(index) } } + + if (StringUtils.isNotBlank(fdef.getInfer)) { + val inferFrom : String = if (StringUtils.isNotBlank(fdef.getInferenceFrom)) fdef.getInferenceFrom else fdef.getPath + res(index) = res(index) match { + case x: Seq[String] => x.map(inference(_, MapDocumentUtil.getJPathString(inferFrom, documentContext), fdef.getInfer)) + case _ => inference(res(index).toString, MapDocumentUtil.getJPathString(inferFrom, documentContext), fdef.getInfer) + } + } + } res + } new GenericRowWithSchema(values, schema) @@ -146,5 +156,17 @@ case class SparkModel(conf: DedupConfig) { res } + def inference(value: String, inferfrom: String, infertype: String) : String = { + val res = infertype match { + case "country" => AbstractPaceFunctions.countryInference(value, inferfrom) + case "city" => AbstractPaceFunctions.cityInference(value) + case "keyword" => AbstractPaceFunctions.keywordInference(value) + case "city_keyword" => AbstractPaceFunctions.cityKeywordInference(value) + case _ => value + } + + res + } + } diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CityMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CityMatch.java deleted file mode 100644 index 1d898ad83..000000000 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CityMatch.java +++ /dev/null @@ -1,48 +0,0 @@ - -package eu.dnetlib.pace.tree; - -import java.util.Map; -import java.util.Set; - -import eu.dnetlib.pace.config.Config; -import eu.dnetlib.pace.tree.support.AbstractStringComparator; -import eu.dnetlib.pace.tree.support.ComparatorClass; - -@ComparatorClass("cityMatch") -public class CityMatch extends AbstractStringComparator { - - private Map params; - - public CityMatch(Map params) { - super(params); - this.params = params; - } - - @Override - public double distance(final String a, final String b, final Config conf) { - - String ca = cleanup(a); - String cb = cleanup(b); - - ca = normalize(ca); - cb = normalize(cb); - - ca = filterAllStopWords(ca); - cb = filterAllStopWords(cb); - - Set cities1 = getCities(ca, Integer.parseInt(params.getOrDefault("windowSize", "4"))); - Set cities2 = getCities(cb, Integer.parseInt(params.getOrDefault("windowSize", "4"))); - - Set codes1 = citiesToCodes(cities1); - Set codes2 = citiesToCodes(cities2); - - // if no cities are detected, the comparator gives 1.0 - if (codes1.isEmpty() && codes2.isEmpty()) - return 1.0; - else { - if (codes1.isEmpty() ^ codes2.isEmpty()) - return -1; // undefined if one of the two has no cities - return commonElementsPercentage(codes1, codes2); - } - } -} diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java new file mode 100644 index 000000000..25a12bcdf --- /dev/null +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java @@ -0,0 +1,51 @@ + +package eu.dnetlib.pace.tree; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import eu.dnetlib.pace.config.Config; +import eu.dnetlib.pace.tree.support.AbstractStringComparator; +import eu.dnetlib.pace.tree.support.ComparatorClass; + +@ComparatorClass("codeMatch") +public class CodeMatch extends AbstractStringComparator { + + private Map params; + + private Pattern CODE_REGEX; + + public CodeMatch(Map params) { + super(params); + this.params = params; + this.CODE_REGEX = Pattern.compile(params.getOrDefault("codeRegex", "[a-zA-Z]::\\d+")); + } + + public Set getRegexList(String input) { + Matcher matcher = this.CODE_REGEX.matcher(input); + Set cities = new HashSet<>(); + while (matcher.find()) { + cities.add(matcher.group()); + } + return cities; + } + + @Override + public double distance(final String a, final String b, final Config conf) { + + Set codes1 = getRegexList(a); + Set codes2 = getRegexList(b); + + // if no codes are detected, the comparator gives 1.0 + if (codes1.isEmpty() && codes2.isEmpty()) + return 1.0; + else { + if (codes1.isEmpty() ^ codes2.isEmpty()) + return -1; // undefined if one of the two has no codes + return commonElementsPercentage(codes1, codes2); + } + } +} diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java index 9cf616356..96a87c455 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java @@ -1,6 +1,8 @@ + package eu.dnetlib.pace.tree; import java.util.Map; +import java.util.Set; import com.wcohen.ss.AbstractStringDistance; @@ -11,37 +13,42 @@ import eu.dnetlib.pace.tree.support.ComparatorClass; @ComparatorClass("countryMatch") public class CountryMatch extends AbstractStringComparator { - public CountryMatch(Map params) { - super(params, new com.wcohen.ss.JaroWinkler()); - } + private Map params; - public CountryMatch(final double weight) { - super(weight, new com.wcohen.ss.JaroWinkler()); - } + public CountryMatch(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + this.params = params; + } - protected CountryMatch(final double weight, final AbstractStringDistance ssalgo) { - super(weight, ssalgo); - } + public CountryMatch(final double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } - @Override - public double distance(final String a, final String b, final Config conf) { - if (a.isEmpty() || b.isEmpty()) { - return -1.0; // return -1 if a field is missing - } - if (a.equalsIgnoreCase("unknown") || b.equalsIgnoreCase("unknown")) { - return -1.0; // return -1 if a country is UNKNOWN - } + protected CountryMatch(final double weight, final AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } - return a.equals(b) ? 1.0 : 0; - } + @Override + public double distance(final String a, final String b, final Config conf) { - @Override - public double getWeight() { - return super.weight; - } + if (a.isEmpty() || b.isEmpty()) { + return -1.0; // return -1 if a field is missing + } + if (a.equalsIgnoreCase("unknown") || b.equalsIgnoreCase("unknown")) { + return -1.0; // return -1 if a country is UNKNOWN + } + + return a.equals(b) ? 1.0 : 0; + } + + @Override + public double getWeight() { + return super.weight; + } + + @Override + protected double normalize(final double d) { + return d; + } - @Override - protected double normalize(final double d) { - return d; - } } diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerLegalname.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerLegalname.java new file mode 100644 index 000000000..e4a48c459 --- /dev/null +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerLegalname.java @@ -0,0 +1,59 @@ + +package eu.dnetlib.pace.tree; + +import java.util.Map; +import java.util.Set; + +import com.wcohen.ss.AbstractStringDistance; + +import eu.dnetlib.pace.config.Config; +import eu.dnetlib.pace.tree.support.AbstractStringComparator; +import eu.dnetlib.pace.tree.support.ComparatorClass; + +@ComparatorClass("jaroWinklerLegalname") +public class JaroWinklerLegalname extends AbstractStringComparator { + + private Map params; + + private final String CITY_CODE_REGEX = "city::\\d+"; + private final String KEYWORD_CODE_REGEX = "key::\\d+"; + + public JaroWinklerLegalname(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + this.params = params; + } + + public JaroWinklerLegalname(double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } + + protected JaroWinklerLegalname(double weight, AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } + + @Override + public double distance(String a, String b, final Config conf) { + + String ca = a.replaceAll(CITY_CODE_REGEX, "").replaceAll(KEYWORD_CODE_REGEX, " "); + String cb = b.replaceAll(CITY_CODE_REGEX, "").replaceAll(KEYWORD_CODE_REGEX, " "); + + ca = ca.replaceAll("[ ]{2,}", " "); + cb = cb.replaceAll("[ ]{2,}", " "); + + if (ca.isEmpty() && cb.isEmpty()) + return 1.0; + else + return normalize(ssalgo.score(ca, cb)); + } + + @Override + public double getWeight() { + return super.weight; + } + + @Override + protected double normalize(double d) { + return d; + } + +} diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerNormalizedName.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerNormalizedName.java deleted file mode 100644 index 576b9281d..000000000 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JaroWinklerNormalizedName.java +++ /dev/null @@ -1,74 +0,0 @@ - -package eu.dnetlib.pace.tree; - -import java.util.Map; -import java.util.Set; - -import com.wcohen.ss.AbstractStringDistance; - -import eu.dnetlib.pace.config.Config; -import eu.dnetlib.pace.tree.support.AbstractStringComparator; -import eu.dnetlib.pace.tree.support.ComparatorClass; - -@ComparatorClass("jaroWinklerNormalizedName") -public class JaroWinklerNormalizedName extends AbstractStringComparator { - - private Map params; - - public JaroWinklerNormalizedName(Map params) { - super(params, new com.wcohen.ss.JaroWinkler()); - this.params = params; - } - - public JaroWinklerNormalizedName(double weight) { - super(weight, new com.wcohen.ss.JaroWinkler()); - } - - protected JaroWinklerNormalizedName(double weight, AbstractStringDistance ssalgo) { - super(weight, ssalgo); - } - - @Override - public double distance(String a, String b, final Config conf) { - String ca = cleanup(a); - String cb = cleanup(b); - - ca = normalize(ca); - cb = normalize(cb); - - ca = filterAllStopWords(ca); - cb = filterAllStopWords(cb); - - Set keywords1 = getKeywords( - ca, conf.translationMap(), Integer.parseInt(params.getOrDefault("windowSize", "4"))); - Set keywords2 = getKeywords( - cb, conf.translationMap(), Integer.parseInt(params.getOrDefault("windowSize", "4"))); - - Set cities1 = getCities(ca, Integer.parseInt(params.getOrDefault("windowSize", "4"))); - Set cities2 = getCities(cb, Integer.parseInt(params.getOrDefault("windowSize", "4"))); - - ca = removeKeywords(ca, keywords1); - ca = removeKeywords(ca, cities1); - cb = removeKeywords(cb, keywords2); - cb = removeKeywords(cb, cities2); - - ca = ca.replaceAll("[ ]{2,}", " "); - cb = cb.replaceAll("[ ]{2,}", " "); - - if (ca.isEmpty() && cb.isEmpty()) - return 1.0; - else - return normalize(ssalgo.score(ca, cb)); - } - - @Override - public double getWeight() { - return super.weight; - } - - @Override - protected double normalize(double d) { - return d; - } - -} diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/KeywordMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/KeywordMatch.java deleted file mode 100644 index 53acb4dc8..000000000 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/KeywordMatch.java +++ /dev/null @@ -1,50 +0,0 @@ - -package eu.dnetlib.pace.tree; - -import java.util.Map; -import java.util.Set; - -import eu.dnetlib.pace.config.Config; -import eu.dnetlib.pace.tree.support.AbstractStringComparator; -import eu.dnetlib.pace.tree.support.ComparatorClass; - -@ComparatorClass("keywordMatch") -public class KeywordMatch extends AbstractStringComparator { - - Map params; - - public KeywordMatch(Map params) { - super(params); - this.params = params; - } - - @Override - public double distance(final String a, final String b, final Config conf) { - - String ca = cleanup(a); - String cb = cleanup(b); - - ca = normalize(ca); - cb = normalize(cb); - - ca = filterAllStopWords(ca); - cb = filterAllStopWords(cb); - - Set keywords1 = getKeywords( - ca, conf.translationMap(), Integer.parseInt(params.getOrDefault("windowSize", "4"))); - Set keywords2 = getKeywords( - cb, conf.translationMap(), Integer.parseInt(params.getOrDefault("windowSize", "4"))); - - Set codes1 = toCodes(keywords1, conf.translationMap()); - Set codes2 = toCodes(keywords2, conf.translationMap()); - - // if no cities are detected, the comparator gives 1.0 - if (codes1.isEmpty() && codes2.isEmpty()) - return 1.0; - else { - if (codes1.isEmpty() ^ codes2.isEmpty()) - return -1.0; // undefined if one of the two has no keywords - return commonElementsPercentage(codes1, codes2); - } - } -} diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java index 0973fdf1e..0ff03f5e1 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java @@ -48,7 +48,7 @@ public class TreeNodeDef implements Serializable { // function for the evaluation of the node public TreeNodeStats evaluate(Row doc1, Row doc2, Config conf) { - TreeNodeStats stats = new TreeNodeStats(); + TreeNodeStats stats = new TreeNodeStats(ignoreUndefined); // for each field in the node, it computes the for (FieldConf fieldConf : fields) { diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java index 2b96048b4..a210c06b2 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java @@ -9,8 +9,11 @@ public class TreeNodeStats implements Serializable { private Map results; // this is an accumulator for the results of the node - public TreeNodeStats() { + private final boolean ignoreUndefined; + + public TreeNodeStats(boolean ignoreUndefined) { this.results = new HashMap<>(); + this.ignoreUndefined = ignoreUndefined; } public Map getResults() { @@ -22,7 +25,10 @@ public class TreeNodeStats implements Serializable { } public int fieldsCount() { - return this.results.size(); + if(ignoreUndefined) + return this.results.size(); + else + return this.results.size() - undefinedCount(); //do not count undefined } public int undefinedCount() { @@ -78,11 +84,25 @@ public class TreeNodeStats implements Serializable { double min = 100.0; // random high value for (FieldStats fs : this.results.values()) { if (fs.getResult() < min) { - if (fs.getResult() >= 0.0 || (fs.getResult() == -1 && fs.isCountIfUndefined())) + if (fs.getResult() == -1) { + if (fs.isCountIfUndefined()) { + min = 0.0; + } + else { + min = -1; + } + } + else { min = fs.getResult(); + } } } - return min; + if (ignoreUndefined) { + return min==-1.0? 0.0 : min; + } + else { + return min; + } } // if at least one is true, return 1.0 @@ -91,7 +111,11 @@ public class TreeNodeStats implements Serializable { if (fieldStats.getResult() >= fieldStats.getThreshold()) return 1.0; } - return 0.0; + if (!ignoreUndefined && undefinedCount()>0){ + return -1.0; + } else { + return 0.0; + } } // if at least one is false, return 0.0 @@ -100,7 +124,7 @@ public class TreeNodeStats implements Serializable { if (fieldStats.getResult() == -1) { if (fieldStats.isCountIfUndefined()) - return 0.0; + return ignoreUndefined? 0.0 : -1.0; } else { if (fieldStats.getResult() < fieldStats.getThreshold()) return 0.0; diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java index 263504dbb..8ae5aa591 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java @@ -43,18 +43,17 @@ public class TreeProcessor { TreeNodeStats stats = currentNode.evaluate(doc1, doc2, config); treeStats.addNodeStats(nextNodeName, stats); - - // if ignoreUndefined=false the miss is considered as undefined - if (!currentNode.isIgnoreUndefined() && stats.undefinedCount() > 0) { + + double finalScore = stats.getFinalScore(currentNode.getAggregation()); + if(finalScore == -1.0) nextNodeName = currentNode.getUndefined(); - } - // if ignoreUndefined=true the miss is ignored and the score computed anyway - else if (stats.getFinalScore(currentNode.getAggregation()) >= currentNode.getThreshold()) { + else if (finalScore >= currentNode.getThreshold()) { nextNodeName = currentNode.getPositive(); } else { nextNodeName = currentNode.getNegative(); } + } while (MatchType.parse(nextNodeName) == MatchType.UNDEFINED); treeStats.setResult(MatchType.parse(nextNodeName)); diff --git a/dhp-pace-core/src/main/resources/eu/dnetlib/pace/config/country_map.csv b/dhp-pace-core/src/main/resources/eu/dnetlib/pace/config/country_map.csv new file mode 100644 index 000000000..01e49979c --- /dev/null +++ b/dhp-pace-core/src/main/resources/eu/dnetlib/pace/config/country_map.csv @@ -0,0 +1,239 @@ +JP;Sapporo;Kobe;Okayama;Maebashi;Nagoya;Yamaguchi;Sendai;Nagano;Saga;Otsu;Niigata;Akita;Fukui;Matsue;Yamagata;Wakayama;Nagasaki;Kumamoto;Shizuoka;Osaka;Chiba;Hiroshima;Yawatahama-shi;Gifu;Takamatsu;Naha;Fukushima;Yokohama;Kagoshima;Kyoto;Aomori;Kanazawa;Miyazaki;Tsu;Tokyo;Toyama;Kochi;Tokushima;Hirado;Fukuoka;Mito;Japan;Utsunomiya;Nara;Kofu;Kozakai-cho;Saitama;Oita;Matsuyama;Morioka +ID;Tahuna;Amuntai;Karanganyar;Pangkalpinang;Makassar;Ranai;Sampit;Melonguane;Kotabumi;Jember;Banjar;Tanjungpandan;Tembagapura;Kisaran;Pangururan;Rangkasbitung;Bandar Lampung;Buntok;Tarutung;Makale;Saumlaki;Larantuka;Raba;Palangkaraya;Airmadidi;Ruteng;Indonesia;Padang;Sanana;Jambi;Bagan Si Api-api;Rantau Prapat;Watampone;Tambolaka;Bandung;Meulaboh;Cilacap;Serui;Denpasar;Manggar;Bengkalis;Piru;Parigi;Pekanbaru;Bengkulu;Tenggarong;Maumere;Majalengka;Soreang;Menggala;Palu;Sumenep;Sumber;Surabaya;Kandangan;Kupang;Betun;Kendari;Tanjungpinang;Mataram;Waingapu;Maba;Ambon;Ciamis;Rantepao;Atambua;Manado;Pelabuhanratu;Nabire;Banyuwangi;Ransiki;Bintuni;Lubuk Sikaping;Ende;Tanjung Selor;Cibinong;Kefamenanu;Wamena;Fakfak;Praya;Dompu;Tobelo;Sumbawa Besar;Raha;Biak;Gunungsitoli;Kuala Kapuas;Tuban;Bangkinang;Yogyakarta;Amahai;Pariaman;Pontianak;Merauke;Putussibau;Leramatang;Sofifi;Kolonodale;Singaraja;Cianjur;Jakarta;Jepara;Tomohon;Medan;Poso;Namlea;Kuningan;Taliwang;Pamekasan;Kuala Pembuang;Purwokerto;Bangkalan;Kaimana;Batang;Muara Teweh;Sorong;Ungaran;Barabai;Waikabubak;Prabumulih;Masamba;Samarinda;Semarang;Puruk Cahu;Pageralam;Barru;Rappang;Martapura;Sentani;Painan;Indramayu;Banjarmasin;Cimerak;Karawang;Tamiang Layang;Bungku;Pinrang;Jayapura;Bebandem;Manokwari;Majene;Ampana;Purwakarta;Palembang;Slateng Dua;Garut;Subang;Sungailiat;Lahat;Mamuju;Soe;Pati;Tentena;Singaparna;Kalabahi;Luwuk;Tarogong;Banda Aceh;Saparua;Teminabuan;Gorontalo;Serang +IN;Ahmedabad;Shimla;Calicut;Jammu;Raipur;Chennai;New Delhi;Itanagar;Dehra Dun;Kolkata;Chandigarh;Jaipur;Thiruvananthapuram;Panchkula;India;Hyderabad;Bhopal;Bhubaneshwar;Kohima;Bangalore;Amaravati;Port Blair;Lucknow;Shillong;Aizawl;Dispur;Kavaratti;Mumbai;Daman;Gangtok;Imphal;Delhi;Srinagar;Agartala;Patna;Panaji;Ghandinagar;Ranchi;Puducherry;Madurai +CN;Changsha;Guangzhou;Fuqing;Dongyang;Yushan;Qingping;Leiyang;Hongshui;Shanhu;Meishan;Suileng;Gyegu;Quyang;Encheng;Weinan;Xushan;Tongliao;Qinzhou;Tengyue;Taiyuan;Tacheng;Tangdong;Puxi;Shuangcheng;Wencheng;Dexing;Beijing;Taohuajiang;Dongping;Jinhua;Wuzhong;Golmud;Boli;Yi Xian;Medog;Yatou;Chaozhou;Zalantun;Xigaze;Anqing;Wangqing;Zhijiang;Linkou;Zhamog;Zhugang;Fuding;Mishan;Jinbi;Longjiang;Neijiang;Yiwu;Fangchenggang;Bayan Hot;Beian;Yuquan;Urumqi;Mianyang;Luocheng;Jinshi;Xinpu;Huishi;Yilong;Hengzhou;Huanggang;Yichun;Tuodian;Zuitou;Xining;Jinan;Jinzhong;Zhongcheng;Lu'an;Dongxing;Yinggen;Fuyuan;Dadukou;Anlu;Weichanglu;Qingquan;Tangdukou;Dongta;Qingdao;Gaoyou;Dalain Hob;Yunzhong;Xiedian;Wuchang;Changde;Minzhu;Yicheng;Jiamusi;Qamdo;Loufan;Zhujiacun;Gutao;Shangzhi;Dianbu;Shacheng;Zhuji;Guigang;Qingan;Ji'an;Huazangsi;Leping;Changling;Dashiqiao;Mazhang;Tailai;Baishan;Jiangmen;Altay;Kaiyuan;Chang'an;Beihai;Suohe;Shenyang;Zhangye;Sanming;Guankou;Ma'erkang;Nanchang;Karakax;Rongcheng;Ningbo;Kaiyun;Linxi;Heshan;Xinyu;Kaifeng Chengguanzhen;Liujiaxia;Guang'an;Mengmeng;Tieli;Chengjiao;Lianzhou;Jian'ou;Yangjiang;Leizhou;Xianyang;Yian;Jianshe;Kunming;Deyang;Liuhe;Yangshe;Xiaping;Dongtai;Huquan;Fujin;Siyang;Yisuhe;Yanqi;Kaiwen;Lingshou;Hechi;Taixing;Jiancheng;Qinggang;Xireg;Yutan;Xinzhou;Laojiezi;Xiayang;Yancheng;Liaocheng;Ningde;Heihe;Gaizhou;Xibeijie;Helixi;Taicheng;Uchturpan;Jiaoxiyakou;Chongqing;Jiayuguan;Yanjiang;Nenjiang;Suonan;Xianshuigu;Yakou;Gexianzhuang;Qiqihar;Lanyi;Shangtianba;Laiyuan;Tianchang;Meizhou;Huichang;Haikou;Zequ;Shuanghe;Shijiazhuang;Mizhou;Luanzhou;Yunfu;Mali;Zanhuang;Hoh Ereg;Xishan;Fucheng;Huaiyang;Qionghu;Wuxi;Anguo;Xinhualu;Nehe;Xinxing;Weifen;Tekes;Longquan;Kuaidamao;Rizhao;Zhangzhou;Zhaozhou;Tongshan;Yongbei;Lecheng;Jinshan;Zhenzhou;Huili Chengguanzhen;Xiangyang;Muping;Baoding;Changzhou;Qianzhou;Huilong;Kaji;Dingzhou;Langzhong;Ailan Mubage;Menglie;Chizhou;Aral;Dianga;Aketao;Suzhou;Linjiang;Longxing;Jincheng;Wuyishan;Xiwanzi;Mangqu;Fenglu;Shishi;Zhenjiang;Nagqu;Gulou;Dongcun;Wanyuan;Zhangjiajie;Hailun;Zhaoqing;Bazhou;Zhuzhou;Zhuhai;Zhoushan;Lhasa;Qarqan;Cangzhou;Dengtalu;Pagqen;Chifeng;Yingshouyingzi;Hong'an;Huludao;Wuwei;Tafeng;Xiaqiaotou;Shaoxing;Chinggil;Chengde;Haomen;Sanhe;Gabasumdo;Changzhi;Keshan;Xinglong;Nanjian;Labuleng;Shanhe;Langfang;Jingdezhen;Shaoyang;Zhongwei;Yucheng;Hangzhou;Guixi;Tongjiang;Fengcheng;Lucheng;Licheng;Taozhou;Nansan;Jiangguanchi;Handan;Dongchuan;Tongyangdao;Xinshi;Mengdong;Youganning;Kaihua;Zhedao;Suining;Meicheng;Chengdu;Genhe;Ghulja;Wenlan;Jinghai;Jingzhou;Jiannan;Shiyan;Taihecun;Luzhou;Gongzhuling;Fengning;Hoxut;Xiongzhou;Hunchun;Fukang;Yinying;Yanji;Linquan;Huinan;Jingcheng;Wusong;Gyangze;Qingnian;Toqsu;Beidao;Qitai;Menglang;Huolu;Weihai;Zijinglu;Xingsha;Hegang;Tangxing;Lishui;Tianjin;Dongning;Dingcheng;Beichengqu;Wenxian Chengguanzhen;Magitang;Aba;Putian;Xicheng;Siping;Alashankou;Huazhou;Tongchuan;Qinhe;Xinyuan;Tanbei;Yuli;Suifenhe;Hejian;Dazhou;Lindong;Zhangjiakou;Chengguan;Chongshan;Sanjiang;Chengxiang;Wudalianchi;Hulin;Mabai;Nantong;Nangandao;Luofeng;Longtoushan Jiezi;Hai'an;Kargilik;Xincheng;Ulan Hua;Renqiu;Sanjiaocheng;Jin'e;Yakeshi;Anda;Yong'an;Daqing;Lianyuan;Qabqa;Xiangcheng;Linxia Chengguanzhen;Wuling;Baiyashi;Yishi;Peyziwat;Xisa;Taoyang;Fuzhou;Wutong;Bama;Yingcheng;Ning'an;Shengping;Shuangluan;Hezuo;Longchuan;Xiangjiaba;Cencheng;Huangshan;Shengli;Heze;Shaoguan;Zhongshan;Xincun;Donghua;Ducheng;Kuiju;Yuxi;Gannan;Dunhua;Pingliang;Yan'an Beilu;Baicheng;Shangrao;Shuozhou;Guozhen;Rongjiawan;Liulin;Lingbao Chengguanzhen;Leling;Zhangjiakou Shi Xuanhua Qu;Khutubi;Nanjing;Nada;Huangshi;Yulin;Oroqen Zizhiqi;Wulan;Ziketan;Lingquan;Hohhot;Dezhou;Guiping;Basuo;Jixi;Fu'an;Wuhai;Jiaji;Jiujiang;Cuihua;Yitiaoshan;Shenzhen;Dahuaishu;Koktokay;Yuanquan;Kumul;Zhuozhou;Jiangna;Xiashi;E'erguna;Xangda;Nong'an;Hotan;Xingcheng;Da'an;Zhuolu;Dingxi;Taihe;Meihekou;Jiaxing;Lanxi;Mudanjiang;Zhengding;Jinjiang;Yongqing;Jiantang;Rulin;Weiyuan;Wuyi;Zhaxi;Xinji;Wuhan;Pingxiang;Shanghai;Longjing;Zhoukou;Bamiantong;Laibin;Pudong;Wenping;Fancheng;Mositai;Yangquan;Puyang Chengguanzhen;Yingchuan;Luzhang;Aihua;Xiaoxita;Zhangping;Fuyang;Nanchong;Qingyang;Zhuangyuan;Jieyang;Miluo Chengguanzhen;Zhaoyu;Shuangqiao;Pingquan;Zhengzhou;Toksun;Tumen;Daiyue;Fangting;Botou;Wuzhou;Mengla;Chaigoubu;Tuncheng;Atushi;Kuqa;Heyin;China;Chenzhou;Sanya;Guiyang;Changting;Rongwo;Ruoqiang;Liaoyuan;Changchun;Baiyin;Ezhou;Huanghua;Karamay;Ji'an Shi;Yushu;Wuhu;Zhongba;Taizhou;Shulan;Longyan;Cili;Lushar;Gaozhou;Hengshui;Songyang;Jinchang;Helong;Shenzhou;Kuytun;Ganzhou;Luohe;Nilka;Puyang;Dali;Guilin;Qapqal;Nanping;Luancheng;Korla;Yingkou;Chengjiao Chengguanzhen;Dongguan;Zhengjiatun;Xihuachi;Xi'an;Harbin;Gar;Leshou;Baiquan;Yaofeng;Quanzhou;Lianran;Guma;Aksu;Xinhua;Gaobeidian;Shangpa;Shihezi;Lianshan;Aheqi;Yiyang;Wuxue;Hezhou;Mingxing;Jizhou;Wenchang;Baocheng;Chuimatan;Haicheng;Liancheng;Jinghong;Shaowu;Heyuan;Ma'anshan;Foshan;Qingyuan;Huzhou;Toli;Zunhua;Xiping;Turpan;Maoming;Huizhou;Hekou;Hefei;Chongzuo;Yanghe;Lechang;Guangming;Yibin;Wancheng;Hongliuwan;Yinchuan;Zhaodong;Lianhe;Kangbao;Qagan Us;Gaoping;Simao;Xichang;Lin'an;Jishi;Huangzhai;Suihua;Xiluodu;Hailin;Zhanjiang;Qincheng;Xiamen;Kashgar;Qiaotou;Nanning;Shantou;Fusui;Gaocheng;Zhanggu;Chuqung;Liuzhou;Shuangyashan;Jingping;Longhua;Shanwei;Shazhou;Lanzhou;Rucheng;Jinhe;Yunxian Chengguanzhen +PH;Ilagan;Poblacion;Baguio City;Kidapawan;Kalibo;Marikina City;Cebu City;Jordan;Perez;Prosperidad;Agdangan;Hinatuan;Mambajao;Iligan;Calapan;Puerto Princesa;Candelaria;Anda;Taguig City;Navotas;Roseller Lim;Pasay City;Tabuk;Jolo;Caloocan City;Cotabato;Tagbilaran City;Davao;Ormoc;Isulan;Tandag;Tuguegarao;Kabugao;Lucena;Mandaluyong City;Tingloy;Laoag;San Antonio;Malaybalay;Monkayo;Maasin;San Juan;Makati City;Butuan;La Trinidad;Iloilo;Datu Odin Sinsuat;Manila;Quezon City;Santiago;Tiaong;Pili;San Jose;Trece Martires City;Legazpi City;Malita;Mamburao;Argao;Valenzuela;Maribojoc;Angeles City;Magalang;Mati;Pagadian;Banaybanay;Muntinlupa City;City of Isabela;Batangas;Dumaguete City;Badian;Baclayon;Talipao;President Quirino;Balanga;Buluan;Vigan;Santa Cruz;Pasig City;Digos;Barobo;Surigao;Oroquieta;Naga City;Kabasalan;Buguey;Boac;Padre Burgos;Tarlac City;Daet;Iba;Malolos;Lapu-Lapu City;Catbalogan;Cabarroguis;Mandaue City;Dauis;Olongapo;Alabel;Marawi City;Bangued;Lagawe;Claveria;Cabadbaran;Naval;Baler;Malabon;Roxas City;Sorsogon;Tagum;San Fernando;Catarman;Dipolog;Koronadal;Bontoc;Siquijor;Philippines;Tacloban;Corella;Palayan City;Basco;Lianga;Tubod;Virac;Bayombong;Panay;Borongan;Zamboanga City;Cagayan de Oro;Romblon;Bacolod;City of Paranaque;Masbate;Antipolo;Nabunturan;Ipil;Dalaguete;Bongao;Lupon;Dagupan City;Alcantara;General Santos;Lingayen +BR;Lagoa do Mato;Sousa;Neopolis;Pirai do Norte;Limoeiro do Norte;Anastacio;Frecheirinha;Varjota;Rio Tinto;Palmeira dos Indios;Cha Grande;Balsas;Araioses;Aracati;Remigio;Floriano;Ico;Guamare;Itiquira;Pedras de Fogo;Ipiau;Conceicao de Jacuipe;Jucas;Cha da Alegria;Itapecuru Mirim;Acopiara;Goianinha;Pao de Acucar;Manaquiri;Paramoti;Colina;Jucurutu;Rio Formoso;Coari;Rio Real;Sao Felix;Aragoiania;Ararenda;Itapiranga;Baiao;Patu;Amapa;Itubera;Atalaia;Passagem Franca;Uropa;Caem;Carnaubal;Gurupi;Alta Floresta;Sao Tome;Jaru;Guajara-Mirim;Primavera;Macajuba;Dianopolis;Queimadas;Madalena;Fonte Boa;Morro Agudo;Morada Nova;Pastos Bons;Goias;Tapiramuta;Acarau;Atalaia do Norte;Cristinapolis;Aratuba;Sao Francisco;Mazagao;Viseu;Aracaju;Feira de Santana;Penedo;Cuite;Parnarama;Boqueirao;Barreira;Conceicao das Alagoas;Juazeirinho;Colorado do Oeste;Brazil;Canguaretama;Lucena;Sapeacu;Feijo;Sao Francisco do Maranhao;Ielmo Marinho;Barra;Santa Quiteria do Maranhao;Guaraci;Castanhal;Grajau;Edeia;Passira;Pimenta Bueno;Nazare da Mata;Acari;Escada;Codajas;Itagi;Beneditinos;Palmacia;Japura;Tracuateua;Ji-Parana;Jaragua;Formosa do Rio Preto;Sao Joao Batista;Santa Luzia do Itanhy;Soledade;Florania;Tuntum;Ibiraci;Altos;Cupira;Lagoa dos Gatos;Luis Correia;Uarini;Lagoa da Confusao;Florianopolis;Alhandra;Baixa Grande;Anguera;Araxa;Mineiros;Pocao de Pedras;Penalva;Joaquim Gomes;Nova Olinda;Mansidao;Governador Dix-Sept Rosado;Simoes Filho;Tucuma;Arari;Varzedo;Sao Pedro do Piaui;Joao Camara;Natividade;Lavras da Mangabeira;Alcantaras;Santana do Cariri;Jaguaretama;Brasnorte;Barao do Grajau;Aragarcas;Itainopolis;Santa Isabel do Para;Carauari;Nina Rodrigues;Nossa Senhora dos Milagres;Sao Miguel das Matas;Aratuipe;Baturite;Colares;Pianco;Messias;Pires Ferreira;Moreno;Campo Maior;Ibateguara;Tapaua;Capao da Canoa;Augustinopolis;Manoel Vitorino;Palhano;Cacu;Torres;Aguas Belas;Ibipetuba;Paulino Neves;Ubata;Tibau do Sul;Capanema;Itumbiara;Palmares;Olimpia;Quixada;Frutal;Tome-Acu;Cururupu;Vitoria;Xapuri;Paripiranga;Paragominas;Juripiranga;Wagner;Marapanim;Rio de Janeiro;Meruoca;Sao Francisco do Conde;Itapipoca;Capim;Sao Jose da Coroa Grande;Canutama;Serra Branca;Colonia Leopoldina;Riachao das Neves;Calcoene;Matoes;Madre de Deus;Santa Ines;Candido Mendes;Valente;Umbauba;Joao Pessoa;Craibas;Urucurituba;Buriti Alegre;Orobo;Goiania;Jaboatao dos Guararapes;Vargem Grande;Sao Bento;Autazes;Coelho Neto;Mauriti;Parnaiba;Crateus;Mata de Sao Joao;Guara;Alto Santo;Mocajuba;Quipapa;Campo Novo do Parecis;Maragogipe;Iraucuba;Indiaroba;Sacramento;Coracao de Maria;Nova Crixas;Arara;Barauna;Formoso do Araguaia;Puxinana;Saubara;Serra Preta;Pedro Velho;Morro do Chapeu;Lagoa do Ouro;Barra de Santa Rosa;Manacapuru;Tiangua;Anori;Sao Miguel do Guapore;Ipojuca;Caninde;Uniao dos Palmares;Jandaira;Sao Vicente Ferrer;Barreirinha;Capitao Poco;Pauini;Alianca;Russas;Maranguape;Baia da Traicao;Aracoiaba;Itamaraca;Gudofredo Viana;Campos Lindos;Araguaina;Santa Rita;Pindare-Mirim;Piracanjuba;Presidente Dutra;Bequimao;Piracuruca;Mancio Lima;Iati;Pacatuba;Paranatama;Buriti do Tocantins;Santo Estevao;Itaberai;Boca da Mata;Tururu;Irara;Pio IX;Amargosa;Altamira;Aracagi;Maraial;Pitimbu;Ceres;Alto Araguaia;Recife;Vicentinopolis;Bela Cruz;Careiro;Alcantara;Mucambo;Angicos;Cruz das Almas;Irituia;Flexeiras;Sao Jose de Mipibu;Curitiba;Araguatins;Olho d'Agua das Cunhas;Moita Bonita;Pintadas;Alto Longa;Campinorte;Varzea Grande;Caxias;Xinguara;Limoeiro de Anadia;Estreito;Igaci;Senador Guiomard;Carire;Reriutaba;Sao Bernardo;Antonio Cardoso;Urucara;Dois Riachos;Areia Branca;Uruara;Belem de Maria;Jatai;Vitoria do Mearim;Novo Aripuana;Mutuipe;Salitre;Gravata;Cajazeiras;Muritiba;Barrocas;Cerejeiras;Bujaru;Joao Alfredo;Maravilha;Portel;Timbauba;Cassia;Conde;Sao Jose do Egito;Boa Nova;Pedra Branca;Vicencia;Orlandia;Mairi;Ararangua;Rio Maria;Borba;Turiacu;Bacabal;Angelim;Santa Juliana;Jacobina;Bacuri;Alto Alegre dos Parecis;Umarizal;Maraba;Coroata;Rio Preto da Eva;Passa e Fica;Conceicao do Almeida;Ibirataia;Sao Luis do Quitunde;Tocantinopolis;Tonantins;Boa Vista;Novo Airao;Itapissuma;Cumaru;Alagoa Grande;Miranorte;Guaira;Itapororoca;Tupanatinga;Monte Alegre;Barretos;Mundo Novo;Tejucuoca;Patrocinio Paulista;Macapa;Presidente Vargas;Currais Novos;Croata;Anapurus;Nilo Pecanha;Senador Jose Porfirio;Ipiranga do Piaui;Porto de Pedras;Paco do Lumiar;Chapada dos Guimaraes;Itamarati;Santo Antonio;Gloria do Goita;Santo Antonio do Ica;Vila do Conde;Jericoacoara;Serrolandia;Sao Joaquim da Barra;Poranga;Centralina;Euclides da Cunha;Braganca;Sao Caetano de Odivelas;Ibicuitinga;Pocone;Bananeiras;Axixa do Tocantins;Cruzeiro do Sul;Sao Sebastiao do Uatuma;Amarante;Caldeirao Grande;Barreirinhas;Costa Marques;Santa Teresinha (2);Brejo do Cruz;Catende;Diamantino;Codo;Esperanca;Maxaranguape;Guiratinga;Pontalina;Nova Brasilandia d'Oeste;Aquidaba;Macaiba;Jaguaribara;Ribeiropolis;Belo Horizonte;Buriti;Ananas;Camocim;Camocim de Sao Felix;Rosario Oeste;Porto Calvo;Nova Olinda do Norte;Nova Cruz;Satuba;Luzilandia;Igarape Grande;Santaluz;Franca;Barro;Goiana;Bom Lugar;Obidos;Aurelino Leal;Alvorada;Vargem da Roca;Bayeux;Areia;Santa Cruz;Ubaira;Primeira Cruz;Anama;Garanhuns;Januario Cicco;Itatira;Chaval;Apodi;Cruz;Ubaitaba;Sao Francisco do Guapore;Coxim;Nazare;Corumba;Gameleira;Mogeiro;Marcacao;Campina Grande;Imperatriz;Itaberaba;Ipora;Acara;Ibirapitanga;Alagoinha;Amaraji;Santana do Matos;Elesbao Veloso;Santanopolis;Santa Helena;Bom Jardim;Itapuranga;Aveiro;Miracema do Tocantins;Itapiuna;Umirim;Araruna;Cuiaba;Buritirama;Olivenca;Morrinhos;Pendencias;Cedro;Teixeira;Ouro Branco;Bom Conselho;Nova Xavantina;Sena Madureira;Bonito de Santa Fe;Santa Quiteria;Guarai;Paripueira;Urucui;Cujubim;Humberto de Campos;Ladario;Quijingue;Agua Branca;Cacoal;Presidente Medici;Cruz do Espirito Santo;Mara Rosa;Agua Azul do Norte;Ipixuna;Presidente Figueiredo;Prata;Tomar do Geru;Cerro Cora;Parelhas;Alta Floresta D'Oeste;Nossa Senhora das Dores;Jussara;Hidrolandia;Rondonopolis;Pindoretama;Jiquirica;Itaparica;Itupiranga;Piripiri;Joao Lisboa;Saude;Dona Ines;Sao Gabriel da Cachoeira;Caracarai;Lajes;Inga;Mozarlandia;Mulungu;Neropolis;Campos Sales;Pacajus;Catole do Rocha;Juara;Caruaru;Santo Antonio do Taua;Oiapoque;Gurupa;Agua Preta;Nova Floresta;Itabaiana;Iranduba;Sao Luis do Curu;Maurilandia;Pentecoste;Cabaceiras do Paraguacu;Caiaponia;Oeiras;Bujari;Vicosa do Ceara;Itaituba;Mombaca;Ipanguacu;Cansancao;Aiuaba;Goiatins;Tenente Ananias Gomes;Sinop;Saboeiro;Itajiba;Granja;Uruacu;Rio Verde;Pe de Serra;Sao Mateus do Maranhao;Iguatu;Sao Lourenco da Mata;Cachoeira;Mujui dos Campos;Varzea Nova;Salgado de Sao Felix;Alenquer;Crato;Normandia;Quatipuru;Varzea Alegre;Cachoeira dos Indios;Governador Archer;Aldeias Altas;Tramandai;Jurua;Ponto Novo;Nova Timboteua;Carnaubais;Sao Jose de Piranhas;Santo Antonio do Leverger;Sonora;Alvorada D'Oeste;Salinopolis;Inhangapi;Jacarau;Milha;Barras;Barra de Santo Antonio;Ribeirao;Estancia;Uirauna;Ipueiras;Iacu;Taquarana;Cairu;Pilar;Senador Canedo;Camacari;Senador Pompeu;Ibia;Monte Alegre de Sergipe;Silves;Aroeiras;Jaciara;Gurinhem;Serra Caiada;Terra de Areia;Mossoro;Gandu;Batalha;Extremoz;Pedro II;Alvaraes;Capistrano;Firminopolis;Esplanada;Santa Isabel do Rio Negro;Coreau;Peritoro;Prainha;Igrapiuna;Timbiras;Joaquim Pires;Barra do Corda;Ferreiros;Sape;Boa Vista do Tupim;Itapage;Afua;Natuba;Acu;Maracanau;Ipua;Tambe;Pombos;Marechal Taumaturgo;Ipecaeta;Colmeia;Jequie;Solanea;Marco;Tamboril;Lagoa de Itaenga;Ibaretama;Carira;Manaus;Patos;Vigia;Miraima;Nhamunda;Pedro Afonso;Esperantina;Porto de Moz;Sao Luis Gonzaga do Maranhao;Branquinha;Olinda;Quebrangulo;Lagoa Seca;Quixere;Beberibe;Filadelfia;Sao Paulo;Oriximina;Matias Olimpio;Ouro Preto d'Oeste;Frei Paulo;Castro Alves;Faro;Porto Alegre;Malhador;Sucupira do Norte;Forquilha;Nossa Senhora da Gloria;Uruana;Rosario;Nossa Senhora dos Remedios;Picui;Cajueiro;Capela do Alto Alegre;Tracunhaem;Sao Miguel de Touros;Duque Bacelar;Assare;Axixa;Miguel Calmon;Rialma;Maragogi;Rondon do Para;Presidente Kennedy;Arapiraca;Paulista;Benevides;Uruoca;Pau dos Ferros;Mirangaba;Uberaba;Chorozinho;Rio Branco;Guadalupe;Pombal;Piacabucu;Brejinho;Piritiba;Santo Antonio de Jesus;Maues;Itiuba;Ipira;Capela;Santana;Boca do Acre;Riachao do Jacuipe;Choro;Trindade;Natal;Nova Russas;Ipu;Machados;Quixelo;Campo Grande;Trairi;Porto Nacional;Comodoro;Araripe;Abreu e Lima;Sao Joao dos Patos;Caapora;Miranda;Lauro de Freitas;Coite do Noia;Barra do Garcas;Espigao D'Oeste;Pirapemas;Pedregulho;Porto;Placido de Castro;Jaguaribe;Bom Jesus;Fortaleza;Lagoa do Carro;Itirucu;Sao Raimundo das Mangabeiras;Inhumas;Itatim;Tucurui;Estrela de Alagoas;Catarina;Barcelos;Brejoes;Sao Goncalo dos Campos;Anadia;Imaculada;Igarassu;Ariquemes;Amontada;Sao Felipe;Canhotinho;Catunda;Goianira;Picos;Sao Felix do Xingu;Itacare;Itapetim;Palmeiras de Goias;Catu;Anajas;Camamu;Limoeiro;Sao Miguel do Araguaia;Carmo do Rio Verde;Caucaia;Sao Joao;Terra Santa;Pio XII;Passos;Campo Formoso;Nossa Senhora do Livramento;Valenca;Careiro da Varzea;Goianapolis;Colinas do Tocantins;Acarape;Barroquinha;Jardim do Serido;Augusto Correa;Guapo;Porto Acre;Tres Cachoeiras;Caridade;Brejo Santo;Alagoa Nova;Sorriso;Matriz de Camarajibe;Canapolis;Maceio;Camarajibe;Fortim;Aracas;Maruim;Delta;Soure;Terra Alta;Colider;Demerval Lobao;Porangatu;Itau de Minas;Belterra;Ipaporanga;Palmeirais;Japaratinga;Sao Sebastiao do Passe;Itaporanga;Porto Franco;Manicore;Salvaterra;Pindobacu;Coremas;Guimaraes;Abaiara;Ceara-Mirim;Rio do Fogo;Seringueiras;Tamandare;Antenor Navarro;Loreto;Campo do Brito;Marau;Jitauna;Montes Altos;Sirinhaem;Missao Velha;Pereiro;Ruy Barbosa;Paraipaba;Porto Alegre do Norte;Caraubas;Touros;Sao Paulo de Olivenca;Wanderlandia;Nazaria;Palmas;Pinheiro;Massaranduba;Ipaumirim;Porto Valter;Sao Domingos do Maranhao;Jutai;Anajatuba;Labrea;Aramari;Carneiros;Confresa;Utinga;Humaita;Cascavel;Buriti dos Lopes;Cocal;Planura;Carutapera;Porto Grande;Aurora;Santo Amaro;Cajari;Chapadinha;Melgaco;Curuca;Dario Meira;Matinha;Amatura;Luis Gomes;Itarema;Anicuns;Eusebio;Massape;Vila Rica;Afonso Bezerra;Monsenhor Gil;Rolim de Moura;Santa Maria do Para;Fronteiras;Itabaianinha;Macaparana;Barreiros;Paraibano;Vitoria de Santo Antao;Timon;Apuiares;Serrinha;Magalhaes de Almeida;Jacareacanga;Acailandia;Viana;Moncao;Peri-Mirim;Curua;Mata Roma;Sao Joaquim do Monte;Mirador;Sao Jose da Tapera;Xexeu;Tartarugalzinho;Dias d'Avila;Capitao de Campos;Guarabira;Banabuiu;Belem;Nossa Senhora Aparecida;Uruburetama;Jacunda;Colinas;Maracacume;Agrestina;Icatu;Pacoti;Sao Benedito;Maracas;Buenos Aires;Sao Jose de Ribamar;Jaicos;Salinas da Margarida;Conceicao do Coite;Valinhos;Lucas do Rio Verde;Sao Luis de Montes Belos;Vera Cruz;Dom Pedro;Oros;Jardim de Piranhas;Martinopole;Nisia Floresta;Marechal Deodoro;Sao Sebastiao de Lagoa de Roca;Ocara;Brasil Novo;Campo Redondo;Santana do Ipanema;Ibiapina;Santa Teresinha;Cedral;Nova Ipixuna;Regeneracao;Groairas;Almeirim;Graca;Goiatuba;Mamanguape;Cortes;Paraiso do Tocantins;Boa Viagem;Taperoa;Benjamin Constant;Baia Formosa;Brasilia;Jaqueira;Parnamirim;Quirinopolis;Tefe;Mirinzal;Ourem;Caapiranga;Itapitanga;Farias Brito;Miguelopolis;Olho d'Agua das Flores;Cacimba de Dentro;Nobres;Conceicao da Feira;Itaitinga;Itaquitinga;Pirpirituba;Conceicao do Araguaia;Jose de Freitas;Barbalha;Cacimbinhas;Maribondo;Pau d'Alho;Juazeiro do Norte;Itapaci;Santa Luzia;Corrente;Capinopolis;Marcolandia;Jaguaquara;Aquiraz;Poco Branco;Quixeramobim;Mirante da Serra;Caico;Condado;Simao Dias;Paracuru;Redencao;Aquidauana;Santana do Acarau;Poxoreo;Rubiataba;Monsenhor Tabosa;Itatuba;Laje;Bonito;Macau;Pocinhos;Carius;Cabedelo;Chupinguaia;Capim Grosso;Alto do Rodrigues;Urbano Santos;Caririacu;Terra Boa;Araua;Icapui;Ourilandia do Norte;Piquet Carneiro;Uniao;Grossos;Buriti Bravo;Eirunepe;Sao Miguel do Guama;Sao Luis;Alexandria;Manuel Urbano;Alto Garcas;Alagoinhas;Upanema;Miguel Alves;Sao Pedro da Agua Branca;Carpina;Santa Helena de Goias;Fortaleza dos Nogueiras;Chaves;Montanhas;Jaguaripe;Amarante do Maranhao;Nova Ubirata;Saire;Igarape-Acu;Santo Antonio dos Lopes;Rosario do Catete;Potengi;Juina;Sao Jose do Campestre;Teresina;Santa Terezinha de Goias;Sao Goncalo do Amarante;Nova Veneza;Saloa;Joaquim Nabuco;Pedra Preta;Itacoatiara;Jaguaruana;Sao Miguel;Sao Felix do Araguaia;Solonopole;Aripuana;Guaraciaba do Norte;Sao Paulo do Potengi;Crixas;Iraci;Valenca do Piaui;Maracana;Sao Geraldo do Araguaia;Tabuleiro do Norte;Candeias;Fortuna;Tarauaca;Ubajara;Piranhas;Parintins;Cidelandia;Cantanhede;Tutoia;Tabatinga;Beruri;Porto Velho;Novo Lino;Fagundes;Major Isidoro;Sao Benedito do Rio Preto;Poco das Trincheiras;Quixabeira;Caetes;Goianesia;Sao Jose da Laje;Sobral;Vilhena;Morros;Igarapava;Itaiba;Brasileia;Nordestina;Ares;Santa Maria das Barreiras;Rio Largo;Correntes;Inhuma;Passo de Camarajibe;Salvador;Brejao;Brejo +KR;Cheongju;Sejong;Daegu;Pocheon;Andong;Jeju;Hongseong;Gimpo;Muan;Jeonju;Changwon;Heunghae;Haeryong;Busan;Daejeon;Gangneung;Korea, South;Bucheon;Mokpo;Chuncheon;Goyang;Paju;Suncheon;Guryongpo;Suwon;Incheon;Boryeong;Sokcho;Naju;Yangsan;Gwangyang;Gwangju;Pohang;Seoul;Ulsan;Yeosu +MX;La Resolana;Nacajuca;Colotlan;Emiliano Zapata;Tlacolula de Matamoros;Coyotepec;Rioverde;Xicotencatl;Amatitan;Chiconcuac;Rafael Delgado;Coacoatzintla;Cuautla;Choix;Ciudad Mante;Tacambaro de Codallos;Quiroga;Carlos A. Carrillo;Ciudad Altamirano;Benito Juarez;Acatic;La Huerta;Tuxtla;Cocotitlan;Acatzingo;Aguascalientes;Tanhuato de Guerrero;Texcatepec;Cardenas;San Jose del Cabo;Moloacan;Ciudad Madero;Juan Aldama;Cuquio;La Trinitaria;Rayon;Ojinaga;Jalapa;Jalpa de Mendez;Perote;Union de San Antonio;Zacatelco;Huandacareo;Ayutla de los Libres;Ixtlahuaca;Candelaria;Santa Ana Jilotzingo;Paraiso;Polotitlan de la Ilustracion;Morelia;Orizaba;Camargo;Acambaro;Santo Domingo Tehuantepec;Tecoanapa;San Pedro Mixtepec;San Andres Timilpan;Jose Maria Morelos;Chiapa de Corzo;Poza Rica de Hidalgo;Temamatla;Tlajomulco de Zuniga;Guadalajara;Candelaria Loxicha;Puerto Penasco;Silao;Chalco;Cuajinicuilapa;Salvatierra;Churumuco de Morelos;San Pedro Garza Garcia;Soledad de Graciano Sanchez;Santiago;Turicato;Escarcega;Chilapa de Alvarez;Zapotlan del Rey;La Huacana;Mexico City;Tamuin;Las Rosas;Capulhuac;Huauchinango;Petatlan;Navolato;Zontecomatlan de Lopez y Fuentes;Sultepec;Teotitlan;Pajapan;Cuencame de Ceniceros;San Francisco del Rincon;Tampamolon Corona;San Julian;Chimalhuacan;Macuspana;Jilotlan de los Dolores;Mineral del Monte;Tecamachalco;Empalme;Tixkokob;Nautla;Guanajuato;Salamanca;Motozintla;Ascension;Citlaltepec;Platon Sanchez;Heroica Nogales;Zapotlan de Juarez;El Pueblito;Teopisca;Atlatlahucan;Cerritos;Valle Hermoso;Coyuca de Catalan;Sonoita;Puebla;Cuernavaca;San Felipe del Progreso;Ixtaczoquitlan;Uriangato;Cochoapa el Grande;Santa Isabel Cholula;Salinas de Hidalgo;Ciudad de Atlixco;Valle de Bravo;Cunduacan;Yuriria;Santa Maria Huatulco;Abasolo;Ixtlahuacan del Rio;Cananea;Vicente Guerrero;Tlacolulan;Altepexi;Tapachula;Nanchital de Lazaro Cardenas del Rio;Ensenada;Ciudad Serdan;Tlalpujahua de Rayon;Jimenez;Tlalnepantla;Tonala;Chiconquiaco;Filomeno Mata;Magdalena de Kino;Coquimatlan;Reynosa;Ciudad Guzman;Acapetahua;Garcia;La Paz;Texistepec;Ocotlan de Morelos;Cosio;Ocuilan de Arteaga;Chilcuautla;San Andres Tuxtla;Doctor Mora;Magdalena;Tecolotlan;Arriaga;Felipe Carrillo Puerto;Nuevo Casas Grandes;San Pedro Pochutla;Tolcayuca;Juchitan de Zaragoza;Tamazunchale;Tejupilco;San Felipe;Valle Nacional;Tala;Villahermosa;San Luis Potosi;Tizimin;Huasca de Ocampo;Nacozari de Garcia;Santa Cruz Atizapan;Ciudad Guadalupe Victoria;Tehuacan;Altamira;Tlapa de Comonfort;Tecate;Tepetzintla;Jiutepec;Ocosingo;Mazapil;Jiquipilas;Encarnacion de Diaz;Etchojoa;San Pablo Huixtepec;Huejucar;Naucalpan de Juarez;Suchiapa;La Cruz;Saucillo;Yecuatla;Chicoloapan;Tres Valles;Delicias;Oteapan;Mazatlan Villa de Flores;Amatlan de los Reyes;Pinotepa;Zapotiltic;Jonuta;Jocotepec;Charcas;El Espinal;Jesus Maria;San Ignacio Cerro Gordo;Venustiano Carranza;Soledad Atzompa;El Rosario;Hecelchakan;Huejotzingo;Soledad de Doblado;Ayutla;Jaltenco;Tepotzotlan;Yahualica de Gonzalez Gallo;Celaya;Putla Villa de Guerrero;Erongaricuaro;Ciudad Santa Catarina;Tomatlan;Villa Corona;Torreon;San Juan del Rio del Centauro del Norte;San Buenaventura;Pantelho;Atoyac de Alvarez;Tlacotepec;Cozumel;Xico;Monclova;Apizaco;Coatepec;Mecatlan;Uman;Castillo de Teayo;Zimatlan de Alvarez;Ciudad Cuauhtemoc;Acambay;Guasave;San Pedro Jicayan;Tixtla de Guerrero;Espita;Jocotitlan;Ignacio de la Llave;Tepic;Joquicingo;Valparaiso;Campeche;Muna;Playa del Carmen;San Juan del Rio;Xalapa;Mazatan;Sabinas;Santiago Suchilquitongo;Guamuchil;Mineral de Angangueo;Temascalapa;Paso del Macho;Nochistlan de Mejia;Ixhuatlan de Madero;Cuautepec de Hinojosa;Hidalgotitlan;Tecamac;Meoqui;Ometepec;Zumpango del Rio;Champoton;Asuncion Nochixtlan;Texcoco;Ozuluama de Mascarenas;Acanceh;Halacho;Hueyotlipan;Tecali;Rincon de Romos;Mazatlan;Villa Luvianos;Cacahoatan;Tezontepec;San Nicolas de los Ranchos;Tatahuicapan;Lagos de Moreno;Colima;Soyalo;Isla Mujeres;Ixtapaluca;Galeana;Ciudad Benito Juarez;Yecapixtla;Durango;Huamantla;Ezequiel Montes;Juarez;Acatlan;San Pedro Huamelula;Actopan;Tepalcatepec;Villaflores;Monterrey;Playas de Rosarito;Tetela de Ocampo;Tepetlixpa;Texcaltitlan;Jamapa;Almoloya de Alquisiras;Zacualpan;Kanasin;Ozumba;San Blas;Queretaro;Degollado;Jaral del Progreso;Tecuala;Tzintzuntzan;Juventino Rosas;Ciudad Apodaca;Balancan;San Jose Villa de Allende;Ixhuatlancillo;Valle de Santiago;Santa Maria Jacatepec;Ayapango;Nuevo Laredo;Cotija de la Paz;Senguio;Mariano Escobedo;Tapalpa;Zacatecas;Cuitzeo del Porvenir;Frontera;Cerro Azul;Totolapan;Ciudad Miguel Aleman;Santa Maria Chilchotla;Tantoyuca;Union de Tula;San Fernando;Pahuatlan de Valle;Apaseo el Alto;San Jose del Rincon Centro;Toluca;Zapotlanejo;Cosamaloapan;Cadereyta Jimenez;Amatan;Huehuetoca;Chichihualco;Ramos Arizpe;Ocoyoacac;Santiago Tulantepec;Lerma;Ilamatlan;La Independencia;San Carlos Yautepec;Jiquilpan de Juarez;Ciudad de Allende;Panuco;San Miguel de Allende;Chanal;Zacatepec;Soyaniquilpan;Mexicaltzingo;Pachuca;Tamiahua;Ciudad Melchor Muzquiz;Heroica Guaymas;Soteapan;Simojovel de Allende;Guachochi;Naolinco de Victoria;Talpa de Allende;Fortin de las Flores;Coatepec Harinas;Tequila;Tepatitlan de Morelos;Calimaya;Cuilapan de Guerrero;Boca del Rio;Ciudad Hidalgo;Jaltipan de Morelos;Navojoa;Ciudad Lerdo;Atenco;Pijijiapan;Tenancingo;Tehuipango;Ciudad Acuna;Santa Ana Maya;San Pablo Villa de Mitla;Agua Prieta;Huatabampo;Calvillo;Huixcolotla;Ahualulco de Mercado;Almoloya;Tototlan;Cuatro Cienegas de Carranza;San Rafael;General Panfilo Natera;Veracruz;Ameca;Coatzacoalcos;Acaxochitlan;Gomez Palacio;Dolores Hidalgo Cuna de la Independencia Nacional;Tlalixcoyan;Jalacingo;Ocozocoautla de Espinosa;Acala;Manlio Fabio Altamirano;Tenango del Aire;Los Reyes de Juarez;Zaragoza;Santa Maria Xadani;Ixmiquilpan;Juchipila;Nuevo San Juan Parangaricutiro;Pihuamo;Compostela;Mazamitla;Pabellon de Arteaga;Pesqueria;Salinas Victoria;Tlahualilo de Zaragoza;Tlaxcala;Santiago Tuxtla;Etzatlan;Matamoros;Santa Maria Colotepec;Ecatzingo;Ursulo Galvan;Huehuetla;Parras de la Fuente;Manzanillo;Chavinda;Amatepec;Ciudad Constitucion;Piedras Negras;Tulum;Escuintla;Tijuana;Villa Aldama;Cosautlan;Acapulco de Juarez;Zinapecuaro;Alamo;Fresnillo;Tierra Colorada;Iguala de la Independencia;Coroneo;Alamos;San Mateo Atenco;Tulancingo;Guadalupe;Escuinapa;Cueramaro;Aldama;Huatusco;Mexicali;Huanimaro;Tuxpan;Monte Escobedo;Tlalnelhuayocan;San Luis Rio Colorado;Jose Cardel;Temascalcingo;Santiago Papasquiaro;Cuauhtemoc;Huixquilucan;Malinalco;Ciudad Rio Bravo;Zumpango;Chilon;Temoaya;Tezoyuca;Jamay;Comapa;Cuichapa;Comala;Teocuitatlan de Corona;Pichucalco;Amacuzac;Jerez de Garcia Salinas;Tlaquepaque;Santa Rosalia;Tepatlaxco;Tuxpam de Rodriguez Cano;Hueyapan de Ocampo;Atlautla;Chihuahua;Tepoztlan;Ciudad Sabinas Hidalgo;Chilpancingo;Parral;Villanueva;Papantla de Olarte;Ejutla de Crespo;Zempoala;Heroica Caborca;Mexico;Villa Purificacion;Cheran;Hunucma;Las Margaritas;Taxco de Alarcon;Villa de Zaachila;Angel R. Cabada;Cuetzalan;Ciudad General Escobedo;Nava;Tlachichilco;Melchor Ocampo;El Salto;Santa Maria del Tule;San Cristobal;Mecayapan;Maxcanu;Cuapiaxtla de Madero;Miacatlan;Tapilula;Comitan;Tempoal de Sanchez;Altamirano;Zacatlan;Comalapa;Zozocolco de Hidalgo;Merida;Puente Nacional;Ixhuatlan del Sureste;Los Mochis;Chapala;Tepetlan;Linares;Juchique de Ferrer;Amatenango del Valle;Santa Barbara;Puerto Vallarta;Zacoalco de Torres;San Gregorio Atzompa;Tlalmanalco;Villagran;Misantla;San Salvador;Tecoman;Tultepec;Coalcoman de Vazquez Pallares;Calpulalpan;Xalatlaco;Almoloya del Rio;Miahuatlan;Teoloyucan;Tlayacapan;San Pedro;Teapa;Playa Vicente;Mascota;Jalpan;Apan;Leon de los Aldama;Huaquechula;Villa Victoria;Las Vigas de Ramirez;Loreto;Santa Ana Chiautempan;Axochiapan;Cihuatlan;San Jose de Gracia;Bochil;Apaxco de Ocampo;Tlaltetela;Yajalon;Huixtla;Irapuato;Cienega de Flores;Nezahualcoyotl;Hostotipaquillo;Singuilucan;Chinampa de Gorostiza;Santa Maria Jalapa del Marques;Zongolica;San Juan Cancuc;San Jose Tenango;Santiago Tianguistenco;San Bartolo Tutotepec;Zihuatanejo;Ojuelos de Jalisco;San Martin de las Piramides;Tultitlan de Mariano Escobedo;Cintalapa de Figueroa;Tenosique;China;Ciudad de Huajuapam de Leon;Sayula de Aleman;Minatitlan;Tequixquiac;Villa Hidalgo;Mixtla de Altamirano;Doctor Arroyo;Tataltepec de Valdes;Sahuayo de Morelos;Progreso;Ciudad Obregon;Miahuatlan de Porfirio Diaz;Tizayuca;Santiago Ixcuintla;Cuautitlan Izcalli;Ciudad de Huitzuco;Reforma;Apaseo el Grande;San Francisco de los Romo;Isla;El Fuerte;Temascaltepec de Gonzalez;El Oro de Hidalgo;Sayula;Villa Donato Guerra;Patzcuaro;Uruapan;San Juan de los Lagos;Tetela del Volcan;Tepeapulco;Chalma;Culiacan;Cortazar;Tecolutla;Alvarado;San Juan Evangelista;Chapa de Mota;Jamiltepec;Ahumada;Cancun;Ixtlahuacan de los Membrillos;Tezoatlan de Segura y Luna;Pantepec;Francisco I. Madero;Vista Hermosa de Negrete;Oxchuc;Metepec;Tlacoachistlahuaca;Hermosillo;Castanos;Matehuala;Canatlan;Ixtapan de la Sal;Amecameca de Juarez;Naranjos;Ciudad Lopez Mateos;Pueblo Nuevo;Berriozabal;Zinacantepec;San Juanito de Escobedo;Maravatio de Ocampo;Ayotoxco de Guerrero;Villa Juarez;Ocotlan;Tecpan de Galeana;Amatenango de la Frontera;Chocaman;Comalcalco;Ciudad Manuel Doblado;Xochistlahuaca;Montemorelos;Tzitzio;Contla;Cruz Grande;Juanacatlan;Jilotepec;Allende;Axapusco;Huejuquilla el Alto;San Blas Atempa;Zacualpan de Amilpas;Tepechitlan;Totutla;Santa Ana Nextlalpan;San Martin Texmelucan de Labastida;La Piedad;Jaltenango;Nicolas Romero;Tezonapa;Atotonilco el Alto;Gonzalez;Tlalixtac de Cabrera;Cuautitlan;Cazones de Herrera;Santo Tomas de los Platanos;Villa Union;Pajacuaran;Epazoyucan;San Nicolas de los Garza;Penamiller;Ciudad del Carmen;Heroica Ciudad de Tlaxiaco;Heroica Matamoros;Autlan de Navarro;Tepetlaoxtoc;El Arenal;Tlapacoyan;Martinez de la Torre;Santa Maria Tonameca;Cadereyta;La Barca;Cordoba;Tenango de Doria;Madera;Otumba;Juan Rodriguez Clara;Tampico;Oaxaca;Huitzilan;Chetumal;Grajales;Benemerito;San Luis de la Paz;Penjamo;Sombrerete;Zapopan;Nuevo Ideal;Villa del Carbon;Tequisquiapan;Ciudad Valles;Temixco;Rio Grande;La Magdalena Tlaltelulco;Villa de Alvarez;Lerdo de Tejada;Villa Corzo;Tlaltenango de Sanchez Roman;Ciudad Tula;Saltillo;Catemaco;Chenalho;Ciudad Victoria;Santiago Tangamandapio;Pinal de Amoles;Nogales;Santa Clara del Cobre;Ecatepec;Acayucan;Paso de Ovejas;Juchitepec;Tila;Coacalco;Calkini;Santa Ana;Mapastepec;Palenque;Purisima de Bustos +EG;Az Zaqaziq;Kafr ash Shaykh;Al Fayyum;Suhaj;Qina;At Tur;Damanhur;Suez;Asyut;Al Minya;Matruh;Rafah;Aswan;Egypt;Al Kharjah;Ash Shaykh Zuwayd;Luxor;Port Said;Bani Suwayf;Ismailia;Al Mansurah;Damietta;Al Ghardaqah;Banha;Alexandria;Shibin al Kawm;Bi'r al `Abd;Al `Arish;Cairo;Giza;Tanta +BD;Rajshahi;Sylhet;Naogaon;Rangpur;Barishal;Dhaka;Khulna;Mymensingh;Chattogram;Gaibandha;Bangladesh +TH;Chaiyo;Phrai Bueng;Loei;Pa Mok;Phrae;Roi Et;Rattaphum;Phak Hai;Nong Khai;Kamphaeng Phet;Bang Sao Thong;Phuket;Phon Charoen;Sai Buri;Chiang Klang;Chum Phae;Mae Hong Son;Lop Buri;Prachuap Khiri Khan;Phra Phutthabat;Sattahip;Chachoengsao;Bang Phae;Pak Thong Chai;Phra Pradaeng;Chumphon;Thong Pha Phum;Bang Phlat;Mae Sot;Uthai Thani;Salaya;Bang Kruai;Chiang Rai;Bueng Kan;Thung Song;Ranong;Si Sa Ket;Singhanakhon;Akat Amnuai;Nong Bua Lamphu;Bua Yai;Sai Mai;Surat Thani;Yala;Tha Bo;Don Sak;Si Wilai;Det Udom;Rong Kwang;Sing Buri;Nakhon Si Thammarat;Samut Songkhram;Tha Chang;Doi Saket;Wang Sombun;Wang Nam Yen;Bueng Khong Long;Narathiwat;Ban Phai;Kanchanaburi;Maha Sarakham;Saraburi;Ban Na San;Chom Thong;Nakhon Phanom;Thoen;Lamphun;Pak Chong;Kalasin;Ban Bueng;Thung Sai;Sa Kaeo;Bang Bua Thong;Thap Khlo;Phayao;Songkhla;Hat Yai;Huai Yot;Chon Buri;Pak Phanang;Phangnga;Phimai;Warin Chamrap;Kanchanadit;Takua Pa;Phichit;Non Sung;Kaset Wisai;Tha Yang;Phitsanulok;Sakon Nakhon;Nakhon Ratchasima;Klaeng;Doi Lo;Ubon Ratchathani;Pa Sang;Buri Ram;Kantharalak;Tha Mai;Khlong Luang;Mae Ai;Photharam;Tak Bai;Sawankhalok;Rayong;Krathum Baen;Chaiyaphum;Phanat Nikhom;Nong Ki;San Kamphaeng;Suphan Buri;Fao Rai;Chai Prakan;Pran Buri;Tak;Aranyaprathet;Khon Kaen;Taphan Hin;Na Klang;Lam Luk Ka;Khao Yoi;Ang Thong;Ban Chang;Si Satchanalai;Bangkok;Bang Khla;Sadao;Nonthaburi;Phra Nakhon Si Ayutthaya;Wiang Haeng;Sai Ngam;Ko Samui;Bang Racham;Betong;Mukdahan;Dok Kham Tai;Phra Samut Chedi;Pua;Chiang Mai;Tha Luang;Udon Thani;Kosum Phisai;Lampang;Samut Prakan;Wichian Buri;Thap Than;Ban Laem;Trat;Chok Chai;Bang Ban;Nam Som;Wang Saphung;Sikhio;Kathu;Pong Nam Ron;Kantang;Si Racha;Kaeng Khro;Trang;Chanthaburi;Wiang Sa;Nong Khae;Mae Wang;Mae Rim;Pathum Thani;Kranuan;Song Phi Nong;Tha Muang;Nong Bua;Prakhon Chai;Suwannaphum;Satuek;Nakhon Nayok;Nan;Samut Sakhon;Ratchaburi;Nakhon Pathom;Pattani;Cha-am;Ban Pong;Lom Sak;Sukhothai;Nakhon Thai;Phetchabun;Mae Sai;Yasothon;Hua Hin;Khlung;Su-ngai Kolok;Sung Noen;Amnat Charoen;Kaeng Khoi;Pak Kret;Nakhon Sawan;Phetchaburi;Sam Phran;Ban Dung;Den Chai;Satun;Kuchinarai;Nong Wua So;Na Wa;Nang Rong;Phai Sali;Prachin Buri;Phon;Sam Ko;Kut Chap;Yang Talat;Chai Nat;Kamalasai;Phatthalung;Krabi;Surin;Lang Suan;Phibun Mangsahan;Nong Kung Si;Kui Buri;Takhli;Na Yung;Uttaradit;Thailand +RU;Magas;Pavlovskiy Posad;Kem;Baksan;Barnaul;Dzerzhinsk;Primorsko-Akhtarsk;Tuapse;Bezhetsk;Mozhaysk;Rybinsk;Gulkevichi;Shatura;Ardon;Akhtubinsk;Izhevsk;Kirovo-Chepetsk;Taldom;Zhukovo;Argun;Apatity;Magadan;Nytva;Zelenogradsk;Derbent;Gorno-Altaysk;Budennovsk;Kostroma;Saint Petersburg;Baltiysk;Novoselitskoye;Khimki;Peschanokopskoye;Maykop;Yeysk;Konstantinovsk;Krasnyy Sulin;Kargopol;Starodub;Kudymkar;Sharya;Tutayev;Stavropol;Kotelniki;Kokhma;Ivanovo;Petropavlovsk-Kamchatskiy;Elkhotovo;Slavyansk-na-Kubani;Zheleznogorsk;Kashin;Vyazma;Astrakhan;Gryazi;Kimovsk;Krasnyy Yar;Komsomol'sk-na-Amure;Uzlovaya;Novopavlovsk;Seltso;Boksitogorsk;Shakhunya;Tomsk;Kondopoga;Vyksa;Kineshma;Kirishi;Kimry;Zhukovskiy;Kochubeyevskoye;Pervomaysk;Sergach;Lyudinovo;Vyatskiye Polyany;Nalchik;Khasavyurt;Ulan-Ude;Shuya;Gvardeysk;Nizhniy Novgorod;Slobodskoy;Gatchina;Akhty;Sergiyev Posad;Zelenokumsk;Rybnoye;Chaplygin;Tyumen;Mirnyy;Bogorodsk;Krasnoyarsk;Kemerovo;Udomlya;Chudovo;Alagir;Zadonsk;Kirov;Novodvinsk;Yaransk;Tver;Privolzhsk;Lytkarino;Nikel;Vladimir;Tula;Ramenskoye;Dankov;Kozelsk;Kalyazin;Abakan;Rasskazovo;Galich;Kuvshinovo;Volokolamsk;Svetlyy;Ostrov;Dylym;Kameshkovo;Belaya Glina;Pochinok;Losino-Petrovskiy;Nar'yan-Mar;Kungur;Yur'yev-Pol'skiy;Uchkeken;Kstovo;Sasovo;Ostashkov;Gudermes;Uglich;Saratov;Valday;Sarapul;Solnechnogorsk;Labinsk;Pitkyaranta;Mozdok;Belorechensk;Pestovo;Kozmodemyansk;Ipatovo;Shchigry;Belaya Kholunitsa;Pionerskiy;Prokhladnyy;Uren;Lebedyan;Chekhov;Kandalaksha;Lyubertsy;Oboyan;Vyazniki;Rylsk;Borovsk;Anapa;Gavrilov-Yam;Unecha;Belgorod;Bologoye;Saransk;Urzhum;Syktyvkar;Kizlyar;Kursavka;Ikryanoye;Aleksandrovskoye;Krasnogorsk;Stupino;Terek;Cheboksary;Digora;Voronezh;Novosibirsk;Mytishchi;Smolensk;Salekhard;Lukoyanov;Volosovo;Kotelnich;Salsk;Danilov;Furmanov;Sobinka;Kislovodsk;Temryuk;Beslan;Azov;Zhukovka;Anadyr;Aksay;Polyarnyy;Blagoveshchensk;Podporozhye;Pskov;Kizilyurt;Nelidovo;Yaroslavl;Podolsk;Belomorsk;Orel;Noginsk;Korenovsk;Reutov;Volgorechensk;Rzhev;Teykovo;Yuzha;Semenov;Krasnogvardeyskoye;Suzdal;Tikhvin;Priozersk;Dubovskoye;Tambov;Kaliningrad;Korolev;Mozhga;Kovdor;Pokrovskoye;Aleksin;Elektrostal;Semikarakorsk;Lodeynoye Pole;Kursk;Bryansk;Khanty-Mansiysk;Volodarsk;Fokino;Yartsevo;Segezha;Domodedovo;Kingisepp;Kolomna;Fryazino;Millerovo;Stroitel;Serpukhov;Slantsy;Moscow;Yoshkar-Ola;Levashi;Naro-Fominsk;Suoyarvi;Nazran;Ulyanovsk;Tikhoretsk;Odintsovo;Tsimlyansk;Botlikh;Kasimov;Pyatigorsk;Torzhok;Luga;Ertil;Bronnitsy;Apsheronsk;Vyshniy Volochek;Pavlovsk;Krymsk;Chernyakhovsk;Nartkala;Makhachkala;Klintsy;Dubna;Babayurt;Yelets;Novoaleksandrovsk;Malaya Vishera;Yefremov;Rudnya;Velsk;Izobil'nyy;Novozybkov;Orenburg;Pochep;Yessentuki;Sudogda;Narimanov;Ust'-Dzheguta;Skopin;Yekaterinburg;Sortavala;Zvenigovo;Ryazhsk;Kamyzyak;Korablino;Kola;Velikiy Novgorod;Manturovo;Konakovo;Shchekino;Onega;Yasnogorsk;Kulebaki;Melenki;Rodniki;Novomoskovsk;Yegoryevsk;Chaltyr;Soltsy;Ryazan;Vologda;Dmitrov;Kamensk-Shakhtinskiy;Lgov;Timashevsk;Neftekumsk;Mezen;Kambarka;Belev;Lyskovo;Tyrnyauz;Samara;Kazan;Volkhov;Levokumskoye;Pavlovo;Kovrov;Balashikha;Omutninsk;Murmansk;Belaya Kalitva;Kirovsk;Vladikavkaz;Zheleznovodsk;Vsevolozhsk;Petrozavodsk;Zavyalovo;Shali;Zernograd;Divnoye;Cherkessk;Dolgoprudnyy;Khabarovsk;Mayskiy;Pudozh;Nevel;Zavolzhsk;Chita;Kashira;Gusev;Velikiye Luki;Krasnoznamensk;Zuyevka;Pereslavl'-Zalesskiy;Irkutsk;Tarusa;Gus'-Khrustal'nyy;Chernogolovka;Istra;Groznyy;Lipetsk;Safonovo;Gorodets;Dorogobuzh;Omsk;Murom;Medvezhyegorsk;Rossosh;Kirzhach;Yuzhno-Sakhalinsk;Borovichi;Arzgir;Vichuga;Mtsensk;Blagodarnyy;Kotlas;Urus-Martan;Kurchaloy;Mikhaylov;Adygeysk;Orekhovo-Zuyevo;Karachev;Suvorov;Gagarin;Shchelkovo;Surazh;Proletarsk;Klin;Kaluga;Guryevsk;Severodvinsk;Gorokhovets;Elista;Karachayevsk;Arzamas;Kireyevsk;Penza;Pushkino;Severomorsk;Sernovodsk;Svetlogorsk;Abinsk;Luza;Perevoz;Porkhov;Kasumkent;Neman;Votkinsk;Nevinnomyssk;Neya;Nyandoma;Sukhinichi;Volgograd;Lobnya;Malgobek;Kirs;Birobidzhan;Petushki;Buynaksk;Pechory;Achkhoy-Martan;Yelnya;Morozovsk;Rostov;Lermontov;Kolchugino;Karabulak;Obninsk;Staraya Russa;Kyzyl;Kurchatov;Sarov;Perm;Vidnoye;Navashino;Toropets;Nolinsk;Livny;Usman;Kurgan;Svetlograd;Georgiyevsk;Voskresensk;Krasnodar;Vladivostok;Volzhsk;Opochka;Aleksandrov;Roslavl;Vyborg;Venev;Ufa;Chelyabinsk;Donskoy;Russia;Buy;Mineral'nyye Vody;Chkalovsk;Plavsk;Tosno;Koryazhma;Glazov;Arkhangelsk;Sovetsk;Nerekhta;Kostomuksha;Bogoroditsk;Kharabali;Kondrovo;Yakutsk;Likhoslavl;Dzerzhinskiy;Ruza;Maloyaroslavets;Kurganinsk +AR;Colon;Famailla;Carhue;Las Flores;San Justo;Pigue;Ranchos;General Jose de San Martin;General Juan Madariaga;Esteban Echeverria;Santiago del Estero;Obera;Chamical;Azul;La Quiaca;Jesus Maria;Berisso;Rio Cuarto;Pilar;General Villegas;El Calafate;Villa Angela;Benito Juarez;San Ramon de la Nueva Oran;San Isidro;Zarate;Quimili;Chacabuco;Bernardo de Irigoyen;San Rafael;Apostoles;Abra Pampa;Bragado;Joaquin V. Gonzalez;Junin de los Andes;Chascomus;Las Heras;Rauch;Gualeguay;Necochea;Concepcion del Uruguay;General Acha;La Leonesa;Gobernador Virasora;San Bernardo;Aguilares;Merlo;Gastre;General Las Heras;Coronel Dorrego;Tres Arroyos;Baradero;Campo Largo;Villa Berthet;Alto Rio Senguer;Anatuya;Santa Rosa de Rio Primero;Mar del Plata;Lomas de Zamora;Esquel;Moron;Malargue;General Rodriguez;Empedrado;Capitan Sarmiento;Villa La Angostura;Esperanza;Tandil;Termas de Rio Hondo;Daireaux;Rosario del Tala;Carmen de Areco;Banda del Rio Sali;Monte Quemado;San Miguel de Tucuman;Reconquista;Diamante;Cerrillos;Formosa;Tostado;Lincoln;General Martin Miguel de Guemes;Nueve de Julio;Andalgala;Berazategui;General Pinedo;Goya;San Juan;Pehuajo;Canuelas;Maipu;Canada de Gomez;Ushuaia;La Cruz;San Luis;Buenos Aires;Jose Maria Ezeiza;General San Martin;Rosario;Resistencia;Monte Caseros;Sierra Colorada;Rodeo;Concepcion;Quitilipi;Balcarce;Las Brenas;Chepes;Puerto Madryn;Miramar;Las Rosas;Comodoro Rivadavia;General Alvear;Puan;Castelli;Nogoya;La Rioja;Federal;Ramallo;Libertador General San Martin;Villa Ojo de Agua;Jose C. Paz;Tornquist;Saladas;Viedma;Santa Fe;Rivadavia;General Conesa;San Julian;Bell Ville;Capilla del Senor;Laboulaye;General Belgrano;Ingeniero Guillermo N. Juarez;Victoria;Guernica;Perito Moreno;Arrecifes;Carmen de Patagones;Caucete;Gualeguaychu;Caseros;San Martin de los Andes;Machagai;Villa Maria;Lanus;Monteros;Belen de Escobar;Hurlingham;Palpala;Villa Paranacito;Campana;Villa del Rosario;Gobernador Gregores;Chimbas;Lujan de Cuyo;Zapala;Frias;Argentina;Lobos;Pergamino;General Viamonte;Posadas;Catamarca;Concordia;Villa Mercedes;San Jose de Jachal;San Roque;Parana;Henderson;Adolfo Gonzales Chaves;Chilecito;Carlos Casares;Santa Rosa;Godoy Cruz;Puerto Tirol;Charata;Rio Gallegos;Villa Constitucion;Federacion;Clorinda;San Vicente;Casilda;Tinogasta;Victorica;Ensenada;Alta Gracia;Rosario de Lerma;Rafaela;San Antonio de Areco;Neuquen;La Banda;Dean Funes;Belen;Cosquin;Villaguay;San Andres de Giles;Presidencia Roque Saenz Pena;Avellaneda;Oliva;Choele Choel;Sauce;San Jose de Feliciano;General Pico;Puerto Rico;San Cristobal;Puerto Deseado;La Plata;Villa Krause;Chumbicha;Mburucuya;Villa Union;San Fernando;Chivilcoy;La Carlota;Marcos Juarez;Marcos Paz;Susques;La Paz;Bahia Blanca;Olavarria;Mercedes;San Francisco;Cruz del Eje;Leandro N. Alem;Vera;Los Polvorines;Corzuela;Pinamar;Tafi Viejo;Veinticinco de Mayo;Ituzaingo;Villa Aberastain;Mendoza;Ayacucho;Cafayate;Rosario de la Frontera;Santa Sylvina;General Roca;San Martin;San Isidro de Lules;Cordoba;Sarmiento;Magdalena;Telsen;Tartagal;Villa Gesell;Coronda;Loberia;San Nicolas de los Arroyos;Lujan;Junin;Comandante Fontana;Tres Isletas;Corrientes;Presidencia de la Plaza;San Salvador;Yerba Buena;San Salvador de Jujuy;Dolores;San Antonio Oeste;San Luis del Palmar;Navarro;Humahuaca;Santa Lucia;Eldorado;San Javier;Pirane;Pampa del Infierno;Salta;Bella Vista;Rinconada;Paso de los Libres;Rio Grande;San Carlos de Bariloche;San Pedro;Tunuyan;Trenque Lauquen;Curuzu Cuatia;Roque Perez;Las Lajas;Florencio Varela;Quilmes;Rawson;Brandsen;Punta Alta;Santa Maria;Coronel Suarez;Chos Malal;Santo Tome;San Antonio de los Cobres;Trancas +NG;Kaduna;Somolu;Auchi;Umuahia;Ede;Abeokuta;Aba;Dutse;Yenagoa;Ilobu;Owo;Damboa;Igboho;Asaba;Chibok;Baure;Zango;Nguru;Ipokia;Nsukka;Biu;Ikot Ekpene;Ibadan;Offa;Kaura Namoda;Orlu;Wukari;Minna;Lokoja;Otukpo;Agege;Jalingo;Lafia;Malumfashi;Isanlu;Ikom;Birnin Kebbi;Epe;Hong;Ugep;Abakaliki;Sokoto;Bama;Mubi;Calabar;Potiskum;Kontagora;Ondo;Kukawa;Sapele;Akure;Apapa;Ile-Ife;Numan;Enugu;Nnewi;Azare;Ikot Abasi;Okene;Iwo;Awka;Makurdi;Zaria;Kano;Funtua;Owerri;Shagamu;Nigeria;Kisi;Gashua;Benin City;Gusau;Osogbo;Bauchi;Gwoza;Ogbomoso;Katsina;Idah;Ilesa;Iseyin;Kuje;Warri;Abuja;Ipetumodu;Osisioma;Maiduguri;Jos;Keffi;Gombe;Lagos;Uyo;Ado-Ekiti;Mushin;Abaji;Burutu;Yola;Mai'Adua;Oyo;Ikeja;Port Harcourt;Saki;Kumo;Ikare;Ilorin;Gwagwalada;Damaturu;Onitsha;Ijebu-Ode;Bida +TR;Alasehir;Harran;Andirin;Selcuk;Kocaeli;Golcuk;Iznik;Duzici;Kepsut;Salihli;Kavaklidere;Gebze;Yatagan;Askale;Turgutlu;Sulakyurt;Alanya;Karapinar;Civril;Bolvadin;Karamursel;Koprukoy;Didim;Sivas;Battalgazi;Karkamis;Sungurlu;Sinanpasa;Tosya;Eruh;Ardesen;Gemerek;Cicekdagi;Soke;Selcuklu;Konak;Sorgun;Ayvalik;Arhavi;Konya;Koycegiz;Karakocan;Sariveliler;Yigilca;Pamukova;Bodrum;Devrek;Turkeli;Ciftlik;Kars;Ankara;Erfelek;Kelkit;Yesilova;Guneysu;Hassa;Hazro;Ozalp;Malkara;Sarkisla;Pendik;Arifiye;Caycuma;Egirdir;Catalpinar;Ceyhan;Kapakli;Catak;Kavak;Gorele;Pazar;Golbasi;Gumushane;Yenimahalle;Of;Beydag;Muradiye;Cesme;Siirt;Cerkezkoy;Unye;Nazilli;Taskopru;Tonya;Erbaa;Kilis;Bitlis;Ceylanpinar;Solhan;Armutlu;Nilufer;Bafra;Kovancilar;Yomra;Aralik;Cayeli;Avanos;Terme;Niksar;Adana;Gomec;Mudanya;Menderes;Simav;Golmarmara;Mazidagi;Taslicay;Istanbul;Kirkagac;Akdagmadeni;Erzin;Buldan;Akhisar;Bilecik;Cobanlar;Baykan;Oltu;Gole;Saraykent;Kumlu;Havsa;Mardin;Aksaray;Kadinhani;Cat;Akkus;Mezitli;Toprakkale;Dursunbey;Osmangazi;Antalya;Ulubey;Huyuk;Gevas;Cankiri;Ayancik;Sebin Karahisar;Susuz;Kayseri;Yakacik;Sultandagi;Gumusova;Safranbolu;Ferizli;Meric;Tunceli;Altinova;Buyukcekmece;Cungus;Dargecit;Kucukkuyu;Espiye;Cubuk;Ortakoy;Vize;Sarkikaraagac;Acigol;Hayrabolu;Golyaka;Sabanozu;Kecioren;Kesan;Uzumlu;Ula;Harbiye;Sultanhani;Suruc;Elmadag;Bayramic;Oguzeli;Ilic;Alacati;Diyadin;Gerze;Yaprakli;Kozan;Polatli;Digor;Bahcesaray;Urla;Mut;Kastamonu;Resadiye;Akyurt;Kizilcahamam;Derinkuyu;Siran;Cal;Saruhanli;Sarioglan;Malatya;Menemen;Gumushacikoy;Kiraz;Isparta;Ahmetli;Besni;Haskoy;Cildir;Can;Cumra;Dortyol;Datca;Golpazari;Soma;Sarayonu;Hinis;Divrigi;Varto;Arapgir;Adakli;Osmancik;Acipayam;Adalar;Kulu;Sivrihisar;Muratli;Dereli;Turkey;Sogut;Carsamba;Tekkekoy;Akcaabat;Yildizeli;Suhut;Bozdogan;Arsuz;Cifteler;Kahramanmaras;Van;Manavgat;Demre;Marmaris;Gordes;Cekerek;Kangal;Denizli;Hidirbey;Samsun;Yakakent;Bergama;Midyat;Korkut;Sariyer;Gonen;Sutculer;Nizip;Tasucu;Karatas;Yumurtalik;Asarcik;Serinhisar;Duzkoy;Camardi;Incirliova;Alapli;Tatvan;Kaynarca;Kirsehir;Osmaneli;Eregli;Cavdir;Karaburun;Tekirdag;Goksun;Yenipazar;Afsin;Sincan;Hani;Manisa;Hatay;Pinarbasi;Dicle;Pazarcik;Beypazari;Alpu;Kocasinan;Umurlu;Sason;Kagizman;Siverek;Sanliurfa;Arakli;Marmara Ereglisi;Kinik;Ortaklar;Eskisehir;Ahlat;Honaz;Yahyali;Tire;Amasra;Sariz;Dogubayazit;Camoluk;Susehri;Ulukisla;Elbistan;Ciftlikkoy;Sarkoy;Mamak;Nigde;Balikesir;Ezine;Karayazi;Guneysinir;Tasova;Bayburt;Gulsehir;Cayiralan;Corlu;Erdemli;Banaz;Kale;Incesu;Talas;Tortum;Eynesil;Dosemealti;Golkoy;Karaman;Bozkurt;Kutahya;Senkaya;Urgup;Belen;Canakkale;Tavas;Kosk;Mersin;Domanic;Kilimli;Kozluk;Genc;Tirebolu;Foca;Lice;Yavuzeli;Bingol;Ermenek;Mecitozu;Zonguldak;Korkuteli;Caldiran;Semdinli;Corum;Silivri;Kulp;Yalova;Serik;Amasya;Gaziantep;Refahiye;Ihsaniye;Uludere;Ayvacik;Gerede;Ivrindi;Kocakoy;Kozlu;Sultanhisar;Sile;Hisarcik;Kursunlu;Sinop;Birecik;Borcka;Sogutlu;Yuksekova;Ispir;Korfez;Cinar;Balya;Araban;Luleburgaz;Meram;Akseki;Gurgentepe;Savsat;Gemlik;Samandag;Aydin;Fatsa;Lapseki;Erdek;Tufanbeyli;Gokcebey;Yozgat;Emirgazi;Bismil;Doganhisar;Zile;Celtik;Pozanti;Ortaca;Adiyaman;Aliaga;Akyazi;Aktepe;Kayapinar;Agri;Dogansehir;Burhaniye;Sakarya;Rize;Salpazari;Caglayancerit;Tokat;Akcakoca;Erzurum;Gaziemir;Palu;Diyarbakir;Reyhanli;Bulancak;Sarikamis;Karpuzlu;Ondokuzmayis;Gazipasa;Bucak;Gallipoli;Anamur;Surmene;Keskin;Ipsala;Pervari;Imamoglu;Selim;Yahsihan;Kemalpasa;Cizre;Hakkari;Hacilar;Trabzon;Camas;Dikili;Halfeti;Bartin;Yaglidere;Cinarcik;Hendek;Narman;Seyitgazi;Sur;Yenisehir;Alaca;Bursa;Kadirli;Turkoglu;Buyukorhan;Vezirkopru;Bayat;Kucukcekmece;Inegol;Cayirova;Aybasti;Hizan;Karabuk;Egil;Kurtalan;Taskent;Mugla;Bor;Artvin;Kumru;Igdir;Kabatas;Mus;Yerkoy;Viransehir;Nurdagi;Sarikaya;Tuzluca;Senirkent;Yildirim;Dalaman;Tut;Zara;Savur;Eleskirt;Karaisali;Gulagac;Aglasun;Iskenderun;Cihanbeyli;Bornova;Sivrice;Guclukonak;Tomarza;Nevsehir;Turhal;Yuregir;Suluova;Patnos;Catalca;Karasu;Fethiye;Batman;Bahce;Cide;Aydincik;Derik;Sereflikochisar;Usak;Edirne;Arpacay;Cerkes;Vakfikebir;Silifke;Torbali;Gorukle;Milas;Cumayeri;Omerli;Susurluk;Gurun;Giresun;Findikli;Piraziz;Sirnak;Kusadasi;Saimbeyli;Alacam;Tekman;Eceabat;Saricam;Ordu;Kocarli;Kirikkale;Babaeski;Elmali;Manyas;Bolu;Yesilhisar;Nurhak;Aslanapa;Sindirgi;Kas;Sarigol;Bayindir;Cankaya;Cermik;Horasan;Baglar;Akcakale;Arsin;Serinyol;Mucur;Bozkir;Bandirma;Baskil;Uzundere;Kirikhan;Germencik;Sirvan;Bala;Goynucek;Selendi;Aladag;Afyonkarahisar;Develi;Gursu;Cay;Cine;Beysehir;Biga;Koyulhisar;Karacabey;Demirozu;Edremit;Tarsus;Emet;Akyaka;Silvan;Altinozu;Kalecik;Besiri;Cayirli;Carsibasi;Emirdag;Osmaniye;Yunak;Altindag;Davutlar;Kose;Hilvan;Nusaybin;Yalvac;Arac;Bogazliyan;Feke;Sapanca;Bozuyuk;Geyve;Caykara;Yayladagi;Cameli;Orhaneli;Aksehir;Mesudiye;Orhangazi;Buharkent;Caybasi;Silopi;Idil;Pasinler;Havran;Kula;Karahalli;Alucra;Karakopru;Bulanik;Mutki;Seyhan;Islahiye;Baskale;Sefaatli;Altinyayla;Altintas;Altunhisar;Aksu;Kesap;Karliova;Saray;Ayas;Gediz;Yenice;Golhisar;Torul;Sivasli;Saraykoy;Duzce;Odemis;Kahta;Adilcevaz;Nallihan;Esme;Bigadic;Yazihan;Basmakci;Ulus;Beykoz;Yusufeli;Savastepe;Izmir;Ilgin;Karsiyaka;Aricak;Seferhisar;Seydisehir;Kocaali;Ekinozu;Mustafakemalpasa;Gurpinar;Almus;Pazaryeri;Uzunkopru;Gokceada;Besikduzu;Ergani;Burdur;Guney;Karapurcek;Yesilli;Ercis;Derince;Havza;Karacasu;Kumluca;Demirci;Altinekin;Etimesgut;Inebolu;Tefenni;Haymana;Koprubasi;Bozyazi;Kartal;Yesilyurt;Evren;Melikgazi;Merzifon;Boyabat;Buca;Persembe;Kandira;Iskilip;Sumbas;Karacoban;Ikizce;Hadim;Kiziltepe;Guroymak;Korgan;Darende;Bunyan;Bozova;Elazig;Kirklareli;Macka;Puturge;Hopa;Kuyucak;Akcadag;Kaman;Keles;Ardahan;Duragan;Finike;Tavsanli;Cilimli;Malazgirt;Erzincan;Cukurca;Kaynasli;Camliyayla;Gulnar +PK;Tando Muhammad Khan;Jacobabad;Chilas;Peshawar;Turbat;Jamshoro;Gujrat;Gujranwala;Sahiwal;Khanewal;New Mirpur;Pishin;Umarkot;Jhelum;Ziarat;Timargara;Dera Allahyar;Jhang City;Uthal;Kohat;Saidu Sharif;Kalat;Vihari;Thatta;Badin;Gakuch;Quetta;Muzaffargarh;Sialkot City;Bhakkar;Khushab;Shekhupura;Bannu;Lahore;Swabi;Hyderabad City;Mardan;Kotli;Khairpur Mir's;Karak;Lodhran;Panjgur;Gwadar;Gilgit;Dera Ghazi Khan;Haripur;Bahawalnagar;Okara;Faisalabad;Bahawalpur;Batgram;Hangu;Chiniot;Abbottabad;Karachi;Ghotki;Sanghar;Mirpur Khas;Sargodha;Bagh;Naushahro Firoz;Pakpattan;Kharan;Tando Allahyar;Pakistan;Dadu;Mandi Bahauddin;Sukkur;Nowshera;Tank;Nawabshah;Chitral;Hafizabad;Zhob;Rawalpindi;Islamabad;Charsadda;Larkana;Toba Tek Singh;Kasur;Mianwali;Multan;Chakwal;Mansehra;Dera Ismail Khan +VN;Thanh Phu;Son La;Hung Yen;Tay Ninh;Thai Binh;Con Dao;Buon Ma Thuot;Cam Ranh;An Chau;Nha Trang;Phu Tho;My Tho;Dong Ha;A Yun Pa;Ha Giang;Ho Chi Minh City;Hoa Binh;Bac Kan;Ky Anh;Da Nang;Kien Giang;Nga Bay;Haiphong;Soc Trang;Tam Ky;Kien Luong;Dong Hoi;Huong Tra;Tuy Hoa;Duc Pho;Cho Moi;Pleiku;Cao Bang;Di Linh;Bien Hoa;Phan Rang-Thap Cham;Long My;Thu Duc;Vung Tau;Thu Dau Mot;An Phu;Yen Bai;Thanh Hoa;Cho Lach;Phu Quoc;Phan Thiet;Trang Bang;Phuc Yen;Lang Son;Vi Thanh;Di An;Long Xuyen;Hanoi;Rach Gia;An Nhon;Tan Chau;Sa Pa;Quang Ngai;Hai Duong;Gia Nghia;Phu Ly;Kon Tum;Dong Xoai;Duyen Hai;Ba Don;Long Thanh;Viet Tri;Ha Long;Ca Mau;Ha Tinh;Bac Giang;Buon Trap;Mong Cai;Thai Nguyen;Vinh;Ninh Hoa;Bo;Tan Phu;Nam Dinh;Long Khanh;Kinh Mon;Quy Nhon;Binh Minh;Son Tay;Bac Ninh;Thi Tran Ngai Giao;Vinh Yen;Vietnam;Huong Canh;Hue;Tam Diep;Dien Bien Phu;Cam Pha;Vinh Chau;Ben Tre;Chon Thanh;Ha Tien;Bao Loc;Lai Chau;Lao Cai;Song Cau;Can Tho;Tan Uyen;Khe Sanh;Tan An;Tuyen Quang;Cao Lanh;Quang Yen;Tra Vinh;Bac Lieu;Vinh Long;Ninh Binh;Hoa Thanh;Sa Dec;Hang Tram;Da Lat +IR;Bastak;Ahram;Qarah Zia' od Din;Lordegan;Quchan;Kavar;Rasak;Kalardasht;Bandar `Abbas;Shirvan;Bandar-e Gaz;Asadabad;Sabzevar;Ramsar;Komijan;Showt;Fannuj;Ravar;Semirom;`Aliabad-e Katul;Qazvin;Qarchak;Shahin Dezh;Zarrin Shahr;Zaboli;Boshruyeh;Heris;Torbat-e Heydariyeh;Ahvaz;Dalgan;Bojnurd;Ardal;Dehloran;Kermanshah;Aligudarz;Neqab;Sonqor;Neyriz;Bahar;Dehaqan;Kaleybar;Bam;Ashtian;Zehak;Arsanjan;Khash;Osku;Piranshahr;Garmsar;Meshgin Shahr;Namin;Robat Karim;Sari;Estahban;Pakdasht;Azarshahr;Basht;Khorramshahr;Mehran;Shahr-e Kord;Abhar;Razan;Poldasht;`Anbarabad;Gerash;Charam;Rabor;Rafsanjan;Zahedan;Kahnuj;Nazarabad;Sa`adat Shahr;Sarableh;Borujerd;Bajestan;Bafq;Landeh;Ilam;Sarakhs;Miandasht;Fereydunshahr;Hamadan;Ramshir;Zanjan;Nowshahr;Varamin;Suran;Mirjaveh;Babolsar;Qorveh;Khomeyn;Tabas;Tabriz;Semnan;Saqqez;Azadshahr;Anar;Qa'en;Birjand;Shahin Shahr;Khalkhal;Hendijan;Shahr-e Herat;Borujen;Tiran;Qom;Paveh;Aleshtar;Jam;Torqabeh;Galugah;Kherameh;Bandar-e Bushehr;Sarbisheh;Ashkhaneh;Saravan;Joghtay;Khorramdarreh;Seyah Cheshmeh;Do Gonbadan;Ahar;Dowlatabad;Eslamshahr;Taft;Yazd;Mashhad;Dehgolan;Farashband;Sarpol-e Zahab;Bukan;Marvdasht;Baneh;Sorkheh;Damavand;Nehbandan;Torbat-e Jam;Darreh Shahr;Sepidan;Ferdows;Kalaleh;Bandar-e Torkaman;Kuhbanan;Bandar-e `Asaluyeh;Kuhdasht;Kangan;Malekan;Farsan;Sarayan;Kashmar;Nikshahr;Ardestan;Jiroft;Shiraz;Mobarakeh;Galikesh;Astaneh-ye Ashrafiyeh;Fariman;Harsin;Germi;Najafabad;Eqlid;Hashtrud;Maragheh;Omidiyeh;Sowme`eh Sara;Marand;Gotvand;Golpayegan;Andimeshk;Ardabil;Shushtar;Taybad;Ravansar;Delijan;Dargaz;Hoveyzeh;Oshnaviyeh;Bardaskan;Fasa;Maraveh Tappeh;Langarud;Aran Bidgol;Kabudarahang;Bileh Savar;Amlash;Susangerd;Feyzabad;Gomishan;Alvand;Khorramabad;Javanrud;Hamidiyeh;Amol;Ben;Mahallat;Damghan;Khvansar;Takab;Fardis;Azna;Esfarayen;Tuyserkan;Jolfa;Karaj;Meybod;Kerman;Shabestar;Siahkal;Bandar-e Khamir;Shahreza;Shush;Fuman;Bijar;Shahr-e Qods;Arak;Sanandaj;Likak;Esfahan;Hashtpar;Ramhormoz;Ashkezar;Dorud;Zarand;`Ajab Shir;Zabol;Takestan;Kut-e `Abdollah;Jask;Masjed Soleyman;Khowrmuj;Eshtehard;Abarkuh;Yasuj;Qasr-e Shirin;Babol;Shahr-e Babak;Neyshabur;Gonabad;Sahneh;Manujan;Khomeyni Shahr;Sardasht;Dezful;Tonekabon;Salmas;Sarvestan;Rasht;Khowy;Rudsar;Malard;Naqadeh;Gilan-e Gharb;Safashahr;Hashtgerd;Kazerun;Orumiyeh;Eyvan;Natanz;Kashan;Mahdishahr;Khonj;Gonbad-e Kavus;Iran;Divandarreh;Shahriar;Bardsir;Konarak;Qeshm;Eslamabad-e Gharb;Arakvaz-e Malekshahi;Bandar-e Mahshahr;Poldokhtar;Qasr-e Qand;Surak;Bandar-e Deylam;Rezvanshahr;Lamerd;Khvaf;Garmeh;Jahrom;Malayer;Tafresh;Jajarm;Mianeh;Bandar-e Lengeh;Shahrud;Dehdasht;Qal`eh Ganj;Sarab;Nahavand;Bandar-e Genaveh;Minudasht;Haftkel;Baft;Mohammadabad;Shazand;Bonab;Fereydun Kenar;Deyr;Behshahr;Tehran;Marivan;Bandar-e Anzali;Behbahan;Sirjan;Famenin;Ardakan;Saveh;Chabahar;Gorgan;Neka;Borazjan;Shadegan;Abyek;Iranshahr;Bagh-e Malek;Falavarjan +CD;Gemena;Inongo;Mbandaka;Kindu;Goma;Isiro;Mbuji-Mayi;Lusambo;Kabinda;Gbadolite;Matadi;Kananga;Bandundu;Lisala;Buta;Kenge;Kisangani;Bunia;Kamina;Kolwezi;Kalemie;Congo (Kinshasa);Tshikapa;Boende;Lubumbashi;Bukavu;Kinshasa +GB;London +FR;Apt;Evreux;Nanterre;Lorient;Fougeres;Vierzon;Arcachon;Chambery;Mantes-la-Jolie;Briancon;Evry;Cholet;Chateau-Gontier;Luneville;Soissons;Parthenay;Avignon;Sarlat-la-Caneda;Saint-Brieuc;Clermont-Ferrand;Paris;Bordeaux;Besancon;Toulon;Arles;Thionville;Grenoble;Albi;Tours;Issoudun;Colmar;Lens;Torcy;Avranches;Ales;Toul;Draguignan;Sarcelles;Boulogne-Billancourt;Nogent-sur-Marne;Bourg-en-Bresse;Pau;Riom;Abbeville;Marseille;Rennes;Bergerac;Cherbourg;Vannes;Tarbes;Montargis;Grasse;Beauvais;Vendome;Cognac;Vienne;Senlis;Metz;Tulle;Bonneville;Foix;Epernay;Meaux;Amiens;Cergy;Caen;Laon;Perpignan;Vesoul;Cahors;Angouleme;Castres;Chalons-en-Champagne;Palaiseau;Aurillac;Dax;Antony;Orleans;Compiegne;Douai;Montauban;Carpentras;Figeac;Limoges;Mulhouse;Bourges;Digne-les-Bains;Strasbourg;Lille;Dijon;Epinal;Rambouillet;Le Mans;Chateauroux;Rodez;Marmande;Blois;Pontarlier;Troyes;La Fleche;Fontainebleau;Bernay;Creteil;Chartres;Saint-Julien-en-Genevois;Bressuire;Alencon;Issoire;Nantes;Nancy;Belfort;Castelsarrasin;Provins;Le Havre;La Roche-sur-Yon;Gap;Cosne sur Loire;Saint-Denis;Boulogne-sur-Mer;Autun;Toulouse;Mayenne;Auxerre;Mont-de-Marsan;Sedan;Saint-Omer;Nevers;Bethune;Perigueux;Les Sables-d'Olonne;Angers;Roanne;Chateaudun;Saint-Quentin;Bayeux;Mende;Laval;Lannion;Montbrison;Saint-Dizier;Verdun;Pamiers;Chatellerault;Oloron-Sainte-Marie;Saint-Gaudens;Vitry-le-Francois;Clermont;Etampes;France;Chalon-sur-Saone;Dole;Villefranche-sur-Saone;Dinan;Chateaubriant;Valence;Limoux;Saint-Lo;Melun;Valenciennes;Bayonne;Thonon-les-Bains;Millau;Chaumont;Chateau-Thierry;Poitiers;Thiers;Le Raincy;Lyon;Sarreguemines;Carcassonne;Saint-Etienne;Aix-en-Provence;Bobigny;Saint-Nazaire;Rochefort;Libourne;Auch;Lons-le-Saunier;Pontivy;Villefranche-de-Rouergue;Istres;Cambrai;Sarrebourg;Le Puy-en-Velay;Molsheim;Dieppe;Sens;Villeneuve-sur-Lot;Vire;Pithiviers;Beziers;Brive-la-Gaillarde;Saint-Die-des-Vosges;Dunkerque;Charleville-Mezieres;Forbach;Albertville;Nice;Ussel;Nogent-le-Rotrou;Brignoles;Saverne;Moulins;Nimes;Annecy;L'Hay-les-Roses;La Rochelle;Bar-le-Duc;Gex;Saumur;Saintes;Argenteuil;Belley;Calais;Montbeliard;Saint-Claude;Lisieux;Redon;Narbonne;Beaune;Fontenay-le-Comte;Reims;Muret;Haguenau;Morlaix;Montlucon;Selestat;Gueret;Brest;Niort;Privas;Versailles;Tournon-sur-Rhone;Rouen;Saint-Germain-en-Laye;Arras;Quimper;Saint-Amand-Montrond;Macon;Agen;Vichy;Dreux;Ajaccio;Argentan;Montpellier;Saint-Malo +PE;Cusco;Chachapoyas;Puno;Trujillo;Abancay;Ayacucho;Pucallpa;Chiclayo;Pampas;Cerro de Pasco;Iquitos;Ica;Huacho;Arequipa;Piura;Huancavelica;Peru;Callao;Tacna;Puerto Maldonado;Huancayo;Huanuco;Tumbes;Moyobamba;Moquegua;Lima;Cajamarca;Contamana;Huaraz +AO;Sumbe;Ndalatando;Malanje;Benguela;Ondjiva;Luanda;Angola;Caxito;Huambo;Uige;Cabinda;Dundo;Luena;Mocamedes;Mbanza Kongo;Lubango;Saurimo;Menongue;Cuito +MY;Labuan;Putrajaya;Kota Bharu;Kuantan;George Town;Kuala Terengganu;Malaysia;Kota Kinabalu;Kangar;Alor Setar;Shah Alam;Johor Bahru;Kuching;Kuala Lumpur;Ipoh;Melaka;Seremban;Tumpat +ZA;Polokwane;Bhisho;Cape Town;Nelspruit;Pretoria;Mahikeng;Pietermaritzburg;Bloemfontein;South Africa;Johannesburg;Kimberley +CO;Yarumal;Cucuta;Neiva;Cajica;Arboledas;Puerto Berrio;Ipiales;Padilla;Pradera;Ospina;Cienaga;Cicuco;Darien;Chia;El Carmen de Bolivar;Mitu;Muzo;Cachira;Piedecuesta;San Jose del Fragua;Tauramena;Morroa;Jerico;Puerto Boyaca;Candelaria;Itagui;San Carlos de Guaroa;El Dovio;San Jose del Guaviare;Popayan;Mistrato;Vistahermosa;Garagoa;Cartago;La Apartada;Manaure;Zarzal;Marquetalia;Combita;Medellin;Tumaco;San Bernardo del Viento;San Zenon;San Jeronimo;Ituango;Ansermanuevo;Chima;Socota;Cartagena;Totoro;Curiti;Sopetran;Moniquira;Tarqui;Abejorral;Genova;Bosconia;Samaniego;Puerto Libertador;Agustin Codazzi;Istmina;Nemocon;Santa Rosa de Cabal;Balboa;Guatica;Tame;Curumani;Fundacion;Sonson;Santo Domingo;Monteria;La Jagua de Ibirico;El Paujil;Angostura;El Carmen;Pacora;Los Cordobas;Aracataca;Buesaco;Susa;Tocaima;Puerto Gaitan;El Retorno;Puerto Lopez;La Argentina;Espinal;Guarne;Toledo;Pereira;Pelaya;Mosquera;Arjona;San Jacinto;San Vicente;Apia;Carmen de Apicala;Guamal;Tocancipa;Mocoa;San Estanislao;El Banco;Maicao;Villagarzon;Sabanalarga;Salgar;Iscuande;Pasca;Leticia;San Juan de Uraba;Cajamarca;Cotorra;Pueblo Bello;Jambalo;Monitos;El Copey;Inirida;Carmen de Viboral;Tuchin;Nobsa;Anori;Pacho;El Cairo;Chinu;Sibate;Juan de Acosta;El Tarra;Guaitarilla;Jurado;Cimitarra;Valdivia;Ponedera;Villahermosa;Pie de Pato;Guadalajara de Buga;San Antonio del Tequendama;Unguia;Sesquile;La Palma;Rionegro;Obando;Morales;Palestina;El Reten;Piamonte;San Juan de Arama;La Cruz;Puerto Concordia;La Plata;Fortul;San Lorenzo;Caparrapi;Acandi;Norosi;Chivolo;Cerete;Ovejas;Montecristo;Momil;Soacha;Caldono;Aipe;Samaca;Rovira;Caicedonia;Tierralta;Arauquita;Bolivar;Palmito;Maria la Baja;Pupiales;Aquitania;Viterbo;Dagua;Guateque;Malaga;San Bernardo;Sabanagrande;La Tebaida;Tado;Pasto;Andalucia;Toribio;Puerto Santander;Zambrano;Entrerrios;Condoto;Sincelejo;Floridablanca;Potosi;Zapatoca;Sampues;Yotoco;San Marcos;Tena;Manzanares;Villapinzon;Cunday;Apartado;Melgar;Roldanillo;Puerto Wilches;Fusagasuga;Guaduas;Tubara;Belen de Umbria;El Carmen de Chucuri;Tuta;Coyaima;Plato;Distraccion;Guayabal;San Sebastian de Buenavista;Tenjo;Santander de Quilichao;Coconuco;Gamarra;Villa de San Diego de Ubate;San Carlos;Baranoa;Saldana;Purisima de la Concepcion;Saravena;Suaita;Zipaquira;Choconta;Chigorodo;Iles;Sogamoso;Armenia;San Benito Abad;Natagaima;Viota;San Martin de Loba;Gachancipa;Pueblo Rico;Santa Genoveva de Docordo;Restrepo;Suarez;Pijino del Carmen;Sucre;Mani;Piendamo;Facatativa;Santuario;Silvia;El Playon;Santo Tomas;Los Patios;Tiquisio;Los Santos;Monterrey;La Montanita;Chachagui;Caqueza;Algarrobo;Caldas;Neira;San Gil;Puerto Lleras;Pizarro;Trinidad;Betania;Barbacoas;Honda;El Carmen de Atrato;La Estrella;Chinacota;Puerto Colombia;El Doncello;Turbana;Suesca;San Pablo;Colombia;Quinchia;Solano;Toca;Barrancabermeja;Paipa;Betulia;La Macarena;Tibu;San Fernando;Suaza;El Cerrito;Nuqui;San Luis de Since;Urumita;Sevilla;San Juan del Cesar;Majagual;Achi;Caucasia;Choachi;La Ceja;San Martin;San Juan Nepomuceno;Donmatias;Purificacion;Gigante;Simijaca;Cisneros;La Virginia;Aguazul;Tesalia;Narino;El Tambo;Supia;Mogotes;Puerto Salgar;Riofrio;Galeras;Santa Marta;Clemencia;Sabana de Torres;Guasca;Agua de Dios;Oiba;Teorama;Cumbal;Castilla La Nueva;Necocli;Florida;Villeta;Caloto;Rio de Oro;Velez;La Gloria;San Jacinto del Cauca;Bello;Bucaramanga;Palmar de Varela;Aguadas;La Mesa;El Tablon;Chitaga;Bocas de Satinga;Puerto Narino;Villa Rica;Covenas;Barbosa;San Rafael;Fresno;Simiti;Mahates;Argelia;Puerto Carreno;Orocue;Villa de Leyva;Santa Rosa de Viterbo;Segovia;Yacuanquer;Cienaga de Oro;San Onofre;Amalfi;Frontino;El Pinon;La Esperanza;El Bordo;Taminango;San Pedro de Uraba;Puerto Nare;Hatillo de Loba;Zaragoza;Ginebra;El Aguila;La Victoria;Santa Catalina;Granada;Montelibano;San Alberto;Fuente de Oro;Nunchia;Fonseca;Ubala;San Antonio;Filadelfia;Palocabildo;Bogota;Puerto Escondido;Guacheta;Carepa;Funza;Valencia;Sardinata;Flandes;Sotomayor;Consaca;Calamar;Tibana;Lerida;Tausa;Ibague;Arauca;Bojaca;Guadalupe;Mutis;Marsella;Turbo;Socorro;Timbiqui;Rioblanco;Sibundoy;Ricaurte;Carlosama;Yumbo;La Calera;Isnos;Dosquebradas;San Sebastian;Timbio;Yondo;Tamesis;Iquira;Malambo;Acacias;Lejanias;Puerto Triunfo;Trujillo;Pamplona;Miranda;Planadas;San Pelayo;Acevedo;Aguachica;San Juan de Betulia;Guatape;Becerril;Santiago de Tolu;Retiro;Tibasosa;Cartagena del Chaira;Turbaco;Algeciras;Rosas;Giron;Usiacuri;El Zulia;San Roque;Yopal;Carmen de Carupa;Villanueva;Arbelaez;San Sebastian de Mariquita;El Colegio;Guaranda;Magangue;Puerto Tejada;Aranzazu;La Dorada;Amaga;Mercaderes;Campamento;Envigado;El Roble;Puerto Rico;Villavicencio;Margarita;Copacabana;Circasia;Puente Nacional;Sandona;Yali;Puerto Asis;Tabio;Linares;Astrea;Santa Barbara de Pinto;Santa Barbara;Raquira;Junin;Vijes;El Paso;Puerto Leguizamo;Convencion;Timana;El Rosal;Santa Lucia;Chiquinquira;Abrego;Altos del Rosario;Talaigua Nuevo;La Union;Canasgordas;Palermo;Valledupar;Guacari;Sopo;Caceres;San Pedro;Alcala;Siachoque;Pivijay;Charala;Arboletes;Nueva Granada;Chita;Tunja;Sahagun;Uribia;El Dificil;La Sierra;Coello;Riosucio;Hatonuevo;Planeta Rica;Cumaral;Salamina;Tuquerres;Ocana;Garzon;Manati;Risaralda;San Andres de Sotavento;Anolaima;Toro;Ayapel;Paz de Ariporo;El Bagre;San Vicente de Chucuri;Suan;Girardota;Galapa;Palmira;Umbita;Campoalegre;Pensilvania;Buenaventura;San Agustin;Chinchina;Cumaribo;Albania;Dibulla;Madrid;San Diego;Subachoque;Taraza;Fredonia;Cogua;Chiriguana;Repelon;Landazuri;Briceno;Pital;Guachucal;Yolombo;Cali;Arroyohondo;Buenos Aires;Fomeque;Alvarado;Rivera;Sitionuevo;Belalcazar;Anapoima;San Jose;Barranco de Loba;Marinilla;Santa Rosa del Sur;Jardin;Saboya;Pitalito;Villamaria;Buenavista;La Florida;Ortega;Orito;Vegachi;Mutata;Cocorna;Anzoategui;Ebejico;Lenguazaque;Ciudad Bolivar;Urrao;La Primavera;Yacopi;Chaparral;Barranquilla;Pueblo Nuevo;Riohacha;Corinto;San Antero;La Vega;Yaguara;Sabaneta;Quibdo;Santa Rosa de Osos;San Vicente del Caguan;Inza;Silvania;La Belleza;Regidor;Villa del Rosario;Bugalagrande;San Calixto;Ramiriqui;Quimbaya;Filandia;Andes;Concordia;Falan;Teruel;Medina;Santa Rosa;Caimito;Montenegro;Hacari;Tolu Viejo;Tenerife;Canalete;Nechi;Pailitas;Puerto Caicedo;Duitama;Jamundi;Anserma;Gacheta;Venecia;Lorica;Soledad;Guapi;Gonzalez;Cajibio;Bellavista;Angelopolis;Campo de la Cruz;Tello;Pesca;Dabeiba;Libano;Almaguer;Puebloviejo;Cachipay;Lloro;Santa Fe de Antioquia;Remedios;Belen de los Andaquies;Chimichagua;Guachaves;Oporapa;Ataco;Cordoba;Venadillo;Manizales;Barrancas;Titiribi;Los Palmitos;Calarca;Tamalameque;Samana;Sasaima;El Charco;Saladoblanco;Pinillos;Polonuevo;Bagado;Guamo;Mesetas;Corozal;Luruaco;Icononzo;Cota;Tulua;Paispamba;San Andres;Florencia;Girardot;Santa Ana;Puerto Guzman;Ventaquemada;Solita;Une +TZ;Chake Chake;Zanzibar;Arusha;Bukoba;Vwawa;Mahonda;Bariadi;Shinyanga;Tanzania;Morogoro;Moshi;Mbeya;Mkokotoni;Lindi;Singida;Njombe;Kigoma;Koani;Iringa;Wete;Mpanda;Geita;Mtwara;Mwanza;Kibaha;Dar es Salaam;Sumbawanga;Dodoma;Musoma;Songea;Tabora;Babati;Tanga +SD;Sudan;Singa;Khartoum;Wad Medani;Ed Daein;Rabak;El Obeid;Port Sudan;Kadugli;Ed Damer;El Fasher;Gedaref;Dongola;Nyala;Kassala;El Geneina;Ed Damazin;Zalingei;El Fula +HK;Hong Kong +SA;Medina;Saudi Arabia;Jazan;Al Qurayyat;Abha;Buraydah;Mecca;Riyadh;Sakaka;Al Bahah;Ha'il;Al Wajh;Arar;Ad Dammam;Tabuk;Najran +CL;Cochrane;Pozo Almonte;Osorno;Coquimbo;Cauquenes;Puerto Montt;Puente Alto;Linares;Valparaiso;San Antonio;Chillan;Arica;Iquique;Antofagasta;La Serena;Los Andes;Angol;Chile;Melipilla;Talagante;Lebu;Valdivia;Santiago;La Ligua;Castro;San Felipe;Colina;Calama;Concepcion;Pichilemu;Copiapo;Illapel;San Bernardo;Curico;La Union;Rancagua;Punta Arenas;Los Angeles;Chanaral;Puerto Williams;Coyhaique;Talca;Ovalle;Temuco;Quillota;Tocopilla;Quilpue;Vallenar;Puerto Aysen;Puerto Natales;San Fernando +ES;Merida;Castellon de la Plana;Barcelona;Bilbao;Granada;Guadalajara;Salamanca;Cuenca;Santander;Sevilla;Valladolid;Pamplona;A Coruna;Huelva;Ciudad Real;Lugo;Cordoba;Vitoria-Gasteiz;Girona;Santa Cruz;Pontevedra;Logrono;Badajoz;Jaen;Valencia;Tarragona;Avila;Santiago de Compostela;Cadiz;Oviedo;Palencia;Teruel;Toledo;Lleida;Malaga;Zamora;Albacete;Murcia;Almeria;Ourense;Donostia;Burgos;Caceres;Segovia;Huesca;Soria;Leon;Spain;Zaragoza;Madrid;Alicante;Las Palmas;Palma +IQ;Erbil;Al Kut;Jamjamal;Ar Rutbah;`Aynkawah;Sawran;Al Kufah;Halabjah;Ba`qubah;Ash Shamiyah;Kifri;Anah;Kuysinjaq;Sayyid Sadiq;Ash Shaykhan;Al Hillah;Karbala';Al Hayy;Kirkuk;Al Hindiyah;Qaladizay;Rawah;Makhmur;Baghdad;Abi al Khasib;Al Basrah;Al Hamzah;Samarra';Al Qa'im;Al `Aziziyah;Al Musayyib;Az Zubayr;Iraq;Saymayl;As Suwayrah;Al Fallujah;Mosul;Kalar;Al Faw;Al Hamdaniyah;Shaqlawah;Ar Ramadi;Al `Amarah;Hit;Tall `Afar;Al Jabayish;Tuz Khurmatu;Al Qurnah;An Nasiriyah;Ash Shatrah;`Aqrah;Zakhu;As Samawah;Rawanduz;Al Hashimiyah;Sinjar;Batufah;Al `Amadiyah;An Najaf;An Nu`maniyah;Abu Ghurayb;Dahuk;Balad;Bayji;Khanaqin;Raniyah;As Sulaymaniyah;Al Miqdadiyah;Hadithah;Tikrit;Ad Diwaniyah;Ar Rumaythah +SG;Singapore +CM;Ebolowa;Bafoussam;Buea;Ngaoundere;Cameroon;Bertoua;Garoua;Maroua;Yaounde;Douala;Bamenda +CA;Ottawa;Quebec City;Halifax;Toronto;Victoria;Regina;Iqaluit;Edmonton;Fredericton;Winnipeg;Yellowknife;St. John's;Whitehorse;Canada;Charlottetown +KE;Kisumu;Lodwar;Iten;Kapenguria;Kajiado;Kerugoya;Murang'a;Kenya;Maralal;Wote;Narok;Bungoma;Meru;Migori;Wajir;Homa Bay;Mandera;Embu;Nyamira;Kitale;Kiambu;Ol Kalou;Mombasa;Kitui;Eldoret;Hola;Rumuruti;Marsabit;Kisii;Nakuru;Lamu;Nairobi;Busia;Garissa;Kapsabet;Kwale;Kericho;Kabarnet;Mwatate;Sotik Post;Kakamega;Siaya;Isiolo;Machakos;Nyeri +MM;Hpa-An;Pathein;Loikaw;Monywa;Burma;Myitkyina;Dawei;Magway;Sittwe;Mandalay;Mawlamyine;Rangoon;Taunggyi;Hakha;Nay Pyi Taw;Bago +US;Salem;Sacramento;Helena;Jackson;Harrisburg;Albany;Little Rock;Lincoln;Hartford;Honolulu;Augusta;Boston;Jefferson City;Columbus;Cheyenne;Charleston;Baton Rouge;Bismarck;Concord;Springfield;Des Moines;St. Paul;Pierre;Boise;Atlanta;Dover;Annapolis;Nashville;Providence;Juneau;Denver;Phoenix;Austin;Richmond;Oklahoma City;Montgomery;Lansing;Frankfort;Montpelier;Tallahassee;Carson City;Salt Lake City;Olympia;Topeka;United States;Santa Fe;Madison;Columbia;Washington;Trenton;Indianapolis;Raleigh +AU;Perth;Hobart;Canberra;Brisbane;Adelaide;Australia;Melbourne;Darwin;Sydney +CI;Duekoue;Adzope;Dimbokro;Minignan;Yamoussoukro;Aboisso;Dabakala;Soubre;Sinfra;Toumodi;Gagnoa;Seguela;San-Pedro;Abengourou;Grand-Bassam;Daoukro;Dabou;Bouna;Man;Kong;Daloa;Ferkessedougou;Côte d’Ivoire;Korhogo;Agboville;Biankouma;Sassandra;Bondoukou;Boundiali;Touba;Abidjan;Bangolo;Oume;Bouafle;Mankono;Divo;Bingerville;Jacqueville;Odienne;Katiola;Bongouanou;Guiglo;Grand-Lahou;Sipilou;Bouake +DE;;Meschede;Freiburg im Breisgau;Ansbach;Aichach;Tauberbischofsheim;Landshut;Lauterbach;Sigmaringen;Sommerda;Chemnitz;Kunzelsau;Stadthagen;Greiz;Kleve;Gottingen;Recklinghausen;Parchim;Esslingen;Ratzeburg;Husum;Wolfenbuttel;Hagen;Lorrach;Peine;Hanau;Helmstedt;Cuxhaven;Lichtenfels;Villingen-Schwenningen;Gorlitz;Hassfurt;Mannheim;Germany;Stade;Osterode;Bad Ems;Augsburg;Heidelberg;Siegen;Halle;Bad Kissingen;Luneburg;Pfarrkirchen;Forst (Lausitz);Wittmund;Bad Schwalbach;Neunkirchen;Prenzlau;Neuwied;Kempten;Jena;Hannover;Ravensburg;Detmold;Bad Neuenahr-Ahrweiler;Euskirchen;Aurich;Bad Segeberg;Celle;Bad Salzungen;Hamburg;Schwabisch Hall;Schwelm;Eisenberg;Leer;Gunzburg;Baden-Baden;Emmendingen;Ulm;Limburg;Erlangen;Stralsund;Bayreuth;Wurzburg;Schwandorf;Salzgitter;Munster;Regensburg;Bad Tolz;Bottrop;Ludwigsburg;Haldensleben;Saarbrucken;Pinneberg;Altenburg;Passau;Kronach;Eisenach;Furth;Tirschenreuth;Heilbronn;Cottbus;Biberach;Gelsenkirchen;Bergisch Gladbach;Kaufbeuren;Tubingen;Saarlouis;Merzig;Plon;Balingen;Meppen;Minden;Kassel;Gera;Eberswalde;Straubing;Sonthofen;Bautzen;Germersheim;Frankenthal;Worms;Meiningen;Reutlingen;Erfurt;Pirmasens;Dusseldorf;Nuremberg;Bitburg;Donauworth;Karlsruhe;Miesbach;Herford;Memmingen;Delmenhorst;Neumunster;Greifswald;Duren;Duisburg;Unna;Vechta;Sondershausen;Hamm;Kelheim;Aschaffenburg;Neu-Ulm;Stuttgart;Oldenburg;Dingolfing;Monchengladbach;Zweibrucken;Schleswig;Nordhorn;Wilhelmshaven;Lauf;Potsdam;Senftenberg;Darmstadt;Kiel;Neubrandenburg;Wiesbaden;Waldshut-Tiengen;Mettmann;Torgau;Solingen;Salzwedel;Apolda;Traunstein;Uelzen;Altotting;Bad Homburg;Kulmbach;Bernburg;Hildburghausen;Gummersbach;Lubeck;Bad Kreuznach;Frankfurt;Bamberg;Weissenburg;Remscheid;Luchow;Saalfeld;Borken;Bremen;Weimar;Dresden;Brake;Arnstadt;Gotha;Neumarkt;Montabaur;Jever;Bad Oldesloe;Ebersberg;Speyer;Heinsberg;Perleberg;Freudenstadt;Diepholz;Neuss;Essen;Burg;Trier;Luckenwalde;Schleiz;Magdeburg;Frankfurt (Oder);Boblingen;Berlin;Siegburg;Wetzlar;Ingolstadt;Rathenow;Rottweil;Pfaffenhofen;Calw;Merseburg;Cham;Waiblingen;Bad Hersfeld;Eschwege;Hof;Herne;Neustadt;Wesel;Mainz;Amberg;Eichstatt;Hameln;Sankt Wendel;Neuburg;Fulda;Soest;Ludwigshafen;Wunsiedel;Northeim;Paderborn;Bad Neustadt;Leipzig;Bad Reichenhall;Marktoberdorf;Giessen;Pforzheim;Annaberg-Buchholz;Viersen;Bad Fallingbostel;Homburg;Neuruppin;Heppenheim;Eutin;Gross-Gerau;Rendsburg;Goslar;Coburg;Korbach;Landsberg;Gutersloh;Meissen;Pirna;Braunschweig;Wolfsburg;Schwerin;Wismar;Plauen;Kaiserslautern;Offenburg;Bochum;Forchheim;Naumburg;Stendal;Heidenheim;Bonn;Kitzingen;Munich;Krefeld;Bielefeld;Mulheim;Mosbach;Sangerhausen;Steinfurt;Cloppenburg;Regen;Aachen;Landau;Coesfeld;Miltenberg;Mindelheim;Weilheim;Hoxter;Bergheim;Verden;Koblenz;Leverkusen;Emden;Alzey;Friedberg;Halberstadt;Ingelheim;Bad Durkheim;Heilbad Heiligenstadt;Oranienburg;Muhlhausen;Goppingen;Lubben (Spreewald);Muhldorf;Brandenburg;Furstenfeldbruck;Freiberg;Osterholz-Scharmbeck;Zwickau;Dillingen;Wittlich;Warendorf;Heide;Gifhorn;Hofheim;Winsen;Dortmund;Erbach;Freising;Deggendorf;Holzminden;Hildesheim;Aalen;Osnabruck;Itzehoe;Dachau;Marburg;Schwabach;Wuppertal;Tuttlingen;Sonneberg;Rosenheim;Rastatt;Suhl;Herzberg;Nordhausen;Olpe;Gustrow;Starnberg;Erding;Oberhausen;Ludenscheid;Flensburg;Westerstede +AF;Tarin Kot;Qalat;Panjab;Ghazni;Mama Khel;Imam Sahib;Sangin;Maimanah;Bazarak;Lashkar Gah;Fayroz Koh;Ghoriyan;Maidan Shahr;Bagrami;Qal`ah-ye Now;Farah;Qarqin;Asadabad;Mazar-e Sharif;Baghlan;Nili;Faizabad;Karukh;Baraki Barak;Taywarah;Sar-e Pul;Sharan;Khulm;Zaranj;Kuhsan;Pul-e `Alam;Charikar;Bamyan;Taluqan;Gardez;Kunduz;Afghanistan;Andkhoy;Khanabad;Balkh;Khost;Kabul;Shibirghan;Aqchah;Herat;Jalalabad;Pul-e Khumri;Mahmud-e Raqi;Mehtar Lam;Parun;Kandahar;Aibak +ML;Nioro;Koulikoro;Kolondieba;Bandiagara;Yelimane;Baroueli;Menaka;Sikasso;Kidal;Kadiolo;Mali;Bamako;Dire;Kangaba;Gourma Rharous;Tominian;Djenne;Gao;Youwarou;Kayes;Douentza;Ansongo;Yorosso;San;Banamba;Tenenkou;Bourem;Bafoulabe;Kita;Koro;Niono;Bougouni;Timbuktu;Koutiala;Nara;Mopti;Goundam;Kati;Segou;Tessalit;Bankass;Diema +JO;Ar Ramtha;Al Juwayyidah;Az Zarqa';Ma`an;Muthallath al Azraq;Al `Aqabah;Jordan;As Salt;Busayra;`Ajlun;Al Hisn;Al Jizah;Ash Shunah ash Shamaliyah;Dayr Abu Sa`id;Irbid;Amman;Jarash;Ar Rusayfah;Al Karak;Al Fuhays;Sahab;Madaba;Al Mazar ash Shamali;Al Mafraq;At Tafilah +MA;Guelmim;Marrakech;Beni Mellal;Agadir;Tangier;Morocco;Oujda-Angad;Tifariti;Errachidia;Casablanca;Fes;Rabat +GH;Kumasi;Techiman;Cape Coast;Damongo;Accra;Sunyani;Ho;Bolgatanga;Dambai;Wa;Ghana;Sekondi;Tamale;Koforidua;Goaso;Nalerigu;Sefwi Wiawso +DZ;El Golea;Mila;Tindouf;Chlef;Bordj Bou Arreridj;Relizane;El Oued;Tlemcen;Khenchela;El Tarf;Oum el Bouaghi;Boumerdes;Skikda;Tizi Ouzou;Guelma;Sidi Bel Abbes;Bordj Mokhtar;El Bayadh;Algeria;I-n-Salah;Bechar;Ouled Djellal;Ain Temouchent;Mostaganem;Laghouat;El Meghaier;Adrar;Setif;Algiers;Blida;Jijel;Mascara;Oran;Ghardaia;Touggourt;Ouargla;Tissemsilt;Naama;Djanet;Djelfa;In Guezzam;Saida;M'Sila;Bouira;Souk Ahras;Annaba;Tamanrasset;Beni Abbes;Ain Defla;Biskra;Tebessa;Constantine;Tiaret;Illizi;Medea;Bejaia;Tipasa;Batna;Timimoun +AE;United Arab Emirates;Ra's al Khaymah;Sharjah;Dubai;Abu Dhabi;Umm al Qaywayn;Al Fujayrah;`Ajman +GR;Kaisariani;Salamina;Drama;Giannitsa;Larisa;Tyrnavos;Palaio Faliro;Alimos;Chios;Katerini;Nea Ionia;Kiato;Kalymnos;Agios Dimitrios;Pefki;Greece;Ano Liosia;Spata;Zacharo;Irakleia;Mesolongi;Karpenisi;Kos;Voula;Kalamaria;Patra;Serres;Aliartos;Nea Filadelfeia;Chalandri;Kilkis;Koufalia;Rethymno;Didymoteicho;Aigaleo;Ptolemaida;Argos;Kifisia;Psychiko;Panorama;Edessa;Nea Moudania;Acharnes;Vyronas;Lamia;Rafina;Veroia;Thessaloniki;Ioannina;Ierapetra;Korinthos;Kavala;Agrinio;Chaidari;Thermi;Kerkyra;Polygyros;Agios Nikolaos;Piraeus;Moschato;Alexandroupoli;Kranidi;Gazi;Korydallos;Agia Varvara;Aigio;Ermoupoli;Amaliada;Elefsina;Skydra;Naousa;Agia Paraskevi;Ano Syros;Aspropyrgos;Sindos;Livadeia;Farsala;Peraia;Xanthi;Nea Smyrni;Stavroupoli;Paiania;Mandra;Metamorfosi;Gerakas;Arta;Sparti;Komotini;Argostoli;Tripoli;Peristeri;Sykies;Zografos;Nikaia;Athens;Evosmos;Siteia;Petroupoli;Agioi Anargyroi;Nafpaktos;Mytilini;Dafni;Florina;Argos Orestiko;Kastoria;Perama;Karditsa;Kozani;Volos;Alexandreia;Kalamata;Kallithea;Vrilissia;Irakleio;Koropi;Orestiada;Argyroupoli;Oraiokastro;Marousi;Rodos;Zakynthos;Pyrgos;Galatsi;Amfilochia;Ilioupoli;Thebes;Xylokastro;Trikala;Ilion;Loutraki;Megara;Chrysoupoli;Chania;Nafplio;Grevena;Cholargos;Lagkadas;Ampelokipoi;Melissia;Glyfada;Messini;Kalyvia Thorikou;Lefkada;Keratsini;Preveza;Chalkida +ET;Harar;Ethiopia;Jijiga;Semera;Mekele;Awasa;Asosa;Dire Dawa;Bahir Dar;Addis Ababa;Gambela;Bonga +TW;Taibao;Taitung;Keelung;Zhubei;Banqiao;Changhua;Tainan;Taoyuan District;Pingtung;Douliu;Taipei;Hualien;Jincheng;Magong;Taichung;Yilan;Hsinchu;Nantou;Nangan;Taiwan;Chiayi;Kaohsiung;Miaoli +GT;Pochuta;San Jose El Idolo;San Carlos Alzatate;Concepcion Tutuapa;San Pedro Carcha;Quezaltepeque;San Juan Chamelco;Comitancillo;San Cristobal Totonicapan;Uspantan;El Asintal;Cabrican;Coatepeque;Sacapulas;Tiquisate;Casillas;Patzicia;San Miguel Ixtahuacan;Magdalena Milpas Altas;Santa Catarina Ixtahuacan;Livingston;Rabinal;Jacaltenango;San Juan Cotzal;Teculutan;Huite;Salama;Santa Maria Chiquimula;Santa Catarina Mita;Pasaco;Concepcion Chiquirichapa;San Pedro Sacatepequez;Raxruha;Tacana;Champerico;San Luis Jilotepeque;San Miguel Panan;El Progreso;Huitan;San Vicente Pacaya;San Lucas Sacatepequez;San Jose Ojetenam;Pastores;La Maquina;San Rafael Petzal;Santo Domingo Xenacoj;Santiago Chimaltenango;Quetzaltenango;Jocotan;Santa Ana Huista;Chuarrancho;Jalapa;Yupiltepeque;Concepcion Las Minas;Santo Tomas La Union;Tucuru;San Sebastian Coatan;San Jose Acatempa;Ciudad Vieja;Guanagazapa;Parramos;San Juan Sacatepequez;La Esperanza;Santa Lucia Cotzumalguapa;Santa Cruz Mulua;San Marcos;Tectitan;Zaragoza;San Juan Ixcoy;Mixco;San Juan Bautista;Almolonga;San Sebastian Huehuetenango;San Antonio La Paz;San Bartolome Milpas Altas;San Pedro Pinula;San Juan La Laguna;Chimaltenango;San Gaspar Ixchil;Pueblo Nuevo Vinas;Zacualpa;Coban;San Andres Sajcabaja;Chicacao;Canilla;Nuevo San Carlos;Santa Lucia La Reforma;Cubulco;Santa Clara La Laguna;San Pedro Ayampuc;Santo Domingo Suchitepequez;Santa Cruz del Quiche;San Pedro Jocopilas;Salcaja;Palestina de los Altos;Patulul;Agua Blanca;Ciudad Melchor de Mencos;Concepcion Huista;Cuyotenango;Solola;Poptun;Playa Grande;San Andres Itzapa;Santa Cruz Naranjo;Huehuetenango;San Luis;San Rafael La Independencia;Colotenango;Chichicastenango;El Chal;Escuintla;Ixchiguan;Santa Catarina Pinula;La Blanca;Moyuta;La Gomera;Momostenango;Jocotenango;Villa Canales;Chajul;San Jorge;Esquipulas Palo Gordo;El Chol;San Francisco El Alto;Tecpan Guatemala;Conguaco;Nahuala;Nenton;Cuilapa;San Agustin Acasaguastlan;Alotenango;Samayac;Santa Cruz Verapaz;Chinautla;Cuilco;Santa Lucia Milpas Altas;El Tumbador;Sumpango;Santa Eulalia;Tamahu;Taxisco;Purulha;San Jose Poaquil;San Sebastian;San Jose Pinula;San Antonio Suchitepequez;Cantel;Amatitlan;Joyabaj;Soloma;San Andres Xecul;Chiquimulilla;Comapa;Antigua Guatemala;Chisec;Sipacapa;Fraijanes;Gualan;Pajapita;San Cristobal Cucho;Malacatan;Pachalum;San Pablo Jocopilas;Cajola;Flores;Santa Lucia Utatlan;Totonicapan;San Martin Jilotepeque;Fray Bartolome de Las Casas;Asuncion Mita;Santa Rosa de Lima;San Raimundo;Zapotitlan;Usumatlan;Barberena;Guazacapan;San Lucas Toliman;Petapa;Palin;San Juan Atitan;Palencia;Esquipulas;San Rafael Pie de la Cuesta;Morazan;San Bartolo;Ixtahuacan;Todos Santos Cuchumatan;Acatenango;Senahu;Zunilito;Guatemala;San Mateo Ixtatan;San Pedro La Laguna;Rio Bravo;Olintepeque;Atescatempa;Tajumulco;San Francisco la Union;Ciudad Tecun Uman;San Pablo;Ostuncalco;Santiago Atitlan;El Palmar;San Francisco Zapotitlan;Patzun;San Antonio Palopo;Chiantla;Panzos;Jalpatagua;Panajachel;San Martin Zapotitlan;El Quetzal;Masagua;Santa Maria de Jesus;Lanquin;San Antonio Huista;Sanarate;San Miguel Duenas;El Estor;San Manuel Chaparron;Sibilia;San Francisco;San Martin Sacatepequez;Chinique;San Antonio Sacatepequez;San Andres Villa Seca;San Bernardino;Villa Nueva;Mataquescuintla;Santa Barbara;Jerez;Iztapa;Puerto Barrios;Siquinala;Santa Maria Ixhuatan;Cabanas;San Antonio Aguas Calientes;Flores Costa Cuca;Sibinal;Chicaman;Estanzuela;Chiche;Zunil;Dolores;San Felipe;Sayaxche;Chiquimula;Retalhuleu;Jutiapa;La Libertad;Tactic;San Pedro Necta;San Miguel Acatan;El Tejar;Nueva Concepcion;San Miguel Siguila;San Antonio Ilotenango;La Union;San Jose La Arada;Yepocapa;San Bartolome Jocotenango;Aguacatan;San Juan Ermita;Sansare;San Juan Tecuaco;Zacapa;San Carlos Sija;Monjas;Malacatancito;Santa Cruz Balanya;Olopa;Quesada;San Rafael Las Flores;Guastatoya;Guatemala City;Colomba;Camotan;Santiago Sacatepequez;Morales;San Miguel Chicaj;Mazatenango;Santa Apolonia;Santa Ana;Ocos;Cunen;Nueva Santa Rosa;Los Amates;La Democracia;San Cristobal Verapaz +KW;Kuwait City +HU;Mezotur;Balatonfured;Mateszalka;Karcag;Zalaegerszeg;Oroszlany;Nyirbator;Siklos;Hajduszoboszlo;Monor;Bicske;Sarkad;Varpalota;Hatvan;Balassagyarmat;Sopron;Nyiregyhaza;Kiskunfelegyhaza;Baja;Pecs;Gyomaendrod;Pilisvorosvar;Szarvas;Szekesfehervar;Szombathely;Tata;Gyongyos;Hodmezovasarhely;Sarbogard;Salgotarjan;Nagykallo;Dabas;Budakeszi;Gyal;Mor;Heves;Kiskunmajsa;Sarospatak;Gardony;Vecses;Celldomolk;Nagykata;Godollo;Dunaujvaros;Sarvar;Tatabanya;Tiszavasvari;Tapolca;Szekszard;Barcs;Kazincbarcika;Hajdunanas;Hungary;Erd;Tolna;Komlo;Komarom;Mohacs;Kiskoros;Esztergom;Bekes;Bekescsaba;Debrecen;Kecskemet;Oroshaza;Balmazujvaros;Batonyterenye;Balatonalmadi;Mosonmagyarovar;Dunakeszi;Rackeve;Marcali;Puspokladany;Bonyhad;Mako;Siofok;Edeleny;Szeged;Veszprem;Hajduboszormeny;Gyula;Ajka;Cegled;Eger;Tiszakecske;Kisvarda;Szentes;Jaszbereny;Mezokovesd;Hajduhadhaz;Csongrad;Papa;Szolnok;Kaposvar;Tiszaujvaros;Kormend;Satoraljaujhely;Szigetvar;Paszto;Nagykanizsa;Szentendre;Tiszafured;Ozd;Paks;Budapest;Torokszentmiklos;Gyor;Miskolc;Kalocsa;Kapuvar;Koszeg;Nagykoros;Dombovar;Keszthely;Nagyatad;Vac;Berettyoujfalu;Kiskunhalas;Csorna;Szigetszentmiklos +YE;Lahij;Raydah;Az Zaydiyah;Al Mukalla;Zinjibar;Hajjah;Aden;At Tawahi;Zabid;Yemen;Al Ma`alla';Dhamar;Bayt al Faqih;Bajil;Al Mahwit;Al Hudaydah;Mocha;Ash Shaykh `Uthman;Al Bayda';Say'un;Tarim;Rada`;Ash Shihr;`Amran;Ad Dali`;Ma`bar;Khamir;Hadibu;Al Mansurah;Ataq;Ta`izz;Yarim;Sa`dah;Jiblah;Al Ghayzah;Marib;Al Jabin;Ja`ar;Ibb;Sanaa;Al Hazm +UZ;Chortoq;Marhamat;Termiz;Dang`ara;Yangiariq;Toyloq Qishlog'i;Jondor Shaharchasi;Nukus;Galaosiyo Shahri;Andijon;Jizzax;Buloqboshi;Shahrixon;Yozyovon;Usmat Shaharchasi;Samarkand;Paxtaobod;Paxtakor Shahri;Qanliko`l;Gurlan;Oqtosh Shahri;Do'stlik Shahri;Gagarin Shahri;Qorako`l Shahri;Jomboy Shahri;Oltiariq;Uchquduq Shahri;Toshloq;Payariq Shahri;Kuyganyor;G'oliblar Qishlog'i;Baliqchi;Bukhara;Asaka;Xonqa;Poytug`;Quvasoy;Yangirabot;Juma Shahri;Yangi Mirishkor;Ishtixon Shahri;Vodil;Qo`rg`ontepa;Loyish Shaharchasi;Beshkent Shahri;G'ijduvon Shahri;Qarshi;Xo`jayli Shahri;Uchqurghon Shahri;Taxtako`pir;Qorovul;Vobkent Shahri;Romitan Shahri;Zomin Shaharchasi;Beshariq;G'allaorol Shahri;Namangan;Payshamba Shahri;Kattaqo'rg'on Shahri;Shofirkon Shahri;Hazorasp;Xo`jaobod;Shovot;Qo'shko'pir;Kegeyli Shahar;Farg`ona;Kogon Shahri;Kosonsoy;Uchtepa Qishlog'i;Nurobod Shahri;Yaypan;Urgut Shahri;Tashkent;Uzbekistan;Qorao`zak;Chimboy Shahri;Qorashina;G`uzor;Navoiy;Xiva;Qo`ng`irot Shahri;Quva;Bulung'ur Shahri;Rishton;Haqqulobod;Guliston;Ziyodin Shaharchasi;Bog'ot;Zarbdor Shaharchasi;Urganch +UA;Skvyra;Sevastopol;Henichesk;Zaporizhzhia;Chortkiv;Krasnohrad;Izmail;Ochakiv;Artsyz;Rozdilna;Horlivka;Svitlovodsk;Bershad;Bakhmut;Kalanchak;Bakhchysarai;Smila;Bilohirsk;Novhorod-Siverskyi;Tiachiv;Bila Tserkva;Kobeliaky;Znamianka;Rakhiv;Kiliia;Nikopol;Shyroke;Lviv;Novomyrhorod;Olevsk;Kalmiuske;Chaplynka;Dolyna;Skadovsk;Izium;Kryvyi Rih;Novohrad-Volynskyi;Storozhynets;Tarashcha;Hlobyne;Pokrovsk;Radomyshl;Tulchyn;Kovel;Tokmak;Dubno;Berehove;Uman;Rokytne;Novoazovsk;Piatykhatky;Zolotonosha;Lutsk;Nova Odesa;Tysmenytsia;Bohodukhiv;Kamianka-Dniprovska;Bilhorod-Dnistrovskyi;Vynohradiv;Kalynivka;Okhtyrka;Simferopol;Poltava;Nemyriv;Khmilnyk;Dolynska;Khotyn;Berezivka;Melitopol;Mykolaiv;Bilozerka;Sokyriany;Sumy;Myronivka;Alchevsk;Pervomaisk;Vyshhorod;Reshetylivka;Kosiv;Reni;Bobrynets;Kurman;Berdychiv;Makiivka;Kramatorsk;Khmelnytskyi;Sievierodonetsk;Yampil;Polohy;Zhytomyr;Shakhtarsk;Oleksandriia;Svatove;Berdiansk;Kamianets-Podilskyi;Drohobych;Marhanets;Ukraine;Chuhuiv;Yuzhnoukrainsk;Volnovakha;Kaharlyk;Bilovodsk;Fastiv;Mizhhiria;Dnipro;Kalush;Horodyshche;Chyhyryn;Tlumach;Zolochiv;Yakymivka;Zhmerynka;Lubny;Novoukrainka;Brovary;Berezhany;Hlyboka;Pohrebyshche;Mukacheve;Vesele;Kherson;Zalishchyky;Chervonohrad;Yavoriv;Novyi Buh;Kamin-Kashyrskyi;Snihurivka;Koriukivka;Prymorsk;Mariupol;Liubashivka;Dovzhansk;Enerhodar;Cherkasy;Boryspil;Synelnykove;Sarny;Balta;Nyzhnohirskyi;Novomoskovsk;Chornomorske;Pustomyty;Stryi;Zhydachiv;Kharkiv;Bar;Odesa;Khust;Zvenyhorodka;Saky;Koziatyn;Kyiv;Chernivtsi;Obukhiv;Kupiansk;Starobilsk;Horodenka;Kolomyia;Romny;Sniatyn;Novotroitske;Kerch;Myrhorod;Kozova;Lokhvytsya;Haivoron;Bolhrad;Podilsk;Voznesensk;Oleshky;Illintsi;Konotop;Yevpatoriia;Bucha;Kryzhopil;Pryluky;Luhansk;Mala Vyska;Rivne;Tatarbunary;Kremenchuk;Ivano-Frankivsk;Kropyvnytskyi;Korosten;Khorol;Verkhnodniprovsk;Yany Kapu;Chernihiv;Nizhyn;Apostolove;Bashtanka;Ovidiopol;Kakhovka;Nadvirna;Kamianske;Shostka;Pavlohrad;Vasylivka;Irshava;Donetsk;Tetiiv;Hola Prystan;Kremenets;Biliaivka;Mykhailivka;Uzhhorod;Varash;Ternivka;Dzhankoi;Makariv;Ternopil;Vinnytsia;Perevalsk;Mohyliv-Podilskyi;Sambir;Haisyn;Lozova +BO;Potosi;La Paz;Santa Cruz;Cochabamba;Bolivia;Tarija;Sucre;Cobija;Trinidad;Oruro +KP;Sariwon;Kanggye;Wonsan;Sariwon-si;Pyongyang;Ch'ongjin;Rajin;Korea, North;Hyesan;Hoeryong;Namsan;Sinuiju;Kaesong;Haeju;P'yongsong-si;Hamhung +IT;Trani;Trento;Trieste;Nuoro;Crotone;Vercelli;Florence;Belluno;Genoa;Brindisi;Viterbo;Pavia;Catanzaro;Pesaro;Latina;Como;Caltanissetta;Frosinone;Matera;Rovigo;Gorizia;Pistoia;Ferrara;Avellino;Tempio Pausania;Vicenza;Cagliari;Catania;Forli;Lucca;L'Aquila;Reggio di Calabria;Campobasso;Agrigento;Arezzo;Foggia;Bolzano;Bologna;Naples;Perugia;Massa;Trapani;Mantova;Potenza;Andria;Bergamo;Pescara;Rieti;Lodi;Reggio Emilia;Caserta;Siena;Taranto;Alessandria;Terni;Sondrio;Rimini;Novara;Varese;Italy;La Spezia;Ascoli Piceno;Prato;Messina;Cremona;Padova;Lecce;Brescia;Teramo;Chieti;Villacidro;Livorno;Ragusa;Grosseto;Ancona;Turin;Biella;Lecco;Vibo Valentia;Monza;Fermo;Salerno;Aosta;Savona;Udine;Ravenna;Tortoli;Benevento;Enna;Bari;Macerata;Pordenone;Rome;Modena;Barletta;Verona;Milan;Oristano;Siracusa;Imperia;Sassari;Pisa;Iglesias;Palermo;Olbia;Cosenza;Carbonia;Treviso;Venice;Parma;Cuneo;Piacenza;Verbania;Asti +EC;Loja;Cuenca;Macas;Tulcan;Latacunga;Guayaquil;Azogues;Santa Elena;Santo Domingo de los Colorados;Zamora;Puerto Baquerizo Moreno;Guaranda;Puyo;Machala;Nueva Loja;Riobamba;Ibarra;Babahoyo;Quito;Esmeraldas;Tena;Ambato;Puerto Francisco de Orellana;Portoviejo;Ecuador +SO;Boosaaso;Xuddur;Mogadishu;Baidoa;Buurhakaba;Kismaayo;Qoryooley;Gaalkacyo;Bu'aale;Afgooye;Hobyo;Jawhar;Dhuusamarreeb;Garbahaarrey;Marka;Laascaanood;Hargeysa;Somalia;Boorama;Beledweyne;Ceerigaabo;Garoowe;Burco +SY;Nubl;Ar Rastan;Khan Shaykhun;Darayya;Darat `Izzah;Aleppo;Tall Abyad;Jisr ash Shughur;Jaramana;Hamah;As Suwayda';Suran;Al Qamishli;Sahnaya;Tallkalakh;Al Mayadin;`Afrin;Ash Shaykh Badr;Damascus;Dayr az Zawr;Muhradah;Tall Rif`at;Armanaz;Dayr Hafir;Izra;Harasta;Babila;Maskanah;Ghabaghib;Salkhad;Homs;`Ayn al `Arab;Al Qunaytirah;Ar Raqqah;Busra ash Sham;Kafr Takharim;Syria;Binnish;Al Qutayfah;As Suqaylibiyah;Hish;Yabrud;Al Hajar al Aswad;Jablah;Al `Asharah;Ash Shaddadah;Taftanaz;As Sanamayn;Jarabulus;Tartus;Nawa;Muh Hasan;As Safirah;At Tall;Sarmin;Khirbat Ghazalah;Manbij;Tallbisah;Tadmur;Ath Thawrah;Madaya;Masyaf;Baniyas;Ma`arrat an Nu`man;Tasil;Al Musayfirah;Jindayris;Qadsayya;An Nabk;Ar Ruhaybah;Hajin;Idlib;Ra's al `Ayn;Harim;Kafr Batna;Kafr Nubl;As Sukhnah;Al Qardahah;Dhiban;Al Malikiyah;Shahba;Ad Duraykish;I`zaz;Shin;Al Atarib;Albu Kamal;Mari`;Al Qusayr;Jasim;Kafr Zayta;Safita;Ma`arratmisrin;Salqin;`Amuda;Duma;Al Kiswah;Al Hasakah;Tall Salhab;Al Muzayrib;Latakia;As Salamiyah;Qatana;Qal`at al Madiq;Al Qaryatayn;Dar`a;Dayr `Atiyah;Az Zabadani;Jayrud;Saraqib;Tadif;Saydnaya;Al Ghizlaniyah;Ad Darbasiyah;Mahin +ZM;Lusaka;Choma;Chinsali;Zambia;Ndola;Kasama;Chipata;Mansa;Kabwe;Mongu;Solwezi +BF;Yako;Boromo;Diapaga;Bobo-Dioulasso;Reo;Ziniare;Manga;Gayeri;Kombissiri;Kaya;Boulsa;Batie;Tougan;Koupela;Burkina Faso;Dedougou;Dano;Djibo;Po;Ouahigouya;Bousse;Hounde;Titao;Banfora;Nouna;Zorgo;Pama;Kokologo;Diebougou;Gorom-Gorom;Gourcy;Leo;Aribinda;Bogande;Fada Ngourma;Toma;Sapone;Ouargaye;Sebba;Koudougou;Tenkodogo;Dori;Ouagadougou;Korsimoro;Tanghin-Dassouri;Gaoua;Sapouy;Orodara;Salanso;Kongoussi +LB;Joubb Jannine;Batroun;Amioun;Bcharre;Beirut;Tripoli;Bent Jbail;El Hermel;Nabatiye;Zgharta;Halba;Baalbek;Lebanon;Zahle;Jounie;Tyre;Aaley;Sidon;Baabda;Jbail +RO;Piatra Neamt;Focsani;Bucharest;Vaslui;Cluj-Napoca;Zalau;Craiova;Resita;Galati;Iasi;Satu Mare;Calarasi;Botosani;Ploiesti;Drobeta-Turnu Severin;Baia Mare;Oradea;Giurgiu;Slatina;Romania;Braila;Buzau;Timisoara;Hunedoara;Constanta;Slobozia;Targu Jiu;Sfantu-Gheorghe;Bistrita;Brasov;Miercurea-Ciuc;Suceava;Arad;Alexandria;Tulcea;Targoviste;Targu-Mures;Bacau;Sibiu;Alba Iulia;Pitesti;Ramnicu Valcea +LK;Trincomalee;Anuradhapura;Sri Jayewardenepura Kotte;Ratnapura;Colombo;Sri Lanka;Badulla;Galle;Jaffna;Kandy;Kurunegala +AZ;Agsu;Saatli;Goygol;Qax;Gadabay;Lerik;Babak;Bilasuvar;Naxcivan;Haciqabul;Ordubad;Samux;Agdam;Sarur;Imisli;Barda;Naftalan;Baku;Ucar;Neftcala;Heydarabad;Samkir;Yevlax;Qusar;Azerbaijan;Susa;Astara;Sabirabad;Zaqatala;Quba;Goycay;Ismayilli;Tovuz;Siyazan;Agstafa;Zardab;Oguz;Goranboy;Cabrayil;Xocali;Qubadli;Davaci;Yardimli;Agcabadi;Sumqayit;Tartar;Culfa;Agdas;Lacin;Sahbuz;Qazax;Saki;Lankaran;Mingacevir;Xocavand;Daskasan;Masalli;Xizi;Xacmaz;Salyan;Kalbacar;Qivraq;Ganca;Fuzuli;Xirdalan;Kurdamir;Beylaqan;Calilabad;Qobustan;Qabala;Balakan +MG;Morondava;Fenoarivo Atsinanana;Maevatanana;Manakara;Antsohihy;Mahajanga;Tsiroanomandidy;Farafangana;Antsiranana;Antananarivo;Maintirano;Tolanaro;Ambatondrazaka;Ambovombe;Antsirabe;Sambava;Ambositra;Fianarantsoa;Toliara;Ihosy;Toamasina;Miarinarivo;Madagascar +VE;Machiques;Colon;Rubio;Aragua de Barcelona;Caracas;Guanare;Tariba;Escuque;Puerto Piritu;La Guaira;Barquisimeto;Motatan;El Limon;Ocumare del Tuy;San Antonio de Los Altos;Ciudad Guayana;El Tocuyo;Cabimas;Caucagua;La Colonia Tovar;Baruta;El Consejo;Ciudad Piar;Maiquetia;Montalban;Chichiriviche;Guatire;San Diego;Guacara;La Victoria;Valera;Quibor;Ejido;Turmero;Altagracia de Orituco;San Jose de Aerocuar;Calabozo;Anaco;Cabudare;Cumana;San Carlos;Valencia;Valle de La Pascua;Cantaura;Caraballeda;Monte Carmelo;Petare;El Callao;San Carlos del Zulia;Tocuyito;Piritu;Palo Negro;Punto Fijo;Guanta;Maturin;Lecherias;El Socorro;Bocono;Las Mercedes;Ciudad Bolivar;Villa de Cura;Guasipati;Yaritagua;Barcelona;Carupano;Puerto La Cruz;Chaguaramas;Trujillo;Onoto;Uracoa;Sabaneta;Chivacoa;El Tigre;Charallave;Puerto Cabello;Guasdualito;Los Teques;Santa Cruz;Coro;Tucupita;Higuerote;San Jose de Guaribe;Caripito;La Asuncion;Maracay;Rio Caribe;Maracaibo;Acarigua;Clarines;Barbacoas;Tucacas;Juangriego;Puerto Ayacucho;El Palmar;Ciudad Ojeda;Venezuela;San Antonio del Tachira;Urena;Isla Raton;Porlamar;San Cristobal;Pampan;Carache;San Fernando;Aguasay;Carvajal;Barinas;Merida;La Grita;Santa Maria de Ipire;Carora;Guigue;Santa Barbara;Araure;Upata;Santa Cruz de Los Taques;El Valle del Espiritu Santo;La Esmeralda;Guarenas;Socopo;Catia La Mar;Cagua;Cocorote;Los Guayos;San Felipe;Ospino;Sarare;Santa Rita;Pampatar;Santa Lucia;Puerto Cumarebo;San Juan de los Morros;Pampanito;Achaguas;Santa Teresa del Tuy;Sanare;Nirgua;Cojedes;San Pedro de Coche;Santa Ana;Cua;Zaraza;Tinaquillo;San Mateo +AT;Zell am See;Feldbach;Austria;Imst;Amstetten;Weiz;Bregenz;Neunkirchen;Vocklabruck;Lienz;Bludenz;Deutschlandsberg;Zwettl;Krems an der Donau;Salzburg;Leoben;Dornbirn;Innsbruck;Baden;Klosterneuburg;Judenburg;Eisenstadt;Korneuburg;Sankt Johann im Pongau;Bruck an der Mur;Ganserndorf;Steyr;Gmunden;Vienna;Volkermarkt;Wolfsberg;Feldkirchen;Schwaz;Mistelbach;Voitsberg;Spittal an der Drau;Linz;Hallein;Modling;Sankt Polten;Braunau am Inn;Wiener Neustadt;Kufstein;Klagenfurt;Sankt Veit an der Glan;Ried im Innkreis;Villach;Graz;Wels;Hollabrunn;Leibnitz +ZW;Chinhoyi;Harare;Marondera;Lupane;Mutare;Gweru;Gwanda;Zimbabwe;Bulawayo;Masvingo;Bindura +KH;Sisophon;Tbeng Meanchey;Kampong Speu;Kampong Cham;Kampong Chhnang;Stung Treng;Prey Veng;Svay Rieng;Kampong Trach;Koh Kong;Takeo;Battambang;Sihanoukville;Sen Monorom;Kratie;Kampot;Pailin;Phnom Penh;Lumphat;Suong;Kampong Thom;Cambodia;Krong Kep;Siem Reap;Samraong;Ta Khmau;Banlung;Pursat +SE;Vanersborg;Kavlinge;Skelleftea;Balsta;Timra;Ronneby;Ornskoldsvik;Gavle;Orebro;Gislaved;Ulricehamn;Ystad;Oxelosund;Mjolby;Uddevalla;Harnosand;Ljungby;Arboga;Soderhamn;Sala;Taby;Staffanstorp;Hassleholm;Stockholm;Karlskrona;Enkoping;Sandviken;Karlstad;Molndal;Jonkoping;Mariestad;Tranas;Vasteras;Malmo;Hagfors;Vaxjo;Hallstahammar;Katrineholm;Motala;Uppsala;Eskilstuna;Nynashamn;Hoor;Falun;Skara;Stenungsund;Kalmar;Vetlanda;Bollnas;Hudiksvall;Kungsangen;Vallentuna;Vasterhaninge;Kumla;Kiruna;Koping;Kinna;Skovde;Vimmerby;Ostersund;Upplands Vasby;Sundsvall;Skoghall;Lomma;Lerum;Sunne;Kungsbacka;Lidkoping;Boden;Habo;Molnlycke;Tibro;Tumba;Gothenburg;Borlange;Kristianstad;Fagersta;Falkoping;Linkoping;Alingsas;Varberg;Arvika;Karlskoga;Ekero;Trelleborg;Strangnas;Amal;Akersberga;Sater;Norrtalje;Marsta;Helsingborg;Saffle;Gagnef;Lidingo;Eslov;Svedala;Sodertalje;Orkelljunga;Nykoping;Nybro;Sjobo;Astorp;Angelholm;Halmstad;Lycksele;Oskarshamn;Eksjo;Pitea;Kungalv;Lulea;Norrkoping;Ludvika;Sweden;Filipstad;Solleftea;Umea;Visby;Kristinehamn;Nassjo;Boras;Avesta;Lindesberg;Trollhattan;Gustavsberg;Arlov;Finspang;Hoganas;Almhult;Landskrona;Mora;Vastervik +CU;Ciro Redondo;Colon;Jobabo;Santa Cruz del Sur;Vinales;Jaruco;Ranchuelo;Manati;Palma Soriano;Cardenas;Sagua de Tanamo;Los Arabos;Perico;Limonar;Guira de Melena;Palmira;Minas;Buey Arriba;Jesus Menendez;Jaguey Grande;Majagua;Aguada de Pasajeros;Nueva Gerona;Sancti Spiritus;Campechuela;Candelaria;Guisa;Moron;Najasa;Marti;Guaimaro;Imias;Cacocum;Baguanos;Jovellanos;Bayamo;Manzanillo;Guanajay;San Luis;Pilon;Havana;Cabaiguan;Quemado de Guines;Carlos Manuel de Cespedes;Media Luna;Guantanamo;Bejucal;Madruga;Taguasco;San Jose de las Lajas;San German;Minas de Matahambre;Guines;Yaguajay;Moa;Cuba;Santa Clara;Chambas;Puerto Padre;Esmeralda;Placetas;San Antonio del Sur;Santo Domingo;Nueva Paz;Batabano;Bolivia;Union de Reyes;Lajas;Artemisa;Trinidad;San Nicolas;La Sierpe;Corralillo;Caimito;Camaguey;Santa Cruz del Norte;Jatibonico;Rio Cauto;Sagua la Grande;Colombia;Venezuela;Bartolome Maso;Yara;Quivican;San Cristobal;Consolacion del Sur;Vertientes;Fomento;Remedios;Sibanicu;Manicaragua;Bahia Honda;Mayari;Baracoa;Melena del Sur;Banes;Primero de Enero;Gibara;Contramaestre;Los Palacios;Jiguani;Cayo Mambi;Las Tunas;Alquizar;Cienfuegos;Matanzas;Cruces;Jimaguayu;Rodas;Cifuentes;Pedro Betancourt;Santa Lucia;Caibarien;Santiago de Cuba;Holguin;Ciego de Avila;San Juan y Martinez;Mariel;Cumanayagua;San Antonio de los Banos;La Palma;Antilla;Amancio;Bauta;Pinar del Rio;Camajuani;Calimete;Encrucijada;Florencia;Niquero;Florida +BY;Baranavichy;Rechytsa;Maladzyechna;Bykhaw;Belarus;Babruysk;Mahilyow;Krychaw;Ivanava;Hantsavichy;Mazyr;Yelsk;Rahachow;Chavusy;Ashmyany;Vyetka;Ivatsevichy;Vawkavysk;Minsk;Pruzhany;Svyetlahorsk;Vilyeyka;Khoyniki;Kobryn;Masty;Zhytkavichy;Klimavichy;Hrodna;Zhlobin;Smarhon;Lyuban;Polatsk;Drahichyn;Pinsk;Pastavy;Slonim;Brest;Orsha;Byalynichy;Talachyn;Homyel';Chachersk;Kalinkavichy;Salihorsk;Mstsislaw;Vitsyebsk;Horki;Horad Zhodzina;Lyakhavichy;Shklow;Haradok;Shchuchyn;Hlybokaye;Kastsyukovichy;Braslaw;Kapyl;Malaryta;Valozhyn;Lyelchytsy;Horad Smalyavichy;Staryya Darohi;Dobrush;Luninyets;Nyasvizh;Astravyets;Buda-Kashalyova;Asipovichy;Pyetrykaw;Lahoysk;Slutsk;Horad Barysaw;Klyetsk;Zhabinka;Byaroza +NL;Winterswijk;Rucphen;Den Helder;Valkenswaard;Oldenzaal;Haren;Kapelle;Bedum;Roermond;Putten;Stein;Sliedrecht;Oosterhout;Houten;Montfoort;Malden;Ede;Brielle;Urk;Wierden;Waalre;Burgum;Bergen op Zoom;Papendrecht;Nijmegen;Arnhem;Heerde;Rijen;Neerijnen;Schiedam;Hulst;Oosterwolde;Purmerend;Beek;Bodegraven;Harlingen;Tiel;Middelburg;Lelystad;Huizen;Alkmaar;Middelharnis;Nijverdal;Zwolle;Pijnacker;Tholen;Hengelo;Aalsmeer;Steenbergen;Franeker;Raamsdonksveer;Doesburg;Apeldoorn;Almere;Beuningen;Brummen;Oirschot;Utrecht;Someren;Tilburg;Hardinxveld-Giessendam;Winsum;Sneek;Goes;Deventer;Hoogeveen;Heeze;Zevenaar;Sittard;Uden;Brunssum;Amstelveen;Katwijk;Wijk bij Duurstede;Appingedam;Spijkenisse;Drachten;Elburg;Bladel;Epe;Venray;Wijchen;Beverwijk;Aalten;Groesbeek;Oss;Zutphen;Oisterwijk;Tubbergen;Vlaardingen;Grave;Simpelveld;Hendrik-Ido-Ambacht;Dordrecht;Heemstede;Breda;Cuijk;Rhenen;Hilversum;Sassenheim;Haaksbergen;Doorn;Boxtel;IJmuiden;Waddinxveen;Joure;Delfzijl;Landgraaf;Lochem;Son en Breugel;Geldermalsen;Wolvega;Deurne;Asten;Zandvoort;Gennep;Uitgeest;Terneuzen;Heerlen;Heerhugowaard;IJsselstein;Meerssen;Staphorst;Enkhuizen;Noordwijkerhout;Boekel;Laren;Heerenveen;Rotterdam;Nieuwegein;Leek;Capelle aan den IJssel;Noordwijk;Woerden;Woudrichem;Strijen;Leiden;Oldebroek;Oostzaan;Nuth;Werkendam;Boskoop;Maastricht;Heemskerk;Blaricum;Schagen;Hellevoetsluis;Woudenberg;Horst;Amersfoort;Zundert;Soest;Eemnes;'s-Hertogenbosch;Ermelo;Haarlem;Hillegom;Scherpenzeel;Beilen;Almelo;Best;Zaandam;Duiven;Baarn;Veenendaal;Emmeloord;Amsterdam;Alblasserdam;Voorschoten;Helmond;Bussum;The Hague;Zierikzee;Waalwijk;Bergeijk;Bergen;Volendam;Nederweert;Kampen;Netherlands;Nuenen;Barneveld;Kaatsheuvel;Leeuwarden;Doetinchem;Wassenaar;Landsmeer;Goirle;Haaren;Dongen;Coevorden;Leusden;Veldhoven;Heiloo;Vianen;Monnickendam;Gemert;Zoetermeer;Weert;Emmen;Beek en Donk;Veghel;Harderwijk;Maarssen;Zeewolde;Hoogezand;Denekamp;Uithoorn;Zaltbommel;Hoofddorp;Boxmeer;Opmeer;Oudenbosch;Roelofarendsveen;Veendam;Nunspeet;Kerkrade;Lopik;Krimpen aan den IJssel;Sint-Michielsgestel;Zuidhorn;Dalfsen;Bergambacht;Heesch;Lisse;Leidschendam;Oud-Beijerland;Rijswijk;Eijsden;Vlissingen;Venlo;Leiderdorp;Lemmer;Hoorn;Gouda;Diemen;Schoonhoven;Zeist;Castricum;Meppel;Maassluis;Etten-Leur;Oegstgeest;Bilthoven;Vught;Borne;Sint Anthonis;Stadskanaal;Geldrop;Delft;Zoeterwoude;Eindhoven;Wageningen;Ommen;Made;Naaldwijk;Raalte;Barendrecht;Hilvarenbeek;Sint-Oedenrode;Westervoort;Schinnen;Dronten;Gorinchem;Assen;Losser;Dokkum;Hattem;Voerendaal;Groningen;Oudewater;Roosendaal;Nijkerk;Culemborg;Ridderkerk;Eersel;Enschede;Alphen aan den Rijn;Weesp;Hardenberg;Vaals;Druten;Zwijndrecht +KZ;Taraz;Beyneu;Shiyeli;Qaratau;Shakhtinsk;Ekibastuz;Qashyr;Alga;Aqtobe;Novoishimskiy;Kazakhstan;Shalqar;Turkistan;Taldyqorghan;Qapshaghay;Qarabalyq;Balqash;Khromtau;Soran;Ertis;Kishkenekol;Qazyqurt;Zhitiqara;Zaysan;Kegen;Pavlodar;Oktyabr'sk;Almaty;Qyzylorda;Makhambet;Petropavl;Qaraghandy;Bulaevo;Osakarovka;Bayanauyl;Zhosaly;Saumalkol;Bayghanin;Aral;Shetpe;Sarykemer;Kentau;Atbasar;Esil;Oral;Atyrau;Zhetisay;Aqsu;Semey;Mangghystau;Esik;Qostanay;Qulsary;Inderbor;Borovskoy;Ayagoz;Shymkent;Temirtau;Astana;Aksay;Ushtobe;Bauyrzhan Momyshuly;Qazaly;Shchuchinsk;Oskemen;Zhangatas;Stepnogorsk;Qarabulaq;Fort-Shevchenko;Satbayev;Aqkol;Talghar;Abay;Derzhavinsk;Shemonaikha;Qulan;Qarqaraly;Rudnyy;Taiynsha;Qaskeleng;Uryzhar;Tekeli;Zyryanovsk;Uzynaghash;Lengir;Zhezqazghan;Torghay;Qarazhal;Makinsk;Atasu;Shongzhy;Ereymentau;Aqtau;Shu;Kokshetau;Sarqan;Zharkent;Arys;Balpyq Bi;Sholaqqorghan;Qusmuryn;Shar;Maqat;Sharbaqty;Arqalyq;Chapaev;Usharal;Ridder;Zhanibek +MW;Kasungu;Dowa;Karonga;Ntcheu;Nkhata Bay;Salima;Ntchisi;Lilongwe;Mangochi;Dedza;Phalombe;Nkhotakota;Mchinji;Nsanje;Machinga;Blantyre;Mzimba;Neno;Mwanza;Chitipa;Mulanje;Thyolo;Rumphi;Zomba;Chikwawa;Chiradzulu;Balaka;Malawi +PL;Bedzin;Gdansk;Pulawy;Wejherowo;Mielec;Glogow;Nysa;Zabkowice Slaskie;Bartoszyce;Wolomin;Walbrzych;Swiecie;Olawa;Otwock;Legionowo;Drawsko Pomorskie;Kamien Pomorski;Garwolin;Zywiec;Miedzychod;Grodzisk Mazowiecki;Zabrze;Rypin;Nowy Targ;Boleslawiec;Swiedbodzin;Siedlce;Gryfino;Klobuck;Przasnysz;Busko-Zdroj;Slupsk;Myslowice;Lidzbark Warminski;Koscian;Tychy;Sieradz;Gizycko;Sandomierz;Ostrzeszow;Siemiatycze;Czluchow;Krosno Odrzanskie;Elk;Klodzko;Lublin;Miedzyrzecz;Zuromin;Wolsztyn;Olesnica;Olesno;Lubin;Strzelce Krajenskie;Lask;Puck;Sroda Wielkopolska;Nisko;Dabrowa Tarnowska;Jedrzejow;Stargard Szczecinski;Szczytno;Kalisz;Kedzierzyn-Kozle;Wschowa;Chrzanow;Ruda Slaska;Radomsko;Swidwin;Wyszkow;Wieliczka;Chelm;Sztum;Parczew;Sokolow Podlaski;Krapkowice;Slupca;Ostroleka;Radom;Rybnik;Zakopane;Czarnkow;Ropczyce;Milicz;Aleksandrow Kujawski;Olsztyn;Legnica;Pinczow;Lowicz;Trzebnica;Sierpc;Skarzysko-Kamienna;Mikolow;Augustow;Polkowice;Jaworzno;Znin;Debica;Naklo nad Notecia;Sochaczew;Poland;Sepolno Krajenskie;Lobez;Zyrardow;Gorzow Wielkopolski;Zawiercie;Glubczyce;Kwidzyn;Pyrzyce;Wielun;Suwalki;Leczna;Wabrzezno;Hrubieszow;Nidzica;Wegorzewo;Nowy Tomysl;Warsaw;Rzeszow;Chorzow;Wloclawek;Police;Wagrowiec;Nowe Miasto Lubawskie;Nowy Sacz;Grojec;Zamosc;Tomaszow Mazowiecki;Tomaszow Lubelski;Nowy Dwor Mazowiecki;Janow Lubelski;Kolbuszowa;Mragowo;Sopot;Bielsko-Biala;Zagan;Slubice;Chojnice;Swinoujscie;Tarnobrzeg;Brzeziny;Wieruszow;Luban;Ciechanow;Tczew;Kamienna Gora;Hajnowka;Belchatow;Krotoszyn;Bytow;Mysliborz;Tarnowskie Gory;Gorlice;Plock;Krosno;Bialogard;Inowroclaw;Walcz;Gryfice;Dabrowa Gornicza;Kartuzy;Pruszkow;Pila;Strzelce Opolskie;Grajewo;Poznan;Torun;Koszalin;Chodziez;Raciborz;Lancut;Minsk Mazowiecki;Konskie;Kolo;Limanowa;Swietochlowice;Jawor;Pabianice;Grudziadz;Kutno;Strzelin;Nowa Sol;Prudnik;Grodzisk Wielkopolski;Zambrow;Mogilno;Wlodawa;Kozienice;Mlawa;Piaseczno;Gora;Cieszyn;Zgorzelec;Lubartow;Zory;Bilgoraj;Piekary Slaskie;Opoczno;Turek;Kolobrzeg;Brzeg;Namyslow;Miechow;Oswiecim;Gostyn;Konin;Dzialdowo;Kolno;Gliwice;Bytom;Jaroslaw;Olecko;Ostrowiec Swietokrzyski;Brzesko;Zary;Sucha Beskidzka;Bierun;Lomza;Zlotow;Myszkow;Gdynia;Ozarow Mazowiecki;Swidnik;Staszow;Przeworsk;Ustrzyki Dolne;Lubaczow;Starachowice;Choszczno;Zielona Gora;Rawicz;Srem;Zgierz;Makow Mazowiecki;Olkusz;Ketrzyn;Bochnia;Brodnica;Przemysl;Bydgoszcz;Leszno;Lipno;Wroclaw;Jelenia Gora;Chelmno;Lebork;Lukow;Ostroda;Oborniki;Szczecinek;Krasnik;Ilawa;Ryki;Opole;Pisz;Ostrow Mazowiecka;Golub-Dobrzyn;Lodz;Slawno;Kielce;Sokolka;Goleniow;Wolow;Tuchola;Szydlowiec;Krakow;Pruszcz Gdanski;Piotrkow Trybunalski;Pszczyna;Kepno;Stalowa Wola;Swidnica;Lubliniec;Wloszczowa;Bielsk Podlaski;Siemianowice Slaskie;Pleszew;Jaslo;Elblag;Rawa Mazowiecka;Szczecin;Lezajsk;Pultusk;Krasnystaw;Sulecin;Sosnowiec;Malbork;Wodzislaw Slaski;Goldap;Sanok;Gostynin;Czestochowa;Biala Podlaska;Starogard Gdanski;Myslenice;Wegrow;Zlotoryja;Plonsk;Bialystok;Zdunska Wola;Dzierzoniow;Radzyn Podlaski;Braniewo;Skierniewice;Monki;Tarnow;Wrzesnia;Szamotuly;Kluczbork;Leczyca;Wadowice;Koscierzyna;Ostrow Wielkopolski;Katowice;Wysokie Mazowieckie;Jastrzebie-Zdroj;Sroda Slaska;Nowy Dwor Gdanski;Jarocin;Gniezno +PR;San Juan +CG;Owando;Djambala;Brazzaville;Madingou;Dolisie;Loango;Kinkala;Congo (Brazzaville);Sibiti;Impfondo;Ouesso;Pointe-Noire;Ewo +UY;Montevideo;Mercedes;Treinta y Tres;Rocha;Maldonado;Canelones;Durazno;Rivera;Uruguay;Melo;Tacuarembo;San Jose;Minas;Paysandu;Salto;Fray Bentos;Colonia del Sacramento;Florida;Artigas +HN;La Paz;La Ceiba;Yuscaran;Trujillo;San Pedro Sula;Honduras;Santa Barbara;Ciudad Choluteca;Juticalpa;Tegucigalpa;Puerto Lempira;Santa Rosa de Copan;Roatan;La Esperanza;Nueva Ocotepeque;Comayagua;Gracias;Nacaome;Yoro +UG;Nakasongola;Rukungiri;Kiryandongo;Lwengo;Kasanda;Ntoroko;Lyantonde;Otuke;Gulu;Namutumba;Kotido;Kanungu;Kyenjojo;Soroti;Bukwo;Nabilatuk;Luuka Town;Mitoma;Adjumani;Bundibugyo;Rubirizi;Kabale;Sironko;Busia;Pader;Moyo;Nsiika;Mukono;Kasaali;Maracha;Kiboga;Bulisa;Kalungu;Lira;Kalangala;Kamwenge;Kyankwanzi;Agago;Ntara;Lamwo;Tororo;Luwero;Busesa;Mpigi;Uganda;Bulambuli;Mityana;Kumi;Kibingo;Alebtong;Ngora;Kayunga;Amolatar;Nwoya;Kagadi;Kibiito;Napak;Kiruhura;Serere;Masaka;Pakwach;Kalaki;Kyegegwa;Rubanda;Abim;Manafwa;Amuria;Mbale;Mayuge;Buikwe;Bukomansimbi;Kapchorwa;Jinja;Wakiso;Zombo;Nakapiripirit;Arua;Koboko;Mbarara;Kakumiro;Buyende;Amudat;Yumbe;Kampala;Ibanda;Kanoni;Mubende;Kisoro;Rakai;Binyin;Apac;Butaleja;Iganga;Palenga;Kinoni;Oyam;Katakwi;Butebo;Hoima;Kitamilo;Fort Portal;Kaliro;Kitgum;Gombe;Butalangu;Nebbi;Kibuku;Bugiri;Bushenyi;Mparo;Namayingo;Dokolo;Amuru;Moroto;Kasese;Bududa;Kole;Ntungamo;Sembabule;Isingiro;Kamuli;Kaberamaido;Bupoto;Masindi;Bukedea;Pallisa;Kaabong +GN;Faranah;Kindia;Gueckedou;Fria;Pita;Kissidougou;Koundara;Macenta;Guinea;Gaoual;Mali;Tougue;Mamou;Forecariah;Dinguiraye;Dalaba;Boke;Conakry;Kouroussa;N'Zerekore;Yomou;Boffa;Kerouane;Telimele;Siguiri;Dabola;Beyla;Kankan;Labe +BG;Sofia;Stara Zagora;Montana;Tutrakan;Veliko Tarnovo;Kyustendil;Blagoevgrad;Bulgaria;Dospat;Sliven;Gabrovo;Rakitovo;Burgas;Knezha;Kavarna;Pazardzhik;Zlatograd;Smolyan;Pernik;Kozloduy;Velingrad;Pleven;Vidin;Sozopol;Lovech;Silistra;Panagyurishte;Haskovo;Razgrad;Plovdiv;Peshtera;Balchik;Berkovitsa;Shumen;Yambol;Lom;Svishtov;Varna;Kardzhali;Ruse;Dobrich;Pomorie;Nesebar;Targovishte;Vratsa +CR;San Jose;Costa Rica;Heredia;Liberia;Puntarenas;Alajuela;Cartago;Puerto Limon +RW;Kigali;Byumba;Rwanda;Nyanza;Kibuye;Rwamagana +PA;Colon;Penonome;Chepo;Jaque;Sabanitas;Las Tablas;Puerto Armuelles;Sieyik;Union Choco;Puerto Pilon;Bocas del Toro;Alcalde Diaz;Buabidi;Pacora;Arraijan;Santiago;Ancon;Bajo Boquete;Kusapin;Tocumen;Cativa;Aguadulce;Nuevo Arraijan;Chilibre;Puerto Caimito;Chitre;Changuinola;Gaigirgordub;David;Panama;La Chorrera;La Palma;Almirante;Panama City;El Coco;San Miguelito +SN;Thies;Diourbel;Kolda;Fatick;Ziguinchor;Dakar;Kaolack;Matam;Sedhiou;Saint-Louis;Louga;Tambacounda;Senegal;Kedougou;Kaffrine +OM;`Ibri;Muscat;Hayma';Oman;Sur;Ibra';Salalah;Ar Rustaq;Suhar;Khasab;Nizwa;Al Buraymi;Al Mazyunah +MN;Uliastay;Ulaangom;Baruun-Urt;Mandalgovi;Ulaan-Uul;Ondorhaan;Dzuunmod;Suhbaatar;Hovd;Bayanhongor;Bulgan;Altay;Hoshoot;Mongolia;Choyr;Saynshand;Choybalsan;Tsetserleg;Arvayheer;Erdenet;Dalanzadgad;Darhan;Olgiy;Dalandzadgad;Harhorin;Moron;Ulaanbaatar +IL;Beersheba;Tel Aviv-Yafo;Nazareth;Jerusalem;Haifa;Israel;Ramla +DK;Taastrup;Frederiksvaerk;Ikast;Thisted;Skive;Copenhagen;Esbjerg;Hjorring;Slagelse;Vejle;Vordingborg;Kalundborg;Svendborg;Hedensted;Kolding;Nykobing Mors;Frederikshavn;Holstebro;Silkeborg;Horsholm;Nykobing Falster;Ringsted;Denmark;Holbaek;Herning;Haslev;Hobro;Grenaa;Bronderslev;Koge;Odense;Aarhus;Stovring;Skanderborg;Nyborg;Odder;Varde;Vejen;Horsens;Sonderborg;Naestved;Frederikssund;Helsingor;Soro;Fredericia;Viborg;Randers;Ringkobing;Helsinge;Dragor;Solrod Strand;Ronne;Aabenraa;Struer;Roskilde;Middelfart;Aalborg;Aars;Hillerod;Haderslev;Farum +FI;Hamina;Imatra;Laukaa;Espoo;Savonlinna;Tuusula;Jarvenpaa;Kempele;Kerava;Sipoo;Kontiolahti;Loimaa;Ii;Kirkkonummi;Alavus;Forssa;Lempaala;Laitila;Karkkila;Masku;Liperi;Ylojarvi;Kitee;Eura;Kajaani;Raahe;Somero;Kuusamo;Jyvaskyla;Joensuu;Tornio;Loviisa;Ylivieska;Jamsa;Sodankyla;Hameenkyro;Kalajoki;Sotkamo;Kurikka;Hanko;Helsinki;Orivesi;Orimattila;Eurajoki;Muhos;Kauniainen;Korsholm;Kangasala;Salo;Kankaanpaa;Nokia;Kuopio;Rauma;Turku;Keuruu;Vaasa;Siilinjarvi;Huittinen;Mantsala;Lieksa;Nivala;Pieksamaki;Lappeenranta;Kaarina;Janakkala;Mariehamn;Raisio;Lahti;Alajarvi;Ulvila;Lapua;Kotka;Porvoo;Kokkola;Leppavirta;Hollola;Paimio;Saarijarvi;Mikkeli;Naantali;Jakobstad;Pargas;Iisalmi;Kauhava;Hausjarvi;Kemijarvi;Sastamala;Heinola;Vantaa;Pirkkala;Uusikaupunki;Kemi;Tampere;Kouvola;Hameenlinna;Valkeakoski;Kauhajoki;Narpes;Riihimaki;Varkaus;Rovaniemi;Lapinlahti;Hyvinkaa;Finland;Muurame;Seinajoki;Lohja;Lieto;Raseborg;Pori;Liminka;Ilmajoki;Aanekoski;Oulu +CZ;Plzen;Liberec;Usti nad Labem;Brno;Pardubice;Hradec Kralove;Prague;Jihlava;Zlin;Ostrava;Czechia;Karlovy Vary;Olomouc;Ceske Budejovice +NZ;Stratford;Auckland;Whangarei;Greymouth;Wellington;Dunedin;Napier;New Zealand;Palmerston North;Hamilton;Gisborne;Richmond;Blenheim;Waitangi;Nelson;Invercargill;Christchurch;Whakatane +DO;Monte Plata;San Francisco de Macoris;Salcedo;La Vega;Bani;Jimani;Santo Domingo;La Romana;Sabaneta;San Juan;Nagua;El Seibo;Monte Cristi;Santiago;Samana;Pedernales;Bonao;Moca;Azua;San Jose de Ocoa;Higuey;Comendador;Puerto Plata;San Pedro de Macoris;Hato Mayor;Santo Domingo Este;Neiba;San Cristobal;Dajabon;Dominican Republic;Barahona;Mao;Cotui +PT;Santiago do Cacem;Mirandela;Albergaria-a-Velha;Olhao;Guarda;Rio Maior;Elvas;Caldas da Rainha;Tondela;Lourinha;Alcobaca;Vouzela;Sines;Silves;Beja;Melgaco;Lousa;Coimbra;Fafe;Aljustrel;Batalha;Peso da Regua;Baiao;Ponte de Lima;Montemor-o-Novo;Serta;Alcanena;Tavira;Gouveia;Portimao;Ponta Delgada;Portalegre;Torres Vedras;Macedo de Cavaleiros;Tomar;Celorico de Basto;Vagos;Gondomar;Ovar;Vila Real;Vendas Novas;Viseu;Miranda do Corvo;Moncao;Ansiao;Esposende;Mangualde;Satao;Vila Real de Santo Antonio;Montalegre;Montijo;Lousada;Vila Verde;Cabeceiras de Basto;Tabua;Vila Nova de Gaia;Castelo Branco;Moura;Alcochete;Idanha-a-Nova;Evora;Vila do Conde;Ponte da Barca;Fundao;Castro Daire;Braganca;Cadaval;Odivelas;Amarante;Moimenta da Beira;Agueda;Viana do Castelo;Mealhada;Pombal;Sao Joao da Madeira;Pacos de Ferreira;Almeirim;Loule;Povoa de Varzim;Barreiro;Seixal;Reguengos de Monsaraz;Trofa;Santarem;Salvaterra de Magos;Vila Pouca de Aguiar;Mortagua;Arruda dos Vinhos;Condeixa-a-Nova;Montemor-o-Velho;Sobral de Monte Agraco;Ourem;Benavente;Oliveira de Azemeis;Entroncamento;Sao Pedro do Sul;Abrantes;Bombarral;Chamusca;Odemira;Maia;Sesimbra;Obidos;Oliveira do Hospital;Porto;Valongo;Mafra;Vieira do Minho;Vinhais;Santo Tirso;Ponte de Sor;Povoa de Lanhoso;Cinfaes;Espinho;Vila Nova de Cerveira;Torres Novas;Loures;Arganil;Arcos de Valdevez;Nazare;Grandola;Paredes de Coura;Figueira da Foz;Barcelos;Mira;Alijo;Pinhel;Anadia;Portugal;Resende;Aveiro;Lagoa;Estremoz;Funchal;Arouca;Leiria;Braga;Ilhavo;Trancoso;Chaves;Azambuja;Lisbon;Moita;Sintra;Alcacer do Sal;Valenca;Oliveira do Bairro;Marinha Grande;Murtosa;Porto de Mos;Torre de Moncorvo;Lagos;Paredes;Sabugal;Amares;Penafiel;Matosinhos;Feira;Penacova;Almada;Seia;Felgueiras;Peniche;Setubal;Vila Franca de Xira;Amadora;Soure;Palmela;Serpa;Cantanhede;Oeiras;Estarreja;Coruche;Guimaraes;Sao Bras de Alportel;Cascais;Lamego;Sever do Vouga;Nelas;Cartaxo;Marco de Canavezes;Famalicao;Oliveira de Frades;Faro;Albufeira;Ferreira do Zezere;Alenquer;Vale de Cambra;Covilha;Santa Comba Dao +IE;Limerick;Dun Dealgan;Nenagh;Tralee;Waterford;Port Laoise;Ennis;Tallaght;Tullamore;Castlebar;Monaghan;Ros Comain;Lifford;Dunleary;Swords;Naas;Wexford;Carrick on Shannon;Galway;Wicklow;Carlow;Kilkenny;Ireland;Cork;An Cabhan;Sligo;Clonmel;Dublin;Mullingar;Longford;Trim +BE;Roeselare;Waremme;Wavre;Sint-Niklaas;Maaseik;Mons;Ostend;Marche-en-Famenne;Mouscron;Bruges;Soignies;Aalst;Namur;Ath;Virton;Veurne;Diksmuide;Belgium;Enghien;Eeklo;Philippeville;Gent;Charleroi;Tielt;Oudenaarde;Turnhout;Brussels;Bastogne;Hannut;Kortrijk;Hasselt;Liege;Arlon;Verviers;Thuin;Vilvoorde;Antwerp;Tournai;Tongeren;Nivelles;Mechelen;Ieper;Dendermonde;Huy;Dinant +RS;Presevo;Gadzin Han;Knic;Priboj;Kragujevac;Zrenjanin;Kikinda;Blace;Bajina Basta;Kovacica;Vladicin Han;Senta;Ljubovija;Bojnik;Smederevska Palanka;Nova Varos;Stara Pazova;Kanjiza;Srbobran;Mali Idos;Sabac;Niska Banja;Sremski Karlovci;Pozarevac;Lazarevac;Serbia;Kostolac;Valjevo;Novi Becej;Crna Trava;Vrbas;Razanj;Bela Crkva;Ada;Bosilegrad;Arilje;Rekovac;Indija;Jagodina;Backa Topola;Novi Pazar;Vrnjacka Banja;Bujanovac;Vladimirci;Malo Crnice;Coka;Krupanj;Dimitrovgrad;Becej;Vrsac;Golubac;Pirot;Sid;Surcin;Nova Crnja;Negotin;Merosina;Cacak;Novi Sad;Kraljevo;Nis;Sjenica;Zabalj;Lajkovac;Kula;Lucani;Koceljeva;Vranje;Arandelovac;Plandiste;Kovin;Belgrade;Varvarin;Cuprija;Medveda;Veliko Gradiste;Mali Zvornik;Titel;Sokobanja;Pancevo;Lapovo;Zitorada;Velika Plana;Pozega;Novi Knezevac;Uzice;Petrovac na Mlavi;Trstenik;Bor;Doljevac;Vlasotince;Leskovac;Bela Palanka;Kursumlija;Ljig;Irig;Svilajnac;Sremska Mitrovica;Ruma;Zagubica;Zabari;Prokuplje;Zajecar;Bogatic;Boljevac;Krusevac;Topola;Majdanpek;Babusnica;Backa Palanka;Svrljig;Osecina;Gornji Milanovac;Brus;Petrovaradin;Ivanjica;Aleksinac;Aleksandrovac;Raska;Loznica;Cicevac;Prijepolje;Smederevo;Zemun;Mionica;Zitiste;Kucevo;Cajetina;Despotovac;Beocin;Batocina;Paracin;Kladovo;Ub;Subotica;Trgoviste;Backi Petrovac;Temerin;Bac;Sombor;Apatin;Knjazevac;Secanj;Lebane;Tutin;Kosjeric;Pecinci;Opovo;Mladenovac;Odzaci;Raca;Alibunar;Surdulica +QA;Ash Shihaniyah;Ar Rayyan;Madinat ash Shamal;Umm Salal `Ali;Al Wakrah;Doha;Az Za`ayin;Qatar;Al Khawr +LY;Tobruk;Libya;Al Jawf;Nalut;Ghat;Al Marj;Zuwarah;Al `Aziziyah;Tripoli;Al Bayda';Surt;Sabha;Hun;Murzuq;Awbari;Gharyan;Idri;Misratah;Benghazi;Ajdabiya;Az Zawiyah;Darnah +BI;Bubanza;Bururi;Ngozi;Kayanza;Karuzi;Gitega;Makamba;Cibitoke;Isale;Ruyigi;Burundi;Rumonge;Kirundo;Muyinga;Muramvya;Rutana;Bujumbura;Cankuzo +MZ;Tete;Beira;Maputo;Quelimane;Chimoio;Pemba;Lichinga;Mozambique;Inhambane;Nampula;Xai-Xai +KG;Kulundu;Belovodskoe;Kerben;Kyrgyzstan;Kayyngdy;Kara-Balta;Kochkor;Kemin;Gulcho;Bishkek;Massy;Batken;Naryn;Aravan;Karakol;Baetov;Chuy;Kara-Suu;Jalal-Abad;Kara-Bak;Isfana;Buzhum;Toktogul;Kant;Talas;Eski-Nookat;Kazarman;Ala-Buka;At-Bashy;Teploklyuchenka;Ivanovka;Pokrovka;Osh;Bazar-Korgon;Kyzyl-Adyr;Kyzyl-Suu;Suzak;Bokonbaev;Uch-Korgon;Cholpon-Ata;Tup;Sokuluk;Lebedinovka;Kara-Kulja +GE;Khashuri;Kutaisi;Rustavi;Senaki;Zugdidi;Ozurgeti;Marneuli;Sokhumi;Chiatura;Poti;Zestaponi;Akhaltsikhe;Telavi;Sagarejo;Kaspi;Gori;Batumi;Ambrolauri;Tbilisi;Kobuleti;Mtskheta;Georgia;Samtredia +TD;Doba;Pala;Bongor;Faya;Mongo;Amdjarass;Goz-Beida;Moundou;Sarh;Moussoro;Chad;Massenya;Koumra;Bol;Abeche;N'Djamena;Ati;Biltine;Massakory;Bardai;Am-Timan;Fada;Lai;Mao +MR;Tidjikja;Nema;Kaedi;Dar Naim;Aleg;Mauritania;Tevragh Zeina;Rosso;Nouadhibou;Kiffa;Zouerate;Selibaby;Aioun;Atar;Akjoujt;Nouakchott;Arafat +AM;Ijevan;Artashat;Ashtarak;Gavarr;Armavir;Gyumri;Yerevan;Armenia;Yeghegnadzor;Hrazdan;Vanadzor;Kapan +NO;Skien;Rorvik;Mandal;Os;Porsgrunn;As;Svolvaer;Volda;Randaberg;Namsos;Rygge;Stavanger;Fredrikstad;Bryne;Voss;Hammerfest;Kristiansand;Oslo;Trondheim;Stord;Sandnes;Narvik;Kopervik;Kongsberg;Drammen;Tonsberg;Stjordal;Moss;Brumunddal;Lyngdal;Harstad;Mosjoen;Bodo;Egersund;Nittedal;Finnsnes;Ski;Alesund;Sandvika;Askim;Bergen;Alta;Kongsvinger;Arendal;Sarpsborg;Tromso;Hamar;Stange;Verdal;Lorenskog;Fauske;Haugesund;Sogndal;Kristiansund;Notodden;Grimstad;Lier;Orsta;Farsund;Gjovik;Kirkenes;Nesoddtangen;Forde;Molde;Asker;Elverum;Halden;Sogne;Vadso;Nannestad;Honefoss;Kleppe;Kragero;Vennesla;Lillehammer;Norway;Levanger;Floro;Eidsvoll;Royken;Steinkjer;Mo i Rana +NI;Altagracia;San Lorenzo;Palacaguina;El Almendro;San Isidro;Greytown;Managua;Ocotal;Tisma;Quezalguaque;Nueva Guinea;Bonanza;Nandaime;Masaya;San Ramon;Santa Rosa del Penon;Jalapa;Siuna;Chichigalpa;Rivas;Prinzapolka;Muy Muy;Potosi;El Ayote;Mulukuku;Rancho Grande;San Marcos;Ticuantepe;San Juan de Rio Coco;San Dionisio;Granada;Quilali;Bilwi;Condega;San Rafael del Sur;Esteli;Somotillo;Juigalpa;La Trinidad;Niquinohomo;San Carlos;Villa Sandino;Ciudad Sandino;El Jicaro;Matagalpa;Jinotepe;Jinotega;Tola;Moyogalpa;Muelle de los Bueyes;Waspan;San Miguelito;Pueblo Nuevo;Matiguas;El Tortuguero;Corinto;San Francisco Libre;Santo Domingo;Rio Blanco;Telica;Sebaco;Villa El Carmen;Totogalpa;El Viejo;Esquipulas;Villanueva;Acoyapa;Terrabona;El Realejo;Masatepe;El Cua;El Rama;El Crucero;Belen;San Jose de Bocay;Murra;Kukrahill;Tipitapa;Wiwili;El Jicaral;Camoapa;San Fernando;Comalapa;San Lucas;San Sebastian de Yali;San Rafael del Norte;Bocana de Paiwas;San Juan de Limay;Nagarote;Nicaragua;Rosita;Teustepe;Boaco;El Sauce;Telpaneca;Posoltega;Achuapa;Yalaguina;Ciudad Dario;La Concepcion;Mateare;La Libertad;La Paz Centro;Nindiri;La Cruz de Rio Grande;Somoto;Chinandega;Leon;Diriomo;Bluefields;Nandasmo;San Juan del Sur;Diriamba +TM;Ashgabat;Mary;Anew;Turkmenistan;Balkanabat;Turkmenabat;Dasoguz +NE;Dosso;Maradi;Gaya;Nguigmi;Arlit;Tillaberi;Niger;Tahoua;Niamey;Diffa;Madaoua;Birnin Konni;Agadez;Guidan Roumdji;Goure;Zinder +LR;Robertsport;Monrovia;Harper;Gbarnga;Cestos City;Voinjama;Tubmanburg;Liberia;Zwedru;Sanniquellie;Bopolu;Barclayville;Kakata;Buchanan;Greenville;Fish Town;Bensonville +HT;Jacmel;Haiti;Port-au-Prince;Cap-Haitien;Gonaives;Miragoane;Hinche;Jeremie;Fort Liberte;Les Cayes;Port-de-Paix +ER;Keren;Massawa;Eritrea;Barentu;Assab;Mendefera;Asmara +SL;Freetown;Bo;Sierra Leone;Kenema;Makeni;Port Loko +LA;Salavan;Savannakhet;Louang Namtha;Xam Nua;Ban Houayxay;Pakxe;Laos;Xekong;Xaignabouli;Phon-Hong;Phongsali;Thakhek;Pakxan;Muang Sing;Phonsavan;Attapu;Anouvong;Xai;Vientiane;Louangphabang +LV;Aizkraukle;Liepaja;Kuldiga;Riga;Valmiera;Marupe;Preili;Ogre;Madona;Saulkrasti;Livani;Tukums;Aluksne;Varaklani;Adazi;Saldus;Dobele;Latvia;Rezekne;Valka;Balvi;Kraslava;Kekava;Ludza;Salaspils;Limbazi;Cesis;Ropazi;Sigulda;Daugavpils;Talsi;Olaine;Jurmala;Gulbene;Smiltene;Jekabpils;Ventspils;Bauska;Ulbroka;Jelgava +CF;Bria;Bimbo;Mbaiki;Bangassou;Bouar;Mobaye;Bossangoa;Berberati;Bangui;Kaga Bandoro;Sibut;Obo;Birao;Nola;Central African Republic;Bozoum;Ndele;Bambari +TJ;Istaravshan;Buston;Shahrinav;Somoniyon;Norak;Vakhsh;Hulbuk;Tursunzoda;Fayzobod;Hisor;Vanj;Konibodom;Danghara;Vahdat;Ayni;Dushanbe;Jilikul;Levakant;Khovaling;Moskva;Khorugh;Ghafurov;Obikiik;Abdurahmoni Jomi;Roghun;Shahritus;Dusti;Mu'minobod;Bokhtar;Shurobod;Ghonchi;Shahriston;Panj;Tajikistan;Yovon;Farkhor;Panjakent;Khujand;Isfara;Rasht;Kulob +NP;Godawari;Biratnagar;Pokhara;Janakpur;Butwal;Bhairahawa;Birendranagar;Kathmandu;Nepal;Hetauda +GA;Koulamoutou;Oyem;Makokou;Lambarene;Port-Gentil;Franceville;Gabon;Tchibanga;Mouila;Libreville +HR;Viskovo;Varazdin;Belisce;Pozega;Solin;Slavonski Brod;Pula;Labin;Vrbovec;Kastav;Cepin;Pazin;Metkovic;Podstrana;Kutina;Duga Resa;Novska;Crikvenica;Rovinj;Sisak;Gospic;Popovaca;Imotski;Trogir;Nova Gradiska;Garesnica;Umag;Koprivnica;Ivanec;Zadar;Slatina;Valpovo;Sibenik;Jastrebarsko;Zupanja;Daruvar;Bjelovar;Novi Marof;Vodice;Sveti Ivan Zelina;Dubrovnik;Porec;Sinj;Ivanic-Grad;Brdovec;Dakovo;Knin;Croatia;Ogulin;Cakovec;Pleternica;Zagreb;Osijek;Nasice;Virovitica;Krizevci;Opatija;Matulji;Rijeka;Nedelisce;Velika Gorica;Samobor;Vinkovci;Krapina;Zapresic;Omis;Split;Zabok;Dugo Selo;Vukovar;Makarska;Benkovac;Petrinja +LT;Taurage;Pagegiai;Kaunas;Kedainiai;Radviliskis;Birstonas;Visaginas;Silute;Zarasai;Salcininkai;Joniskis;Elektrenai;Ignalina;Plunge;Prienai;Sakiai;Skuodas;Naujoji Akmene;Kazlu Ruda;Vilkaviskis;Svencionys;Kaisiadorys;Klaipeda;Lazdijai;Mazeikiai;Rokiskis;Kelme;Marijampole;Silale;Druskininkai;Utena;Anyksciai;Varena;Pakruojis;Vilnius;Lithuania;Raseiniai;Alytus;Ukmerge;Trakai;Siauliai;Kupiskis;Telsiai;Rietavas;Jurbarkas;Birzai;Kalvarija;Nida;Panevezys;Moletai;Pasvalys;Palanga;Jonava;Sirvintos;Kretinga +MD;Telenesti;Comrat;Criuleni;Nisporeni;Moldova;Cocieri;Drochia;Briceni;Soldanesti;Calarasi;Floresti;Causeni;Hincesti;Soroca;Balti;Edinet;Basarabeasca;Glodeni;Chisinau;Cantemir;Falesti;Stefan Voda;Taraclia;Donduseni;Riscani;Ialoveni;Anenii Noi;Leova;Singerei;Cimislia;Bender;Tiraspol;Orhei;Ungheni;Rezina;Straseni;Ocnita;Cahul +PG;Kundiawa;Port Moresby;Wewak;Alotau;Madang;Buka;Wabag;Vanimo;Tari;Lorengau;Kimbe;Goroka;Papua New Guinea;Daru;Popondetta;Kurumul;Kokopo;Lae;Kavieng;Mendi;Kerema;Mount Hagen +BJ;Ouidah;Kandi;Savalou;Parakou;Abomey;Djougou;Natitingou;Porto-Novo;Sakete;Lokossa;Benin;Dogbo;Cotonou +BH;Manama +EE;Kohtla-Jarve;Narva;Johvi;Paide;Rakvere;Tartu;Tallinn;Kardla;Viljandi;Parnu;Keila;Sillamae;Polva;Estonia;Jogeva;Rapla;Valga;Haapsalu;Kuressaare;Maardu;Voru +DJ;Dikhil;Djibouti;Ali Sabieh;Obock;Arta;Tadjourah +TN;Kasserine;Monastir;Kairouan;Sidi Bouzid;Gafsa;Sousse;Ben Arous;Beja;Tataouine;Manouba;El Kef;Medenine;Zaghouan;Jendouba;Kebili;Gabes;Ariana;Tunis;Bizerte;Tozeur;Sfax;Tunisia;Siliana;Mahdia;Nabeul +XG;Gaza +JM;Falmouth;Lucea;Port Antonio;Morant Bay;Kingston;Spanish Town;Jamaica;Port Maria;Savanna-la-Mar;Mandeville;May Pen;Half Way Tree;Black River;Saint Ann's Bay;Montego Bay +MK;Zelenikovo;Brvenica;Vrapciste;Makedonska Kamenica;Novo Selo;Bogovinje;Cucer-Sandevo;Rosoman;Kicevo;Kocani;Gostivar;Lipkovo;Vasilevo;Stip;Petrovec;Jegunovce;Konce;Pehcevo;Ilinden;Gradsko;North Macedonia;Bogdanci;Plasnica;Lozovo;Star Dojran;Kratovo;Centar Zupa;Radovis;Debar;Kumanovo;Ohrid;Vevcani;Sopiste;Rankovce;Sveti Nikole;Veles;Tearce;Vinica;Makedonski Brod;Probistip;Kavadarci;Zrnovci;Rostusa;Prilep;Aracinovo;Berovo;Zelino;Bosilovo;Struga;Mogila;Studenicani;Novaci;Strumica;Tetovo;Belcista;Bitola;Demir Kapija;Resen;Gevgelija;Dolneni;Krusevo;Valandovo;Skopje;Kriva Palanka;Demir Hisar;Staro Nagoricane;Delcevo;Krivogastani;Negotino;Karbinci;Oblesevo +GW;Cacheu;Guinea-Bissau;Quinhamel;Bissau;Buba;Gabu;Farim;Catio;Bolama;Bafata +MT;Victoria;Lija;Safi;Valletta;Ghasri;Gzira;Ghaxaq;Ghajnsielem;Mellieha;Tarxien;Xewkija;Imtarfa;Mosta;San Giljan;Siggiewi;Iklin;Qormi;Qrendi;Ta' Xbiex;Santa Lucija;Imsida;Gharb;Nadur;Birzebbuga;Xaghra;Marsaskala;Saint Paul's Bay;Imdina;Gudja;Sliema;Swieqi;Marsaxlokk;Attard;San Gwann;Floriana;Birkirkara;Fontana;Imqabba;Vittoriosa;Senglea;Marsa;Qala;Munxar;Dingli;Balzan;Cospicua;Fgura;Luqa;Zurrieq;San Lawrenz;Zabbar;Imgarr;Rabat;Pieta;Kalkara;Gharghur;Naxxar;Sannat;Malta;Hamrun;Kercem;Zebbug;Kirkop;Zejtun;Pembroke;Xghajra;Santa Venera;Paola +PY;Caazapa;Fuerte Olimpo;Filadelfia;Villa Hayes;Pilar;San Pedro de Ycuamandiyu;Aregua;Caacupe;Ciudad del Este;Pedro Juan Caballero;Concepcion;Asuncion;Encarnacion;Paraguari;Villarrica;Paraguay;San Juan Bautista;Salto del Guaira;Coronel Oviedo +SK;Hlohovec;Poprad;Nove Zamky;Dunajska Streda;Presov;Senica;Ziar nad Hronom;Sabinov;Malacky;Rimavska Sobota;Detva;Vel'ky Krtis;Michalovce;Humenne;Piest'any;Trebisov;Bratislava;Kysucke Nove Mesto;Pezinok;Zvolen;Liptovsky Mikulas;Senec;Partizanske;Zilina;Svidnik;Puchov;Vranov nad Topl'ou;Zlate Moravce;Trencin;Ruzomberok;Banska Stiavnica;Bardejov;Dolny Kubin;Nitra;Lucenec;Revuca;Slovakia;Topol'cany;Spisska Nova Ves;Galanta;Snina;Skalica;Cadca;Nove Mesto nad Vahom;Prievidza;Povazska Bystrica;Banska Bystrica;Levoca;Stropkov;Kosice;Bytca;Levice;Brezno;Trnava;Sal'a;Kezmarok;Tvrdosin;Banovce nad Bebravou;Myjava;Martin;Roznava;Stara L'ubovna +SS;Wau;South Sudan;Bor;Malakal;Rumbek;Aweil;Juba;Kuacjok;Yambio;Torit;Bentiu +CH;Aarau;Altdorf;Rheinfelden;Kussnacht;Switzerland;Einsiedeln;Solothurn;Rapperswil-Jona;Brig-Glis;Herisau;Rorschach;Yverdon-les-Bains;Geneva;Sankt Gallen;Schaffhausen;Wil;Hinwil;Chur;La Chaux-de-Fonds;Bern;Weinfelden;Lachen;Pfaffikon;Bremgarten;Glarus;Frauenfeld;Renens;Winterthur;Basel;Sion;Schwyz;Thun;Lausanne;Uster;Stans;Appenzell;Kreuzlingen;Altstatten;Davos;Sursee;Bellinzona;Zug;Lucerne;Neuchatel;Fribourg;Langnau;Liestal;Delemont;Sarnen;Zurich +NA;Otjiwarongo;Windhoek;Keetmanshoop;Oshakati;Eenhana;Gobabis;Outapi;Nkurenkuru;Omuthiya;Katima Mulilo;Opuwo;Rundu;Namibia;Mariental;Swakopmund +BA;Gradacac;Zepce;Banja Luka;Celic;Donji Vakuf;Kladanj;Doboj;Ilijas;Gacko;Zivinice;Jajce;Gracanica;Bileca;Odzak;Gradiska;Vares;Bugojno;Cazin;Tesanj;Lopare;Brcko;Vitez;Kljuc;Sarajevo;Posusje;Bihac;Tuzla;Vogosca;Banovici;Rogatica;Srebrenik;Prijedor;Visoko;Kiseljak;Bijeljina;Srbac;Fojnica;Olovo;Stolac;Bratunac;Sapna;Srebrenica;Vlasenica;Pale;Bosanska Krupa;Mostar;Lukavac;Maglaj;Breza;Capljina;Siroki Brijeg;Busovaca;Gorazde;Gornji Vakuf;Trebinje;Jablanica;Bosnia and Herzegovina;Travnik;Citluk;Livno;Novi Travnik;Orasje;Milici;Zenica;Zvornik;Sanski Most;Hadzici;Ilidza;Foca;Ljubuski;Derventa;Brod;Novi Grad +AL;Shkoder;Peshkopi;Albania;Lezhe;Vlore;Berat;Durres;Kukes;Korce;Elbasan;Gjirokaster;Fier;Tirana +GM;Mansa Konko;Basse Santa Su;Brikama;Kanifing;Kerewan;Gambia, The;Janjanbureh;Banjul +LS;Thaba-Tseka;Qacha's Nek;Mohale's Hoek;Mafeteng;Teyateyaneng;Quthing;Mokhotlong;Butha-Buthe;Lesotho;Leribe;Maseru +CY;Famagusta;Kyrenia;Cyprus;Larnaca;Limassol;Paphos;Nicosia +SV;Zacatecoluca;Sonsonate;Sensuntepeque;La Union;El Salvador;Ahuachapan;Santa Tecla;Chalatenango;San Francisco;San Vicente;San Miguel;Usulutan;Santa Ana;San Salvador;Cojutepeque +RE;Saint-Denis +GQ;Pale;Luba;Djibloho;Evinayong;Mongomo;Ebebiyin;Malabo;Equatorial Guinea;Bata +SI;Ljubljana;Vojnik;Piran;Slovenska Bistrica;Sredisce ob Dravi;Dobrovnik;Skofljica;Vransko;Race;Kamnik;Dragomer;Naklo;Starse;Trzin;Brezice;Sempeter pri Gorici;Nazarje;Kocevje;Prevalje;Grad;Lasko;Zgornja Hajdina;Radece;Ig;Gornja Radgona;Bohinjska Bistrica;Slovenske Konjice;Benedikt;Vrhnika;Podlehnik;Sveta Trojica v Slovenskih Goricah;Gornji Petrovci;Smarjeske Toplice;Trbovlje;Kidricevo;Polzela;Moravce;Velike Lasce;Cerkno;Kuzma;Cerklje na Gorenjskem;Markovci;Lukovica;Jesenice;Nova Gorica;Odranci;Recica;Destrnik;Apace;Salovci;Bistrica ob Sotli;Gorenja Vas;Kozje;Podcetrtek;Braslovce;Vitomarci;Hodos;Velenje;Beltinci;Trzic;Ravne na Koroskem;Mirna Pec;Mirna;Cirkulane;Vipava;Selnica ob Dravi;Majsperk;Videm pri Ptuju;Mislinja;Nova Vas;Verzej;Jursinci;Sencur;Bled;Ilirska Bistrica;Straza;Sevnica;Ziri;Maribor;Kranjska Gora;Sentrupert;Medvode;Ankaran;Spodnje Hoce;Komen;Dravograd;Zelezniki;Puconci;Izola;Krizevci;Crnomelj;Cerknica;Rogasovci;Trebnje;Metlika;Borovnica;Kranj;Gornji Grad;Kostanjevica na Krki;Vitanje;Prebold;Semic;Zagorje;Sodrazica;Sostanj;Crna na Koroskem;Zgornja Kungota;Videm;Brezovica;Ajdovscina;Radenci;Radlje ob Dravi;Lovrenc na Pohorju;Logatec;Muta;Sentjur;Luce;Kobilje;Turnisce;Skofja Loka;Solcava;Sentilj;Postojna;Mezica;Poljcane;Ljubno;Store;Zgornje Gorje;Zalec;Velika Polana;Pesnica;Dobrna;Novo Mesto;Ormoz;Vuzenica;Slovenia;Osilnica;Celje;Zgornje Jezersko;Tabor;Ljutomer;Skocjan;Smartno;Ribnica;Komenda;Oplotnica;Rogatec;Dol;Trnovska Vas;Krsko;Domzale;Zetale;Jurovski Dol;Podvelka;Ptuj;Sezana;Kostel;Kobarid;Tolmin;Smarje;Ruse;Zavrc;Miklavz na Dravskem Polju;Slovenj Gradec;Litija;Rogaska Slatina;Razkrizje;Miren;Horjul;Dobrova;Preddvor;Murska Sobota;Bovec;Mokronog;Tisina;Menges;Zrece;Kanal;Divaca;Gorisnica;Cankova;Crensovci;Zuzemberk;Stari Trg;Koper;Dolenjske Toplice;Ivancna Gorica;Spodnji Duplek;Vodice;Hrastnik;Tomaz pri Ormozu;Dobrovo;Hrib-Loski Potok;Lenart v Slovenskih Goricah;Cerkvenjak;Sveta Ana;Sveti Jurij;Dornava;Makole;Moravske-Toplice;Mozirje;Sentjernej;Lendava;Dobje;Pivka;Grosuplje;Idrija;Radovljica +BS;Nassau +MQ;Fort-de-France +BW;Selibe Phikwe;Kasane;Kanye;Masunga;Maun;Jwaneng;Serowe;Lobatse;Gaborone;Tshabong;Francistown;Botswana;Mochudi;Sowa Town;Molepolole;Ghanzi;Ramotswa +SR;Groningen;Onverwacht;Totness;Lelydorp;Albina;Nieuw Nickerie;Brokopondo;Paramaribo;Nieuw Amsterdam;Suriname +TL;Viqueque;Same;Manatuto;Lospalos;Baucau;Aileu;Timor-Leste;Maliana;Pante Macassar;Dili;Ainaro;Gleno;Suai;Liquica +XK;Kamenice;Peje;Lipjan;Istog;Podujeve;Pristina;Junik;Ranillug;Mitrovice;Gjilan;Shtime;Hani i Elezit;Ferizaj;Zubin Potok;Kosovo;Rahovec;Kllokot;Partesh;Gracanice;Mamushe;Kline;Decan;Fushe Kosove;Malisheve;Suhareke;Shterpce;Dragash;Prizren;Viti;Obiliq;Kacanik;Gjakove;Leposaviq;Zvecan;Vushtrri;Novoberde;Gllogovc;Skenderaj +GY;Vreed-en-Hoop;New Amsterdam;Lethem;Mahdia;Fort Wellington;Linden;Georgetown;Guyana;Mabaruma;Bartica +FJ;Suva +NC;We;Noumea;Kone;New Caledonia +ME;Savnik;Cetinje;Zabljak;Golubovci;Ulcinj;Rozaje;Herceg Novi;Pluzine;Danilovgrad;Plav;Pljevlja;Petnjica;Tivat;Andrijevica;Gusinje;Niksic;Montenegro;Budva;Bijelo Polje;Bar;Kolasin;Tuzi;Podgorica;Kotor;Mojkovac;Berane +CW;Willemstad +MU;Port Louis +IS;Reykjavik +MV;Viligili;Foammulah;Dhihdhoo;Hithadhoo;Felidhoo;Kudahuvadhoo;Rasdhoo;Eydhafushi;Veymandoo;Funadhoo;Thinadhoo;Fonadhoo;Mahibadhoo;Male;Muli;Nilandhoo;Thulusdhoo;Kulhudhuffushi;Manadhoo;Maldives;Ungoofaaru;Naifaru +LU;Capellen;Vianden;Echternach;Luxembourg;Grevenmacher;Remich;Mersch;Redange-sur-Attert;Clervaux;Esch-sur-Alzette;Dudelange;Wiltz;Diekirch;Differdange +PF;Papeete +BT;Daga;Samdrup Jongkhar;Lhuentse;Wangdue Phodrang;Samtse;Zhemgang;Bhutan;Tsirang;Thimphu;Sarpang;Mongar;Trongsa;Tsimasham;Paro;Jakar;Pemagatshel;Haa;Trashigang;Trashi Yangtse;Gasa;Punakha +TG;Atakpame;Sokode;Kara;Togo;Dapaong;Lome +SZ;Eswatini;Mbabane;Nhlangano;Lobamba;Manzini;Siteki +BB;Bridgetown +TT;Couva;Tunapuna;Scarborough;Rio Claro;Point Fortin;Port of Spain;Aranguez;Trinidad and Tobago;Siparia;Debe;Arima;San Fernando;Diego Martin;Princes Town;Chaguanas;Sangre Grande +SB;Honiara;Buala;Taro;Tigoa;Tulagi;Kirakira;Solomon Islands;Gizo;Auki;Lata +YT;Mamoudzou +CV;Espargos;Assomada;Calheta de Sao Miguel;Cidade Velha;Sao Filipe;Tarrafal;Praia;Pombas;Porto Ingles;Porto Novo;Mindelo;Cova Figueira;Joao Teves;Ponta do Sol;Pedra Badejo;Ribeira Brava;Sal Rei;Igreja;Picos;Ribeira Grande;Sao Domingos;Cabo Verde;Nova Sintra +LC;Castries +ST;Sao Tome and Principe;Santo Antonio;Sao Tome;Neves;Santana;Guadalupe;Trindade;Sao Joao dos Angolares +GF;Saint-Laurent-du-Maroni;Saint-Georges;Kourou;Sinnamary;Cayenne;Roura;French Guiana;Iracoubo +BZ;Corozal;San Ignacio;Belize City;Punta Gorda;Belmopan;Orange Walk;Belize;Dangriga +BN;Brunei;Bangar;Tutong;Kuala Belait;Bandar Seri Begawan +VU;Isangel;Lakatoro;Luganville;Saratamata;Vanuatu;Port-Vila;Sola +WS;Samoa;Saleaula;Safotulafai;Safotu;Lufilufi;Samamea;Satupa`itea;Apia;Vailoa;Mulifanua;Afega;Asau;Leulumoega +MC;Monaco +AW;Oranjestad +GI;Gibraltar +JE;Saint Helier +MH;Majuro +KM;Mutsamudu;Fomboni;Moroni;Comoros +KI;Tarawa +IM;Douglas +KY;George Town +SC;Victoria +TO;Nuku`alofa +AD;La Massana;Canillo;Ordino;Andorra;Escaldes-Engordany;Encamp;Andorra la Vella;Sant Julia de Loria +AG;Saint John's +GG;Saint Peter Port +GL;Qaqortoq;Nuuk;Sisimiut;Ilulissat;Aasiaat;Greenland +DM;Roseau +VC;Kingstown +FM;Kolonia;Colonia;Tofol;Palikir;Micronesia, Federated States of;Weno +FO;Sorvagur;Kunoy;Oyrarbakki;Skalavik;Fuglafjordhur;Toftir;Kirkja;Skopun;Kvivik;Sandur;Saltangara;Tvoroyri;Skuvoy;Hov;Vestmanna;Famjin;Torshavn;Husavik;Hvalba;Sandavagur;Klaksvik;Vagur;Hvannasund;Strendur;Sumba;Faroe Islands;Porkeri;Nordhragota;Vidhareidhi;Eidhi +KN;Basseterre +VG;Road Town +AS;Pago Pago +SM;Faetano;Serravalle;Borgo Maggiore;Fiorentino;San Marino;Domagnano;Montegiardino;Acquaviva;Chiesanuova +BQ;Oranjestad;The Bottom;Bonaire, Sint Eustatius, and Saba;Kralendijk +TV;Funafuti +LI;Balzers;Triesenberg;Ruggell;Schellenberg;Mauren;Triesen;Vaduz;Planken;Schaan;Eschen;Liechtenstein;Gamprin +MF;Marigot +PM;Saint-Pierre +GD;Saint George's +CK;Avarua +TC;Grand Turk +AI;The Valley +MP;Capitol Hill +BL;Gustavia +FK;Stanley +SX;Philipsburg +XR;Longyearbyen +CX;Flying Fish Cove +GU;Hagta +WF;Leava;Wallis and Futuna;Mata-Utu +BM;Hamilton +VA;Vatican City +NR;Yaren +SH;Edinburgh of the Seven Seas;Saint Helena, Ascension, and Tristan da Cunha;Jamestown;Georgetown +NU;Alofi +MS;Montserrat;Brades;Plymouth +NF;Kingston +GP;Basse-Terre +PW;Ngerulmud +PN;Adamstown +GS;King Edward Point +VI;Charlotte Amalie \ No newline at end of file diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java index 80e349a3f..1f1827ab3 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java @@ -3,6 +3,7 @@ package eu.dnetlib.pace.clustering; import java.util.Map; +import com.mongodb.connection.Cluster; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -177,41 +178,16 @@ public class ClusteringFunctionTest extends AbstractPaceTest { } @Test - public void testKeywordsClustering() { + public void legalnameClustering() { - final ClusteringFunction cf = new KeywordsClustering(params); - final String s = "Polytechnic University of Turin"; + final ClusteringFunction cf = new LegalnameClustering(params); + String s = "key::1 key::2 city::1"; System.out.println(s); System.out.println(cf.apply(conf, Lists.newArrayList(s))); - final String s1 = "POLITECNICO DI TORINO"; - System.out.println(s1); - System.out.println(cf.apply(conf, Lists.newArrayList(s1))); - - final String s2 = "Universita farmaceutica culturale di milano bergamo"; - System.out.println("s2 = " + s2); - System.out.println(cf.apply(conf, Lists.newArrayList(s2))); - - final String s3 = "universita universita milano milano"; - System.out.println("s3 = " + s3); - System.out.println(cf.apply(conf, Lists.newArrayList(s3))); - - final String s4 = "Politechniki Warszawskiej (Warsaw University of Technology)"; - System.out.println("s4 = " + s4); - System.out.println(cf.apply(conf, Lists.newArrayList(s4))); - - final String s5 = "İstanbul Ticarət Universiteti"; - System.out.println("s5 = " + s5); - System.out.println(cf.apply(conf, Lists.newArrayList(s5))); - - final String s6 = "National and Kapodistrian University of Athens"; - System.out.println("s6 = " + s6); - System.out.println(cf.apply(conf, Lists.newArrayList(s6))); - - final String s7 = "Εθνικό και Καποδιστριακό Πανεπιστήμιο Αθηνών"; - System.out.println("s7 = " + s7); - System.out.println(cf.apply(conf, Lists.newArrayList(s7))); - + s = "key::1 key::2 city::1 city::2"; + System.out.println(s); + System.out.println(cf.apply(conf, Lists.newArrayList(s))); } @Test diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java index 7fd81d975..4ec120f4a 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java @@ -54,4 +54,47 @@ public class PaceFunctionTest extends AbstractPaceFunctions { System.out.println("Fixed aliases : " + fixAliases(TEST_STRING)); } + @Test + public void countryInferenceTest() { + assertEquals("IT", countryInference("UNKNOWN", "Università di Bologna")); + assertEquals("UK", countryInference("UK", "Università di Bologna")); + assertEquals("IT", countryInference("UNKNOWN", "Universiteé de Naples")); + assertEquals("UNKNOWN", countryInference("UNKNOWN", "Università del Lavoro")); + } + + @Test + public void cityInferenceTest() { + assertEquals("universita city::3181928", cityInference("Università di Bologna")); + assertEquals("university city::3170647", cityInference("University of Pisa")); + assertEquals("universita", cityInference("Università del lavoro")); + assertEquals("universita city::3173331 city::3169522", cityInference("Università di Modena e Reggio Emilia")); + } + + @Test + public void keywordInferenceTest() { + assertEquals("key::41 turin", keywordInference("Polytechnic University of Turin")); + assertEquals("key::41 torino", keywordInference("POLITECNICO DI TORINO")); + assertEquals( + "key::1 key::60 key::81 milano bergamo", + keywordInference("Universita farmaceutica culturale di milano bergamo")); + assertEquals("key::1 key::1 milano milano", keywordInference("universita universita milano milano")); + assertEquals( + "key::10 kapodistriako panepistemio athenon", + keywordInference("Εθνικό και Καποδιστριακό Πανεπιστήμιο Αθηνών")); + } + + @Test + public void cityKeywordInferenceTest() { + assertEquals("key::41 city::3165524", cityKeywordInference("Polytechnic University of Turin")); + assertEquals("key::41 city::3165524", cityKeywordInference("POLITECNICO DI TORINO")); + assertEquals( + "key::1 key::60 key::81 city::3173435 city::3182164", + cityKeywordInference("Universita farmaceutica culturale di milano bergamo")); + assertEquals( + "key::1 key::1 city::3173435 city::3173435", cityKeywordInference("universita universita milano milano")); + assertEquals( + "key::10 kapodistriako panepistemio city::264371", + cityKeywordInference("Εθνικό και Καποδιστριακό Πανεπιστήμιο Αθηνών")); + } + } diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index 8e72f4efc..fe0dca7f1 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -35,6 +35,7 @@ public class ComparatorTest extends AbstractPaceTest { params.put("name_th", "0.95"); params.put("jpath_value", "$.value"); params.put("jpath_classid", "$.qualifier.classid"); + params.put("codeRegex", "key::\\d+"); } @Test @@ -44,52 +45,23 @@ public class ComparatorTest extends AbstractPaceTest { } @Test - public void cityMatchTest() { - final CityMatch cityMatch = new CityMatch(params); + public void codeMatchTest() { + CodeMatch codeMatch = new CodeMatch(params); - // both names with no cities - assertEquals(1.0, cityMatch.distance("Università", "Centro di ricerca", conf)); + // both names with no codes + assertEquals(1.0, codeMatch.distance("testing1", "testing2", conf)); - // one of the two names with no cities - assertEquals(-1.0, cityMatch.distance("Università di Bologna", "Centro di ricerca", conf)); + // one of the two names with no codes + assertEquals(-1.0, codeMatch.distance("testing1 key::1", "testing", conf)); - // both names with cities (same) - assertEquals(1.0, cityMatch.distance("Universita di Bologna", "Biblioteca di Bologna", conf)); + // both names with codes (same) + assertEquals(1.0, codeMatch.distance("testing1 key::1", "testing2 key::1", conf)); - // both names with cities (different) - assertEquals(0.0, cityMatch.distance("Universita di Bologna", "Universita di Torino", conf)); - assertEquals(0.0, cityMatch.distance("Franklin College", "Concordia College", conf)); + // both names with codes (different) + assertEquals(0.0, codeMatch.distance("testing1 key::1", "testing2 key::2", conf)); - // particular cases - assertEquals(1.0, cityMatch.distance("Free University of Bozen-Bolzano", "Università di Bolzano", conf)); - assertEquals( - 1.0, - cityMatch - .distance( - "Politechniki Warszawskiej (Warsaw University of Technology)", "Warsaw University of Technology", - conf)); - - // failing becasuse 'Allen' is a transliterrated greek stopword - // assertEquals(-1.0, cityMatch.distance("Allen (United States)", "United States Military Academy", conf)); - assertEquals(-1.0, cityMatch.distance("Washington (United States)", "United States Military Academy", conf)); - } - - @Test - public void keywordMatchTest() { - params.put("threshold", "0.5"); - - final KeywordMatch keywordMatch = new KeywordMatch(params); - - assertEquals( - 0.5, keywordMatch.distance("Biblioteca dell'Universita di Bologna", "Università di Bologna", conf)); - assertEquals(1.0, keywordMatch.distance("Universita degli studi di Pisa", "Universita di Pisa", conf)); - assertEquals(1.0, keywordMatch.distance("Polytechnic University of Turin", "POLITECNICO DI TORINO", conf)); - assertEquals(1.0, keywordMatch.distance("Istanbul Commerce University", "İstanbul Ticarət Universiteti", conf)); - assertEquals(1.0, keywordMatch.distance("Franklin College", "Concordia College", conf)); - assertEquals(2.0 / 3.0, keywordMatch.distance("University of Georgia", "Georgia State University", conf)); - assertEquals(0.5, keywordMatch.distance("University College London", "University of London", conf)); - assertEquals(0.5, keywordMatch.distance("Washington State University", "University of Washington", conf)); - assertEquals(-1.0, keywordMatch.distance("Allen (United States)", "United States Military Academy", conf)); + // both names with codes (1 same, 1 different) + assertEquals(0.5,codeMatch.distance("key::1 key::2 testing1", "key::1 testing", conf)); } @@ -155,15 +127,15 @@ public class ComparatorTest extends AbstractPaceTest { } @Test - public void jaroWinklerNormalizedNameTest() { + public void jaroWinklerLegalnameTest() { - final JaroWinklerNormalizedName jaroWinklerNormalizedName = new JaroWinklerNormalizedName(params); + final JaroWinklerLegalname jaroWinklerLegalname = new JaroWinklerLegalname(params); - double result = jaroWinklerNormalizedName - .distance("AT&T (United States)", "United States Military Academy", conf); + double result = jaroWinklerLegalname + .distance("AT&T (United States)", "United States key::2 key::1", conf); System.out.println("result = " + result); - result = jaroWinklerNormalizedName.distance("NOAA - Servicio Meteorol\\u00f3gico Nacional", "NOAA - NWS", conf); + result = jaroWinklerLegalname.distance("NOAA - Servicio Meteorol\\u00f3gico Nacional", "NOAA - NWS", conf); System.out.println("result = " + result); } @@ -344,13 +316,13 @@ public class ComparatorTest extends AbstractPaceTest { double result = countryMatch.distance("UNKNOWN", "UNKNOWN", conf); assertEquals(-1.0, result); - result = countryMatch.distance("CHILE", "UNKNOWN", conf); + result = countryMatch.distance("CL", "UNKNOWN", conf); assertEquals(-1.0, result); - result = countryMatch.distance("CHILE", "ITALY", conf); + result = countryMatch.distance("CL", "IT", conf); assertEquals(0.0, result); - result = countryMatch.distance("CHILE", "CHILE", conf); + result = countryMatch.distance("CL", "CL", conf); assertEquals(1.0, result); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java new file mode 100644 index 000000000..28f2bfc66 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java @@ -0,0 +1,83 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.io.IOException; + +import eu.dnetlib.dhp.oa.dedup.SparkOpenorgsDedupTest; +import eu.dnetlib.pace.tree.support.TreeProcessor; +import eu.dnetlib.pace.tree.support.TreeStats; +import org.apache.commons.io.IOUtils; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.platform.commons.util.StringUtils; + +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.SparkModel; + +class DecisionTreeTest { + + @Test + void testJPath() throws IOException { + + DedupConfig conf = DedupConfig + .load(IOUtils.toString(getClass().getResourceAsStream("dedup_conf_organization.json"))); + + final String org = IOUtils.toString(getClass().getResourceAsStream("organization.json")); + + Row row = SparkModel.apply(conf).rowFromJson(org); + + System.out.println("row = " + row); + Assertions.assertNotNull(row); + Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); + + System.out.println("row = " + row.getAs("countrytitle")); + } + + @Test + void jsonToModelTest() throws IOException{ + DedupConfig conf = DedupConfig + .load(IOUtils + .toString( + SparkOpenorgsDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + + final String org = IOUtils.toString(getClass().getResourceAsStream("organization_example1.json")); + + Row row = SparkModel.apply(conf).rowFromJson(org); + // to check that the same parsing returns the same row + Row row1 = SparkModel.apply(conf).rowFromJson(org); + + Assertions.assertEquals(row, row1); + System.out.println("row = " + row); + Assertions.assertNotNull(row); + Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); + } + + @Test + void organizationDecisionTreeTest() throws Exception { + DedupConfig conf = DedupConfig + .load(IOUtils + .toString( + SparkOpenorgsDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + + final String org1 = "{\"eclegalbody\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecresearchorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"SUSF - Universit\\\\u00e9 internationale de floride\"}, \"pid\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"qualifier\": {\"classid\": \"grid\", \"classname\": \"grid\", \"schemename\": \"dnet:pid_types\", \"schemeid\": \"dnet:pid_types\"}, \"value\": \"grid.65456.34\"}], \"websiteurl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"http://www.fiu.edu/\"}, \"ecnutscode\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"logourl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"collectedfrom\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"GRID - Global Research Identifier Database\", \"key\": \"10|openaire____::ff4a008470319a22d9cf3d14af485977\"}], \"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"UNKNOWN\", \"classname\": \"UNKNOWN\", \"schemename\": \"dnet:provenanceActions\", \"schemeid\": \"dnet:provenanceActions\"}, \"inferred\": true, \"inferenceprovenance\": \"dedup-similarity-organization-simple\", \"invisible\": false, \"trust\": \"0.89\"}, \"alternativeNames\": [], \"echighereducation\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"id\": \"20|grid________::f22e08fb7bd544b4355f99bef2c43ad5\", \"eclegalperson\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"lastupdatetimestamp\": 1566902405602, \"ecinternationalorganizationeurinterests\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"dateofcollection\": \"\", \"dateoftransformation\": \"\", \"ecnonprofit\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecenterprise\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecinternationalorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalshortname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"FIU\"}, \"country\": {\"classid\": \"US\", \"classname\": \"United States\", \"schemename\": \"dnet:countries\", \"schemeid\": \"dnet:countries\"}, \"extraInfo\": [], \"originalId\": [], \"ecsmevalidated\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}}"; + final String org2 = "{\"eclegalbody\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecresearchorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"SUSF - Universidad Internacional de Florida\"}, \"pid\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"qualifier\": {\"classid\": \"grid\", \"classname\": \"grid\", \"schemename\": \"dnet:pid_types\", \"schemeid\": \"dnet:pid_types\"}, \"value\": \"grid.65456.34\"}], \"websiteurl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"http://www.fiu.edu/\"}, \"ecnutscode\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"logourl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"collectedfrom\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"GRID - Global Research Identifier Database\", \"key\": \"10|openaire____::ff4a008470319a22d9cf3d14af485977\"}], \"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"UNKNOWN\", \"classname\": \"UNKNOWN\", \"schemename\": \"dnet:provenanceActions\", \"schemeid\": \"dnet:provenanceActions\"}, \"inferred\": true, \"inferenceprovenance\": \"dedup-similarity-organization-simple\", \"invisible\": false, \"trust\": \"0.89\"}, \"alternativeNames\": [], \"echighereducation\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"id\": \"20|grid________::2b261e9d8c2a63abbfd5826918c23b6d\", \"eclegalperson\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"lastupdatetimestamp\": 1566902405602, \"ecinternationalorganizationeurinterests\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"dateofcollection\": \"\", \"dateoftransformation\": \"\", \"ecnonprofit\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecenterprise\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecinternationalorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalshortname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"FIU\"}, \"country\": {\"classid\": \"US\", \"classname\": \"United States\", \"schemename\": \"dnet:countries\", \"schemeid\": \"dnet:countries\"}, \"extraInfo\": [], \"originalId\": [], \"ecsmevalidated\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}}"; + + Row row1 = SparkModel.apply(conf).rowFromJson(org1); + Row row2 = SparkModel.apply(conf).rowFromJson(org2); + + System.out.println("row1 = " + row1); + System.out.println("row2 = " + row2); + TreeProcessor tree = new TreeProcessor(conf); + + boolean result = tree.compare(row1, row2); + + System.out.println("result = " + result); + + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java index c80c98bb7..3bfd861f8 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java @@ -190,7 +190,7 @@ public class SparkDedupTest implements Serializable { System.out.println("orp_simrel = " + orp_simrel); if (CHECK_CARDINALITIES) { - assertEquals(751, orgs_simrel); + assertEquals(742, orgs_simrel); assertEquals(566, pubs_simrel); assertEquals(113, sw_simrel); assertEquals(148, ds_simrel); @@ -251,7 +251,7 @@ public class SparkDedupTest implements Serializable { // entities simrels supposed to be equal to the number of previous step (no rels in whitelist) if (CHECK_CARDINALITIES) { - assertEquals(751, orgs_simrel); + assertEquals(742, orgs_simrel); assertEquals(566, pubs_simrel); assertEquals(148, ds_simrel); assertEquals(280, orp_simrel); @@ -442,7 +442,7 @@ public class SparkDedupTest implements Serializable { final List merges = pubs .filter("source == '50|arXiv_dedup_::c93aeb433eb90ed7a86e29be00791b7c'") .collectAsList(); - assertEquals(3, merges.size()); + assertEquals(1, merges.size()); Set dups = Sets .newHashSet( "50|doi_________::3b1d0d8e8f930826665df9d6b82fbb73", @@ -451,7 +451,7 @@ public class SparkDedupTest implements Serializable { merges.forEach(r -> { assertEquals(ModelConstants.RESULT_RESULT, r.getRelType()); assertEquals(ModelConstants.DEDUP, r.getSubRelType()); - assertEquals(ModelConstants.MERGES, r.getRelClass()); + assertEquals(ModelConstants.IS_MERGED_IN, r.getRelClass()); assertTrue(dups.contains(r.getTarget())); }); @@ -561,7 +561,7 @@ public class SparkDedupTest implements Serializable { System.out.println("orp_mergerel = " + orp_mergerel); if (CHECK_CARDINALITIES) { - assertEquals(1268, orgs_mergerel); + assertEquals(1278, orgs_mergerel); assertEquals(1156, pubs.count()); assertEquals(292, sw_mergerel); assertEquals(476, ds_mergerel); @@ -618,7 +618,7 @@ public class SparkDedupTest implements Serializable { System.out.println("orp_deduprecord = " + orp_deduprecord); if (CHECK_CARDINALITIES) { - assertEquals(86, orgs_deduprecord); + assertEquals(78, orgs_deduprecord); assertEquals(96, pubs.count()); assertEquals(47, sw_deduprecord); assertEquals(97, ds_deduprecord); @@ -761,7 +761,7 @@ public class SparkDedupTest implements Serializable { if (CHECK_CARDINALITIES) { assertEquals(930, publications); - assertEquals(839, organizations); + assertEquals(831, organizations); assertEquals(100, projects); assertEquals(100, datasource); assertEquals(196, softwares); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java index 6f2a6904b..b2b5d824b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java @@ -22,8 +22,11 @@ import java.util.Properties; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.*; import org.junit.jupiter.api.extension.ExtendWith; @@ -143,7 +146,7 @@ public class SparkOpenorgsDedupTest implements Serializable { .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) .count(); - assertEquals(86, orgs_simrel); + assertEquals(92, orgs_simrel); } @Test @@ -172,7 +175,7 @@ public class SparkOpenorgsDedupTest implements Serializable { .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) .count(); - assertEquals(122, orgs_simrel); + assertEquals(128, orgs_simrel); } @Test @@ -207,7 +210,7 @@ public class SparkOpenorgsDedupTest implements Serializable { .read() .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") .count(); - assertEquals(132, orgs_mergerel); + assertEquals(128, orgs_mergerel); // verify that a DiffRel is in the mergerels (to be sure that the job supposed to remove them has something to // do) diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java index 705c2cc84..40a73c9b3 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup.jpath; import java.io.IOException; +import eu.dnetlib.dhp.oa.dedup.SparkOpenorgsDedupTest; import org.apache.commons.io.IOUtils; import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; @@ -24,6 +25,30 @@ class JsonPathTest { Row row = SparkModel.apply(conf).rowFromJson(org); + System.out.println("row = " + row); + Assertions.assertNotNull(row); + Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); + + System.out.println("row = " + row.getAs("countrytitle")); + } + + @Test + void jsonToModelTest() throws IOException{ + DedupConfig conf = DedupConfig + .load(IOUtils + .toString( + SparkOpenorgsDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + + final String org = IOUtils.toString(getClass().getResourceAsStream("organization_example1.json")); + + Row row = SparkModel.apply(conf).rowFromJson(org); + // to check that the same parsing returns the same row + Row row1 = SparkModel.apply(conf).rowFromJson(org); + + Assertions.assertEquals(row, row1); + System.out.println("row = " + row); Assertions.assertNotNull(row); Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json index 917c9426b..f00f6198e 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json @@ -4,8 +4,8 @@ "dedupRun" : "001", "entityType" : "organization", "subEntityValue": "organization", - "orderField" : "legalname", - "queueMaxSize" : "2000", + "orderField" : "original_legalname", + "queueMaxSize" : "100000", "groupMaxSize" : "50", "slidingWindowSize" : "200", "idPath":"$.id", @@ -15,10 +15,10 @@ }, "pace" : { "clustering" : [ - { "name" : "sortedngrampairs", "fields" : [ "legalname" ], "params" : { "max" : 2, "ngramLen" : "3"} }, - { "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : 1, "len" : "3" } }, + { "name" : "sortedngrampairs", "fields" : [ "original_legalname" ], "params" : { "max" : 2, "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "original_legalname" ], "params" : { "max" : 1, "len" : "3" } }, { "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }, - { "name" : "keywordsclustering", "fields" : [ "legalname" ], "params" : { "max": 2, "windowSize": 4} } + { "name" : "legalnameclustering", "fields" : [ "legalname" ], "params" : { "max": 2} } ], "decisionTree" : { "start": { @@ -29,16 +29,23 @@ "weight": 1, "countIfUndefined": "false", "params": {} + }, + { + "field": "rorid", + "comparator": "exactMatch", + "weight": 1, + "countIfUndefined": "false", + "params": {} } ], "threshold": 1, - "aggregation": "AVG", + "aggregation": "OR", "positive": "MATCH", "negative": "NO_MATCH", - "undefined": "layer2", + "undefined": "necessaryConditions", "ignoreUndefined": "false" }, - "layer2": { + "necessaryConditions": { "fields": [ { "field": "websiteurl", @@ -55,14 +62,14 @@ "params": {} }, { - "field": "legalname", + "field": "original_legalname", "comparator": "numbersMatch", "weight": 1, "countIfUndefined": "true", "params": {} }, { - "field": "legalname", + "field": "original_legalname", "comparator": "romansMatch", "weight": 1, "countIfUndefined": "true", @@ -71,68 +78,64 @@ ], "threshold": 1, "aggregation": "AND", - "positive": "layer3", + "positive": "cityCheck", "negative": "NO_MATCH", - "undefined": "layer3", + "undefined": "cityCheck", "ignoreUndefined": "true" }, - "layer3": { + "cityCheck": { "fields": [ { "field": "legalname", - "comparator": "cityMatch", + "comparator": "codeMatch", "weight": 1.0, "countIfUndefined": "true", "params": { - "windowSize": "4" + "codeRegex": "city::\\d+" } } ], "threshold": 0.1, "aggregation": "AVG", - "positive": "layer4", + "positive": "keywordCheck", "negative": "NO_MATCH", "undefined": "NO_MATCH", "ignoreUndefined": "true" }, - "layer4": { + "keywordCheck": { "fields": [ { "field": "legalname", - "comparator": "keywordMatch", + "comparator": "codeMatch", "weight": 1.0, "countIfUndefined": "true", "params": { - "windowSize": "4" + "codeRegex": "key::\\d+" } } ], "threshold": 0.7, "aggregation": "AVG", - "positive": "layer5", + "positive": "nameCheck", "negative": "NO_MATCH", - "undefined": "layer5", + "undefined": "nameCheck", "ignoreUndefined": "true" }, - "layer5": { + "nameCheck": { "fields": [ { "field": "legalname", - "comparator": "jaroWinklerNormalizedName", + "comparator": "jaroWinklerLegalname", "weight": 0.9, "countIfUndefined": "true", - "params": { - "windowSize": "4" - } + "params": {} }, { "field": "legalshortname", - "comparator": "jaroWinklerNormalizedName", + "comparator": "jaroWinklerLegalname", "weight": 0.1, "countIfUndefined": "false", - "params": { - "windowSize": 4 - } + "params": {} } ], "threshold": 0.9, @@ -144,126 +147,16 @@ } }, "model" : [ - { "name" : "country", "type" : "String", "path" : "$.country.classid"}, - { "name" : "legalshortname", "type" : "String", "path" : "$.legalshortname.value"}, - { "name" : "legalname", "type" : "String", "path" : "$.legalname.value" }, + { "name" : "country", "type" : "String", "path" : "$.country.classid", "infer" : "country", "inferenceFrom" : "$.legalname.value"}, + { "name" : "legalshortname", "type" : "String", "path" : "$.legalshortname.value", "infer" : "city_keyword"}, + { "name" : "original_legalname", "type" : "String", "path" : "$.legalname.value" }, + { "name" : "legalname", "type" : "String", "path" : "$.legalname.value", "infer" : "city_keyword"}, { "name" : "websiteurl", "type" : "URL", "path" : "$.websiteurl.value" }, { "name" : "gridid", "type" : "String", "path" : "$.pid[?(@.qualifier.classid =='grid')].value"}, + { "name" : "rorid", "type" : "String", "path" : "$.pid[?(@.qualifier.classid =='ROR')].value"}, { "name" : "originalId", "type" : "String", "path" : "$.id" } ], - "blacklists" : { - "legalname" : [] - }, - "synonyms": { - "key::1": ["university","università", "universitas", "università studi","universitario","universitaria","université", "universite", "universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti", "universiti"], - "key::2": ["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"], - "key::3": ["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"], - "key::4": ["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"], - "key::5": ["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"], - "key::6": ["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"], - "key::7": ["college","collegio","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","κολλέγιο"], - "key::8": ["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"], - "key::9": ["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"], - "key::10": ["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"], - "key::11": ["association","associazione","association","asociación","associação","Verein","verband","stowarzyszenie","ассоциация","associatie"], - "key::12": ["society","societa","société","sociedad","sociedade","gesellschaft","społeczeństwo","общество","maatschappij","κοινωνία"], - "key::13": ["international","internazionale","international","internacional","internacional","international","międzynarodowy","Международный","internationaal","internationale","διεθνής","διεθνή","διεθνές"], - "key::14": ["community","comunita","communauté","comunidad","comunidade","Gemeinschaft","społeczność","сообщество","gemeenschap","κοινότητα"], - "key::15": ["school","scuola","école","escuela","escola","schule","Szkoła","школа","school","σχολείο"], - "key::16": ["education","educazione","éducation","educacion","Educação","Bildung","Edukacja","образование","opleiding","εκπαίδευση"], - "key::17": ["academy","accademia","académie","academia","academia","Akademie","akademie","академия","academie","ακαδημία"], - "key::18": ["public","pubblico","public","publique","publics","publiques","publico","publico","Öffentlichkeit","publiczny","публичный","publiek","publieke","δημόσιος","δημόσια","δημόσιο"], - "key::19": ["museum","museo","musée","mueso","museu","museum","muzeum","музей","museum","μουσείο"], - "key::20": ["group","gruppo","groupe","grupo","grupo","gruppe","grupa","группа","groep","ομάδα","όμιλος"], - "key::21": ["department","dipartimento","département","departamento","departamento","abteilung","departament","отдел","afdeling","τμήμα"], - "key::22": ["council","consiglio","conseil","Consejo","conselho","gesellschaft","rada","совет","raad","συμβούλιο"], - "key::23": ["library","biblioteca","bibliothèque","biblioteca","biblioteca","Bibliothek","biblioteka","библиотека","bibliotheek","βιβλιοθήκη"], - "key::24": ["ministry","ministero","ministère","ministerio","ministério","Ministerium","ministerstwo","министерство","ministerie","υπουργείο"], - "key::25": ["services","servizi","services","servicios","Serviços","Dienstleistungen","usługi","услуги","diensten","υπηρεσίες"], - "key::26": ["central","centrale","central","centrale","centrales","central","central","zentral","centralny","цетральный","centraal","κεντρικός","κεντρική","κεντρικό","κεντρικά"], - "key::27": ["general","generale","général","générale","généraux","générales","general","geral","general","Allgemeines","general","общий","algemeen","algemene","γενικός","γενική","γενικό","γενικά"], - "key::28": ["applied","applicati","appliqué","appliquée","appliqués","appliquées","aplicado","aplicada","angewendet","stosowany","прикладной","toegepast","toegepaste","εφαρμοσμένος","εφαρμοσμένη","εφαρμοσμένο","εφαρμοσμένα"], - "key::29": ["european","europee","europea","européen","européenne","européens","européennes","europeo","europeu","europäisch","europejski","европейский","Europees","Europese","ευρωπαϊκός","ευρωπαϊκή","ευρωπαϊκό","ευρωπαϊκά"], - "key::30": ["agency","agenzia","agence","agencia","agencia","agentur","agencja","агенция","agentschap","πρακτορείο"], - "key::31": ["laboratory","laboratorio","laboratoire","laboratorio","laboratorio","labor","laboratorium","лаборатория","laboratorium","εργαστήριο"], - "key::32": ["industry","industria","industrie","индустрия","industrie","βιομηχανία"], - "key::33": ["industrial","industriale","industriel","industrielle","industriels","industrielles","индустриальный","industrieel","βιομηχανικός","βιομηχανική","βιομηχανικό","βιομηχανικά","βιομηχανικές"], - "key::34": ["consortium","consorzio","consortium","консорциум","consortium","κοινοπραξία"], - "key::35": ["organization","organizzazione","organisation","organización","organização","organizacja","организация","organisatie","οργανισμός"], - "key::36": ["authority","autorità","autorité","авторитет","autoriteit"], - "key::37": ["federation","federazione","fédération","федерация","federatie","ομοσπονδία"], - "key::38": ["observatory","osservatorio","observatoire","обсерватория","observatorium","αστεροσκοπείο"], - "key::39": ["bureau","ufficio","bureau","офис","bureau","γραφείο"], - "key::40": ["company","impresa","compagnie","société","компания","bedrijf","εταιρία"], - "key::41": ["polytechnic","politecnico","polytechnique","политехника","polytechnisch","πολυτεχνείο","universita politecnica","polytechnic university","universidad politecnica","universitat politecnica","politechnika","politechniki","university technology","university science technology"], - "key::42": ["coalition","coalizione","coalition","коалиция","coalitie","συνασπισμός"], - "key::43": ["initiative","iniziativa","initiative","инициатива","initiatief","πρωτοβουλία"], - "key::44": ["academic","accademico","académique","universitaire","акадеческий academisch","ακαδημαϊκός","ακαδημαϊκή","ακαδημαϊκό","ακαδημαϊκές","ακαδημαϊκοί"], - "key::45": ["institution","istituzione","institution","институциональный","instelling","ινστιτούτο"], - "key::46": ["division","divisione","division","отделение","divisie","τμήμα"], - "key::47": ["committee","comitato","comité","комитет","commissie","επιτροπή"], - "key::48": ["promotion","promozione","продвижение","proothisis","forderung"], - "key::49": ["medical","medicine","clinical","medicina","clinici","médico","medicina","clínica","médico","medicina","clínica","medizinisch","Medizin","klinisch","medisch","geneeskunde","klinisch","ιατρικός","ιατρική","ιατρικό","ιατρικά","κλινικός","κλινική","κλινικό","κλινικά","tıbbi","tıp","klinik","orvosi","orvostudomány","klinikai","zdravniški","medicinski","klinični","meditsiini","kliinik","kliiniline"], - "key::50": ["technology","technological","tecnologia","tecnologie","tecnología","tecnológico","tecnologia","tecnológico","Technologie","technologisch","technologie","technologisch","τεχνολογία","τεχνολογικός","τεχνολογική","τεχνολογικό","teknoloji","teknolojik","technológia","technológiai","tehnologija","tehnološki","tehnoloogia","tehnoloogiline","technologii","technical","texniki","teknik"], - "key::51": ["science","scientific","scienza","scientifiche","scienze","ciencia","científico","ciência","científico","Wissenschaft","wissenschaftlich","wetenschap","wetenschappelijk","επιστήμη","επιστημονικός","επιστημονική","επιστημονικό","επιστημονικά","bilim","bilimsel","tudomány","tudományos","znanost","znanstveni","teadus","teaduslik",""], - "key::52": ["engineering","ingegneria","ingeniería","engenharia","Ingenieurwissenschaft","ingenieurswetenschappen","bouwkunde","μηχανικός","μηχανική","μηχανικό","mühendislik","mérnöki","Inženirstvo","inseneeria","inseneri",""], - "key::53": ["management","gestione","gestionale","gestionali","gestión","administración","gestão","administração","Verwaltung","management","διαχείριση","yönetim","menedzsment","vodstvo","upravljanje","management","juhtkond","juhtimine","haldus",""], - "key::54": ["energy","energia","energía","energia","Energie","energie","ενέργεια","enerji","energia","energija","energia",""], - "key::55": ["agricultural","agriculture","agricoltura","agricole","agrícola","agricultura","agrícola","agricultura","landwirtschaftlich","Landwirtschaft","landbouwkundig","landbouw","αγροτικός","αγροτική","αγροτικό","γεωργικός","γεωργική","γεωργικό","γεωργία","tarımsal","tarım","mezőgazdasági","mezőgazdaság","poljedelski","poljedelstvo","põllumajandus","põllumajanduslik",""], - "key::56": ["information","informazione","información","informação","Information","informatie","πληροφορία","bilgi","információ","informacija","informatsioon","informatycznych",""], - "key::57": ["social","sociali","social","social","Sozial","sociaal","maatschappelijk","κοινωνικός","κοινωνική","κοινωνικό","κοινωνικά","sosyal","szociális","družbeni","sotsiaal","sotsiaalne",""], - "key::58": ["environmental","ambiente","medioambiental","ambiente","medioambiente","meioambiente","Umwelt","milieu","milieuwetenschap","milieukunde","περιβαλλοντικός","περιβαλλοντική","περιβαλλοντικό","περιβαλλοντικά","çevre","környezeti","okoliški","keskonna",""], - "key::59": ["business","economia","economiche","economica","negocio","empresa","negócio","Unternehmen","bedrijf","bedrijfskunde","επιχείρηση","iş","üzleti","posel","ettevõte/äri",""], - "key::60": ["pharmaceuticals","pharmacy","farmacia","farmaceutica","farmacéutica","farmacia","farmacêutica","farmácia","Pharmazeutika","Arzneimittelkunde","farmaceutica","geneesmiddelen","apotheek","φαρμακευτικός","φαρμακευτική","φαρμακευτικό","φαρμακευτικά","φαρμακείο","ilaç","eczane","gyógyszerészeti","gyógyszertár","farmacevtika","lekarništvo","farmaatsia","farmatseutiline",""], - "key::61": ["healthcare","health services","salute","atenciónmédica","cuidadodelasalud","cuidadoscomasaúde","Gesundheitswesen","gezondheidszorg","ιατροφαρμακευτικήπερίθαλψη","sağlıkhizmeti","egészségügy","zdravstvo","tervishoid","tervishoiu",""], - "key::62": ["history","storia","historia","história","Geschichte","geschiedenis","geschiedkunde","ιστορία","tarih","történelem","zgodovina","ajalugu",""], - "key::63": ["materials","materiali","materia","materiales","materiais","materialen","υλικά","τεκμήρια","malzemeler","anyagok","materiali","materjalid","vahendid",""], - "key::64": ["economics","economia","economiche","economica","economía","economia","Wirtschaft","economie","οικονομικά","οικονομικέςεπιστήμες","ekonomi","közgazdaságtan","gospodarstvo","ekonomija","majanduslik","majandus",""], - "key::65": ["therapeutics","terapeutica","terapéutica","terapêutica","therapie","θεραπευτική","tedavibilimi","gyógykezelés","terapevtika","terapeutiline","ravi",""], - "key::66": ["oncology","oncologia","oncologico","oncología","oncologia","Onkologie","oncologie","ογκολογία","onkoloji","onkológia","onkologija","onkoloogia",""], - "key::67": ["natural","naturali","naturale","natural","natural","natürlich","natuurlijk","φυσικός","φυσική","φυσικό","φυσικά","doğal","természetes","naraven","loodus",""], - "key::68": ["educational","educazione","pedagogia","educacional","educativo","educacional","pädagogisch","educatief","εκπαιδευτικός","εκπαιδευτική","εκπαιδευτικό","εκπαιδευτικά","eğitimsel","oktatási","izobraževalen","haridus","hariduslik",""], - "key::69": ["biomedical","biomedica","biomédico","biomédico","biomedizinisch","biomedisch","βιοιατρικός","βιοιατρική","βιοιατρικό","βιοιατρικά","biyomedikal","orvosbiológiai","biomedicinski","biomeditsiiniline",""], - "key::70": ["veterinary","veterinaria","veterinarie","veterinaria","veterinária","tierärtzlich","veterinair","veeartsenijlkunde","κτηνιατρικός","κτηνιατρική","κτηνιατρικό","κτηνιατρικά","veteriner","állatorvosi","veterinar","veterinarski","veterinaaria",""], - "key::71": ["chemistry","chimica","química","química","Chemie","chemie","scheikunde","χημεία","kimya","kémia","kemija","keemia",""], - "key::72": ["security","sicurezza","seguridad","segurança","Sicherheit","veiligheid","ασφάλεια","güvenlik","biztonsági","varnost","turvalisus","julgeolek",""], - "key::73": ["biotechnology","biotecnologia","biotecnologie","biotecnología","biotecnologia","Biotechnologie","biotechnologie","βιοτεχνολογία","biyoteknoloji","biotechnológia","biotehnologija","biotehnoloogia",""], - "key::74": ["military","militare","militari","militar","militar","Militär","militair","leger","στρατιωτικός","στρατιωτική","στρατιωτικό","στρατιωτικά","askeri","katonai","vojaški","vojni","militaar","wojskowa",""], - "key::75": ["theological","teologia","teologico","teológico","tecnológica","theologisch","theologisch","θεολογικός","θεολογική","θεολογικό","θεολογικά","teolojik","technológiai","teološki","teoloogia","usuteadus","teoloogiline",""], - "key::76": ["electronics","elettronica","electrónica","eletrônicos","Elektronik","elektronica","ηλεκτρονική","elektronik","elektronika","elektronika","elektroonika",""], - "key::77": ["forestry","forestale","forestali","silvicultura","forestal","floresta","Forstwirtschaft","bosbouw","δασοκομία","δασολογία","ormancılık","erdészet","gozdarstvo","metsandus",""], - "key::78": ["maritime","marittima","marittime","marittimo","marítimo","marítimo","maritiem","ναυτικός","ναυτική","ναυτικό","ναυτικά","ναυτιλιακός","ναυτιλιακή","ναυτιλιακό","ναυτιλιακά","θαλάσσιος","θαλάσσια","θαλάσσιο","denizcilik","tengeri","morski","mere","merendus",""], - "key::79": ["sports","sport","deportes","esportes","Sport","sport","sportwetenschappen","άθληση","γυμναστικήδραστηριότητα","spor","sport","šport","sport","spordi",""], - "key::80": ["surgery","chirurgia","chirurgiche","cirugía","cirurgia","Chirurgie","chirurgie","heelkunde","εγχείρηση","επέμβαση","χειρουργικήεπέμβαση","cerrahi","sebészet","kirurgija","kirurgia",""], - "key::81": ["cultural","culturale","culturali","cultura","cultural","cultural","kulturell","cultureel","πολιτιστικός","πολιτιστική","πολιτιστικό","πολιτισμικός","πολιτισμική","πολιτισμικό","kültürel","kultúrális","kulturni","kultuuri","kultuuriline",""], - "key::82": ["computerscience","informatica","ordenador","computadora","informática","computación","cienciasdelacomputación","ciênciadacomputação","Computer","computer","υπολογιστής","ηλεκτρονικόςυπολογιστής","bilgisayar","számítógép","računalnik","arvuti",""], - "key::83": ["finance","financial","finanza","finanziarie","finanza","financiero","finanças","financeiro","Finanzen","finanziell","financiën","financieel","χρηματοοικονομικά","χρηματοδότηση","finanse","finansal","pénzügy","pénzügyi","finance","finančni","finants","finantsiline",""], - "key::84": ["communication","comunicazione","comuniciación","comunicação","Kommunikation","communication","επικοινωνία","iletişim","kommunikáció","komuniciranje","kommunikatsioon",""], - "key::85": ["justice","giustizia","justicia","justiça","Recht","Justiz","justitie","gerechtigheid","δικαιοσύνη","υπουργείοδικαιοσύνης","δίκαιο","adalet","igazságügy","pravo","õigus",""], - "key::86": ["aerospace","aerospaziale","aerospaziali","aeroespacio","aeroespaço","Luftfahrt","luchtvaart","ruimtevaart","αεροπορικός","αεροπορική","αεροπορικό","αεροναυπηγικός","αεροναυπηγική","αεροναυπηγικό","αεροναυπηγικά","havacılıkveuzay","légtér","zrakoplovstvo","atmosfäär","kosmos",""], - "key::87": ["dermatology","dermatologia","dermatología","dermatologia","Dermatologie","dermatologie","δρματολογία","dermatoloji","bőrgyógyászat","dermatológia","dermatologija","dermatoloogia",""], - "key::88": ["architecture","architettura","arquitectura","arquitetura","Architektur","architectuur","αρχιτεκτονική","mimarlık","építészet","arhitektura","arhitektuur",""], - "key::89": ["mathematics","matematica","matematiche","matemáticas","matemáticas","Mathematik","wiskunde","mathematica","μαθηματικά","matematik","matematika","matematika","matemaatika",""], - "key::90": ["language","lingue","linguistica","linguistiche","lenguaje","idioma","língua","idioma","Sprache","taal","taalkunde","γλώσσα","dil","nyelv","jezik","keel",""], - "key::91": ["neuroscience","neuroscienza","neurociencia","neurociência","Neurowissenschaft","neurowetenschappen","νευροεπιστήμη","nörobilim","idegtudomány","nevroznanost","neuroteadused",""], - "key::92": ["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""], - "key::93": ["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""], - "key::94": ["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""], - "key::95": ["mechanics", "mechanical", "meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""], - "key::96": ["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""], - "key::97": ["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""], - "key::98": ["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""], - "key::99": ["neurology","neurologia","neurologiche","neurología","neurologia","Neurologie","neurologie","zenuwleer","νευρολογία","nöroloji","neurológia","ideggyógyászat","nevrologija","neuroloogia",""], - "key::100": ["geology","geologia","geologiche","geología","geologia","Geologie","geologie","aardkunde","γεωλογία","jeoloji","geológia","földtudomány","geologija","geoloogia",""], - "key::101": ["microbiology","microbiologia","micro-biologia","microbiologiche","microbiología","microbiologia","Mikrobiologie","microbiologie","μικροβιολογία","mikrobiyoloji","mikrobiológia","mikrobiologija","mikrobioloogia",""], - "key::102": ["informatics","informatica","informática","informática","informatica",""], - "key::103": ["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"], - "key::104": ["commerce","ticaret","ticarət","commercio","trade","handel","comercio"], - "key::105" : ["state", "stato", "etade", "estado", "statale", "etat", "zustand", "estado"], - "key::106" : ["seminary", "seminario", "seminaire", "seminar"], - "key::107" : ["agricultural forestry", "af", "a f"], - "key::108" : ["agricultural mechanical", "am", "a m"], - "key::109" : ["catholic", "catholique", "katholische", "catolica", "cattolica", "catolico"] - } + "blacklists" : {}, + "synonyms": {} } } \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/oa/dedup/jpath/organization_example1.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/oa/dedup/jpath/organization_example1.json new file mode 100644 index 000000000..52fb304a9 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/oa/dedup/jpath/organization_example1.json @@ -0,0 +1 @@ +{"collectedfrom":[{"key":"10|openaire____::0362fcdb3076765d9c0041ad331553e8","value":"OpenOrgs Database","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1617006357185,"id":"20|openorgsmesh::8536807f92ca146e6f0990b07d712406","originalId":["openorgsmesh::0000098375-0dcc2286c99400f0ac63d0db84a57759"],"pid":[{"value":"0000 0004 1757 1758","qualifier":{"classid":"ISNI","classname":"International Standard Name Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"263329","qualifier":{"classid":"OrgRef","classname":"OrgRef","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"501100005969","qualifier":{"classid":"FundRef","classname":"FundRef","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"grid.6292.f","qualifier":{"classid":"GRID","classname":"GRID","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"https://ror.org/01111rn36","qualifier":{"classid":"ROR","classname":"ROR","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Q131262","qualifier":{"classid":"Wikidata","classname":"Wikidata","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2021-03-17","dateoftransformation":"2021-03-17","extraInfo":[],"oaiprovenance":null,"legalshortname":{"value":"Università di Bologna","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"legalname":{"value":"Università di Bologna","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"alternativeNames":[],"websiteurl":{"value":"http://www.unibo.it/en/homepage","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"logourl":null,"eclegalbody":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"eclegalperson":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecnonprofit":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecresearchorganization":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"echighereducation":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecinternationalorganizationeurinterests":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecinternationalorganization":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecenterprise":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecsmevalidated":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"ecnutscode":{"value":"false","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.880","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"country":{"classid":"IT","classname":"Italy","schemeid":"dnet:countries","schemename":"dnet:countries"}} \ No newline at end of file From 5cdba9172bb68c4f842efaddebc644e9cdef1b21 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 10 Jul 2024 14:53:13 +0200 Subject: [PATCH 062/239] implementeation of the new collector plugin: research_fi --- .../researchfi/ResearchFiCollectorPlugin.java | 72 +++++++++++ .../plugin/researchfi/ResearchFiIterator.java | 115 ++++++++++++++++++ .../ResearchFiCollectorPluginTest.java | 55 +++++++++ 3 files changed, 242 insertions(+) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java new file mode 100644 index 000000000..c5961c598 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java @@ -0,0 +1,72 @@ +package eu.dnetlib.dhp.collection.plugin.researchfi; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.http.NameValuePair; +import org.apache.http.client.entity.UrlEncodedFormEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.message.BasicNameValuePair; +import org.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; +import eu.dnetlib.dhp.common.collection.CollectorException; + +public class ResearchFiCollectorPlugin implements CollectorPlugin { + + private static final Logger log = LoggerFactory.getLogger(ResearchFiCollectorPlugin.class); + + @Override + public Stream collect(final ApiDescriptor api, final AggregatorReport report) + throws CollectorException { + + final String authUrl = api.getParams().get("auth_url"); + final String clientId = api.getParams().get("auth_client_id"); + final String clientSecret = api.getParams().get("auth_client_secret"); + + final String authToken = authenticate(authUrl, clientId, clientSecret); + + final Iterator iter = new ResearchFiIterator(api.getBaseUrl(), authToken); + + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iter, Spliterator.ORDERED), false); + } + + private String authenticate(final String authUrl, final String clientId, final String clientSecret) throws CollectorException { + try (final CloseableHttpClient client = HttpClients.createDefault()) { + final HttpPost req = new HttpPost(authUrl); + final List params = new ArrayList<>(); + params.add(new BasicNameValuePair("grant_type", "client_credentials")); + params.add(new BasicNameValuePair("client_id", clientId)); + params.add(new BasicNameValuePair("client_secret", clientSecret)); + + req.setEntity(new UrlEncodedFormEntity(params, "UTF-8")); + + try (final CloseableHttpResponse response = client.execute(req)) { + final String content = IOUtils.toString(response.getEntity().getContent()); + final JSONObject obj = new JSONObject(content); + final String token = obj.getString("access_token"); + if (StringUtils.isNotBlank(token)) { return token; } + } + } catch (final Throwable e) { + log.warn("Error obtaining access token", e); + throw new CollectorException("Error obtaining access token", e); + } + throw new CollectorException("Access token is missing"); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java new file mode 100644 index 000000000..38a3cece8 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java @@ -0,0 +1,115 @@ +package eu.dnetlib.dhp.collection.plugin.researchfi; + +import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.PriorityBlockingQueue; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.http.Header; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.json.JSONArray; + +import eu.dnetlib.dhp.collection.plugin.utils.JsonUtils; +import eu.dnetlib.dhp.common.collection.CollectorException; + +public class ResearchFiIterator implements Iterator { + + private static final Log log = LogFactory.getLog(ResearchFiIterator.class); + + private static final int PAGE_SIZE = 100; + + private final String baseUrl; + private final String authToken; + private int currPage; + private int nPages; + + private final Queue queue = new PriorityBlockingQueue<>(); + + public ResearchFiIterator(final String baseUrl, final String authToken) { + this.baseUrl = baseUrl; + this.authToken = authToken; + this.currPage = 0; + this.nPages = 0; + } + + private void verifyStarted() { + if (this.currPage == 0) { + try { + nextCall(); + } catch (final CollectorException e) { + throw new IllegalStateException(e); + } + } + } + + @Override + public boolean hasNext() { + synchronized (this.queue) { + verifyStarted(); + return !this.queue.isEmpty(); + } + } + + @Override + public String next() { + synchronized (this.queue) { + verifyStarted(); + final String res = this.queue.poll(); + while (this.queue.isEmpty() && (this.currPage < this.nPages)) { + try { + nextCall(); + } catch (final CollectorException e) { + throw new IllegalStateException(e); + } + } + return res; + } + } + + private void nextCall() throws CollectorException { + + this.currPage += 1; + + try (final CloseableHttpClient client = HttpClients.createDefault()) { + final String url; + if (!this.baseUrl.contains("?")) { + url = String.format("%s?PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); + } else if (!this.baseUrl.contains("PageSize=")) { + url = String.format("%s&PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); + } else { + url = String.format("%s&PageNumber=%d", this.baseUrl, this.currPage); + } + log.info("Calling url: " + url); + + final HttpGet req = new HttpGet(url); + req.addHeader("Authorization", "Bearer " + this.authToken); + try (final CloseableHttpResponse response = client.execute(req)) { + for (final Header header : response.getAllHeaders()) { + log.debug("HEADER: " + header.getName() + " = " + header.getValue()); + if ("x-page-count".equals(header.getName())) { + final int totalPages = NumberUtils.toInt(header.getValue()); + if (this.nPages != totalPages) { + this.nPages = NumberUtils.toInt(header.getValue()); + log.info("Total pages: " + totalPages); + } + } + } + + final String content = IOUtils.toString(response.getEntity().getContent()); + final JSONArray jsonArray = new JSONArray(content); + + jsonArray.forEach(obj -> this.queue.add(JsonUtils.convertToXML(obj.toString()))); + } + } catch (final Throwable e) { + log.warn("Error obtaining access token", e); + throw new CollectorException("Error obtaining access token", e); + } + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java new file mode 100644 index 000000000..47c77796b --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java @@ -0,0 +1,55 @@ +package eu.dnetlib.dhp.collection.plugin.researchfi; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +import org.dom4j.DocumentException; +import org.dom4j.DocumentHelper; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; +import eu.dnetlib.dhp.common.collection.CollectorException; + +public class ResearchFiCollectorPluginTest { + + private final ResearchFiCollectorPlugin plugin = new ResearchFiCollectorPlugin(); + + @Test + @Disabled + void testCollect() throws CollectorException { + final ApiDescriptor api = new ApiDescriptor(); + api.setBaseUrl("https://research.fi/api/rest/v1/funding-decisions?FunderName=AKA&FundingStartYearFrom=2022"); + api.setProtocol("research_fi"); + api.getParams().put("auth_url", "https://researchfi-auth.2.rahtiapp.fi/realms/publicapi/protocol/openid-connect/token"); + api.getParams().put("auth_client_id", ""); + api.getParams().put("auth_client_secret", ""); + + final AtomicLong count = new AtomicLong(0); + final Set ids = new HashSet<>(); + + this.plugin.collect(api, new AggregatorReport()).forEach(s -> { + + if (count.getAndIncrement() == 0) { + System.out.println("First: " + s); + } + + try { + final String id = DocumentHelper.parseText(s).valueOf("/recordWrap/funderProjectNumber"); + if (ids.contains(id)) { + System.out.println("Id already present: " + id); + } + ids.add(id); + } catch (final DocumentException e) { + throw new RuntimeException(e); + } + }); + + System.out.println("Total records: " + count); + System.out.println("Total identifiers: " + ids.size()); + + } + +} From bbe52584f7b2d301408d73a69ee5ff63f20581cc Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 11 Jul 2024 15:14:34 +0200 Subject: [PATCH 063/239] log message --- .../plugin/researchfi/ResearchFiIterator.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java index 38a3cece8..e08823c46 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java @@ -76,16 +76,17 @@ public class ResearchFiIterator implements Iterator { this.currPage += 1; + final String url; + if (!this.baseUrl.contains("?")) { + url = String.format("%s?PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); + } else if (!this.baseUrl.contains("PageSize=")) { + url = String.format("%s&PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); + } else { + url = String.format("%s&PageNumber=%d", this.baseUrl, this.currPage); + } + log.info("Calling url: " + url); + try (final CloseableHttpClient client = HttpClients.createDefault()) { - final String url; - if (!this.baseUrl.contains("?")) { - url = String.format("%s?PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); - } else if (!this.baseUrl.contains("PageSize=")) { - url = String.format("%s&PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); - } else { - url = String.format("%s&PageNumber=%d", this.baseUrl, this.currPage); - } - log.info("Calling url: " + url); final HttpGet req = new HttpGet(url); req.addHeader("Authorization", "Bearer " + this.authToken); @@ -107,8 +108,8 @@ public class ResearchFiIterator implements Iterator { jsonArray.forEach(obj -> this.queue.add(JsonUtils.convertToXML(obj.toString()))); } } catch (final Throwable e) { - log.warn("Error obtaining access token", e); - throw new CollectorException("Error obtaining access token", e); + log.warn("Error calling url: " + url, e); + throw new CollectorException("Error calling url: " + url, e); } } From f9ed2ae33c95159072a913f2b235e7b0876ae63f Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 11 Jul 2024 15:24:29 +0200 Subject: [PATCH 064/239] [metadata collection] added the possibility to specify the JAVA_HOME and the JAVA_OPTS parameters --- .../dhp/collection/oozie_app/workflow.xml | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml index 0678eed11..9df88b4b6 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml @@ -48,12 +48,31 @@ Used to configure the heap size for the map JVM process. Should be 80% of mapreduce.map.memory.mb. + + JAVA_HOME + /srv/java/openjdk-17 + Used to configure the Java home location for oozie.launcher.mapreduce.map.env + ${jobTracker} ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.launcher.mapreduce.map.env + JAVA_HOME=${JAVA_HOME} + + @@ -99,7 +118,7 @@ eu.dnetlib.dhp.collection.CollectorWorkerApplication - ${collection_java_xmx} + ${JAVA_OPTS} ${collection_java_xmx} --apidescriptor${apiDescription} --namenode${nameNode} --workflowId${workflowId} From 61d1fa9b9fe84850c5c21de6bc8027773df81368 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 12 Jul 2024 10:25:51 +0200 Subject: [PATCH 065/239] [metadata collection] added -Dcom.sun.security.enableAIAcaIssuers=true as a default for metadata collection --- .../eu/dnetlib/dhp/collection/oozie_app/workflow.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml index 9df88b4b6..f1364b63c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml @@ -54,6 +54,12 @@ Used to configure the Java home location for oozie.launcher.mapreduce.map.env + + JAVA_OPTS + -Dcom.sun.security.enableAIAcaIssuers=true + Used to configure the JAVA_OPTS parameter + + From e17edb25817e184f8bfbc4f2c21b73684ffd5f4b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 12 Jul 2024 10:27:24 +0200 Subject: [PATCH 066/239] [broker] fine tuned the workflow memory settings --- .../oa/generate_all/oozie_app/workflow.xml | 45 +++++++++++++------ 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index bc6778f52..3e709117c 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -7,7 +7,7 @@ outputDir - the path where the the generated data will be stored + the path where the generated data will be stored datasourceIdWhitelist @@ -185,11 +185,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=5000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -209,11 +210,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -234,11 +236,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -258,11 +261,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=5000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -282,11 +286,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=10000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -306,11 +311,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=2000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -332,11 +338,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -356,11 +363,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -380,11 +388,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -404,11 +413,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -428,11 +438,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -452,11 +463,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --graphPath${graphInputPath} --workingDir${workingDir} @@ -476,11 +488,12 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --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 + --conf spark.sql.shuffle.partitions=8000 --workingDir${workingDir} --outputDir${outputDir} @@ -503,6 +516,7 @@ --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.dynamicAllocation.maxExecutors=${sparkMaxExecutorsForIndexing} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} @@ -535,6 +549,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -562,6 +577,7 @@ --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -585,6 +601,7 @@ --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} --conf spark.dynamicAllocation.maxExecutors=${sparkMaxExecutorsForIndexing} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} From f99fb2104007aab8aeee9d5491c66d4148a781a5 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Fri, 12 Jul 2024 15:29:45 +0200 Subject: [PATCH 067/239] tests --- .../simple/EnrichMissingAuthorOrcidTest.java | 64 +++++++++++++++++++ .../broker/oa/util/ConversionUtilsTest.java | 50 +++++++++++++-- 2 files changed, 110 insertions(+), 4 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java new file mode 100644 index 000000000..e61299800 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java @@ -0,0 +1,64 @@ +package eu.dnetlib.dhp.broker.oa.matchers.simple; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.List; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.broker.objects.OaBrokerAuthor; +import eu.dnetlib.broker.objects.OaBrokerMainEntity; + +class EnrichMissingAuthorOrcidTest { + + final EnrichMissingAuthorOrcid matcher = new EnrichMissingAuthorOrcid(); + + @BeforeEach + void setUp() throws Exception {} + + @Test + void testFindDifferences_1() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_2() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + + source.getCreators().add(new OaBrokerAuthor("Claudio Atzori", "0000-0001-9613-6639")); + target.getCreators().add(new OaBrokerAuthor("Claudio Atzori", null)); + + final List list = this.matcher.findDifferences(source, target); + assertEquals(1, list.size()); + } + + @Test + void testFindDifferences_3() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + + source.getCreators().add(new OaBrokerAuthor("Claudio Atzori", null)); + target.getCreators().add(new OaBrokerAuthor("Claudio Atzori", "0000-0001-9613-6639")); + + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_4() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.getCreators().add(new OaBrokerAuthor("Claudio Atzori", "0000-0001-9613-6639")); + target.getCreators().add(new OaBrokerAuthor("Claudio Atzori", "0000-0001-9613-6639")); + + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java index fc630df05..ee1bfed05 100644 --- a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java @@ -2,27 +2,31 @@ package eu.dnetlib.dhp.broker.oa.util; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerTypedValue; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Instance; import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -class ConversionUtilsTest { +public class ConversionUtilsTest { @BeforeEach - void setUp() throws Exception { - } + public void setUp() throws Exception {} @Test - void testAllResultPids() { + public void testAllResultPids() { final Qualifier qf = new Qualifier(); qf.setClassid("test"); qf.setClassname("test"); @@ -91,4 +95,42 @@ class ConversionUtilsTest { assertEquals(6, list.size()); } + public void testOafResultToBrokerResult() { + + final Author a1 = createAuthor("Michele Artini", "0000-0002-4406-428X"); + final Author a2 = createAuthor("Claudio Atzori", "http://orcid.org/0000-0001-9613-6639"); + final Author a3 = createAuthor("Alessia Bardi", null); + + final Result r = new Result(); + r.setAuthor(Arrays.asList(a1, a2, a3)); + + final OaBrokerMainEntity br = ConversionUtils.oafResultToBrokerResult(r); + + assertEquals(3, br.getCreators().size()); + assertEquals("0000-0002-4406-428X", br.getCreators().get(0).getOrcid()); + assertEquals("0000-0001-9613-6639", br.getCreators().get(1).getOrcid()); + assertNull(br.getCreators().get(2).getOrcid()); + } + + private Author createAuthor(final String name, final String orcid) { + + final Author a = new Author(); + a.setFullname("Michele Artini"); + + if (orcid != null) { + final Qualifier q = new Qualifier(); + q.setClassid(ModelConstants.ORCID); + q.setClassname(ModelConstants.ORCID); + q.setSchemeid("dnet:pids"); + q.setSchemename("dnet:pids"); + + final StructuredProperty pid = new StructuredProperty(); + pid.setQualifier(q); + pid.setValue(orcid); + + a.setPid(Arrays.asList(pid)); + } + return a; + } + } From 7fa3d512000250e2d549bac581a964d06930d4a5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 12 Jul 2024 17:09:01 +0200 Subject: [PATCH 068/239] renamed class, updated criteria to consider the ORCIDs used in the matchers --- ...mpleEntititiesJob.java => PrepareSimpleEntitiesJob.java} | 6 +++--- .../java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java | 3 +-- .../dhp/broker/oa/generate_all/oozie_app/workflow.xml | 2 +- 3 files changed, 5 insertions(+), 6 deletions(-) rename dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/{PrepareSimpleEntititiesJob.java => PrepareSimpleEntitiesJob.java} (96%) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntitiesJob.java similarity index 96% rename from dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java rename to dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntitiesJob.java index 4cb3ecfd9..30e50931b 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntitiesJob.java @@ -26,15 +26,15 @@ import eu.dnetlib.dhp.schema.oaf.Publication; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.Software; -public class PrepareSimpleEntititiesJob { +public class PrepareSimpleEntitiesJob { - private static final Logger log = LoggerFactory.getLogger(PrepareSimpleEntititiesJob.class); + private static final Logger log = LoggerFactory.getLogger(PrepareSimpleEntitiesJob.class); public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - PrepareSimpleEntititiesJob.class + PrepareSimpleEntitiesJob.class .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json"))); parser.parseArgument(args); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index 4bf4d3341..237c21db8 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -160,8 +160,7 @@ public class ConversionUtils { .stream() .filter(Objects::nonNull) .filter(pid -> pid.getQualifier() != null) - .filter(pid -> pid.getQualifier().getClassid() != null) - .filter(pid -> pid.getQualifier().getClassid().equalsIgnoreCase(ModelConstants.ORCID)) + .filter(pid -> StringUtils.startsWithIgnoreCase(pid.getQualifier().getClassid(), ModelConstants.ORCID)) .map(StructuredProperty::getValue) .map(ConversionUtils::cleanOrcid) .filter(StringUtils::isNotBlank) diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index 3e709117c..f5409617b 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -179,7 +179,7 @@ yarn cluster PrepareSimpleEntititiesJob - eu.dnetlib.dhp.broker.oa.PrepareSimpleEntititiesJob + eu.dnetlib.dhp.broker.oa.PrepareSimpleEntitiesJob dhp-broker-events-${projectVersion}.jar --executor-cores=${sparkExecutorCores} From 6f6e85ddf445842d775f715b6d1ae7f14d37919d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 15 Jul 2024 09:32:04 +0200 Subject: [PATCH 069/239] code formatting --- .../oa/matchers/simple/EnrichMissingAuthorOrcidTest.java | 4 +++- .../eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java index e61299800..2a378ddd6 100644 --- a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcidTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.broker.oa.matchers.simple; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -16,7 +17,8 @@ class EnrichMissingAuthorOrcidTest { final EnrichMissingAuthorOrcid matcher = new EnrichMissingAuthorOrcid(); @BeforeEach - void setUp() throws Exception {} + void setUp() throws Exception { + } @Test void testFindDifferences_1() { diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java index ee1bfed05..4c7891576 100644 --- a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtilsTest.java @@ -23,7 +23,8 @@ import eu.dnetlib.dhp.schema.oaf.StructuredProperty; public class ConversionUtilsTest { @BeforeEach - public void setUp() throws Exception {} + public void setUp() throws Exception { + } @Test public void testAllResultPids() { From 1fb44198fb5fc732f8ff592cfc6e22722c8bf45c Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 15 Jul 2024 15:24:18 +0200 Subject: [PATCH 070/239] renamed workflow to better reflect its purpose --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 605c86ac9..697e30a09 100644 --- a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + stats_db_name From 38f8ed27fd7f53549637f9062c8028fe63e42437 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 15 Jul 2024 16:30:43 +0200 Subject: [PATCH 071/239] [graph provision] log the Solr admin application operations for alias deletion and creation --- .../java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java index 44426e8c5..2bf7d3fbb 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java @@ -128,12 +128,14 @@ public class SolrAdminApplication implements Closeable { public SolrResponse deleteAlias(String aliasName) throws SolrServerException, IOException { CollectionAdminRequest.DeleteAlias deleteAliasRequest = CollectionAdminRequest.deleteAlias(aliasName); + log.info("deleting alias: {}", aliasName); return deleteAliasRequest.process(solrClient); } public SolrResponse createAlias(String aliasName, String collection) throws IOException, SolrServerException { CollectionAdminRequest.CreateAlias createAliasRequest = CollectionAdminRequest .createAlias(aliasName, collection); + log.info("creating alias: {} for collection: {}", aliasName, collection); return createAliasRequest.process(solrClient); } From 5aa7847ea626be914f719464b69cf6e1b111e0f1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 16 Jul 2024 10:38:50 +0200 Subject: [PATCH 072/239] consider the transformative agreement text when merging results --- .../main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index 74fd13ae7..ea402ecbf 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -464,6 +464,10 @@ public class MergeUtils { merge.setIsInDiamondJournal(booleanOR(merge.getIsInDiamondJournal(), enrich.getIsInDiamondJournal())); merge.setPubliclyFunded(booleanOR(merge.getPubliclyFunded(), enrich.getPubliclyFunded())); + if (StringUtils.isBlank(merge.getTransformativeAgreement())) { + merge.setTransformativeAgreement(enrich.getTransformativeAgreement()); + } + return merge; } From beb93cdfe9563ae01102e6d105a82ef03a80fff1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 16 Jul 2024 11:43:48 +0200 Subject: [PATCH 073/239] [graph provision] expand the context info for each entity type --- .../model/ProvisionModelSupport.java | 20 ++++--- .../oa/provision/utils/XmlRecordFactory.java | 19 +++---- .../oa/provision/SolrConfigExploreTest.java | 56 ++++++++++--------- .../dhp/oa/provision/SolrConfigTest.java | 3 +- .../oa/provision/XmlRecordFactoryTest.java | 41 +++++++++++++- .../dnetlib/dhp/oa/provision/project_aka.json | 1 + 6 files changed, 91 insertions(+), 49 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index b450b95be..abcf4992f 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.solr.ExternalReference; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -31,6 +30,7 @@ import eu.dnetlib.dhp.schema.solr.Context; import eu.dnetlib.dhp.schema.solr.Country; import eu.dnetlib.dhp.schema.solr.Datasource; import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines; +import eu.dnetlib.dhp.schema.solr.ExternalReference; import eu.dnetlib.dhp.schema.solr.Instance; import eu.dnetlib.dhp.schema.solr.Journal; import eu.dnetlib.dhp.schema.solr.Measure; @@ -562,10 +562,16 @@ public class ProvisionModelSupport { .orElse(null); } - private static List mapExternalReference(List externalReference) { - return Optional.ofNullable(externalReference) - .map(ext -> ext.stream() - .map(e -> ExternalReference.newInstance( + private static List mapExternalReference( + List externalReference) { + return Optional + .ofNullable(externalReference) + .map( + ext -> ext + .stream() + .map( + e -> ExternalReference + .newInstance( e.getSitename(), e.getLabel(), e.getAlternateLabel(), @@ -573,8 +579,8 @@ public class ProvisionModelSupport { mapCodeLabel(e.getQualifier()), e.getRefidentifier(), e.getQuery())) - .collect(Collectors.toList())) - .orElse(Lists.newArrayList()); + .collect(Collectors.toList())) + .orElse(Lists.newArrayList()); } private static List asContext(List ctxList, diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 3d89b1e34..899dad221 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -219,6 +219,13 @@ public class XmlRecordFactory implements Serializable { if (entity.getMeasures() != null) { metadata.addAll(measuresAsXml(entity.getMeasures())); } + if (entity.getContext() != null) { + contexts.addAll(entity.getContext().stream().map(Context::getId).collect(Collectors.toList())); + /* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */ + if (contexts.contains("dh-ch::subcommunity::2")) { + contexts.add("clarin"); + } + } if (ModelSupport.isResult(type)) { final Result r = (Result) entity; @@ -245,14 +252,6 @@ public class XmlRecordFactory implements Serializable { .collect(Collectors.toList())); } - if (r.getContext() != null) { - contexts.addAll(r.getContext().stream().map(c -> c.getId()).collect(Collectors.toList())); - /* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */ - if (contexts.contains("dh-ch::subcommunity::2")) { - contexts.add("clarin"); - } - } - if (r.getTitle() != null) { metadata .addAll( @@ -1603,9 +1602,7 @@ public class XmlRecordFactory implements Serializable { private List buildContexts(final String type, final Set contexts) { final List res = Lists.newArrayList(); - if (contextMapper != null - && !contextMapper.isEmpty() - && MainEntityType.result.toString().equals(type)) { + if (contextMapper != null && !contextMapper.isEmpty()) { XMLTag document = XMLDoc.newDocument(true).addRoot("contextRoot"); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java index 41eac2a30..90aef5adc 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java @@ -1,12 +1,13 @@ package eu.dnetlib.dhp.oa.provision; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.File; import java.io.IOException; import java.net.URI; import java.nio.file.Path; -import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrQuery; @@ -32,14 +33,13 @@ import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; - -import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.jupiter.api.Assertions.assertEquals; +import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; +import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) public class SolrConfigExploreTest { @@ -91,7 +91,7 @@ public class SolrConfigExploreTest { SparkConf conf = new SparkConf(); conf.setAppName(XmlIndexingJobTest.class.getSimpleName()); conf.registerKryoClasses(new Class[] { - SerializableSolrInputDocument.class + SerializableSolrInputDocument.class }); conf.setMaster("local[1]"); @@ -101,10 +101,10 @@ public class SolrConfigExploreTest { conf.set("spark.sql.warehouse.dir", workingDir.resolve("spark").toString()); spark = SparkSession - .builder() - .appName(SolrConfigExploreTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); + .builder() + .appName(SolrConfigExploreTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); // random unassigned HTTP port final int jettyPort = 0; @@ -134,35 +134,35 @@ public class SolrConfigExploreTest { log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); NamedList res = createCollection( - miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); // miniCluster.getSolrClient().setDefaultCollection(SHADOW_COLLECTION); res = createCollection( - miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); admin = new SolrAdminApplication(miniCluster.getZkClient().getZkServerAddress()); CollectionAdminResponse rsp = (CollectionAdminResponse) admin - .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); assertEquals(0, rsp.getStatus()); rsp = (CollectionAdminResponse) admin.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); } @@ -180,7 +180,8 @@ public class SolrConfigExploreTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions + .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", @@ -200,7 +201,8 @@ public class SolrConfigExploreTest { // System.out.println(rsp.getExplainMap()); for (SolrDocument doc : rsp.getResults()) { - log.info( + log + .info( doc.get("score") + "\t" + doc.get("__indexrecordidentifier") + "\t" + doc.get("resultidentifier") + "\t" + @@ -216,7 +218,7 @@ public class SolrConfigExploreTest { } protected static NamedList createCollection(CloudSolrClient client, String name, int numShards, - int replicationFactor, int maxShardsPerNode, String configName) throws Exception { + int replicationFactor, int maxShardsPerNode, String configName) throws Exception { ModifiableSolrParams modParams = new ModifiableSolrParams(); modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name()); modParams.set("name", name); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java index 2c62389c6..c04fa1cc6 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java @@ -85,7 +85,8 @@ public class SolrConfigTest extends SolrTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions + .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java index d617991a1..ab4301f9a 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java @@ -1,8 +1,7 @@ package eu.dnetlib.dhp.oa.provision; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.*; import java.io.IOException; import java.io.StringReader; @@ -22,6 +21,7 @@ import com.google.common.collect.Lists; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; +import eu.dnetlib.dhp.oa.provision.utils.ContextDef; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; import eu.dnetlib.dhp.schema.oaf.*; @@ -51,7 +51,7 @@ public class XmlRecordFactoryTest { assertNotNull(doc); - // System.out.println(doc.asXML()); + System.out.println(doc.asXML()); assertEquals("0000-0001-9613-6638", doc.valueOf("//creator[@rank = '1']/@orcid")); assertEquals("0000-0001-9613-6639", doc.valueOf("//creator[@rank = '1']/@orcid_pending")); @@ -267,4 +267,39 @@ public class XmlRecordFactoryTest { } + @Test + public void test_AKA_project() throws DocumentException, IOException { + final ContextMapper contextMapper = new ContextMapper(); + + contextMapper + .put("dh-ch", new ContextDef("dh-ch", "Digital Humanities and Cultural Heritage", "context", "community")); + contextMapper.put("dh-ch::projects", new ContextDef("dh-ch::projects", "DH-CH Projects", "category", "")); + contextMapper + .put("dh-ch::projects::2", new ContextDef("dh-ch::projects::2", "ARIADNE", "concept", "community")); + + final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, + PayloadConverterJob.schemaLocation); + + final Project p = OBJECT_MAPPER + .readValue( + IOUtils.toString(getClass().getResourceAsStream("project_aka.json")), + Project.class); + + assertNotNull(p.getContext()); + assertEquals(1, p.getContext().size()); + assertEquals("dh-ch::projects::2", p.getContext().get(0).getId()); + + final String xml = xmlRecordFactory.build(new JoinedEntity(p)); + + assertNotNull(xml); + + final Document doc = new SAXReader().read(new StringReader(xml)); + + assertNotNull(doc); + + assertEquals("dh-ch", doc.valueOf("//context/@id")); + assertEquals("dh-ch::projects", doc.valueOf("//context/category/@id")); + assertEquals("dh-ch::projects::2", doc.valueOf("//context/category/concept/@id")); + } + } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json new file mode 100644 index 000000000..18bd5adf4 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json @@ -0,0 +1 @@ +{"context" : [{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"bulktagging","inferred":true,"invisible":false,"provenanceaction":{"classid":"bulktagging:community:subject","classname":"Bulk Tagging for Communities","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"}],"id":"dh-ch::projects::2"}], "callidentifier":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Virkapäätöksiin liittyvä yleiskustannusosuus KY"},"code":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"204684"},"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"currency":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"EUR"},"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2019-01-25","dateoftransformation":"2022-02-08","duration":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0"},"ecarticle29_3":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsc39":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"enddate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2002-12-31"},"extraInfo":[],"fundedamount":3230.0,"fundingtree":[{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"\n aka_________::AKA\n AKA\n Academy of Finland\n Academy of Finland\n FI\n "}],"id":"40|aka_________::00f0012ac67a2f826f2e98dbdfd6b058","jsonextrainfo":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"{}"},"lastupdatetimestamp":1719997513126,"oamandatepublications":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"optional1":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"3,230 €"},"originalId":["aka_________::204684"],"pid":[],"startdate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2002-12-01"},"subjects":[],"title":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Tutkijan virkaan liittyvä yleiskustannusosuus suorituspaikalle"},"totalcost":0.0} \ No newline at end of file From 9d279101446a71236fe3edb53c5e51946aa2c2fd Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 16 Jul 2024 13:48:48 +0200 Subject: [PATCH 074/239] [BulkTag]added tagging for the organization relevant for the community. Added test. Changed the tagging variables. --- .../eu/dnetlib/pace/tree/CountryMatch.java | 55 ++++---- .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 45 +++--- .../bulktag/community/TaggingConstants.java | 6 + .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 132 ++++++++++++++++++ .../update_datasource/organization | 4 + .../publication/update_datasource/project | 4 + .../model/ProvisionModelSupport.java | 20 ++- .../oa/provision/SolrConfigExploreTest.java | 56 ++++---- .../dhp/oa/provision/SolrConfigTest.java | 3 +- 9 files changed, 243 insertions(+), 82 deletions(-) create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/organization diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java index 9cf616356..c02381983 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java @@ -1,3 +1,4 @@ + package eu.dnetlib.pace.tree; import java.util.Map; @@ -11,37 +12,37 @@ import eu.dnetlib.pace.tree.support.ComparatorClass; @ComparatorClass("countryMatch") public class CountryMatch extends AbstractStringComparator { - public CountryMatch(Map params) { - super(params, new com.wcohen.ss.JaroWinkler()); - } + public CountryMatch(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + } - public CountryMatch(final double weight) { - super(weight, new com.wcohen.ss.JaroWinkler()); - } + public CountryMatch(final double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } - protected CountryMatch(final double weight, final AbstractStringDistance ssalgo) { - super(weight, ssalgo); - } + protected CountryMatch(final double weight, final AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } - @Override - public double distance(final String a, final String b, final Config conf) { - if (a.isEmpty() || b.isEmpty()) { - return -1.0; // return -1 if a field is missing - } - if (a.equalsIgnoreCase("unknown") || b.equalsIgnoreCase("unknown")) { - return -1.0; // return -1 if a country is UNKNOWN - } + @Override + public double distance(final String a, final String b, final Config conf) { + if (a.isEmpty() || b.isEmpty()) { + return -1.0; // return -1 if a field is missing + } + if (a.equalsIgnoreCase("unknown") || b.equalsIgnoreCase("unknown")) { + return -1.0; // return -1 if a country is UNKNOWN + } - return a.equals(b) ? 1.0 : 0; - } + return a.equals(b) ? 1.0 : 0; + } - @Override - public double getWeight() { - return super.weight; - } + @Override + public double getWeight() { + return super.weight; + } - @Override - protected double normalize(final double d) { - return d; - } + @Override + protected double normalize(final double d) { + return d; + } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 354741690..e9a45be40 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -33,10 +33,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.bulktag.community.*; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.Context; -import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Project; -import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import scala.Tuple2; @@ -114,27 +111,35 @@ public class SparkBulkTagJob { extendCommunityConfigurationForEOSC(spark, inputPath, cc); execBulkTag( spark, inputPath, outputPath, protoMap, cc); + execEntityTag( + spark, inputPath + "organization", outputPath + "organization", + Utils.getCommunityOrganization(baseURL), Organization.class, TaggingConstants.CLASS_ID_ORGANIZATION, + TaggingConstants.CLASS_NAME_BULKTAG_ORGANIZATION); + execEntityTag( + spark, inputPath + "project", outputPath + "project", Utils.getCommunityProjects(baseURL), + Project.class, TaggingConstants.CLASS_ID_PROJECT, TaggingConstants.CLASS_NAME_BULKTAG_PROJECT); execDatasourceTag(spark, inputPath, outputPath, Utils.getDatasourceCommunities(baseURL)); - execProjectTag(spark, inputPath, outputPath, Utils.getCommunityProjects(baseURL)); + }); } - private static void execProjectTag(SparkSession spark, String inputPath, String outputPath, - CommunityEntityMap communityProjects) { - Dataset projects = readPath(spark, inputPath + "project", Project.class); + private static void execEntityTag(SparkSession spark, String inputPath, String outputPath, + CommunityEntityMap communityEntity, Class entityClass, + String classID, String calssName) { + Dataset entity = readPath(spark, inputPath, entityClass); Dataset pc = spark .createDataset( - communityProjects + communityEntity .keySet() .stream() - .map(k -> EntityCommunities.newInstance(k, communityProjects.get(k))) + .map(k -> EntityCommunities.newInstance(k, communityEntity.get(k))) .collect(Collectors.toList()), Encoders.bean(EntityCommunities.class)); - projects - .joinWith(pc, projects.col("id").equalTo(pc.col("entityId")), "left") - .map((MapFunction, Project>) t2 -> { - Project ds = t2._1(); + entity + .joinWith(pc, entity.col("id").equalTo(pc.col("entityId")), "left") + .map((MapFunction, E>) t2 -> { + E ds = t2._1(); if (t2._2() != null) { List context = Optional .ofNullable(ds.getContext()) @@ -156,8 +161,8 @@ public class SparkBulkTagJob { false, TaggingConstants.BULKTAG_DATA_INFO_TYPE, true, false, OafMapperUtils .qualifier( - TaggingConstants.CLASS_ID_DATASOURCE, - TaggingConstants.CLASS_NAME_BULKTAG_DATASOURCE, + classID, + calssName, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "1"))); @@ -166,17 +171,17 @@ public class SparkBulkTagJob { }); } return ds; - }, Encoders.bean(Project.class)) + }, Encoders.bean(entityClass)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(outputPath + "project"); + .json(outputPath); - readPath(spark, outputPath + "project", Project.class) + readPath(spark, outputPath, entityClass) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(inputPath + "project"); + .json(inputPath); } private static void execDatasourceTag(SparkSession spark, String inputPath, String outputPath, diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java index aea21f8e5..6e6f12cfb 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java @@ -13,6 +13,9 @@ public class TaggingConstants { public static final String CLASS_ID_CZENODO = "community:zenodocommunity"; public static final String CLASS_ID_ADVANCED_CONSTRAINT = "community:advconstraint"; + public static final String CLASS_ID_PROJECT = "community:project"; + public static final String CLASS_ID_ORGANIZATION = "community:organization"; + public static final String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; public static final String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; @@ -20,5 +23,8 @@ public class TaggingConstants { public static final String CLASS_NAME_BULKTAG_ZENODO = "Bulktagging for Community - Zenodo"; public static final String CLASS_NAME_BULKTAG_ADVANCED_CONSTRAINT = "Bulktagging for Community - Advanced Constraints"; + public static final String CLASS_NAME_BULKTAG_PROJECT = "Bulktagging for Community - Project"; + public static final String CLASS_NAME_BULKTAG_ORGANIZATION = "Bulktagging for Community - Organization"; + public static final String TAGGING_TRUST = "0.8"; } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index a5280a3b3..f49774574 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -465,6 +465,138 @@ public class BulkTagJobTest { } + @Test + void organizationTag() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/") + .getPath(); + LocalFileSystem fs = FileSystem.getLocal(new Configuration()); + fs + .copyFromLocalFile( + false, new org.apache.hadoop.fs.Path(getClass() + .getResource("/eu/dnetlib/dhp/bulktag/pathMap/") + .getPath()), + new org.apache.hadoop.fs.Path(workingDir.toString() + "/data/bulktagging/protoMap")); + SparkBulkTagJob + .main( + new String[] { + + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + + "-outputPath", workingDir.toString() + "/", + "-baseURL", "https://services.openaire.eu/openaire/community/", + + "-pathMap", workingDir.toString() + "/data/bulktagging/protoMap/pathMap", + "-nameNode", "local" + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/organization") + .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)); + + Assertions.assertEquals(4, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Organization.class)); + + verificationDataset.createOrReplaceTempView("organization"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from organization " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + + idExplodeCommunity.show(false); + + Assertions.assertEquals(3, idExplodeCommunity.count()); + Assertions + .assertEquals( + 3, idExplodeCommunity.filter("provenance = 'community:organization'").count()); + Assertions + .assertEquals( + 3, + idExplodeCommunity + .filter("name = 'Bulktagging for Community - Organization'") + .count()); + + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'netherlands'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'beopen'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); + + } + + @Test + void projectTag() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/") + .getPath(); + LocalFileSystem fs = FileSystem.getLocal(new Configuration()); + fs + .copyFromLocalFile( + false, new org.apache.hadoop.fs.Path(getClass() + .getResource("/eu/dnetlib/dhp/bulktag/pathMap/") + .getPath()), + new org.apache.hadoop.fs.Path(workingDir.toString() + "/data/bulktagging/protoMap")); + SparkBulkTagJob + .main( + new String[] { + + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + + "-outputPath", workingDir.toString() + "/", + "-baseURL", "https://services.openaire.eu/openaire/community/", + + "-pathMap", workingDir.toString() + "/data/bulktagging/protoMap/pathMap", + "-nameNode", "local" + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/project") + .map(item -> OBJECT_MAPPER.readValue(item, Project.class)); + + Assertions.assertEquals(4, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Project.class)); + + verificationDataset.createOrReplaceTempView("project"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from project " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + + idExplodeCommunity.show(false); + + Assertions.assertEquals(4, idExplodeCommunity.count()); + Assertions + .assertEquals( + 4, idExplodeCommunity.filter("provenance = 'community:project'").count()); + Assertions + .assertEquals( + 4, + idExplodeCommunity + .filter("name = 'Bulktagging for Community - Project'") + .count()); + + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'enermaps'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'clarin'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'dh-ch'").count()); + + } + @Test void bulktagByZenodoCommunityTest() throws Exception { final String sourcePath = getClass() diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/organization b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/organization new file mode 100644 index 000000000..cf7a2f90e --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/organization @@ -0,0 +1,4 @@ +{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2024-03-29","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|openorgs____::2b9422d5f5bac4aae99b24e61586d3d3","lastupdatetimestamp":1719997513126,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"Laboratoire de Physique Statistique, UMR 8550 - Ecole Normale Supérieure"},"organizationType":"Unknown","originalId":["anr_________::9fdd505a0c45c6ad239c7c1406145820"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"qualifier":{"classid":"RNSR","classname":"RNSR","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"RNSR:199812878L"}]} +{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2023-12-04","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|openorgs____::00010e003d3ce9c05e038cadfb6d4f03","lastupdatetimestamp":1719997513126,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"National Institute for Agricultural and Food Research and Technology"},"organizationType":"Unknown","originalId":["anr_________::16ba9bdd7a02f3e5f38a9c6a0ca24be6"],"pid":[]} +{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2022-10-11","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|openorgs____::ad863df6deda1619a25e7fad4a534891","lastupdatetimestamp":1719997513126,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"INSTITUT DE VEILLE SANITAIRE"},"organizationType":"Unknown","originalId":["anr_________::27a60b3e005f647d173309729e4c62ae"],"pid":[]} +{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::b30dac7baac631f3da7c2bb18dd9891f","value":"CORDA - COmmon Research DAta Warehouse"}],"country":{"classid":"DE","classname":"Germany","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"dateofcollection":"2018-03-12","dateoftransformation":"2022-05-24","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"true"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"false"},"extraInfo":[],"id":"20|corda_______::0dfa837c57e43e0528db09543a8c4dac","lastupdatetimestamp":1719997513126,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"ALLGEMEINER DEUTSCHER AUTOMOBIL CLUB E.V."},"legalshortname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"value":"ADAC"},"organizationType":"Unknown","originalId":["corda_______::999607990"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.800"},"qualifier":{"classid":"PIC","classname":"Participant Identification Code","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"999607990"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/project b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/project index e69de29bb..2b9628ea1 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/project +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/project @@ -0,0 +1,4 @@ +{"code":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"unidentified"},"collectedfrom":[{"key":"10|openaire____::718b1e0b0206401b66e8d87f68876121","value":"Stroke Association"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2024-04-20","dateoftransformation":"2024-04-20","duration":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0"},"ecarticle29_3":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsc39":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"extraInfo":[],"fundedamount":0.0,"fundingtree":[],"id":"40|ukri________::6937e06c766521c0629d8a6ca20ca345","jsonextrainfo":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"{}"},"lastupdatetimestamp":1719997513126,"oamandatepublications":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"optional1":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"uk_funders_import"},"originalId":["501100000364::unidentified"],"pid":[],"subjects":[],"title":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"unidentified"},"totalcost":0.0} +{"callidentifier":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Academy Project BY"},"code":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"276480"},"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"currency":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"EUR"},"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2024-07-05T18:46:23.059","dateoftransformation":"2022-02-08","duration":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0"},"ecarticle29_3":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsc39":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"enddate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2018-08-31"},"extraInfo":[],"fundedamount":471335.0,"fundingtree":[{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"\n aka_________::AKA\n AKA\n Academy of Finland\n Academy of Finland\n FI\n "}],"id":"40|corda_______::ef782b2d85676aa3e5a907427feb18c4","jsonextrainfo":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"{}"},"lastupdatetimestamp":1719997513126,"measures":[{"id":"numOfInfluentialResults","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"19"}]},{"id":"numOfPopularResults","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"19"}]},{"id":"totalImpulse","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"486"}]},{"id":"totalCitationCount","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"840"}]},{"id":"downloads","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:usage_counts","classname":"measure:usage_counts","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"count","value":"17"}]},{"id":"views","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:usage_counts","classname":"measure:usage_counts","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"count","value":"9"}]}],"oamandatepublications":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"optional1":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"471,335 €"},"originalId":["aka_________::276480"],"pid":[],"startdate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2014-09-01"},"subjects":[],"title":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Yeast hybrid vigour: elucidation of mechanisms governing competitive advantage and sub-genome compatibility after interspecific hybridization"},"totalcost":0.0} +{"callidentifier":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Subsidy for organising of international conferences KY"},"code":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"107956"},"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"currency":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"EUR"},"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2019-01-25","dateoftransformation":"2022-02-08","duration":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0"},"ecarticle29_3":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsc39":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"enddate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2005-12-31"},"extraInfo":[],"fundedamount":2000.0,"fundingtree":[{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"\n aka_________::AKA\n AKA\n Academy of Finland\n Academy of Finland\n FI\n "}],"id":"40|aka_________::017ddc7873b184db49337b2472cdc83f","jsonextrainfo":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"{}"},"lastupdatetimestamp":1719997513126,"oamandatepublications":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"optional1":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2,000 €"},"originalId":["aka_________::107956"],"pid":[],"startdate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2005-01-01"},"subjects":[],"title":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"NODALIDA 2005 15th Nordic Conference on Computational Linguistics, 19.-22.5.2005"},"totalcost":0.0} +{"callidentifier":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Biofuture 2025, second stage LT"},"code":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"307624"},"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"currency":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"EUR"},"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2024-07-05T18:46:23.059","dateoftransformation":"2022-02-08","duration":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0"},"ecarticle29_3":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsc39":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"enddate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2020-12-31"},"extraInfo":[],"fundedamount":331277.0,"fundingtree":[{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"\n aka_________::AKA\n AKA\n Academy of Finland\n Academy of Finland\n FI\n "}],"id":"40|snsf________::911db67dc5eebd24d7787d69d4974ca4","jsonextrainfo":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"{}"},"lastupdatetimestamp":1719997513126,"measures":[{"id":"numOfInfluentialResults","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"8"}]},{"id":"numOfPopularResults","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"8"}]},{"id":"totalImpulse","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"28"}]},{"id":"totalCitationCount","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"measure:bip","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"30"}]}],"oamandatepublications":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"optional1":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"331,277 €"},"originalId":["aka_________::307624"],"pid":[],"startdate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2017-01-01"},"subjects":[],"title":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Overcoming technology barriers with tailored catalysts: Design of molecularly functionalized heterogeneous catalysts for selective reductions of biomass-derived materials / Consortium: FunCat"},"totalcost":0.0} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index b450b95be..abcf4992f 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.solr.ExternalReference; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -31,6 +30,7 @@ import eu.dnetlib.dhp.schema.solr.Context; import eu.dnetlib.dhp.schema.solr.Country; import eu.dnetlib.dhp.schema.solr.Datasource; import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines; +import eu.dnetlib.dhp.schema.solr.ExternalReference; import eu.dnetlib.dhp.schema.solr.Instance; import eu.dnetlib.dhp.schema.solr.Journal; import eu.dnetlib.dhp.schema.solr.Measure; @@ -562,10 +562,16 @@ public class ProvisionModelSupport { .orElse(null); } - private static List mapExternalReference(List externalReference) { - return Optional.ofNullable(externalReference) - .map(ext -> ext.stream() - .map(e -> ExternalReference.newInstance( + private static List mapExternalReference( + List externalReference) { + return Optional + .ofNullable(externalReference) + .map( + ext -> ext + .stream() + .map( + e -> ExternalReference + .newInstance( e.getSitename(), e.getLabel(), e.getAlternateLabel(), @@ -573,8 +579,8 @@ public class ProvisionModelSupport { mapCodeLabel(e.getQualifier()), e.getRefidentifier(), e.getQuery())) - .collect(Collectors.toList())) - .orElse(Lists.newArrayList()); + .collect(Collectors.toList())) + .orElse(Lists.newArrayList()); } private static List asContext(List ctxList, diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java index 41eac2a30..90aef5adc 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java @@ -1,12 +1,13 @@ package eu.dnetlib.dhp.oa.provision; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.File; import java.io.IOException; import java.net.URI; import java.nio.file.Path; -import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrQuery; @@ -32,14 +33,13 @@ import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; - -import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.jupiter.api.Assertions.assertEquals; +import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; +import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) public class SolrConfigExploreTest { @@ -91,7 +91,7 @@ public class SolrConfigExploreTest { SparkConf conf = new SparkConf(); conf.setAppName(XmlIndexingJobTest.class.getSimpleName()); conf.registerKryoClasses(new Class[] { - SerializableSolrInputDocument.class + SerializableSolrInputDocument.class }); conf.setMaster("local[1]"); @@ -101,10 +101,10 @@ public class SolrConfigExploreTest { conf.set("spark.sql.warehouse.dir", workingDir.resolve("spark").toString()); spark = SparkSession - .builder() - .appName(SolrConfigExploreTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); + .builder() + .appName(SolrConfigExploreTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); // random unassigned HTTP port final int jettyPort = 0; @@ -134,35 +134,35 @@ public class SolrConfigExploreTest { log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); NamedList res = createCollection( - miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); // miniCluster.getSolrClient().setDefaultCollection(SHADOW_COLLECTION); res = createCollection( - miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); admin = new SolrAdminApplication(miniCluster.getZkClient().getZkServerAddress()); CollectionAdminResponse rsp = (CollectionAdminResponse) admin - .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); assertEquals(0, rsp.getStatus()); rsp = (CollectionAdminResponse) admin.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); } @@ -180,7 +180,8 @@ public class SolrConfigExploreTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions + .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", @@ -200,7 +201,8 @@ public class SolrConfigExploreTest { // System.out.println(rsp.getExplainMap()); for (SolrDocument doc : rsp.getResults()) { - log.info( + log + .info( doc.get("score") + "\t" + doc.get("__indexrecordidentifier") + "\t" + doc.get("resultidentifier") + "\t" + @@ -216,7 +218,7 @@ public class SolrConfigExploreTest { } protected static NamedList createCollection(CloudSolrClient client, String name, int numShards, - int replicationFactor, int maxShardsPerNode, String configName) throws Exception { + int replicationFactor, int maxShardsPerNode, String configName) throws Exception { ModifiableSolrParams modParams = new ModifiableSolrParams(); modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name()); modParams.set("name", name); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java index 2c62389c6..c04fa1cc6 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java @@ -85,7 +85,8 @@ public class SolrConfigTest extends SolrTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions + .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", From 9246bdec1ce5d34495370c1652103a3d8aa3c4bf Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 16 Jul 2024 14:07:37 +0200 Subject: [PATCH 075/239] [WebCrawlAffiliation]remove from the creation of the action set the relations for pmc and pmid. Only doi are allowed --- .../eu/dnetlib/dhp/actionmanager/Constants.java | 3 +++ .../webcrawl/CreateActionSetFromWebEntries.java | 15 ++++++++------- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java index 006d3af76..73b4b77cb 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -42,6 +42,9 @@ public class Constants { public static final String NULL = "NULL"; public static final String NA = "N/A"; + public static final String WEB_CRAWL_ID = "10|openaire____::fb98a192f6a055ba495ef414c330834b"; + public static final String WEB_CRAWL_NAME = "Web Crawl"; + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private Constants() { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index 27970f2c3..f2a73889f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -21,6 +21,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.actionmanager.Constants; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; @@ -29,6 +30,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; import eu.dnetlib.dhp.schema.oaf.utils.PidType; +import io.netty.util.Constant; import scala.Tuple2; /** @@ -44,8 +46,7 @@ public class CreateActionSetFromWebEntries implements Serializable { private static final String PMID_PREFIX = "50|pmid________::"; private static final String PMCID_PREFIX = "50|pmc_________::"; - private static final String WEB_CRAWL_ID = "10|openaire____::fb98a192f6a055ba495ef414c330834b"; - private static final String WEB_CRAWL_NAME = "Web Crawl"; + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { @@ -104,8 +105,7 @@ public class CreateActionSetFromWebEntries implements Serializable { final String ror = ROR_PREFIX + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror"))); ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror)); - ret.addAll(createAffiliationRelationPairPMID(row.getAs("pmid"), ror)); - ret.addAll(createAffiliationRelationPairPMCID(row.getAs("pmcid"), ror)); + return ret .iterator(); @@ -139,8 +139,9 @@ public class CreateActionSetFromWebEntries implements Serializable { "institution", functions .explode( functions.col("institutions"))) + .selectExpr( - "id", "doi", "ids.pmcid as pmcid", "ids.pmid as pmid", "institution.ror as ror", + "id", "doi", "institution.ror as ror", "institution.country_code as country_code", "publication_year") .distinct(); @@ -214,7 +215,7 @@ public class CreateActionSetFromWebEntries implements Serializable { ModelConstants.IS_AUTHOR_INSTITUTION_OF, Arrays .asList( - OafMapperUtils.keyValue(WEB_CRAWL_ID, WEB_CRAWL_NAME)), + OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), OafMapperUtils .dataInfo( false, null, false, false, @@ -233,7 +234,7 @@ public class CreateActionSetFromWebEntries implements Serializable { ModelConstants.HAS_AUTHOR_INSTITUTION, Arrays .asList( - OafMapperUtils.keyValue(WEB_CRAWL_ID, WEB_CRAWL_NAME)), + OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), OafMapperUtils .dataInfo( false, null, false, false, From d96215cb9b6c03198f723bf9f60261b283cb127a Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 16 Jul 2024 18:17:32 +0200 Subject: [PATCH 076/239] [UnpayWall]added othe : in the identifier construction --- .../scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index c4aa64fd4..27eaa61b5 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -93,7 +93,7 @@ case object Crossref2Oaf { val cf = new KeyValue cf.setValue("UnpayWall") - cf.setKey(s"10|openaire____:${DHPUtils.md5("UnpayWall".toLowerCase)}") + cf.setKey(s"10|openaire____::${DHPUtils.md5("UnpayWall".toLowerCase)}") cf } From f17e1243ba4974942d2ce5062d85b90305320b70 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 17 Jul 2024 10:23:50 +0200 Subject: [PATCH 077/239] reverted changed contens under dhp-graph-provision --- .../oa/provision/SolrAdminApplication.java | 2 + .../model/ProvisionModelSupport.java | 20 +++---- .../oa/provision/SolrConfigExploreTest.java | 54 +++++++++---------- .../dhp/oa/provision/SolrConfigTest.java | 3 +- 4 files changed, 36 insertions(+), 43 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java index 44426e8c5..2bf7d3fbb 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SolrAdminApplication.java @@ -128,12 +128,14 @@ public class SolrAdminApplication implements Closeable { public SolrResponse deleteAlias(String aliasName) throws SolrServerException, IOException { CollectionAdminRequest.DeleteAlias deleteAliasRequest = CollectionAdminRequest.deleteAlias(aliasName); + log.info("deleting alias: {}", aliasName); return deleteAliasRequest.process(solrClient); } public SolrResponse createAlias(String aliasName, String collection) throws IOException, SolrServerException { CollectionAdminRequest.CreateAlias createAliasRequest = CollectionAdminRequest .createAlias(aliasName, collection); + log.info("creating alias: {} for collection: {}", aliasName, collection); return createAliasRequest.process(solrClient); } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index abcf4992f..b450b95be 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,6 +5,7 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.solr.ExternalReference; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -30,7 +31,6 @@ import eu.dnetlib.dhp.schema.solr.Context; import eu.dnetlib.dhp.schema.solr.Country; import eu.dnetlib.dhp.schema.solr.Datasource; import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines; -import eu.dnetlib.dhp.schema.solr.ExternalReference; import eu.dnetlib.dhp.schema.solr.Instance; import eu.dnetlib.dhp.schema.solr.Journal; import eu.dnetlib.dhp.schema.solr.Measure; @@ -562,16 +562,10 @@ public class ProvisionModelSupport { .orElse(null); } - private static List mapExternalReference( - List externalReference) { - return Optional - .ofNullable(externalReference) - .map( - ext -> ext - .stream() - .map( - e -> ExternalReference - .newInstance( + private static List mapExternalReference(List externalReference) { + return Optional.ofNullable(externalReference) + .map(ext -> ext.stream() + .map(e -> ExternalReference.newInstance( e.getSitename(), e.getLabel(), e.getAlternateLabel(), @@ -579,8 +573,8 @@ public class ProvisionModelSupport { mapCodeLabel(e.getQualifier()), e.getRefidentifier(), e.getQuery())) - .collect(Collectors.toList())) - .orElse(Lists.newArrayList()); + .collect(Collectors.toList())) + .orElse(Lists.newArrayList()); } private static List asContext(List ctxList, diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java index 90aef5adc..41eac2a30 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigExploreTest.java @@ -1,13 +1,12 @@ package eu.dnetlib.dhp.oa.provision; -import static org.junit.jupiter.api.Assertions.assertEquals; - import java.io.File; import java.io.IOException; import java.net.URI; import java.nio.file.Path; +import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrQuery; @@ -33,13 +32,14 @@ import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.jupiter.api.Assertions.assertEquals; @ExtendWith(MockitoExtension.class) public class SolrConfigExploreTest { @@ -91,7 +91,7 @@ public class SolrConfigExploreTest { SparkConf conf = new SparkConf(); conf.setAppName(XmlIndexingJobTest.class.getSimpleName()); conf.registerKryoClasses(new Class[] { - SerializableSolrInputDocument.class + SerializableSolrInputDocument.class }); conf.setMaster("local[1]"); @@ -101,10 +101,10 @@ public class SolrConfigExploreTest { conf.set("spark.sql.warehouse.dir", workingDir.resolve("spark").toString()); spark = SparkSession - .builder() - .appName(SolrConfigExploreTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); + .builder() + .appName(SolrConfigExploreTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); // random unassigned HTTP port final int jettyPort = 0; @@ -134,35 +134,35 @@ public class SolrConfigExploreTest { log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); NamedList res = createCollection( - miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), SHADOW_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); // miniCluster.getSolrClient().setDefaultCollection(SHADOW_COLLECTION); res = createCollection( - miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); + miniCluster.getSolrClient(), PUBLIC_COLLECTION, 4, 2, 20, CONFIG_NAME); res.forEach(o -> log.info(o.toString())); admin = new SolrAdminApplication(miniCluster.getZkClient().getZkServerAddress()); CollectionAdminResponse rsp = (CollectionAdminResponse) admin - .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); + .createAlias(ProvisionConstants.PUBLIC_ALIAS_NAME, PUBLIC_COLLECTION); assertEquals(0, rsp.getStatus()); rsp = (CollectionAdminResponse) admin.createAlias(ProvisionConstants.SHADOW_ALIAS_NAME, SHADOW_COLLECTION); assertEquals(0, rsp.getStatus()); log - .info( - CollectionAdminRequest.ClusterStatus - .getClusterStatus() - .process(miniCluster.getSolrClient()) - .toString()); + .info( + CollectionAdminRequest.ClusterStatus + .getClusterStatus() + .process(miniCluster.getSolrClient()) + .toString()); } @@ -180,8 +180,7 @@ public class SolrConfigExploreTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions - .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", @@ -201,8 +200,7 @@ public class SolrConfigExploreTest { // System.out.println(rsp.getExplainMap()); for (SolrDocument doc : rsp.getResults()) { - log - .info( + log.info( doc.get("score") + "\t" + doc.get("__indexrecordidentifier") + "\t" + doc.get("resultidentifier") + "\t" + @@ -218,7 +216,7 @@ public class SolrConfigExploreTest { } protected static NamedList createCollection(CloudSolrClient client, String name, int numShards, - int replicationFactor, int maxShardsPerNode, String configName) throws Exception { + int replicationFactor, int maxShardsPerNode, String configName) throws Exception { ModifiableSolrParams modParams = new ModifiableSolrParams(); modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name()); modParams.set("name", name); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java index c04fa1cc6..2c62389c6 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/SolrConfigTest.java @@ -85,8 +85,7 @@ public class SolrConfigTest extends SolrTest { new XmlIndexingJob(spark, inputPath, SHADOW_FORMAT, ProvisionConstants.SHADOW_ALIAS_NAME, batchSize) .run(isLookupClient); - Assertions - .assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); + Assertions.assertEquals(0, miniCluster.getSolrClient().commit(ProvisionConstants.SHADOW_ALIAS_NAME).getStatus()); String[] queryStrings = { "cancer", From 78b5e4bb6f44be063fdbf9a49ac966fb4b17353f Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 17 Jul 2024 10:48:20 +0200 Subject: [PATCH 078/239] reverted changed contens under dhp-graph-provision --- .../oa/provision/utils/XmlRecordFactory.java | 19 ++++----- .../oa/provision/XmlRecordFactoryTest.java | 41 +++++++++++++++++-- .../dnetlib/dhp/oa/provision/project_aka.json | 1 + 3 files changed, 47 insertions(+), 14 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 3d89b1e34..899dad221 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -219,6 +219,13 @@ public class XmlRecordFactory implements Serializable { if (entity.getMeasures() != null) { metadata.addAll(measuresAsXml(entity.getMeasures())); } + if (entity.getContext() != null) { + contexts.addAll(entity.getContext().stream().map(Context::getId).collect(Collectors.toList())); + /* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */ + if (contexts.contains("dh-ch::subcommunity::2")) { + contexts.add("clarin"); + } + } if (ModelSupport.isResult(type)) { final Result r = (Result) entity; @@ -245,14 +252,6 @@ public class XmlRecordFactory implements Serializable { .collect(Collectors.toList())); } - if (r.getContext() != null) { - contexts.addAll(r.getContext().stream().map(c -> c.getId()).collect(Collectors.toList())); - /* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */ - if (contexts.contains("dh-ch::subcommunity::2")) { - contexts.add("clarin"); - } - } - if (r.getTitle() != null) { metadata .addAll( @@ -1603,9 +1602,7 @@ public class XmlRecordFactory implements Serializable { private List buildContexts(final String type, final Set contexts) { final List res = Lists.newArrayList(); - if (contextMapper != null - && !contextMapper.isEmpty() - && MainEntityType.result.toString().equals(type)) { + if (contextMapper != null && !contextMapper.isEmpty()) { XMLTag document = XMLDoc.newDocument(true).addRoot("contextRoot"); diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java index d617991a1..ab4301f9a 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java @@ -1,8 +1,7 @@ package eu.dnetlib.dhp.oa.provision; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.*; import java.io.IOException; import java.io.StringReader; @@ -22,6 +21,7 @@ import com.google.common.collect.Lists; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; +import eu.dnetlib.dhp.oa.provision.utils.ContextDef; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; import eu.dnetlib.dhp.schema.oaf.*; @@ -51,7 +51,7 @@ public class XmlRecordFactoryTest { assertNotNull(doc); - // System.out.println(doc.asXML()); + System.out.println(doc.asXML()); assertEquals("0000-0001-9613-6638", doc.valueOf("//creator[@rank = '1']/@orcid")); assertEquals("0000-0001-9613-6639", doc.valueOf("//creator[@rank = '1']/@orcid_pending")); @@ -267,4 +267,39 @@ public class XmlRecordFactoryTest { } + @Test + public void test_AKA_project() throws DocumentException, IOException { + final ContextMapper contextMapper = new ContextMapper(); + + contextMapper + .put("dh-ch", new ContextDef("dh-ch", "Digital Humanities and Cultural Heritage", "context", "community")); + contextMapper.put("dh-ch::projects", new ContextDef("dh-ch::projects", "DH-CH Projects", "category", "")); + contextMapper + .put("dh-ch::projects::2", new ContextDef("dh-ch::projects::2", "ARIADNE", "concept", "community")); + + final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, + PayloadConverterJob.schemaLocation); + + final Project p = OBJECT_MAPPER + .readValue( + IOUtils.toString(getClass().getResourceAsStream("project_aka.json")), + Project.class); + + assertNotNull(p.getContext()); + assertEquals(1, p.getContext().size()); + assertEquals("dh-ch::projects::2", p.getContext().get(0).getId()); + + final String xml = xmlRecordFactory.build(new JoinedEntity(p)); + + assertNotNull(xml); + + final Document doc = new SAXReader().read(new StringReader(xml)); + + assertNotNull(doc); + + assertEquals("dh-ch", doc.valueOf("//context/@id")); + assertEquals("dh-ch::projects", doc.valueOf("//context/category/@id")); + assertEquals("dh-ch::projects::2", doc.valueOf("//context/category/concept/@id")); + } + } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json new file mode 100644 index 000000000..18bd5adf4 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/project_aka.json @@ -0,0 +1 @@ +{"context" : [{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"bulktagging","inferred":true,"invisible":false,"provenanceaction":{"classid":"bulktagging:community:subject","classname":"Bulk Tagging for Communities","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"}],"id":"dh-ch::projects::2"}], "callidentifier":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Virkapäätöksiin liittyvä yleiskustannusosuus KY"},"code":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"204684"},"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"currency":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"EUR"},"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2019-01-25","dateoftransformation":"2022-02-08","duration":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0"},"ecarticle29_3":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsc39":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"enddate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2002-12-31"},"extraInfo":[],"fundedamount":3230.0,"fundingtree":[{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"\n aka_________::AKA\n AKA\n Academy of Finland\n Academy of Finland\n FI\n "}],"id":"40|aka_________::00f0012ac67a2f826f2e98dbdfd6b058","jsonextrainfo":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"{}"},"lastupdatetimestamp":1719997513126,"oamandatepublications":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"optional1":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"3,230 €"},"originalId":["aka_________::204684"],"pid":[],"startdate":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"2002-12-01"},"subjects":[],"title":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Tutkijan virkaan liittyvä yleiskustannusosuus suorituspaikalle"},"totalcost":0.0} \ No newline at end of file From 6c98d692159f992c5018b3cc0a29381468b4a296 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 17 Jul 2024 11:09:37 +0200 Subject: [PATCH 079/239] reverted changed contens under dhp-pace-core --- .../src/main/java/eu/dnetlib/pace/tree/CountryMatch.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java index c02381983..96a87c455 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CountryMatch.java @@ -2,6 +2,7 @@ package eu.dnetlib.pace.tree; import java.util.Map; +import java.util.Set; import com.wcohen.ss.AbstractStringDistance; @@ -12,8 +13,11 @@ import eu.dnetlib.pace.tree.support.ComparatorClass; @ComparatorClass("countryMatch") public class CountryMatch extends AbstractStringComparator { + private Map params; + public CountryMatch(Map params) { super(params, new com.wcohen.ss.JaroWinkler()); + this.params = params; } public CountryMatch(final double weight) { @@ -26,6 +30,7 @@ public class CountryMatch extends AbstractStringComparator { @Override public double distance(final String a, final String b, final Config conf) { + if (a.isEmpty() || b.isEmpty()) { return -1.0; // return -1 if a field is missing } @@ -45,4 +50,5 @@ public class CountryMatch extends AbstractStringComparator { protected double normalize(final double d) { return d; } + } From db9c54c944d704f3e775093cda4f4b0b5574b970 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 17 Jul 2024 11:27:43 +0200 Subject: [PATCH 080/239] Revert "removed legacy actionmanager dependencies" This reverts commit bb12d0b4df50919fea19d5a4d25dcb39718fdfb1. --- dhp-workflows/dhp-actionmanager/pom.xml | 4 +- .../dnetlib/dhp/actionmanager/ISClient.java | 29 +- dhp-workflows/dhp-swh/pom.xml | 43 + pom.xml | 2029 +++++++++-------- 4 files changed, 1168 insertions(+), 937 deletions(-) diff --git a/dhp-workflows/dhp-actionmanager/pom.xml b/dhp-workflows/dhp-actionmanager/pom.xml index 1ef55a060..ce13502b6 100644 --- a/dhp-workflows/dhp-actionmanager/pom.xml +++ b/dhp-workflows/dhp-actionmanager/pom.xml @@ -50,12 +50,11 @@ org.apache.hadoop hadoop-distcp - 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 24d50029c..088e618c7 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 @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.actionmanager; import java.io.Serializable; import java.io.StringReader; import java.util.List; +import java.util.NoSuchElementException; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -21,14 +22,13 @@ import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import eu.dnetlib.actionmanager.rmi.ActionManagerException; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; public class ISClient implements Serializable { - private static final long serialVersionUID = 4632443200867340872L; - private static final Logger log = LoggerFactory.getLogger(ISClient.class); private static final String INPUT_ACTION_SET_ID_SEPARATOR = ","; @@ -65,7 +65,7 @@ public class ISClient implements Serializable { .map(t -> buildDirectory(basePath, t)) .collect(Collectors.toList())) .orElseThrow(() -> new IllegalStateException("empty set list")); - } catch (ISLookUpException e) { + } catch (ActionManagerException | ISLookUpException e) { throw new IllegalStateException("unable to query ActionSets info from the IS"); } } @@ -89,18 +89,31 @@ public class ISClient implements Serializable { return Joiner.on("/").join(basePath, t.getMiddle(), t.getRight()); } - private String getBasePathHDFS(ISLookUpService isLookup) throws ISLookUpException { + private String getBasePathHDFS(ISLookUpService isLookup) throws ActionManagerException { return queryServiceProperty(isLookup, "basePath"); } private String queryServiceProperty(ISLookUpService isLookup, final String propertyName) - throws ISLookUpException { + 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); - - final List value = isLookup.quickSearchProfile(q); - return Iterables.getOnlyElement(value); + 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-swh/pom.xml b/dhp-workflows/dhp-swh/pom.xml index ef9049b20..80fff4587 100644 --- a/dhp-workflows/dhp-swh/pom.xml +++ b/dhp-workflows/dhp-swh/pom.xml @@ -51,6 +51,49 @@ hadoop-distcp + + eu.dnetlib + dnet-actionmanager-api + + + eu.dnetlib + dnet-actionmanager-common + + + eu.dnetlib + dnet-openaireplus-mapping-utils + + + saxonica + saxon + + + saxonica + saxon-dom + + + jgrapht + jgrapht + + + net.sf.ehcache + ehcache + + + org.springframework + spring-test + + + org.apache.* + * + + + apache + * + + + + org.apache.httpcomponents httpclient diff --git a/pom.xml b/pom.xml index 47b7d5f38..a5bc6cedd 100644 --- a/pom.xml +++ b/pom.xml @@ -1,930 +1,1107 @@ - 4.0.0 - eu.dnetlib.dhp - dhp - 1.2.5-SNAPSHOT - pom - - - - GNU Affero General Public License v3.0 or later - https://spdx.org/licenses/AGPL-3.0-or-later.html#licenseText - repo - This program is free software: you can redistribute it and/or modify it under the terms of the - GNU Affero General Public License as published by the Free Software Foundation, either version 3 of the - License, or (at your option) any later version. - - - - - dhp-build - dhp-pace-core - dhp-common - dhp-workflows - - - - Redmine - https://support.openaire.eu/projects/openaire - - - - jenkins - https://jenkins-dnet.d4science.org/ - - - - scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git - scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git - https://code-repo.d4science.org/D-Net/dnet-hadoop/ - HEAD - - - This module is the root descriptor for the dnet-hadoop project - - - - - - - dnet45-releases - D-Net 45 releases - https://maven.d4science.org/nexus/content/repositories/dnet45-releases - default - - false - - - true - - - - dnet45-snapshots - D-Net 45 snapshots - https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots - default - - true - - - false - - - - dnet45-bootstrap-snapshot - D-Net 45 Bootstrap Snapshot - https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-snapshot/ - - false - - - true - - default - - - dnet45-bootstrap-release - D-Net 45 Bootstrap Release - https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-release/ - - true - - - false - - default - - - cloudera - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos - - true - - - false - - - - dnet-deps - dnet-dependencies - https://maven.d4science.org/nexus/content/repositories/dnet-deps - default - - - maven-restlet - Restlet repository - https://maven.restlet.talend.com - - - conjars - conjars - https://conjars.wensel.net/repo/ - - - - - - org.junit.jupiter - junit-jupiter - ${junit-jupiter.version} - test - - - - org.mockito - mockito-core - ${mockito-core.version} - test - - - - org.mockito - mockito-junit-jupiter - ${mockito-core.version} - test - - - - - - - - eu.dnetlib.dhp - ${dhp-schemas.artifact} - ${dhp-schemas.version} - - - org.apache.hadoop - hadoop-hdfs - ${dhp.hadoop.version} - provided - - - org.apache.hadoop - hadoop-common - ${dhp.hadoop.version} - provided - - - org.apache.hadoop - hadoop-client - ${dhp.hadoop.version} - provided - - - org.apache.hadoop - hadoop-distcp - ${dhp.hadoop.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${dhp.spark.version} - provided - - - org.apache.spark - spark-sql_${scala.binary.version} - ${dhp.spark.version} - provided - - - org.apache.spark - spark-graphx_${scala.binary.version} - ${dhp.spark.version} - provided - - - org.apache.spark - spark-hive_${scala.binary.version} - ${dhp.spark.version} - test - - - - org.slf4j - jcl-over-slf4j - 1.7.25 - provided - - - - org.apache.commons - commons-lang3 - ${dhp.commons.lang.version} - - - - commons-validator - commons-validator - 1.7 - - - - com.github.sisyphsu - dateparser - 1.0.7 - - - - me.xuender - unidecode - 0.0.7 - - - - com.google.guava - guava - ${dhp.guava.version} - - - - - commons-codec - commons-codec - 1.9 - - - - commons-io - commons-io - 2.4 - - - - commons-cli - commons-cli - 1.2 - provided - - - - net.sf.saxon - Saxon-HE - 9.9.1-6 - - - - dom4j - dom4j - 1.6.1 - - - - xml-apis - xml-apis - 1.4.01 - - - - jaxen - jaxen - 1.1.6 - - - - com.mycila.xmltool - xmltool - 3.3 - - - - org.apache.solr - solr-solrj - ${solr.version} - - - * - * - - - - - com.lucidworks.spark - spark-solr - ${sparksolr.version} - - - * - * - - - - - org.apache.solr - solr-test-framework - ${solr.version} - test - - - io.dropwizard.metrics - metrics-core - 3.2.6 - test - - - - - org.apache.httpcomponents - httpclient - ${org.apache.httpcomponents.version} - - - org.apache.httpcomponents - httpmime - ${org.apache.httpcomponents.version} - - - org.noggit - noggit - 0.8 - - - org.apache.zookeeper - zookeeper - 3.4.11 - - - - net.schmizz - sshj - 0.10.0 - test - - - - com.fasterxml.jackson.core - jackson-core - ${dhp.jackson.version} - provided - - - - com.fasterxml.jackson.core - jackson-annotations - ${dhp.jackson.version} - provided - - - com.fasterxml.jackson.core - jackson-databind - ${dhp.jackson.version} - provided - - - - eu.dnetlib - cnr-rmi-api - ${cnr-rmi-api.version} - - - - eu.dnetlib.dhp - dnet-openaire-broker-common - ${dnet-openaire-broker-common.version} - - - - org.apache.cxf - cxf-rt-transports-http - 3.1.5 - - - javax.persistence - javax.persistence-api - 2.2 - provided - - - - com.jayway.jsonpath - json-path - 2.4.0 - - - com.arakelian - java-jq - 0.10.1 - - - edu.cmu - secondstring - 1.0.0 - - - org.mongodb - mongo-java-driver - ${mongodb.driver.version} - - - io.fares.junit.mongodb - mongodb-junit-test - 1.1.0 - - - org.postgresql - postgresql - 42.2.10 - - - - org.antlr - stringtemplate - 3.2.1 - - - - org.antlr - ST4 - 4.3.4 - - - - com.ximpleware - vtd-xml - ${vtd.version} - - - - org.elasticsearch - elasticsearch-hadoop - 7.6.0 - - - - - org.apache.oozie - oozie-client - ${dhp.oozie.version} - provided - - - - slf4j-simple - org.slf4j - - - - - - - com.squareup.okhttp3 - okhttp - ${okhttp.version} - - - - org.apache.commons - commons-compress - ${common.compress.version} - - - - - org.apache.commons - commons-csv - ${common.csv.version} - - - - - - org.apache.poi - poi-ooxml - ${apache.poi.version} - - - - org.json - json - 20180813 - - - - org.json4s - json4s-jackson_${scala.binary.version} - ${json4s.version} - - - - com.github.victools - jsonschema-generator - ${jsonschemagenerator.version} - - - - org.apache.commons - commons-text - ${common.text.version} - - - - com.opencsv - opencsv - 5.5 - - - io.github.classgraph - classgraph - 4.8.71 - - - - com.fasterxml.jackson.dataformat - jackson-dataformat-xml - ${jackson.version} - provided - - - com.fasterxml.jackson.module - jackson-module-jsonSchema - ${jackson.version} - provided - - - - - org.apache.commons - commons-math3 - 3.6.1 - - - - - com.google.code.gson - gson - ${google.gson.version} - - - - commons-collections - commons-collections - ${commons.collections.version} - - - commons-logging - commons-logging - ${commons.logging.version} - - - - org.reflections - reflections - 0.9.10 - - - - org.scala-lang - scala-library - ${scala.version} - - - - com.ibm.icu - icu4j - 70.1 - - - - - - target - target/classes - ${project.artifactId}-${project.version} - target/test-classes - - - - org.apache.maven.plugins - maven-plugin-plugin - 3.3 - - - org.apache.maven.plugins - maven-project-info-reports-plugin - 3.0.0 - - - org.apache.maven.plugins - maven-site-plugin - 3.9.1 - - ${dhp.site.skip} - - - - - org.apache.maven.plugins - maven-compiler-plugin - ${maven.compiler.plugin.version} - - 1.8 - 1.8 - ${project.build.sourceEncoding} - - - - - org.apache.maven.plugins - maven-jar-plugin - 3.0.2 - - - - org.apache.maven.plugins - maven-source-plugin - 3.0.1 - - - attach-sources - verify - - jar-no-fork - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 3.0.0-M4 - - true - - - - org.apache.maven.plugins - maven-javadoc-plugin - 3.2.0 - - true - none - - - - org.apache.maven.plugins - maven-dependency-plugin - 3.6.0 - - - - net.revelc.code.formatter - formatter-maven-plugin - 2.11.0 - - - eu.dnetlib.dhp - dhp-code-style - ${project.version} - - - - - org.antipathy - mvn-scalafmt_${scala.binary.version} - 1.0.1640073709.733712b - - - eu.dnetlib.dhp - dhp-code-style - ${project.version} - - - - - - - - org.apache.maven.plugins - maven-site-plugin - - - org.apache.maven.plugins - maven-project-info-reports-plugin - - - net.revelc.code.formatter - formatter-maven-plugin - - - - format - - - eclipse/formatter_dnet.xml - - - - - - net.revelc.code - impsort-maven-plugin - 1.4.1 - - java.,javax.,org.,com. - java,* - - **/thrift/*.java - - - - - sort-imports - - sort - - - - - - org.antipathy - mvn-scalafmt_${scala.binary.version} - - https://code-repo.d4science.org/D-Net/dnet-hadoop/raw/branch/beta/dhp-build/dhp-code-style/src/main/resources/scalafmt/scalafmt.conf - false - false - - ${project.basedir}/src/main/scala - - - ${project.basedir}/src/test/scala - - false - false - : git rev-parse --abbrev-ref HEAD - false - - - - validate - - format - - - - - - org.apache.maven.plugins - maven-release-plugin - 2.5.3 - - - org.jacoco - jacoco-maven-plugin - 0.7.9 - - - **/schemas/* - **/com/cloudera/**/* - **/org/apache/avro/io/**/* - - - - - default-prepare-agent - - prepare-agent - - - - default-report - prepare-package - - report - - - - - - - - - - org.apache.maven.wagon - wagon-ssh - 2.10 - - - - - - dnet45-snapshots - DNet45 Snapshots - https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots - default - - - dnet45-releases - https://maven.d4science.org/nexus/content/repositories/dnet45-releases - - - DHPSite - ${dhp.site.stage.path}/ - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - true - none - - - - - - - sftp://dnet-hadoop@static-web.d4science.org/dnet-hadoop - UTF-8 - UTF-8 - 3.6.0 - 1.8 - 1.8 - 2.22.2 - 2.0.1 - cdh5.9.2 - 2.6.0-${dhp.cdh.version} - 4.1.0-${dhp.cdh.version} - dhp-schemas - 3.6.0 - 2.4.0.cloudera2 - 2.9.6 - 3.5 - true - 11.0.2 - 2.11.12 - 2.11 - 1.3.0 - 5.6.1 - 3.3.3 - 3.4.2 - [2.12,3.0) - [6.1.3] - [3.1.6] - [2.6.1] - 7.5.0 - 4.7.2 - 1.20 - 3.5.3 - 4.13.0 - 1.8 - 4.1.2 - 1.8 - 4.5.3 - 4.0.1 - 2.2.2 - 1.1.3 - 3.2.1 - - - - - - scala-2.12 - - 2.12 - 2.12.18 - - - - 4.0.2 - 3.4.1 - 2.14.2 - 3.12.0 - 3.7.0-M11 - 4.8.1 - - - - - - - - arm-silicon-mac - - - aarch64 - mac - - - - - - org.xerial.snappy - snappy-java - 1.1.8.4 - - - - + 4.0.0 + eu.dnetlib.dhp + dhp + 1.2.5-SNAPSHOT + pom + + + + GNU Affero General Public License v3.0 or later + https://spdx.org/licenses/AGPL-3.0-or-later.html#licenseText + repo + This program is free software: you can redistribute it and/or modify it under the terms of the + GNU Affero General Public License as published by the Free Software Foundation, either version 3 of the + License, or (at your option) any later version. + + + + + + dhp-build + dhp-pace-core + dhp-common + dhp-workflows + dhp-shade-package + + + + Redmine + https://support.openaire.eu/projects/openaire + + + + jenkins + https://jenkins-dnet.d4science.org/ + + + + scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git + scm:git:gitea@code-repo.d4science.org:D-Net/dnet-hadoop.git + https://code-repo.d4science.org/D-Net/dnet-hadoop/ + HEAD + + + This module is the root descriptor for the dnet-hadoop project + + + + + + + + Openaire-third-parties-snaphot + Openaire third parties Snapshot + https://maven.d4science.org/nexus/content/repositories/Openaire-third-parties-snaphot/ + + false + + + true + + + + + dnet45-releases + D-Net 45 releases + https://maven.d4science.org/nexus/content/repositories/dnet45-releases + default + + false + + + true + + + + dnet45-snapshots + D-Net 45 snapshots + https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots + default + + true + + + false + + + + dnet45-bootstrap-snapshot + D-Net 45 Bootstrap Snapshot + https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-snapshot/ + + false + + + true + + default + + + dnet45-bootstrap-release + D-Net 45 Bootstrap Release + https://maven.d4science.org/nexus/content/repositories/dnet45-bootstrap-release/ + + true + + + false + + default + + + cloudera + Cloudera Repository + https://repository.cloudera.com/artifactory/cloudera-repos + + true + + + false + + + + dnet-deps + dnet-dependencies + https://maven.d4science.org/nexus/content/repositories/dnet-deps + default + + + maven-restlet + Restlet repository + https://maven.restlet.talend.com + + + conjars + conjars + https://conjars.wensel.net/repo/ + + + + + + + org.projectlombok + lombok + 1.18.28 + provided + + + org.junit.jupiter + junit-jupiter + ${junit-jupiter.version} + test + + + + org.mockito + mockito-core + ${mockito-core.version} + test + + + + org.mockito + mockito-junit-jupiter + ${mockito-core.version} + test + + + + + + + + eu.dnetlib.dhp + dhp-schemas + ${dhp-schemas.version} + + + org.apache.hadoop + hadoop-hdfs + ${dhp.hadoop.version} + provided + + + org.apache.hadoop + hadoop-common + ${dhp.hadoop.version} + provided + + + org.apache.hadoop + hadoop-client + ${dhp.hadoop.version} + provided + + + org.apache.hadoop + hadoop-distcp + ${dhp.hadoop.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${dhp.spark.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${dhp.spark.version} + provided + + + org.apache.spark + spark-graphx_${scala.binary.version} + ${dhp.spark.version} + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + ${dhp.spark.version} + test + + + + org.slf4j + slf4j-api + ${org.slf4j.version} + provided + + + + org.slf4j + slf4j-log4j12 + ${org.slf4j.version} + provided + + + + org.slf4j + jcl-over-slf4j + ${org.slf4j.version} + provided + + + + org.apache.logging.log4j + log4j-slf4j2-impl + ${log4j.version} + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.apache.commons + commons-lang3 + ${dhp.commons.lang.version} + + + + org.apache.commons + commons-beanutils + ${commons-beanutils.version} + + + + + commons-validator + commons-validator + ${commons-validator.version} + + + + com.github.sisyphsu + dateparser + ${dateparser.version} + + + + me.xuender + unidecode + ${unidecode.version} + + + + com.google.guava + guava + ${dhp.guava.version} + + + + + commons-codec + commons-codec + ${commons-codec.version} + + + + commons-io + commons-io + ${commons-io.version} + + + + commons-cli + commons-cli + 1.2 + provided + + + + net.sf.saxon + Saxon-HE + 9.9.1-6 + + + + dom4j + dom4j + 1.6.1 + + + + xml-apis + xml-apis + 1.4.01 + + + + jaxen + jaxen + 1.1.6 + + + + com.mycila.xmltool + xmltool + 3.3 + + + + org.apache.solr + solr-solrj + ${solr.version} + + + * + * + + + + + com.lucidworks.spark + spark-solr + ${sparksolr.version} + + + * + * + + + + + org.apache.solr + solr-test-framework + ${solr.version} + test + + + io.dropwizard.metrics + metrics-core + 3.2.6 + test + + + + + org.apache.httpcomponents + httpclient + ${org.apache.httpcomponents.version} + + + org.apache.httpcomponents + httpmime + ${org.apache.httpcomponents.version} + + + org.noggit + noggit + 0.8 + + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + + + + net.schmizz + sshj + 0.10.0 + test + + + + com.fasterxml.jackson.core + jackson-core + ${dhp.jackson.version} + provided + + + + com.fasterxml.jackson.core + jackson-annotations + ${dhp.jackson.version} + provided + + + com.fasterxml.jackson.core + jackson-databind + ${dhp.jackson.version} + provided + + + + eu.dnetlib + dnet-actionmanager-common + ${dnet-actionmanager-common.version} + + + org.apache.hadoop + hadoop-common + + + + + eu.dnetlib + dnet-actionmanager-api + ${dnet-actionmanager-api.version} + + + eu.dnetlib + cnr-misc-utils + + + + + + eu.dnetlib + cnr-rmi-api + ${cnr-rmi-api.version} + + + + eu.dnetlib.dhp + dnet-openaire-broker-common + ${dnet-openaire-broker-common.version} + + + + org.apache.cxf + cxf-rt-transports-http + 3.1.5 + + + + javax.persistence + javax.persistence-api + 2.2 + provided + + + + com.jayway.jsonpath + json-path + 2.4.0 + + + com.arakelian + java-jq + 0.10.1 + + + edu.cmu + secondstring + 1.0.0 + + + org.mongodb + mongo-java-driver + ${mongodb.driver.version} + + + io.fares.junit.mongodb + mongodb-junit-test + 1.1.0 + + + org.postgresql + postgresql + 42.2.10 + + + + org.antlr + stringtemplate + 3.2.1 + + + + org.antlr + ST4 + 4.3.4 + + + + com.ximpleware + vtd-xml + ${vtd.version} + + + + org.elasticsearch + elasticsearch-hadoop + 7.6.0 + + + + + org.apache.oozie + oozie-client + ${dhp.oozie.version} + provided + + + + slf4j-simple + org.slf4j + + + + + + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + + + org.apache.commons + commons-compress + ${common.compress.version} + + + org.apache.commons + commons-csv + ${common.csv.version} + + + org.apache.poi + poi-ooxml + ${apache.poi.version} + + + + org.json + json + 20180813 + + + + org.json4s + json4s-jackson_${scala.binary.version} + ${json4s.version} + + + + com.github.victools + jsonschema-generator + ${jsonschemagenerator.version} + + + + org.apache.commons + commons-text + ${common.text.version} + + + + com.opencsv + opencsv + 5.5 + + + io.github.classgraph + classgraph + 4.8.71 + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-xml + ${jackson.version} + provided + + + com.fasterxml.jackson.module + jackson-module-jsonSchema + ${jackson.version} + provided + + + + org.apache.commons + commons-math3 + 3.6.1 + + + + com.google.code.gson + gson + ${google.gson.version} + + + + commons-collections + commons-collections + ${commons.collections.version} + + + commons-logging + commons-logging + ${commons.logging.version} + + + + org.reflections + reflections + ${reflections.version} + + + + org.scala-lang + scala-library + ${scala.version} + + + + com.ibm.icu + icu4j + 70.1 + + + + org.javassist + javassist + ${javassist.version} + + + + + + target + target/classes + ${project.artifactId}-${project.version} + target/test-classes + + + + org.apache.maven.plugins + maven-plugin-plugin + 3.3 + + + org.apache.maven.plugins + maven-project-info-reports-plugin + 3.0.0 + + + org.apache.maven.plugins + maven-site-plugin + 3.9.1 + + ${dhp.site.skip} + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven.compiler.plugin.version} + + 1.8 + 1.8 + ${project.build.sourceEncoding} + + + + + org.apache.maven.plugins + maven-jar-plugin + 3.0.2 + + + + org.apache.maven.plugins + maven-source-plugin + 3.0.1 + + + attach-sources + verify + + jar-no-fork + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M4 + + true + false + + + + org.apache.maven.plugins + maven-javadoc-plugin + 3.2.0 + + true + none + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.6.0 + + + + net.revelc.code.formatter + formatter-maven-plugin + 2.11.0 + + + eu.dnetlib.dhp + dhp-code-style + ${project.version} + + + + + org.antipathy + mvn-scalafmt_${scala.binary.version} + 1.0.1640073709.733712b + + + eu.dnetlib.dhp + dhp-code-style + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-site-plugin + + + org.apache.maven.plugins + maven-project-info-reports-plugin + + + net.revelc.code.formatter + formatter-maven-plugin + + + + format + + + eclipse/formatter_dnet.xml + + + + + + net.revelc.code + impsort-maven-plugin + 1.6.2 + + java.,javax.,org.,com. + java,* + + **/thrift/*.java + + + + + sort-imports + + sort + + + + + + org.antipathy + mvn-scalafmt_${scala.binary.version} + + + https://code-repo.d4science.org/D-Net/dnet-hadoop/raw/branch/beta/dhp-build/dhp-code-style/src/main/resources/scalafmt/scalafmt.conf + + false + false + + ${project.basedir}/src/main/scala + + + ${project.basedir}/src/test/scala + + false + false + : git rev-parse --abbrev-ref HEAD + false + + + + validate + + format + + + + + + org.apache.maven.plugins + maven-release-plugin + 2.5.3 + + + org.jacoco + jacoco-maven-plugin + 0.8.10 + + + **/schemas/* + **/com/cloudera/**/* + **/org/apache/avro/io/**/* + + + + + default-prepare-agent + + prepare-agent + + + + default-report + prepare-package + + report + + + + + + + + + + org.apache.maven.wagon + wagon-ssh + 2.10 + + + + + + dnet45-snapshots + DNet45 Snapshots + https://maven.d4science.org/nexus/content/repositories/dnet45-snapshots + default + + + dnet45-releases + https://maven.d4science.org/nexus/content/repositories/dnet45-releases + + + DHPSite + ${dhp.site.stage.path}/ + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + none + + + + + + + sftp://dnet-hadoop@static-web.d4science.org/dnet-hadoop + UTF-8 + UTF-8 + 1.8 + 1.8 + + + 2.11.12 + 2.11 + + + 3.6.0 + 2.22.2 + 2.0.1 + 4.0.1 + + + 4.1.2 + [2.6.1] + 1.20 + 1.8 + 1.8 + 1.9.4 + 1.9 + 3.2.1 + 2.4 + 1.1.3 + 1.7 + 1.0.7 + [6.1.4-SNAPSHOT] + cdh5.9.2 + 3.5 + 11.0.2 + 2.6.0-${dhp.cdh.version} + 2.9.6 + 4.1.0-${dhp.cdh.version} + true + 2.4.0.cloudera2 + [4.0.3] + [6.0.5] + [3.1.6] + 2.2.2 + 1.2.17 + 3.19.0-GA + 3.5.3 + 4.13.0 + 5.6.1 + 3.3.3 + 3.4.2 + 4.7.2 + 4.5.3 + 1.7.25 + 0.9.10 + 1.3.0 + 7.5.0 + 3.6.0 + 0.0.7 + [2.12,3.0) + 3.4.6 + + + + + + spark-34 + + 2.12 + 2.12.18 + 1.3.0 + + + 4.8.1 + + + 1.22 + 1.8 + 1.10.0 + 1.9.4 + 1.15 + 3.2.2 + 2.11.0 + 1.1.3 + 1.7 + + 14.0.1 + 8.11.0 + 4.0.4 + 3.4.2.openaire + 2.14.2 + 3.12.0 + 2.19.0 + 3.7.0-M11 + 3.25.0-GA + 4.10.0 + 2.0.6 + 0.10.2 + 3.6.3 + + + + + spark-35 + + 2.12 + 2.12.18 + 1.3.0 + + + 4.8.1 + + + 1.23.0 + 1.8 + 1.10.0 + 1.9.4 + 1.16.0 + 3.2.2 + 2.13.0 + 1.1.3 + 1.7 + + 14.0.1 + 8.11.0 + 4.0.4 + 3.5.1.openaire-SNAPSHOT + 2.15.2 + 3.12.0 + 2.20.0 + 3.7.0-M11 + 3.25.0-GA + 4.10.0 + 2.0.7 + 0.10.2 + 3.6.3 + + + + + java11 + + [11 + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M4 + + + --add-opens=java.base/java.lang=ALL-UNNAMED + --add-opens=java.base/java.lang.invoke=ALL-UNNAMED + --add-opens=java.base/java.lang.reflect=ALL-UNNAMED + --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED + --add-opens=java.base/java.nio=ALL-UNNAMED + --add-opens=java.base/java.util=ALL-UNNAMED + --add-opens=java.base/java.util.concurrent=ALL-UNNAMED + --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + --add-opens=java.base/sun.nio.ch=ALL-UNNAMED + --add-opens=java.base/sun.nio.cs=ALL-UNNAMED + --add-opens=java.base/sun.security.action=ALL-UNNAMED + --add-opens=java.base/sun.util.calendar=ALL-UNNAMED + + true + false + + + + + + \ No newline at end of file From 83327239de91fe0e32902c48cc8cf52bc7eb6433 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 17 Jul 2024 11:58:48 +0200 Subject: [PATCH 081/239] fixed pom definitions, bumped dependency version for the dhp-schema module, removed unnecessary dependencies --- .../pace/tree/support/TreeNodeStats.java | 19 ++-- .../pace/tree/support/TreeProcessor.java | 5 +- .../clustering/ClusteringFunctionTest.java | 2 +- .../pace/comparators/ComparatorTest.java | 2 +- dhp-workflows/dhp-actionmanager/pom.xml | 43 -------- .../dnetlib/dhp/actionmanager/ISClient.java | 27 ++--- .../CreateActionSetFromWebEntries.java | 1 - .../researchfi/ResearchFiCollectorPlugin.java | 10 +- .../plugin/researchfi/ResearchFiIterator.java | 1 + .../ResearchFiCollectorPluginTest.java | 5 +- .../dhp/oa/dedup/DecisionTreeTest.java | 100 +++++++++--------- .../dhp/oa/dedup/jpath/JsonPathTest.java | 15 +-- .../model/ProvisionModelSupport.java | 1 + dhp-workflows/dhp-swh/pom.xml | 43 -------- pom.xml | 27 +---- 15 files changed, 91 insertions(+), 210 deletions(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java index a210c06b2..f6b210a8c 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java @@ -25,10 +25,10 @@ public class TreeNodeStats implements Serializable { } public int fieldsCount() { - if(ignoreUndefined) + if (ignoreUndefined) return this.results.size(); else - return this.results.size() - undefinedCount(); //do not count undefined + return this.results.size() - undefinedCount(); // do not count undefined } public int undefinedCount() { @@ -87,20 +87,17 @@ public class TreeNodeStats implements Serializable { if (fs.getResult() == -1) { if (fs.isCountIfUndefined()) { min = 0.0; - } - else { + } else { min = -1; } - } - else { + } else { min = fs.getResult(); } } } if (ignoreUndefined) { - return min==-1.0? 0.0 : min; - } - else { + return min == -1.0 ? 0.0 : min; + } else { return min; } } @@ -111,7 +108,7 @@ public class TreeNodeStats implements Serializable { if (fieldStats.getResult() >= fieldStats.getThreshold()) return 1.0; } - if (!ignoreUndefined && undefinedCount()>0){ + if (!ignoreUndefined && undefinedCount() > 0) { return -1.0; } else { return 0.0; @@ -124,7 +121,7 @@ public class TreeNodeStats implements Serializable { if (fieldStats.getResult() == -1) { if (fieldStats.isCountIfUndefined()) - return ignoreUndefined? 0.0 : -1.0; + return ignoreUndefined ? 0.0 : -1.0; } else { if (fieldStats.getResult() < fieldStats.getThreshold()) return 0.0; diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java index 8ae5aa591..28b3a82af 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java @@ -43,9 +43,9 @@ public class TreeProcessor { TreeNodeStats stats = currentNode.evaluate(doc1, doc2, config); treeStats.addNodeStats(nextNodeName, stats); - + double finalScore = stats.getFinalScore(currentNode.getAggregation()); - if(finalScore == -1.0) + if (finalScore == -1.0) nextNodeName = currentNode.getUndefined(); else if (finalScore >= currentNode.getThreshold()) { nextNodeName = currentNode.getPositive(); @@ -53,7 +53,6 @@ public class TreeProcessor { nextNodeName = currentNode.getNegative(); } - } while (MatchType.parse(nextNodeName) == MatchType.UNDEFINED); treeStats.setResult(MatchType.parse(nextNodeName)); diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java index 1f1827ab3..e62f742f8 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java @@ -3,12 +3,12 @@ package eu.dnetlib.pace.clustering; import java.util.Map; -import com.mongodb.connection.Cluster; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.mongodb.connection.Cluster; import eu.dnetlib.pace.AbstractPaceTest; import eu.dnetlib.pace.common.AbstractPaceFunctions; diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index fe0dca7f1..c008902c4 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -61,7 +61,7 @@ public class ComparatorTest extends AbstractPaceTest { assertEquals(0.0, codeMatch.distance("testing1 key::1", "testing2 key::2", conf)); // both names with codes (1 same, 1 different) - assertEquals(0.5,codeMatch.distance("key::1 key::2 testing1", "key::1 testing", conf)); + assertEquals(0.5, codeMatch.distance("key::1 key::2 testing1", "key::1 testing", conf)); } diff --git a/dhp-workflows/dhp-actionmanager/pom.xml b/dhp-workflows/dhp-actionmanager/pom.xml index ce13502b6..e0bba29d2 100644 --- a/dhp-workflows/dhp-actionmanager/pom.xml +++ b/dhp-workflows/dhp-actionmanager/pom.xml @@ -51,48 +51,5 @@ hadoop-distcp - - eu.dnetlib - dnet-actionmanager-api - - - eu.dnetlib - dnet-actionmanager-common - - - eu.dnetlib - dnet-openaireplus-mapping-utils - - - saxonica - saxon - - - saxonica - saxon-dom - - - jgrapht - jgrapht - - - net.sf.ehcache - ehcache - - - org.springframework - spring-test - - - org.apache.* - * - - - apache - * - - - - 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 088e618c7..7ae2901e3 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 @@ -4,7 +4,6 @@ package eu.dnetlib.dhp.actionmanager; import java.io.Serializable; import java.io.StringReader; import java.util.List; -import java.util.NoSuchElementException; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -22,7 +21,6 @@ import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import eu.dnetlib.actionmanager.rmi.ActionManagerException; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -65,7 +63,7 @@ public class ISClient implements Serializable { .map(t -> buildDirectory(basePath, t)) .collect(Collectors.toList())) .orElseThrow(() -> new IllegalStateException("empty set list")); - } catch (ActionManagerException | ISLookUpException e) { + } catch (ISLookUpException e) { throw new IllegalStateException("unable to query ActionSets info from the IS"); } } @@ -89,31 +87,18 @@ public class ISClient implements Serializable { return Joiner.on("/").join(basePath, t.getMiddle(), t.getRight()); } - private String getBasePathHDFS(ISLookUpService isLookup) throws ActionManagerException { + private String getBasePathHDFS(ISLookUpService isLookup) throws ISLookUpException { return queryServiceProperty(isLookup, "basePath"); } private String queryServiceProperty(ISLookUpService isLookup, final String propertyName) - throws ActionManagerException { + throws ISLookUpException { 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); - } + + final List value = isLookup.quickSearchProfile(q); + return Iterables.getOnlyElement(value); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index f2a73889f..07668f53b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -106,7 +106,6 @@ public class CreateActionSetFromWebEntries implements Serializable { + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror"))); ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror)); - return ret .iterator(); }, Encoders.bean(Relation.class)) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java index c5961c598..2603ecab1 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPlugin.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.plugin.researchfi; import java.util.ArrayList; @@ -32,7 +33,7 @@ public class ResearchFiCollectorPlugin implements CollectorPlugin { @Override public Stream collect(final ApiDescriptor api, final AggregatorReport report) - throws CollectorException { + throws CollectorException { final String authUrl = api.getParams().get("auth_url"); final String clientId = api.getParams().get("auth_client_id"); @@ -45,7 +46,8 @@ public class ResearchFiCollectorPlugin implements CollectorPlugin { return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iter, Spliterator.ORDERED), false); } - private String authenticate(final String authUrl, final String clientId, final String clientSecret) throws CollectorException { + private String authenticate(final String authUrl, final String clientId, final String clientSecret) + throws CollectorException { try (final CloseableHttpClient client = HttpClients.createDefault()) { final HttpPost req = new HttpPost(authUrl); final List params = new ArrayList<>(); @@ -59,7 +61,9 @@ public class ResearchFiCollectorPlugin implements CollectorPlugin { final String content = IOUtils.toString(response.getEntity().getContent()); final JSONObject obj = new JSONObject(content); final String token = obj.getString("access_token"); - if (StringUtils.isNotBlank(token)) { return token; } + if (StringUtils.isNotBlank(token)) { + return token; + } } } catch (final Throwable e) { log.warn("Error obtaining access token", e); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java index e08823c46..269a89f71 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.plugin.researchfi; import java.util.Iterator; diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java index 47c77796b..3f715c2eb 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiCollectorPluginTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.plugin.researchfi; import java.util.HashSet; @@ -23,7 +24,9 @@ public class ResearchFiCollectorPluginTest { final ApiDescriptor api = new ApiDescriptor(); api.setBaseUrl("https://research.fi/api/rest/v1/funding-decisions?FunderName=AKA&FundingStartYearFrom=2022"); api.setProtocol("research_fi"); - api.getParams().put("auth_url", "https://researchfi-auth.2.rahtiapp.fi/realms/publicapi/protocol/openid-connect/token"); + api + .getParams() + .put("auth_url", "https://researchfi-auth.2.rahtiapp.fi/realms/publicapi/protocol/openid-connect/token"); api.getParams().put("auth_client_id", ""); api.getParams().put("auth_client_secret", ""); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java index 28f2bfc66..6acc65e05 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DecisionTreeTest.java @@ -3,81 +3,83 @@ package eu.dnetlib.dhp.oa.dedup; import java.io.IOException; -import eu.dnetlib.dhp.oa.dedup.SparkOpenorgsDedupTest; -import eu.dnetlib.pace.tree.support.TreeProcessor; -import eu.dnetlib.pace.tree.support.TreeStats; import org.apache.commons.io.IOUtils; import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.platform.commons.util.StringUtils; +import eu.dnetlib.dhp.oa.dedup.SparkOpenorgsDedupTest; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.model.SparkModel; +import eu.dnetlib.pace.tree.support.TreeProcessor; +import eu.dnetlib.pace.tree.support.TreeStats; class DecisionTreeTest { - @Test - void testJPath() throws IOException { + @Test + void testJPath() throws IOException { - DedupConfig conf = DedupConfig - .load(IOUtils.toString(getClass().getResourceAsStream("dedup_conf_organization.json"))); + DedupConfig conf = DedupConfig + .load(IOUtils.toString(getClass().getResourceAsStream("dedup_conf_organization.json"))); - final String org = IOUtils.toString(getClass().getResourceAsStream("organization.json")); + final String org = IOUtils.toString(getClass().getResourceAsStream("organization.json")); - Row row = SparkModel.apply(conf).rowFromJson(org); + Row row = SparkModel.apply(conf).rowFromJson(org); - System.out.println("row = " + row); - Assertions.assertNotNull(row); - Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); + System.out.println("row = " + row); + Assertions.assertNotNull(row); + Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); - System.out.println("row = " + row.getAs("countrytitle")); - } + System.out.println("row = " + row.getAs("countrytitle")); + } - @Test - void jsonToModelTest() throws IOException{ - DedupConfig conf = DedupConfig - .load(IOUtils - .toString( - SparkOpenorgsDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + @Test + void jsonToModelTest() throws IOException { + DedupConfig conf = DedupConfig + .load( + IOUtils + .toString( + SparkOpenorgsDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); - final String org = IOUtils.toString(getClass().getResourceAsStream("organization_example1.json")); + final String org = IOUtils.toString(getClass().getResourceAsStream("organization_example1.json")); - Row row = SparkModel.apply(conf).rowFromJson(org); - // to check that the same parsing returns the same row - Row row1 = SparkModel.apply(conf).rowFromJson(org); + Row row = SparkModel.apply(conf).rowFromJson(org); + // to check that the same parsing returns the same row + Row row1 = SparkModel.apply(conf).rowFromJson(org); - Assertions.assertEquals(row, row1); - System.out.println("row = " + row); - Assertions.assertNotNull(row); - Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); - } + Assertions.assertEquals(row, row1); + System.out.println("row = " + row); + Assertions.assertNotNull(row); + Assertions.assertTrue(StringUtils.isNotBlank(row.getAs("identifier"))); + } - @Test - void organizationDecisionTreeTest() throws Exception { - DedupConfig conf = DedupConfig - .load(IOUtils - .toString( - SparkOpenorgsDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + @Test + void organizationDecisionTreeTest() throws Exception { + DedupConfig conf = DedupConfig + .load( + IOUtils + .toString( + SparkOpenorgsDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); - final String org1 = "{\"eclegalbody\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecresearchorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"SUSF - Universit\\\\u00e9 internationale de floride\"}, \"pid\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"qualifier\": {\"classid\": \"grid\", \"classname\": \"grid\", \"schemename\": \"dnet:pid_types\", \"schemeid\": \"dnet:pid_types\"}, \"value\": \"grid.65456.34\"}], \"websiteurl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"http://www.fiu.edu/\"}, \"ecnutscode\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"logourl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"collectedfrom\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"GRID - Global Research Identifier Database\", \"key\": \"10|openaire____::ff4a008470319a22d9cf3d14af485977\"}], \"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"UNKNOWN\", \"classname\": \"UNKNOWN\", \"schemename\": \"dnet:provenanceActions\", \"schemeid\": \"dnet:provenanceActions\"}, \"inferred\": true, \"inferenceprovenance\": \"dedup-similarity-organization-simple\", \"invisible\": false, \"trust\": \"0.89\"}, \"alternativeNames\": [], \"echighereducation\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"id\": \"20|grid________::f22e08fb7bd544b4355f99bef2c43ad5\", \"eclegalperson\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"lastupdatetimestamp\": 1566902405602, \"ecinternationalorganizationeurinterests\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"dateofcollection\": \"\", \"dateoftransformation\": \"\", \"ecnonprofit\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecenterprise\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecinternationalorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalshortname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"FIU\"}, \"country\": {\"classid\": \"US\", \"classname\": \"United States\", \"schemename\": \"dnet:countries\", \"schemeid\": \"dnet:countries\"}, \"extraInfo\": [], \"originalId\": [], \"ecsmevalidated\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}}"; - final String org2 = "{\"eclegalbody\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecresearchorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"SUSF - Universidad Internacional de Florida\"}, \"pid\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"qualifier\": {\"classid\": \"grid\", \"classname\": \"grid\", \"schemename\": \"dnet:pid_types\", \"schemeid\": \"dnet:pid_types\"}, \"value\": \"grid.65456.34\"}], \"websiteurl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"http://www.fiu.edu/\"}, \"ecnutscode\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"logourl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"collectedfrom\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"GRID - Global Research Identifier Database\", \"key\": \"10|openaire____::ff4a008470319a22d9cf3d14af485977\"}], \"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"UNKNOWN\", \"classname\": \"UNKNOWN\", \"schemename\": \"dnet:provenanceActions\", \"schemeid\": \"dnet:provenanceActions\"}, \"inferred\": true, \"inferenceprovenance\": \"dedup-similarity-organization-simple\", \"invisible\": false, \"trust\": \"0.89\"}, \"alternativeNames\": [], \"echighereducation\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"id\": \"20|grid________::2b261e9d8c2a63abbfd5826918c23b6d\", \"eclegalperson\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"lastupdatetimestamp\": 1566902405602, \"ecinternationalorganizationeurinterests\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"dateofcollection\": \"\", \"dateoftransformation\": \"\", \"ecnonprofit\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecenterprise\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecinternationalorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalshortname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"FIU\"}, \"country\": {\"classid\": \"US\", \"classname\": \"United States\", \"schemename\": \"dnet:countries\", \"schemeid\": \"dnet:countries\"}, \"extraInfo\": [], \"originalId\": [], \"ecsmevalidated\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}}"; + final String org1 = "{\"eclegalbody\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecresearchorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"SUSF - Universit\\\\u00e9 internationale de floride\"}, \"pid\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"qualifier\": {\"classid\": \"grid\", \"classname\": \"grid\", \"schemename\": \"dnet:pid_types\", \"schemeid\": \"dnet:pid_types\"}, \"value\": \"grid.65456.34\"}], \"websiteurl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"http://www.fiu.edu/\"}, \"ecnutscode\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"logourl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"collectedfrom\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"GRID - Global Research Identifier Database\", \"key\": \"10|openaire____::ff4a008470319a22d9cf3d14af485977\"}], \"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"UNKNOWN\", \"classname\": \"UNKNOWN\", \"schemename\": \"dnet:provenanceActions\", \"schemeid\": \"dnet:provenanceActions\"}, \"inferred\": true, \"inferenceprovenance\": \"dedup-similarity-organization-simple\", \"invisible\": false, \"trust\": \"0.89\"}, \"alternativeNames\": [], \"echighereducation\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"id\": \"20|grid________::f22e08fb7bd544b4355f99bef2c43ad5\", \"eclegalperson\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"lastupdatetimestamp\": 1566902405602, \"ecinternationalorganizationeurinterests\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"dateofcollection\": \"\", \"dateoftransformation\": \"\", \"ecnonprofit\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecenterprise\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecinternationalorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalshortname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"FIU\"}, \"country\": {\"classid\": \"US\", \"classname\": \"United States\", \"schemename\": \"dnet:countries\", \"schemeid\": \"dnet:countries\"}, \"extraInfo\": [], \"originalId\": [], \"ecsmevalidated\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}}"; + final String org2 = "{\"eclegalbody\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecresearchorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"SUSF - Universidad Internacional de Florida\"}, \"pid\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"qualifier\": {\"classid\": \"grid\", \"classname\": \"grid\", \"schemename\": \"dnet:pid_types\", \"schemeid\": \"dnet:pid_types\"}, \"value\": \"grid.65456.34\"}], \"websiteurl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"http://www.fiu.edu/\"}, \"ecnutscode\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"logourl\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"collectedfrom\": [{\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"GRID - Global Research Identifier Database\", \"key\": \"10|openaire____::ff4a008470319a22d9cf3d14af485977\"}], \"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"UNKNOWN\", \"classname\": \"UNKNOWN\", \"schemename\": \"dnet:provenanceActions\", \"schemeid\": \"dnet:provenanceActions\"}, \"inferred\": true, \"inferenceprovenance\": \"dedup-similarity-organization-simple\", \"invisible\": false, \"trust\": \"0.89\"}, \"alternativeNames\": [], \"echighereducation\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"id\": \"20|grid________::2b261e9d8c2a63abbfd5826918c23b6d\", \"eclegalperson\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"lastupdatetimestamp\": 1566902405602, \"ecinternationalorganizationeurinterests\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"dateofcollection\": \"\", \"dateoftransformation\": \"\", \"ecnonprofit\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecenterprise\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"ecinternationalorganization\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}, \"legalshortname\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"FIU\"}, \"country\": {\"classid\": \"US\", \"classname\": \"United States\", \"schemename\": \"dnet:countries\", \"schemeid\": \"dnet:countries\"}, \"extraInfo\": [], \"originalId\": [], \"ecsmevalidated\": {\"dataInfo\": {\"deletedbyinference\": false, \"provenanceaction\": {\"classid\": \"\", \"classname\": \"\", \"schemename\": \"\", \"schemeid\": \"\"}, \"inferred\": false, \"inferenceprovenance\": \"\", \"invisible\": false, \"trust\": \"\"}, \"value\": \"\"}}"; - Row row1 = SparkModel.apply(conf).rowFromJson(org1); - Row row2 = SparkModel.apply(conf).rowFromJson(org2); + Row row1 = SparkModel.apply(conf).rowFromJson(org1); + Row row2 = SparkModel.apply(conf).rowFromJson(org2); - System.out.println("row1 = " + row1); - System.out.println("row2 = " + row2); - TreeProcessor tree = new TreeProcessor(conf); + System.out.println("row1 = " + row1); + System.out.println("row2 = " + row2); + TreeProcessor tree = new TreeProcessor(conf); - boolean result = tree.compare(row1, row2); + boolean result = tree.compare(row1, row2); - System.out.println("result = " + result); + System.out.println("result = " + result); - } + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java index 40a73c9b3..18c9ce18d 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java @@ -3,13 +3,13 @@ package eu.dnetlib.dhp.oa.dedup.jpath; import java.io.IOException; -import eu.dnetlib.dhp.oa.dedup.SparkOpenorgsDedupTest; import org.apache.commons.io.IOUtils; import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.platform.commons.util.StringUtils; +import eu.dnetlib.dhp.oa.dedup.SparkOpenorgsDedupTest; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.model.SparkModel; @@ -33,13 +33,14 @@ class JsonPathTest { } @Test - void jsonToModelTest() throws IOException{ + void jsonToModelTest() throws IOException { DedupConfig conf = DedupConfig - .load(IOUtils - .toString( - SparkOpenorgsDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + .load( + IOUtils + .toString( + SparkOpenorgsDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); final String org = IOUtils.toString(getClass().getResourceAsStream("organization_example1.json")); diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index abcf4992f..1a75deafc 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -37,6 +37,7 @@ import eu.dnetlib.dhp.schema.solr.Measure; import eu.dnetlib.dhp.schema.solr.OpenAccessColor; import eu.dnetlib.dhp.schema.solr.OpenAccessRoute; import eu.dnetlib.dhp.schema.solr.Organization; +import eu.dnetlib.dhp.schema.solr.Pid; import eu.dnetlib.dhp.schema.solr.Project; import eu.dnetlib.dhp.schema.solr.Result; import eu.dnetlib.dhp.schema.solr.Subject; diff --git a/dhp-workflows/dhp-swh/pom.xml b/dhp-workflows/dhp-swh/pom.xml index 80fff4587..ef9049b20 100644 --- a/dhp-workflows/dhp-swh/pom.xml +++ b/dhp-workflows/dhp-swh/pom.xml @@ -51,49 +51,6 @@ hadoop-distcp - - eu.dnetlib - dnet-actionmanager-api - - - eu.dnetlib - dnet-actionmanager-common - - - eu.dnetlib - dnet-openaireplus-mapping-utils - - - saxonica - saxon - - - saxonica - saxon-dom - - - jgrapht - jgrapht - - - net.sf.ehcache - ehcache - - - org.springframework - spring-test - - - org.apache.* - * - - - apache - * - - - - org.apache.httpcomponents httpclient diff --git a/pom.xml b/pom.xml index a5bc6cedd..e3e57eb4c 100644 --- a/pom.xml +++ b/pom.xml @@ -440,29 +440,6 @@ provided - - eu.dnetlib - dnet-actionmanager-common - ${dnet-actionmanager-common.version} - - - org.apache.hadoop - hadoop-common - - - - - eu.dnetlib - dnet-actionmanager-api - ${dnet-actionmanager-api.version} - - - eu.dnetlib - cnr-misc-utils - - - - eu.dnetlib cnr-rmi-api @@ -960,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [6.1.4-SNAPSHOT] + [7.0.0] cdh5.9.2 3.5 11.0.2 @@ -969,8 +946,6 @@ 4.1.0-${dhp.cdh.version} true 2.4.0.cloudera2 - [4.0.3] - [6.0.5] [3.1.6] 2.2.2 1.2.17 From 79985ad197ef1cdc9ca2d0401301432a1b96e3f9 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Wed, 17 Jul 2024 18:30:24 +0200 Subject: [PATCH 082/239] [Crossref]added mapping for DFG versus the unidentified project [https://support.openaire.eu/issues/9926?next_issue_id=9924&prev_issue_id=9927#note-4] --- .../eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index 27eaa61b5..ebe72ae5b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -888,7 +888,11 @@ case object Crossref2Oaf { val targetId = getProjectId("cihr________", "1e5e62235d094afd01cd56e65112fc63") queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) - +// Added mapping for DFG + case "10.13039/501100001659" => + val targetId = getProjectId("dfgf________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) case "10.13039/100020031" => val targetId = getProjectId("tara________", "1e5e62235d094afd01cd56e65112fc63") queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) From 4f4c73d65b59118f194a263fb53272bd39160f3f Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 22 Jul 2024 15:19:02 +0200 Subject: [PATCH 083/239] minor change: addition of missing parameter in sql query --- .../eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql index 3451333c0..2a7cbb9fc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql @@ -25,7 +25,8 @@ SELECT o.country || '@@@dnet:countries' AS country, array[]::text[] AS alternativenames, 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction, - array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@' || i.issuertype), NULL) AS pid + array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@' || i.issuertype), NULL) AS pid, + 'Unknown' AS typology FROM dsm_organizations o LEFT OUTER JOIN dsm_services d ON (d.id = o.collectedfrom) LEFT OUTER JOIN dsm_organizationpids p ON (p.organization = o.id) From d27e9ea50f7650df75fc4eb7dee1723693b35a35 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 23 Jul 2024 09:56:27 +0200 Subject: [PATCH 084/239] added ODF invisible stores in raw_all workflow --- .../oa/graph/raw_all/oozie_app/workflow.xml | 30 +++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) 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 b74562284..ff927fe52 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 @@ -368,6 +368,32 @@ --mdLayoutstore --mdInterpretationcleaned + + + + + + + yarn + cluster + ImportODF_hdfs_invisible + eu.dnetlib.dhp.oa.graph.raw.MigrateHdfsMdstoresApplication + 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} + + --hdfsPath${contentPath}/odf_mdstore_hdfs_invisible + --mdstoreManagerUrl${mdstoreManagerUrl} + --mdFormatODF + --mdLayoutstore + --mdInterpretationintersection + @@ -529,7 +555,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePaths${contentPath}/db_openaire,${contentPath}/db_openorgs,${contentPath}/oaf_mdstore_hdfs,${contentPath}/odf_mdstore_hdfs,${contentPath}/mdstore/*/* + --sourcePaths${contentPath}/db_openaire,${contentPath}/db_openorgs,${contentPath}/oaf_mdstore_hdfs,${contentPath}/odf_mdstore_hdfs,${contentPath}/odf_mdstore_hdfs_invisible,${contentPath}/mdstore/*/* --invalidPath${workingDir}/invalid_records --isLookupUrl${isLookupUrl} @@ -553,7 +579,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePaths${contentPath}/db_openaire,${contentPath}/db_openorgs,${contentPath}/oaf_mdstore_hdfs,${contentPath}/odf_mdstore_hdfs,${contentPath}/mdstore/*/* + --sourcePaths${contentPath}/db_openaire,${contentPath}/db_openorgs,${contentPath}/oaf_mdstore_hdfs,${contentPath}/odf_mdstore_hdfs,${contentPath}/odf_mdstore_hdfs_invisible,${contentPath}/mdstore/*/* --targetPath${workingDir}/entities --isLookupUrl${isLookupUrl} --shouldHashId${shouldHashId} From 9573bf576d540d30246bda2c1c7a7857f101b412 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 23 Jul 2024 12:47:57 +0200 Subject: [PATCH 085/239] [SDG]added code to ingest also the SDG without DOI --- .../PrepareSDGSparkJob.java | 96 ++++++++++---- .../fosnodoi/CreateActionSetSparkJob.java | 7 - .../sdgnodoi/CreateActionSetSparkJob.java | 86 ++++++++++++ .../actionmanager/sdgnodoi/as_parameters.json | 20 +++ .../sdgnodoi/oozie_app/config-default.xml | 30 +++++ .../sdgnodoi/oozie_app/workflow.xml | 125 ++++++++++++++++++ 6 files changed, 328 insertions(+), 36 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/as_parameters.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java index a88607986..012178c1e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java @@ -6,26 +6,23 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.Serializable; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; +import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; -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.apache.spark.sql.*; +import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.SDGDataModel; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.dhp.schema.oaf.Subject; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import eu.dnetlib.dhp.utils.DHPUtils; public class PrepareSDGSparkJob implements Serializable { @@ -52,42 +49,83 @@ public class PrepareSDGSparkJob implements Serializable { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); + final Boolean distributeDOI = Optional + .ofNullable(parser.get("distributeDoi")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("distribute doi {}", distributeDOI); + SparkConf conf = new SparkConf(); runWithSparkSession( conf, isSparkSessionManaged, spark -> { - doPrepare( - spark, - sourcePath, + if (distributeDOI) + doPrepare( + spark, + sourcePath, + + outputPath); + else + doPrepareoaid(spark, sourcePath, outputPath); - outputPath); }); } - private static void doPrepare(SparkSession spark, String sourcePath, String outputPath) { - Dataset sdgDataset = readPath(spark, sourcePath, SDGDataModel.class); + private static void doPrepareoaid(SparkSession spark, String sourcePath, String outputPath) { + Dataset sdgDataset = spark + .read() + .format("csv") + .option("sep", DEFAULT_DELIMITER) + .option("inferSchema", "true") + .option("header", "true") + .option("quotes", "\"") + .load(sourcePath); sdgDataset - .groupByKey((MapFunction) r -> r.getDoi().toLowerCase(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, it) -> { - Result r = new Result(); - r.setId(DHPUtils.generateUnresolvedIdentifier(k, DOI)); - SDGDataModel first = it.next(); - List sbjs = new ArrayList<>(); - sbjs.add(getSubject(first.getSbj(), SDG_CLASS_ID, SDG_CLASS_NAME, UPDATE_SUBJECT_SDG_CLASS_ID)); - it - .forEachRemaining( - s -> sbjs - .add(getSubject(s.getSbj(), SDG_CLASS_ID, SDG_CLASS_NAME, UPDATE_SUBJECT_SDG_CLASS_ID))); - r.setSubject(sbjs); - - return r; - }, Encoders.bean(Result.class)) + .groupByKey((MapFunction) v -> ((String) v.getAs("oaid")).toLowerCase(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction) (k, + it) -> getResult( + DHPUtils + .generateUnresolvedIdentifier( + ModelSupport.entityIdPrefix.get(Result.class.getSimpleName().toLowerCase()) + "|" + k, + DOI), + it), + Encoders.bean(Result.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputPath + "/sdg"); } + private static void doPrepare(SparkSession spark, String sourcePath, String outputPath) { + Dataset sdgDataset = spark.read().csv(sourcePath); + + sdgDataset + .groupByKey((MapFunction) r -> ((String) r.getAs("doi")).toLowerCase(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction) PrepareSDGSparkJob::getResult, Encoders.bean(Result.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + "/sdg"); + } + + private static @NotNull Result getResult(String id, Iterator it) { + Result r = new Result(); + r.setId(id); + Row first = it.next(); + List sbjs = new ArrayList<>(); + sbjs.add(getSubject(first.getAs("sdg"), SDG_CLASS_ID, SDG_CLASS_NAME, UPDATE_SUBJECT_SDG_CLASS_ID)); + it + .forEachRemaining( + s -> sbjs + .add(getSubject(s.getAs("sdg"), SDG_CLASS_ID, SDG_CLASS_NAME, UPDATE_SUBJECT_SDG_CLASS_ID))); + r.setSubject(sbjs); + + return r; + } + } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java index e86fccb84..fceed2008 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java @@ -13,9 +13,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.FilterFunction; -import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; @@ -24,13 +21,9 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.schema.oaf.utils.*; import scala.Tuple2; public class CreateActionSetSparkJob implements Serializable { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java new file mode 100644 index 000000000..a2f8cfa39 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java @@ -0,0 +1,86 @@ + +package eu.dnetlib.dhp.actionmanager.sdgnodoi; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; +import java.util.Optional; + +import org.apache.commons.cli.ParseException; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.Result; +import scala.Tuple2; + +public class CreateActionSetSparkJob implements Serializable { + + private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(final String[] args) throws IOException, ParseException { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + CreateActionSetSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json")))); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("sourcePath"); + log.info("inputPath {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}", outputPath); + + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> createActionSet(spark, inputPath, outputPath)); + + } + + private static void createActionSet(SparkSession spark, String inputPath, String outputPath) { + spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, Result.class), + Encoders.bean(Result.class)) + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/as_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/as_parameters.json new file mode 100644 index 000000000..3f056edf7 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/as_parameters.json @@ -0,0 +1,20 @@ +[ + { + "paramName": "sp", + "paramLongName": "sourcePath", + "paramDescription": "the zipped opencitations file", + "paramRequired": true + }, + { + "paramName": "op", + "paramLongName": "outputPath", + "paramDescription": "the working path", + "paramRequired": true + }, + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the hdfs name node", + "paramRequired": false + } +] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/config-default.xml new file mode 100644 index 000000000..d262cb6e0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/config-default.xml @@ -0,0 +1,30 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hiveJdbcUrl + jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 + + + hiveDbName + openaire + + + oozie.launcher.mapreduce.user.classpath.first + true + + diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/workflow.xml new file mode 100644 index 000000000..82144d0d6 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/sdgnodoi/oozie_app/workflow.xml @@ -0,0 +1,125 @@ + + + + + sdgPath + the input path of the resources to be extended + + + outputPath + the path where to store the actionset + + + 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())}] + + + + + yarn + cluster + Produces the results from FOS + eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareSDGSparkJob + dhp-aggregation-${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.warehouse.dir=${sparkSqlWarehouseDir} + + --sourcePath${sdgPath} + --outputPath${workingDir}/prepared + --distributeDoifalse + + + + + + + + + + yarn + cluster + Save the action set grouping results with the same id + eu.dnetlib.dhp.actionmanager.sdgnodoi.CreateActionSetSparkJob + dhp-aggregation-${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.warehouse.dir=${sparkSqlWarehouseDir} + + --sourcePath${workingDir}/prepared/sdg + --outputPath${outputPath} + + + + + + + \ No newline at end of file From 7d2c0a3723f8abaf7fa23cbb5f729db70b8a9c72 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Tue, 23 Jul 2024 15:10:17 +0300 Subject: [PATCH 086/239] added new institutions --- .../scripts/updateMonitorDB_institutions.sql | 12 +++++++++++- .../scripts/step20-createMonitorDB_institutions.sql | 10 ++++++++-- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB_institutions.sql index 5ab8c88b5..a3f29a9e3 100644 --- a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB_institutions.sql @@ -61,7 +61,17 @@ create table TARGET.result stored as parquet as 'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development 'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology 'openorgs____::b5ca9d4340e26454e367e2908ef3872f', -- Alma Mater Studiorum University of Bologna - 'openorgs____::a6340e6ecf60f6bba163659df985b0f2' -- TU Dresden + 'openorgs____::a6340e6ecf60f6bba163659df985b0f2', -- TU Dresden + 'openorgs____::64badd35233ba2cd4946368ef2f4cf57', -- University of Vienna + 'openorgs____::7501d66d2297a963ebfb075c43fff88e', -- Royal Institute of Technology + 'openorgs____::d5eb679abdd31f70fcd4c8ba711148bf', -- Sorbonne University + 'openorgs____::b316f25380d106aac402f5ae8653910d', -- Centre for Research on Ecology and Forestry Applications + 'openorgs____::45a2076eee3013e0e85625ce61bcd272', -- Institut d'Investigació Sanitària Illes Balears + 'openorgs____::00b20b0a743a96169e6cf135e6e2bd7c', -- Universidad Publica De Navarra + 'openorgs____::0f398605c2459294d125ff23473a97dc', -- Aalto University + 'openorgs____::25b1fa62c7fd8e409d3a83c07e04b2d4', -- WHU-Otto Beisheim School of Management + 'openorgs____::d6eec313417f11205db4e736a34c0db6', -- KEMPELENOV INSTITUT INTELIGENTNYCH TECHNOLOGII + 'openorgs____::c2dfb90e797a2dc52f0084c549289d0c' -- National Research Institute for Agriculture, Food and Environment ))) foo; --ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql index 62c68c625..1326979d8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql @@ -65,5 +65,11 @@ create table TARGET.result stored as parquet as 'openorgs____::64badd35233ba2cd4946368ef2f4cf57', -- University of Vienna 'openorgs____::7501d66d2297a963ebfb075c43fff88e', -- Royal Institute of Technology 'openorgs____::d5eb679abdd31f70fcd4c8ba711148bf', -- Sorbonne University - 'openorgs____::b316f25380d106aac402f5ae8653910d' -- Centre for Research on Ecology and Forestry Applications - ))) foo; \ No newline at end of file + 'openorgs____::b316f25380d106aac402f5ae8653910d', -- Centre for Research on Ecology and Forestry Applications + 'openorgs____::45a2076eee3013e0e85625ce61bcd272', -- Institut d'Investigació Sanitària Illes Balears + 'openorgs____::00b20b0a743a96169e6cf135e6e2bd7c', -- Universidad Publica De Navarra + 'openorgs____::0f398605c2459294d125ff23473a97dc', -- Aalto University + 'openorgs____::25b1fa62c7fd8e409d3a83c07e04b2d4', -- WHU-Otto Beisheim School of Management + 'openorgs____::d6eec313417f11205db4e736a34c0db6', -- KEMPELENOV INSTITUT INTELIGENTNYCH TECHNOLOGII + 'openorgs____::c2dfb90e797a2dc52f0084c549289d0c' -- National Research Institute for Agriculture, Food and Environment + ))) foo; \ No newline at end of file From d0590e0e4994834a3db618547735e0e080bb2b02 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Tue, 23 Jul 2024 15:17:15 +0300 Subject: [PATCH 087/239] added latest institutions --- .../oozie_app/scripts/updateMonitorDBAll.sql | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDBAll.sql b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDBAll.sql index 35ab42029..064b5425b 100644 --- a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDBAll.sql @@ -81,7 +81,17 @@ create table TARGET.result stored as parquet as 'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development 'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology 'openorgs____::b5ca9d4340e26454e367e2908ef3872f', -- Alma Mater Studiorum University of Bologna - 'openorgs____::a6340e6ecf60f6bba163659df985b0f2' -- TU Dresden + 'openorgs____::a6340e6ecf60f6bba163659df985b0f2', -- TU Dresden + 'openorgs____::64badd35233ba2cd4946368ef2f4cf57', -- University of Vienna + 'openorgs____::7501d66d2297a963ebfb075c43fff88e', -- Royal Institute of Technology + 'openorgs____::d5eb679abdd31f70fcd4c8ba711148bf', -- Sorbonne University + 'openorgs____::b316f25380d106aac402f5ae8653910d', -- Centre for Research on Ecology and Forestry Applications + 'openorgs____::45a2076eee3013e0e85625ce61bcd272', -- Institut d'Investigació Sanitària Illes Balears + 'openorgs____::00b20b0a743a96169e6cf135e6e2bd7c', -- Universidad Publica De Navarra + 'openorgs____::0f398605c2459294d125ff23473a97dc', -- Aalto University + 'openorgs____::25b1fa62c7fd8e409d3a83c07e04b2d4', -- WHU-Otto Beisheim School of Management + 'openorgs____::d6eec313417f11205db4e736a34c0db6', -- KEMPELENOV INSTITUT INTELIGENTNYCH TECHNOLOGII + 'openorgs____::c2dfb90e797a2dc52f0084c549289d0c' -- National Research Institute for Agriculture, Food and Environment ))) foo; --ANALYZE TABLE TARGET.result COMPUTE STATISTICS; From 19806c2ae3267f0e1080b2afffba4c51b4fe07c7 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 23 Jul 2024 17:12:55 +0200 Subject: [PATCH 088/239] [SDG]fixed switch of methods --- .../PrepareSDGSparkJob.java | 18 +++++++++++++----- .../sdgnodoi/CreateActionSetSparkJob.java | 7 ++++++- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java index 012178c1e..be22077c3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareSDGSparkJob.java @@ -73,7 +73,7 @@ public class PrepareSDGSparkJob implements Serializable { }); } - private static void doPrepareoaid(SparkSession spark, String sourcePath, String outputPath) { + private static void doPrepare(SparkSession spark, String sourcePath, String outputPath) { Dataset sdgDataset = spark .read() .format("csv") @@ -84,7 +84,7 @@ public class PrepareSDGSparkJob implements Serializable { .load(sourcePath); sdgDataset - .groupByKey((MapFunction) v -> ((String) v.getAs("oaid")).toLowerCase(), Encoders.STRING()) + .groupByKey((MapFunction) v -> ((String) v.getAs("doi")).toLowerCase(), Encoders.STRING()) .mapGroups( (MapGroupsFunction) (k, it) -> getResult( @@ -100,11 +100,19 @@ public class PrepareSDGSparkJob implements Serializable { .json(outputPath + "/sdg"); } - private static void doPrepare(SparkSession spark, String sourcePath, String outputPath) { - Dataset sdgDataset = spark.read().csv(sourcePath); + private static void doPrepareoaid(SparkSession spark, String sourcePath, String outputPath) { + Dataset sdgDataset = spark + .read() + .format("csv") + .option("sep", DEFAULT_DELIMITER) + .option("inferSchema", "true") + .option("header", "true") + .option("quotes", "\"") + .load(sourcePath); + ; sdgDataset - .groupByKey((MapFunction) r -> ((String) r.getAs("doi")).toLowerCase(), Encoders.STRING()) + .groupByKey((MapFunction) r -> "50|" + ((String) r.getAs("oaid")), Encoders.STRING()) .mapGroups( (MapGroupsFunction) PrepareSDGSparkJob::getResult, Encoders.bean(Result.class)) .write() diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java index a2f8cfa39..0bc3b524b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/sdgnodoi/CreateActionSetSparkJob.java @@ -10,6 +10,7 @@ import java.util.Optional; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.Hdfs; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; @@ -23,6 +24,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.oaf.Result; import scala.Tuple2; @@ -63,7 +65,10 @@ public class CreateActionSetSparkJob implements Serializable { runWithSparkSession( conf, isSparkSessionManaged, - spark -> createActionSet(spark, inputPath, outputPath)); + spark -> { + HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration()); + createActionSet(spark, inputPath, outputPath); + }); } From 6f1801d7d1f1b1470813479905a94f618385f3d6 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 23 Jul 2024 17:34:48 +0200 Subject: [PATCH 089/239] [webcrawl]- --- .../webcrawl/RemoveRelationFromActionSet.java | 244 ++++++++++++++++++ 1 file changed, 244 insertions(+) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java new file mode 100644 index 000000000..074311f1f --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java @@ -0,0 +1,244 @@ +package eu.dnetlib.dhp.actionmanager.webcrawl; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.actionmanager.Constants; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; +import eu.dnetlib.dhp.schema.oaf.utils.PidType; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +public class RemoveRelationFromActionSet + implements Serializable { + private static final Logger log = LoggerFactory.getLogger(CreateActionSetFromWebEntries.class); + private static final String DOI_PREFIX = "50|doi_________::"; + + + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + CreateActionSetFromWebEntries.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/webcrawl/as_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("actionSetPath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String blackListInputPath = parser.get("blackListPath"); + log.info("blackListInputPath: {}", blackListInputPath); + + SparkConf conf = new SparkConf(); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + + removeFromActionSet(spark, inputPath, outputPath, blackListInputPath); + + }); + } + + private static void removeFromActionSet(SparkSession spark, String inputPath, String outputPath, String blackListInputPath) { + + } + + public static void createActionSet(SparkSession spark, String inputPath, + String outputPath, String blackListInputPath) { + + final Dataset dataset = readWebCrawl(spark, inputPath) + .filter("country_code=='IE'") + .drop("publication_year"); + + final Dataset blackList = readBlackList(spark, blackListInputPath); + + dataset + .join(blackList, dataset.col("id").equalTo(blackList.col("OpenAlexId")), "left") + .filter((FilterFunction) r -> r.getAs("OpenAlexId") == null) + .drop("OpenAlexId") + .flatMap((FlatMapFunction) row -> { + List ret = new ArrayList<>(); + final String ror = ROR_PREFIX + + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror"))); + ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror)); + + return ret + .iterator(); + }, Encoders.bean(Relation.class)) + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + + } + + private static Dataset readWebCrawl(SparkSession spark, String inputPath) { + StructType webInfo = StructType + .fromDDL( + "`id` STRING , `doi` STRING, `ids` STRUCT<`pmid` :STRING, `pmcid`: STRING >, `publication_year` STRING, " + + + "`authorships` ARRAY>>>"); + + return spark + .read() + .schema(webInfo) + .json(inputPath) + .withColumn( + "authors", functions + .explode( + functions.col("authorships"))) + .selectExpr("id", "doi", "ids", "publication_year", "authors.institutions as institutions") + .withColumn( + "institution", functions + .explode( + functions.col("institutions"))) + + .selectExpr( + "id", "doi", "institution.ror as ror", + "institution.country_code as country_code", "publication_year") + .distinct(); + + } + + private static Dataset readBlackList(SparkSession spark, String inputPath) { + + return spark + .read() + .option("header", true) + .csv(inputPath) + .select("OpenAlexId"); + } + + private static List createAffiliationRelationPairPMCID(String pmcid, String ror) { + if (pmcid == null) + return new ArrayList<>(); + + return createAffiliatioRelationPair( + PMCID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), removeResolver("PMC", pmcid))), + ror); + } + + private static List createAffiliationRelationPairPMID(String pmid, String ror) { + if (pmid == null) + return new ArrayList<>(); + + return createAffiliatioRelationPair( + PMID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), removeResolver("PMID", pmid))), + ror); + } + + private static String removeResolver(String pidType, String pid) { + switch (pidType) { + case "PMID": + return pid.substring(33); + case "PMC": + return "PMC" + pid.substring(43); + case "DOI": + return pid.substring(16); + } + + throw new RuntimeException(); + + } + + private static List createAffiliationRelationPairDOI(String doi, String ror) { + if (doi == null) + return new ArrayList<>(); + + return createAffiliatioRelationPair( + DOI_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), removeResolver("DOI", doi))), + ror); + + } + + private static List createAffiliatioRelationPair(String resultId, String orgId) { + ArrayList newRelations = new ArrayList(); + + newRelations + .add( + OafMapperUtils + .getRelation( + orgId, resultId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION, + ModelConstants.IS_AUTHOR_INSTITUTION_OF, + Arrays + .asList( + OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + "sysimport:crasswalk:webcrawl", "Imported from Webcrawl", + ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.9"), + null)); + + newRelations + .add( + OafMapperUtils + .getRelation( + resultId, orgId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION, + ModelConstants.HAS_AUTHOR_INSTITUTION, + Arrays + .asList( + OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + "sysimport:crasswalk:webcrawl", "Imported from Webcrawl", + ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.9"), + null)); + + return newRelations; + + } +} From 01958a3e07f141a223094e96210603893f4ea8b8 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 24 Jul 2024 10:00:10 +0200 Subject: [PATCH 090/239] [graph provision] addded filter to exclude records marked with datainfo.deletedbyinference = true --- .../eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java index d46ab1404..351526336 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java @@ -11,6 +11,7 @@ import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; +import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.apache.spark.util.LongAccumulator; @@ -29,6 +30,8 @@ import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; import eu.dnetlib.dhp.oa.provision.model.TupleWrapper; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Oaf; import eu.dnetlib.dhp.schema.solr.SolrRecord; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -115,6 +118,12 @@ public class PayloadConverterJob { .read() .load(toSeq(paths)) .as(Encoders.kryo(JoinedEntity.class)) + .filter( + (FilterFunction) je -> !Optional + .ofNullable(je.getEntity()) + .map(Oaf::getDataInfo) + .map(DataInfo::getDeletedbyinference) + .orElse(false)) .map( (MapFunction>) je -> new Tuple2<>( recordFactory.build(je, validateXML), From d771a883f933f6142d6cc3160f2135bdb2b22d55 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 25 Jul 2024 09:53:48 +0200 Subject: [PATCH 091/239] [dedup] updated sql query used to read organizations from the OpenOrgs DB to include their typology --- .../dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql index d9e4b855d..933c26356 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql @@ -25,12 +25,14 @@ SELECT null AS ecinternationalorganization, null AS ecenterprise, null AS ecsmevalidated, - null AS ecnutscode + null AS ecnutscode, + org_types.name AS typology FROM organizations o LEFT OUTER JOIN acronyms a ON (a.id = o.id) LEFT OUTER JOIN urls u ON (u.id = o.id) LEFT OUTER JOIN other_ids i ON (i.id = o.id) LEFT OUTER JOIN other_names n ON (n.id = o.id) + LEFT OUTER JOIN org_types ON (org_types.val = o.type) WHERE o.status = 'approved' GROUP BY @@ -38,7 +40,8 @@ GROUP BY o.name, o.creation_date, o.modification_date, - o.country + o.country, + org_types.name UNION ALL @@ -69,13 +72,15 @@ SELECT (array_remove(array_cat(ARRAY[o.ec_internationalorganization], array_agg(od.ec_internationalorganization)), NULL))[1] AS ecinternationalorganization, (array_remove(array_cat(ARRAY[o.ec_enterprise], array_agg(od.ec_enterprise)), NULL))[1] AS ecenterprise, (array_remove(array_cat(ARRAY[o.ec_smevalidated], array_agg(od.ec_smevalidated)), NULL))[1] AS ecsmevalidated, - (array_remove(array_cat(ARRAY[o.ec_nutscode], array_agg(od.ec_nutscode)), NULL))[1] AS ecnutscode + (array_remove(array_cat(ARRAY[o.ec_nutscode], array_agg(od.ec_nutscode)), NULL))[1] AS ecnutscode, + org_types.name AS typology FROM other_names n LEFT OUTER JOIN organizations o ON (n.id = o.id) LEFT OUTER JOIN urls u ON (u.id = o.id) LEFT OUTER JOIN other_ids i ON (i.id = o.id) LEFT OUTER JOIN oa_duplicates d ON (o.id = d.local_id) LEFT OUTER JOIN organizations od ON (d.oa_original_id = od.id) + LEFT OUTER JOIN org_types ON (org_types.val = o.type) WHERE o.status = 'approved' GROUP BY @@ -83,4 +88,5 @@ GROUP BY o.creation_date, o.modification_date, o.country, + org_types.name, n.name; \ No newline at end of file From fc60661ac5d676c8ec9475def8c3dd03cb617749 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 25 Jul 2024 12:25:14 +0200 Subject: [PATCH 092/239] [webcrawl] added code and test (code/resource) to verify the deletion of the relations related to results put in blacklist --- .../CreateActionSetFromWebEntries.java | 4 +- .../webcrawl/RemoveRelationFromActionSet.java | 323 +++++++----------- .../dhp/actionmanager/webcrawl/job.properties | 14 +- .../webcrawl/oozie_app/workflow.xml | 34 +- .../actionmanager/webcrawl/CreateASTest.java | 23 +- .../webcrawl/RemoveFromASTest.java | 108 ++++++ .../webcrawl/blackListRemove/not_irish.csv | 2 + 7 files changed, 291 insertions(+), 217 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveFromASTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index 9828ad907..7607cfc76 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -8,7 +8,7 @@ import java.util.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FilterFunction; @@ -112,7 +112,7 @@ public class CreateActionSetFromWebEntries implements Serializable { .mapToPair( aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java index 074311f1f..33dfbacf1 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java @@ -1,244 +1,159 @@ + package eu.dnetlib.dhp.actionmanager.webcrawl; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.Constants; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; -import eu.dnetlib.dhp.schema.oaf.utils.PidType; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static org.apache.spark.sql.functions.*; + +import java.io.File; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Optional; + +import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; +import org.apache.commons.io.filefilter.DirectoryFileFilter; +import org.apache.commons.io.filefilter.FileFileFilter; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FilterFunction; -import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; -import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import scala.Tuple2; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; - -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - public class RemoveRelationFromActionSet - implements Serializable { - private static final Logger log = LoggerFactory.getLogger(CreateActionSetFromWebEntries.class); - private static final String DOI_PREFIX = "50|doi_________::"; + implements Serializable { + private static final Logger log = LoggerFactory.getLogger(CreateActionSetFromWebEntries.class); + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final StructType KV_SCHEMA = StructType$.MODULE$ + .apply( + Arrays + .asList( + StructField$.MODULE$.apply("key", DataTypes.StringType, false, Metadata.empty()), + StructField$.MODULE$.apply("value", DataTypes.StringType, false, Metadata.empty()))); - public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final StructType ATOMIC_ACTION_SCHEMA = StructType$.MODULE$ + .apply( + Arrays + .asList( + StructField$.MODULE$.apply("clazz", DataTypes.StringType, false, Metadata.empty()), + StructField$.MODULE$ + .apply( + "payload", DataTypes.StringType, false, Metadata.empty()))); - public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils - .toString( - CreateActionSetFromWebEntries.class - .getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/webcrawl/as_parameters.json")); + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + CreateActionSetFromWebEntries.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/webcrawl/as_parameters.json")); - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); - final String inputPath = parser.get("actionSetPath"); - log.info("inputPath: {}", inputPath); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + // the actionSet path + final String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String blackListInputPath = parser.get("blackListPath"); - log.info("blackListInputPath: {}", blackListInputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - SparkConf conf = new SparkConf(); + final String blackListInputPath = parser.get("blackListPath"); + log.info("blackListInputPath: {}", blackListInputPath); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { + SparkConf conf = new SparkConf(); - removeFromActionSet(spark, inputPath, outputPath, blackListInputPath); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { - }); - } + removeFromActionSet(spark, inputPath, outputPath, blackListInputPath); - private static void removeFromActionSet(SparkSession spark, String inputPath, String outputPath, String blackListInputPath) { + }); + } - } + private static void removeFromActionSet(SparkSession spark, String inputPath, String outputPath, + String blackListInputPath) { + // read the blacklist + Dataset blackList = readBlackList(spark, blackListInputPath) + .map( + (MapFunction) r -> IdentifierFactory + .idFromPid("50", "doi", ((String) r.getAs("DOI / PMID")).substring(16), true), + Encoders.STRING()); - public static void createActionSet(SparkSession spark, String inputPath, - String outputPath, String blackListInputPath) { + // read the old actionset and get the relations in the payload + JavaPairRDD seq = JavaSparkContext + .fromSparkContext(spark.sparkContext()) + .sequenceFile(inputPath, Text.class, Text.class); - final Dataset dataset = readWebCrawl(spark, inputPath) - .filter("country_code=='IE'") - .drop("publication_year"); + JavaRDD rdd = seq + .map(x -> RowFactory.create(x._1().toString(), x._2().toString())); - final Dataset blackList = readBlackList(spark, blackListInputPath); + Dataset actionSet = spark + .createDataFrame(rdd, KV_SCHEMA) + .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) + .select(expr("atomic_action.*")); - dataset - .join(blackList, dataset.col("id").equalTo(blackList.col("OpenAlexId")), "left") - .filter((FilterFunction) r -> r.getAs("OpenAlexId") == null) - .drop("OpenAlexId") - .flatMap((FlatMapFunction) row -> { - List ret = new ArrayList<>(); - final String ror = ROR_PREFIX - + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror"))); - ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror)); + Dataset relation = actionSet + .map( + (MapFunction) r -> MAPPER.readValue((String) r.getAs("payload"), Relation.class), + Encoders.bean(Relation.class)); - return ret - .iterator(); - }, Encoders.bean(Relation.class)) - .toJavaRDD() - .map(p -> new AtomicAction(p.getClass(), p)) - .mapToPair( - aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), - new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + // select only the relation not matching any pid in the blacklist as source for the relation + Dataset relNoSource = relation + .joinWith(blackList, relation.col("source").equalTo(blackList.col("value")), "left") + .filter((FilterFunction>) t2 -> t2._2() == null) + .map((MapFunction, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class)); - } + // select only the relation not matching any pid in the blacklist as target of the relation + relNoSource + .joinWith(blackList, relNoSource.col("target").equalTo(blackList.col("value")), "left") + .filter((FilterFunction>) t2 -> t2._2() == null) + .map((MapFunction, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class)) + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); + ; - private static Dataset readWebCrawl(SparkSession spark, String inputPath) { - StructType webInfo = StructType - .fromDDL( - "`id` STRING , `doi` STRING, `ids` STRUCT<`pmid` :STRING, `pmcid`: STRING >, `publication_year` STRING, " - + - "`authorships` ARRAY>>>"); + } - return spark - .read() - .schema(webInfo) - .json(inputPath) - .withColumn( - "authors", functions - .explode( - functions.col("authorships"))) - .selectExpr("id", "doi", "ids", "publication_year", "authors.institutions as institutions") - .withColumn( - "institution", functions - .explode( - functions.col("institutions"))) + private static Dataset readBlackList(SparkSession spark, String inputPath) { - .selectExpr( - "id", "doi", "institution.ror as ror", - "institution.country_code as country_code", "publication_year") - .distinct(); + return spark + .read() + .option("header", true) + .csv(inputPath) + .select("DOI / PMID"); + } - } - - private static Dataset readBlackList(SparkSession spark, String inputPath) { - - return spark - .read() - .option("header", true) - .csv(inputPath) - .select("OpenAlexId"); - } - - private static List createAffiliationRelationPairPMCID(String pmcid, String ror) { - if (pmcid == null) - return new ArrayList<>(); - - return createAffiliatioRelationPair( - PMCID_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), removeResolver("PMC", pmcid))), - ror); - } - - private static List createAffiliationRelationPairPMID(String pmid, String ror) { - if (pmid == null) - return new ArrayList<>(); - - return createAffiliatioRelationPair( - PMID_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), removeResolver("PMID", pmid))), - ror); - } - - private static String removeResolver(String pidType, String pid) { - switch (pidType) { - case "PMID": - return pid.substring(33); - case "PMC": - return "PMC" + pid.substring(43); - case "DOI": - return pid.substring(16); - } - - throw new RuntimeException(); - - } - - private static List createAffiliationRelationPairDOI(String doi, String ror) { - if (doi == null) - return new ArrayList<>(); - - return createAffiliatioRelationPair( - DOI_PREFIX - + IdentifierFactory - .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), removeResolver("DOI", doi))), - ror); - - } - - private static List createAffiliatioRelationPair(String resultId, String orgId) { - ArrayList newRelations = new ArrayList(); - - newRelations - .add( - OafMapperUtils - .getRelation( - orgId, resultId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION, - ModelConstants.IS_AUTHOR_INSTITUTION_OF, - Arrays - .asList( - OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), - OafMapperUtils - .dataInfo( - false, null, false, false, - OafMapperUtils - .qualifier( - "sysimport:crasswalk:webcrawl", "Imported from Webcrawl", - ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), - "0.9"), - null)); - - newRelations - .add( - OafMapperUtils - .getRelation( - resultId, orgId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION, - ModelConstants.HAS_AUTHOR_INSTITUTION, - Arrays - .asList( - OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)), - OafMapperUtils - .dataInfo( - false, null, false, false, - OafMapperUtils - .qualifier( - "sysimport:crasswalk:webcrawl", "Imported from Webcrawl", - ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), - "0.9"), - null)); - - return newRelations; - - } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/job.properties b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/job.properties index d7bd709fc..641e72610 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/job.properties +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/job.properties @@ -1,3 +1,11 @@ -sourcePath=/user/miriam.baglioni/openalex-snapshot/data/works/ -outputPath=/tmp/miriam/webcrawlComplete/ -blackListPath=/user/miriam.baglioni/openalex-blackList +#PROPERTIES TO CREATE THE ACTION SET +#sourcePath=/user/miriam.baglioni/openalex-snapshot/data/works/ +#outputPath=/tmp/miriam/webcrawlComplete/ +#blackListPath=/user/miriam.baglioni/openalex-blackList +#resumeFrom=create + +#PROPERTIES TO REMOVE FROM THE ACTION SET +sourcePath=/var/lib/dnet/actionManager_PROD/webcrawl/rawset_28247629-468b-478e-9a42-bc540877125d_1718121542061/ +outputPath=/tmp/miriam/webcrawlRemoved/ +blackListPath=/user/miriam.baglioni/oalexBlackListNormalized +resumeFrom=remove \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/oozie_app/workflow.xml index b9394c7e6..ccf34c557 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/webcrawl/oozie_app/workflow.xml @@ -20,12 +20,19 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + ${wf:conf('resumeFrom') eq 'create'} + + + + yarn @@ -50,5 +57,30 @@ + + + + yarn + cluster + Removes some relations found to be wrong from the AS + eu.dnetlib.dhp.actionmanager.webcrawl.RemoveRelationFromActionSet + dhp-aggregation-${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.warehouse.dir=${sparkSqlWarehouseDir} + + --sourcePath${sourcePath} + --outputPath${outputPath} + --blackListPath${blackListPath} + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateASTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateASTest.java index e9291f93c..d23b7faa2 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateASTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateASTest.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.actionmanager.webcrawl; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.nio.file.Files; @@ -101,7 +102,10 @@ public class CreateASTest { .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) .map(aa -> ((Relation) aa.getPayload())); - Assertions.assertEquals(58, tmp.count()); + tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r))); + tmp.foreach(r -> assertTrue(r.getSource().startsWith("20|ror") || r.getSource().startsWith("50|doi"))); + tmp.foreach(r -> assertTrue(r.getTarget().startsWith("20|ror") || r.getTarget().startsWith("50|doi"))); + Assertions.assertEquals(24, tmp.count()); } @@ -112,7 +116,7 @@ public class CreateASTest { String inputPath = getClass() .getResource( - "/eu/dnetlib/dhp/actionmanager/webcrawl/") + "/eu/dnetlib/dhp/actionmanager/webcrawl/input/") .getPath(); String blackListPath = getClass() .getResource( @@ -194,7 +198,7 @@ public class CreateASTest { Assertions .assertEquals( - 2, tmp + 1, tmp .filter( r -> r .getSource() @@ -207,7 +211,7 @@ public class CreateASTest { Assertions .assertEquals( - 2, tmp + 1, tmp .filter( r -> r .getTarget() @@ -228,13 +232,13 @@ public class CreateASTest { "20|ror_________::" + IdentifierFactory .md5( PidCleaner - .normalizePidValue(PidType.doi.toString(), "https://ror.org/03265fv13"))) + .normalizePidValue("ROR", "https://ror.org/03265fv13"))) && r.getSource().startsWith("50|doi")) .count()); Assertions .assertEquals( - 1, tmp + 0, tmp .filter( r -> r .getTarget() @@ -268,6 +272,10 @@ public class CreateASTest { .getResource( "/eu/dnetlib/dhp/actionmanager/webcrawl") .getPath(); + String blackListPath = getClass() + .getResource( + "/eu/dnetlib/dhp/actionmanager/webcrawl/blackList/") + .getPath(); CreateActionSetFromWebEntries .main( @@ -277,7 +285,8 @@ public class CreateASTest { "-sourcePath", inputPath, "-outputPath", - workingDir.toString() + "/actionSet1" + workingDir.toString() + "/actionSet1", + "-blackListPath", blackListPath }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveFromASTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveFromASTest.java new file mode 100644 index 000000000..bc78804f2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveFromASTest.java @@ -0,0 +1,108 @@ + +package eu.dnetlib.dhp.actionmanager.webcrawl; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; +import eu.dnetlib.dhp.schema.oaf.utils.PidType; + +/** + * @author miriam.baglioni + * @Date 22/04/24 + */ +public class RemoveFromASTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + private static final Logger log = LoggerFactory + .getLogger(RemoveFromASTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files + .createTempDirectory(RemoveFromASTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(RemoveFromASTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(RemoveFromASTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testNumberofRelations() throws Exception { + + String inputPath = getClass() + .getResource( + "/eu/dnetlib/dhp/actionmanager/webcrawl/actionSet/") + .getPath(); + String blackListPath = getClass() + .getResource( + "/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/") + .getPath(); + + RemoveRelationFromActionSet + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + inputPath, + "-outputPath", + workingDir.toString() + "/actionSet1", + "-blackListPath", blackListPath + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class) + .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) + .map(aa -> ((Relation) aa.getPayload())); + + Assertions.assertEquals(22, tmp.count()); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv new file mode 100644 index 000000000..009925839 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv @@ -0,0 +1,2 @@ +DOI / PMID,OpenAlexId,Comments, +https://doi.org/10.1098/rstl.1684.0023,https://openalex.org/W2124362779,, \ No newline at end of file From d4bf449e8c64ec1b46b6aa44764b4e8781fe55f0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 25 Jul 2024 14:53:06 +0200 Subject: [PATCH 093/239] minor --- .../eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java index da3915aee..a0692cb34 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java @@ -5,14 +5,12 @@ import java.io.Serializable; import java.util.LinkedList; import java.util.List; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; - -import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.oaf.*; public class JoinedEntity implements Serializable { + private static final long serialVersionUID = -6337458773099581114L; + private OafEntity entity; private List links; From 7cff281d3e39d153c0e90cd2dccd88fc836e7e5e Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 25 Jul 2024 15:16:42 +0200 Subject: [PATCH 094/239] [webcrawl] the blacklist is now in json and no more in csv after the normalization process --- .../webcrawl/RemoveRelationFromActionSet.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java index 33dfbacf1..08d543218 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/RemoveRelationFromActionSet.java @@ -105,7 +105,7 @@ public class RemoveRelationFromActionSet Dataset blackList = readBlackList(spark, blackListInputPath) .map( (MapFunction) r -> IdentifierFactory - .idFromPid("50", "doi", ((String) r.getAs("DOI / PMID")).substring(16), true), + .idFromPid("50", "doi", ((String) r.getAs("doi")).substring(16), true), Encoders.STRING()); // read the old actionset and get the relations in the payload @@ -151,9 +151,8 @@ public class RemoveRelationFromActionSet return spark .read() - .option("header", true) - .csv(inputPath) - .select("DOI / PMID"); + .json(inputPath) + .select("doi"); } } From c7f6669f1a8fc9596c76b242f041249c4c778b2c Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 25 Jul 2024 15:20:18 +0200 Subject: [PATCH 095/239] [webcrawl] the blacklist is now in json and no more in csv after the normalization process --- .../actionmanager/webcrawl/CreateActionSetFromWebEntries.java | 3 +-- .../dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv | 2 -- .../dhp/actionmanager/webcrawl/blackListRemove/not_irish.json | 1 + 3 files changed, 2 insertions(+), 4 deletions(-) delete mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.json diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java index 7607cfc76..b5aed6ea2 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/webcrawl/CreateActionSetFromWebEntries.java @@ -148,8 +148,7 @@ public class CreateActionSetFromWebEntries implements Serializable { return spark .read() - .option("header", true) - .csv(inputPath) + .json(inputPath) .select("OpenAlexId"); } diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv deleted file mode 100644 index 009925839..000000000 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.csv +++ /dev/null @@ -1,2 +0,0 @@ -DOI / PMID,OpenAlexId,Comments, -https://doi.org/10.1098/rstl.1684.0023,https://openalex.org/W2124362779,, \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.json new file mode 100644 index 000000000..2c470c555 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/not_irish.json @@ -0,0 +1 @@ +{"doi":"https://doi.org/10.1098/rstl.1684.0023","OpenAlexId":"https://openalex.org/W2124362779"} \ No newline at end of file From 359b8ebda81abb5fd82fa26028b65dd0fa7bead0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 25 Jul 2024 15:22:29 +0200 Subject: [PATCH 096/239] [graph provision] include only FoS L1..L2 in the record serialization --- .../model/ProvisionModelSupport.java | 11 ++++++++ .../oa/provision/utils/XmlRecordFactory.java | 6 ++-- .../utils/XmlSerializationUtils.java | 4 --- .../oa/provision/XmlRecordFactoryTest.java | 2 ++ .../dnetlib/dhp/oa/provision/publication.json | 28 +++++++++++++++++-- 5 files changed, 40 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 1a75deafc..277d0deb6 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -702,6 +702,7 @@ public class ProvisionModelSupport { .stream() .filter(s -> Objects.nonNull(s.getQualifier())) .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) + .filter(ProvisionModelSupport::filterFosL1L2) .map( s -> Subject .newInstance(s.getValue(), s.getQualifier().getClassid(), s.getQualifier().getClassname())) @@ -709,6 +710,16 @@ public class ProvisionModelSupport { .orElse(null); } + public static boolean filterFosL1L2(StructuredProperty s) { + final String subjectType = Optional.ofNullable(s.getQualifier()).map(Qualifier::getClassid).orElse(""); + if (ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(subjectType)) { + String code = StringUtils.substringBefore(s.getValue(), " "); + return code.matches("^\\d{2}$|^\\d{4}$"); + } + + return true; + } + private static Country asCountry(eu.dnetlib.dhp.schema.oaf.Qualifier country) { return Optional .ofNullable(country) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 899dad221..44004faf3 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -20,6 +20,7 @@ import javax.xml.transform.*; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; +import eu.dnetlib.dhp.oa.provision.model.*; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -41,10 +42,6 @@ import com.google.common.collect.Sets; import com.mycila.xmltool.XMLDoc; import com.mycila.xmltool.XMLTag; -import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; -import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; -import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; -import eu.dnetlib.dhp.oa.provision.model.XmlInstance; import eu.dnetlib.dhp.schema.common.*; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Result; @@ -389,6 +386,7 @@ public class XmlRecordFactory implements Serializable { .getSubject() .stream() .filter(Objects::nonNull) + .filter(ProvisionModelSupport::filterFosL1L2) .map(s -> XmlSerializationUtils.mapStructuredProperty("subject", s)) .collect(Collectors.toList())); } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java index fbd647ae4..b4517002c 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java @@ -5,11 +5,7 @@ import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.removePrefix; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import java.util.HashSet; import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java index ab4301f9a..dcd021db1 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java @@ -97,6 +97,8 @@ public class XmlRecordFactoryTest { assertEquals("bronze", doc.valueOf("//*[local-name() = 'result']/openaccesscolor/text()")); assertEquals("true", doc.valueOf("//*[local-name() = 'result']/isindiamondjournal/text()")); assertEquals("true", doc.valueOf("//*[local-name() = 'result']/publiclyfunded/text()")); + + assertEquals(15, doc.selectNodes("//*[local-name() = 'result']/*[local-name() = 'subject']").size()); } @Test diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json index a89ec62d5..a073fbebd 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json @@ -1886,12 +1886,34 @@ "trust": "" }, "qualifier": { - "classid": "keyword", - "classname": "keyword", + "classid": "FOS", + "classname": "Fields of Science and Technology classification", "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies" }, - "value": "Thermal conductivity" + "value": "0101 mathematics" + }, + { + "dataInfo": { + "deletedbyinference": false, + "inferenceprovenance": "", + "inferred": false, + "invisible": false, + "provenanceaction": { + "classid": "", + "classname": "", + "schemeid": "", + "schemename": "" + }, + "trust": "" + }, + "qualifier": { + "classid": "FOS", + "classname": "Fields of Science and Technology classification", + "schemeid": "dnet:subject_classification_typologies", + "schemename": "dnet:subject_classification_typologies" + }, + "value": "010101 applied mathematics" } ], "title": [ From a81c555fe6bfa23b7c4108eac2d0415d78c8a630 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 25 Jul 2024 15:26:47 +0200 Subject: [PATCH 097/239] [graph provision] include only FoS L1..L2 in the record serialization --- .../dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 277d0deb6..4a2326453 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -687,6 +687,7 @@ public class ProvisionModelSupport { .stream() .filter(s -> Objects.nonNull(s.getQualifier())) .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) + .filter(ProvisionModelSupport::filterFosL1L2) .map( s -> Subject .newInstance(s.getValue(), s.getQualifier().getClassid(), s.getQualifier().getClassname())) @@ -702,7 +703,6 @@ public class ProvisionModelSupport { .stream() .filter(s -> Objects.nonNull(s.getQualifier())) .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) - .filter(ProvisionModelSupport::filterFosL1L2) .map( s -> Subject .newInstance(s.getValue(), s.getQualifier().getClassid(), s.getQualifier().getClassname())) From 75a11d0ba5a3ff9a362c7c160ace11d0965a57a5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 25 Jul 2024 16:34:32 +0200 Subject: [PATCH 098/239] [dedup] avoid NPEs in the countryInference dedup utility --- .../dnetlib/pace/common/AbstractPaceFunctions.java | 2 +- .../eu/dnetlib/pace/common/PaceFunctionTest.java | 12 ++++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java index 6ef550c50..12a54bade 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/common/AbstractPaceFunctions.java @@ -90,7 +90,7 @@ public class AbstractPaceFunctions extends PaceCommonUtils { inferFrom = normalize(inferFrom); inferFrom = filterAllStopWords(inferFrom); Set cities = getCities(inferFrom, 4); - return citiesToCountry(cities).stream().findFirst().orElse("UNKNOWN"); + return citiesToCountry(cities).stream().filter(Objects::nonNull).findFirst().orElse("UNKNOWN"); } public static String cityInference(String original) { diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java index 4ec120f4a..92f7bf6ff 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/common/PaceFunctionTest.java @@ -1,8 +1,7 @@ package eu.dnetlib.pace.common; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.*; import org.junit.jupiter.api.*; @@ -54,8 +53,17 @@ public class PaceFunctionTest extends AbstractPaceFunctions { System.out.println("Fixed aliases : " + fixAliases(TEST_STRING)); } + @Test() + public void countryInferenceTest_NPE() { + assertThrows( + NullPointerException.class, + () -> countryInference("UNKNOWN", null), + "Expected countryInference() to throw an NPE"); + } + @Test public void countryInferenceTest() { + assertEquals("UNKNOWN", countryInference("UNKNOWN", "")); assertEquals("IT", countryInference("UNKNOWN", "Università di Bologna")); assertEquals("UK", countryInference("UK", "Università di Bologna")); assertEquals("IT", countryInference("UNKNOWN", "Universiteé de Naples")); From 64740475d0a8f0209165cb15fc7f44b125700a23 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 29 Jul 2024 11:51:42 +0200 Subject: [PATCH 099/239] depending on dhp-schemas:7.0.1 --- .../personentity/ExtractPerson.java | 18 ++++++++++++------ .../actionmanager/person/CreatePersonAS.java | 13 +------------ pom.xml | 2 +- 3 files changed, 14 insertions(+), 19 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 064fb41a1..d381ed176 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -13,7 +13,6 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.BZip2Codec; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.*; @@ -25,8 +24,6 @@ import org.spark_project.jetty.util.StringUtil; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.Constants; -import eu.dnetlib.dhp.actionmanager.transformativeagreement.model.TransformativeAgreementModel; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.collection.orcid.model.Author; import eu.dnetlib.dhp.collection.orcid.model.Employment; @@ -37,7 +34,6 @@ import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Person; -import eu.dnetlib.dhp.schema.oaf.Pid; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; @@ -181,10 +177,20 @@ public class ExtractPerson implements Serializable { .map( v -> v .stream() - .map(p -> Pid.newInstance(p.getSchema(), p.getValue())) + .map( + p -> OafMapperUtils + .structuredProperty( + p.getValue(), p.getSchema(), p.getSchema(), ModelConstants.DNET_PID_TYPES, + ModelConstants.DNET_PID_TYPES, null)) .collect(Collectors.toList())) .orElse(new ArrayList<>())); - person.getPid().add(Pid.newInstance(ModelConstants.ORCID, op.getOrcid())); + person + .getPid() + .add( + OafMapperUtils + .structuredProperty( + op.getOrcid(), ModelConstants.ORCID, ModelConstants.ORCID_CLASSNAME, + ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, null)); person.setDateofcollection(op.getLastModifiedDate()); person.setOriginalId(Arrays.asList(op.getOrcid())); return person; diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java index 2e7b21010..b5333c2fb 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/person/CreatePersonAS.java @@ -1,22 +1,15 @@ package eu.dnetlib.dhp.actionmanager.person; -import static org.junit.jupiter.api.Assertions.assertEquals; - import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.util.Optional; import org.apache.commons.io.FileUtils; import org.apache.hadoop.io.Text; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FilterFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; @@ -27,15 +20,11 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob; import eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson; -import eu.dnetlib.dhp.collection.orcid.model.Author; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Person; import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.utils.DHPUtils; public class CreatePersonAS { @@ -167,7 +156,7 @@ public class CreatePersonAS { .getPid() .stream() .anyMatch( - p -> p.getSchema().equalsIgnoreCase("Scopus Author ID") + p -> p.getQualifier().getSchemename().equalsIgnoreCase("Scopus Author ID") && p.getValue().equalsIgnoreCase("15119405200"))); Assertions diff --git a/pom.xml b/pom.xml index dc1dd7308..666ba2350 100644 --- a/pom.xml +++ b/pom.xml @@ -937,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [7.0.0] + [7.0.1] cdh5.9.2 3.5 11.0.2 From 9486e21a44f9c8e13919e7d43f34983866e3874f Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 30 Jul 2024 14:25:31 +0200 Subject: [PATCH 100/239] copy or process the person records throughout the graph pipeline --- .../oaf/utils/GraphCleaningFunctions.java | 2 + .../wf/main/oozie_app/import.txt | 1 + .../wf/main/oozie_app/workflow.xml | 16 +++ .../wf/person/oozie_app/workflow.xml | 130 ++++++++++++++++++ .../dhp/blacklist/oozie_app/workflow.xml | 10 ++ .../oozie_app/workflow.xml | 12 ++ .../dhp/enrich/orcid/oozie_app/workflow.xml | 8 ++ .../dhp/oa/graph/clean/oozie_app/workflow.xml | 36 +++++ 8 files changed, 215 insertions(+) create mode 100644 dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index 2be4e8e0c..b9dc3253b 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -363,6 +363,8 @@ public class GraphCleaningFunctions extends CleaningFunctions { // nothing to clean here } else if (value instanceof Project) { // nothing to clean here + } else if (value instanceof Person) { + // nothing to clean here } else if (value instanceof Organization) { Organization o = (Organization) value; if (Objects.isNull(o.getCountry()) || StringUtils.isBlank(o.getCountry().getClassid())) { diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/import.txt b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/import.txt index dd8f5e14e..14409a42a 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/import.txt +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/import.txt @@ -7,3 +7,4 @@ promote_action_payload_for_project_table classpath eu/dnetlib/dhp/actionmanager/ promote_action_payload_for_publication_table classpath eu/dnetlib/dhp/actionmanager/wf/publication/oozie_app promote_action_payload_for_relation_table classpath eu/dnetlib/dhp/actionmanager/wf/relation/oozie_app promote_action_payload_for_software_table classpath eu/dnetlib/dhp/actionmanager/wf/software/oozie_app +promote_action_payload_for_person_table classpath eu/dnetlib/dhp/actionmanager/wf/person/oozie_app 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 65ddd402b..7ccfb342e 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 @@ -148,6 +148,7 @@ + @@ -270,6 +271,21 @@ + + + ${wf:appPath()}/promote_action_payload_for_person_table + + + + inputActionPayloadRootPath + ${workingDir}/action_payload_by_type + + + + + + + diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml new file mode 100644 index 000000000..7c119b305 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml @@ -0,0 +1,130 @@ + + + + activePromotePersonActionPayload + when true will promote actions with eu.dnetlib.dhp.schema.oaf.Person payload + + + inputGraphRootPath + root location of input materialized graph + + + inputActionPayloadRootPath + root location of action payloads to promote + + + outputGraphRootPath + root location for output materialized graph + + + mergeAndGetStrategy + strategy for merging graph table objects with action payload instances, MERGE_FROM_AND_GET or SELECT_NEWER_AND_GET + + + 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} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + ${(activePromotePersonActionPayload eq "true") and + (fs:exists(concat(concat(concat(concat(wf:conf('nameNode'),'/'),wf:conf('inputGraphRootPath')),'/'),'person')) eq "true") and + (fs:exists(concat(concat(concat(concat(wf:conf('nameNode'),'/'),wf:conf('inputActionPayloadRootPath')),'/'),'clazz=eu.dnetlib.dhp.schema.oaf.Person')) eq "true")} + + + + + + + + yarn-cluster + cluster + PromotePersonActionPayloadForPersonTable + eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJob + dhp-actionmanager-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --inputGraphTablePath${inputGraphRootPath}/person + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Person + --inputActionPayloadPath${inputActionPayloadRootPath}/clazz=eu.dnetlib.dhp.schema.oaf.Person + --actionPayloadClassNameeu.dnetlib.dhp.schema.oaf.Person + --outputGraphTablePath${outputGraphRootPath}/person + --mergeAndGetStrategy${mergeAndGetStrategy} + --promoteActionStrategy${promoteActionStrategy} + + + + + + + + + + + -pb + ${inputGraphRootPath}/person + ${outputGraphRootPath}/person + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index dd7827da4..563a549f3 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -63,6 +63,7 @@ + @@ -120,6 +121,15 @@ + + + ${nameNode}/${sourcePath}/person + ${nameNode}/${outputPath}/person + + + + + ${nameNode}/${sourcePath}/datasource diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/orcidtoresultfromsemrel/oozie_app/workflow.xml index ba3633e07..8eaa79c53 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -34,6 +34,7 @@ + @@ -80,6 +81,17 @@ + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/person + ${nameNode}/${outputPath}/person + + + + + diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml index 72fc9e338..4031da15a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml @@ -89,6 +89,14 @@ ${nameNode}/${graphPath}/project ${nameNode}/${targetPath}/project + + + + + + ${nameNode}/${graphPath}/person + ${nameNode}/${targetPath}/person + diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml index 4188cb018..2512fc5bc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml @@ -142,6 +142,7 @@ + @@ -390,6 +391,41 @@ + + + yarn + cluster + Clean person + eu.dnetlib.dhp.oa.graph.clean.CleanGraphSparkJob + dhp-graph-mapper-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} + --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=2000 + + --inputPath${graphInputPath}/person + --outputPath${graphOutputPath}/person + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Person + --isLookupUrl${isLookupUrl} + --contextId${contextId} + --verifyParam${verifyParam} + --country${country} + --verifyCountryParam${verifyCountryParam} + --hostedBy${workingDir}/working/hostedby + --collectedfrom${collectedfrom} + --masterDuplicatePath${workingDir}/masterduplicate + --deepClean${shouldClean} + + + + + yarn From 6bdb8643e6531ea0acf004f14a10a8baf55fa308 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 31 Jul 2024 11:02:22 +0200 Subject: [PATCH 101/239] ActionManager promote: allow to ingest person records in a graph that did not contain them, bumped dhp-schemas version --- .../PromoteActionPayloadForGraphTableJob.java | 21 ++++++++++++------- .../PromoteActionPayloadFunctions.java | 2 +- .../wf/person/oozie_app/workflow.xml | 1 - pom.xml | 2 +- 4 files changed, 16 insertions(+), 10 deletions(-) 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 56cbda4d6..f72fd4269 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 @@ -151,12 +151,17 @@ public class PromoteActionPayloadForGraphTableJob { SparkSession spark, 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)); + if (HdfsSupport.exists(path, spark.sparkContext().hadoopConfiguration())) { + return spark + .read() + .textFile(path) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, rowClazz), + Encoders.bean(rowClazz)); + } else { + logger.info("Found empty graph table from path: {}", path); + return spark.emptyDataset(Encoders.bean(rowClazz)); + } } private static Dataset readActionPayload( @@ -223,7 +228,7 @@ public class PromoteActionPayloadForGraphTableJob { rowClazz, actionPayloadClazz); - if (shouldGroupById) { + if (Boolean.TRUE.equals(shouldGroupById)) { return PromoteActionPayloadFunctions .groupGraphTableByIdAndMerge( joinedAndMerged, rowIdFn, mergeRowsAndGetFn, zeroFn, isNotZeroFn, rowClazz); @@ -250,6 +255,8 @@ public class PromoteActionPayloadForGraphTableJob { return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Relation()); case "eu.dnetlib.dhp.schema.oaf.Software": return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Software()); + case "eu.dnetlib.dhp.schema.oaf.Person": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Person()); default: throw new RuntimeException("unknown class: " + clazz.getCanonicalName()); } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java index f0b094240..a3b975d0a 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java @@ -50,7 +50,7 @@ public class PromoteActionPayloadFunctions { PromoteAction.Strategy promoteActionStrategy, Class rowClazz, Class actionPayloadClazz) { - if (!isSubClass(rowClazz, actionPayloadClazz)) { + if (Boolean.FALSE.equals(isSubClass(rowClazz, actionPayloadClazz))) { throw new RuntimeException( "action payload type must be the same or be a super type of table row type"); } diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml index 7c119b305..1bacd09f1 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/person/oozie_app/workflow.xml @@ -77,7 +77,6 @@ ${(activePromotePersonActionPayload eq "true") and - (fs:exists(concat(concat(concat(concat(wf:conf('nameNode'),'/'),wf:conf('inputGraphRootPath')),'/'),'person')) eq "true") and (fs:exists(concat(concat(concat(concat(wf:conf('nameNode'),'/'),wf:conf('inputActionPayloadRootPath')),'/'),'clazz=eu.dnetlib.dhp.schema.oaf.Person')) eq "true")} diff --git a/pom.xml b/pom.xml index 666ba2350..175cb9e7c 100644 --- a/pom.xml +++ b/pom.xml @@ -937,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [7.0.1] + [7.0.2] cdh5.9.2 3.5 11.0.2 From 975d44cac7e9fa617c9b00070eba88edafc98c7d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 2 Aug 2024 16:14:10 +0200 Subject: [PATCH 102/239] [graph provision] added person to the provision workflow --- .../model/ProvisionModelSupport.java | 11 ++++ .../dhp/oa/provision/oozie_app/workflow.xml | 59 +++++++++++++++++++ 2 files changed, 70 insertions(+) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 1a75deafc..196faf9ca 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,6 +5,7 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.solr.Person; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -89,6 +90,8 @@ public class ProvisionModelSupport { r.setOrganization(mapOrganization((eu.dnetlib.dhp.schema.oaf.Organization) e)); } else if (e instanceof eu.dnetlib.dhp.schema.oaf.Project) { r.setProject(mapProject((eu.dnetlib.dhp.schema.oaf.Project) e, vocs)); + } else if (e instanceof eu.dnetlib.dhp.schema.oaf.Person) { + r.setPerson(mapPerson((eu.dnetlib.dhp.schema.oaf.Person) e)); } r .setLinks( @@ -185,6 +188,14 @@ public class ProvisionModelSupport { return ps; } + private static Person mapPerson(eu.dnetlib.dhp.schema.oaf.Person p) { + Person ps = new Person(); + ps.setFamilyName(p.getFamilyName()); + ps.setGivenName(p.getGivenName()); + ps.setAlternativeNames(p.getAlternativeNames()); + return ps; + } + private static Funding mapFunding(List fundingtree, VocabularyGroup vocs) { SAXReader reader = new SAXReader(); return Optional diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 15d3b6300..879911ccc 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -180,6 +180,7 @@ + @@ -378,6 +379,34 @@ + + + yarn + cluster + Join[relation.target = person.id] + eu.dnetlib.dhp.oa.provision.CreateRelatedEntitiesJob_phase1 + dhp-graph-provision-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining} + --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=5000 + --conf spark.network.timeout=${sparkNetworkTimeout} + + --inputRelationsPath${workingDir}/relation + --inputEntityPath${inputGraphRootPath}/person + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Person + --outputPath${workingDir}/join_partial/person + + + + + @@ -388,6 +417,7 @@ + @@ -593,6 +623,35 @@ + + + yarn + cluster + Join[person.id = relatedEntity.source] + eu.dnetlib.dhp.oa.provision.CreateRelatedEntitiesJob_phase2 + dhp-graph-provision-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining} + --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=5000 + --conf spark.network.timeout=${sparkNetworkTimeout} + + --inputEntityPath${inputGraphRootPath}/person + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Person + --inputRelatedEntitiesPath${workingDir}/join_partial + --outputPath${workingDir}/join_entities/person + --numPartitions10000 + + + + + From 0bf76f2a3401c550dea7da6c1fd4c38ca3903527 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 5 Aug 2024 09:35:07 +0200 Subject: [PATCH 103/239] [graph provision] added person to the graph2hive workflow --- .../dhp/oa/graph/hive/oozie_app/workflow.xml | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml index eec67fc5c..872ef8a2d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml @@ -102,6 +102,7 @@ + @@ -308,6 +309,35 @@ + + + yarn + cluster + Import table person + eu.dnetlib.dhp.oa.graph.hive.GraphHiveTableImporterJob + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=1000 + + --inputPath${inputPath}/person + --hiveDbName${hiveDbName} + --classNameeu.dnetlib.dhp.schema.oaf.Person + --hiveMetastoreUris${hiveMetastoreUris} + --numPartitions1000 + + + + + yarn From 985ca15264dbe3e7a3407451b2c883c76d2d87dd Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 5 Aug 2024 12:10:40 +0200 Subject: [PATCH 104/239] [openaire-affiliation]removes matchings without DOI --- .../bipaffiliations/PrepareAffiliationRelations.java | 3 ++- .../dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 8f911e980..633e53d46 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -129,7 +129,8 @@ public class PrepareAffiliationRelations implements Serializable { Dataset df = spark .read() .schema("`DOI` STRING, `Matchings` ARRAY>") - .json(inputPath); + .json(inputPath) + .where("DOI is not NULL"); // unroll nested arrays df = df diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala index f3a5fe77c..4e5ad5365 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala @@ -31,5 +31,11 @@ class ORCIDAuthorMatchersTest { assertTrue(matchOrderedTokenAndAbbreviations("孙林 Sun Lin", "Sun Lin")) // assertTrue(AuthorsMatchRevised.compare("孙林 Sun Lin", "孙林")); // not yet implemented } + @Test def testDocumentationNames(): Unit = { + assertTrue(matchOrderedTokenAndAbbreviations("James C. A. Miller-Jones", "James Antony Miller-Jones")) + } + @Test def testDocumentationNames2(): Unit = { + assertTrue(matchOrderedTokenAndAbbreviations("James C. A. Miller-Jones", "James Antony Miller Jones")) + } } From 8e7ef79ce09d41d57d9d70f90875563bd2799e40 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 5 Aug 2024 12:13:48 +0200 Subject: [PATCH 105/239] [bip affiliations] considers only DOI based records --- .../bipaffiliations/PrepareAffiliationRelations.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 8f911e980..98915bdc5 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -129,7 +129,8 @@ public class PrepareAffiliationRelations implements Serializable { Dataset df = spark .read() .schema("`DOI` STRING, `Matchings` ARRAY>") - .json(inputPath); + .json(inputPath) + .where("DOI is not null"); // unroll nested arrays df = df From e16616b9646b77622a1a035574f2e8a39932294d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 5 Aug 2024 15:57:37 +0200 Subject: [PATCH 106/239] added dataInfo to person records --- .../personentity/ExtractPerson.java | 52 +++++++------------ 1 file changed, 20 insertions(+), 32 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index d381ed176..7e82698f7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -32,6 +32,7 @@ import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Person; import eu.dnetlib.dhp.schema.oaf.Relation; @@ -62,6 +63,20 @@ public class ExtractPerson implements Serializable { public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + public static final DataInfo DATAINFO = OafMapperUtils + .dataInfo( + false, + null, + false, + false, + OafMapperUtils + .qualifier( + ORCID_AUTHORS_CLASSID, + ORCID_AUTHORS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"); + public static void main(final String[] args) throws IOException, ParseException { final ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -193,6 +208,7 @@ public class ExtractPerson implements Serializable { ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, null)); person.setDateofcollection(op.getLastModifiedDate()); person.setOriginalId(Arrays.asList(op.getOrcid())); + person.setDataInfo(DATAINFO); return person; }, Encoders.bean(Person.class)) .write() @@ -307,14 +323,7 @@ public class ExtractPerson implements Serializable { source, target, ModelConstants.ORG_PERSON_RELTYPE, ModelConstants.ORG_PERSON_SUBRELTYPE, ModelConstants.ORG_PERSON_PARTICIPATES, Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils - .dataInfo( - false, null, false, false, - OafMapperUtils - .qualifier( - ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - "0.91"), + DATAINFO, null); if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())) { @@ -348,14 +357,7 @@ public class ExtractPerson implements Serializable { ModelConstants.PERSON_PERSON_SUBRELTYPE, ModelConstants.PERSON_PERSON_HASCOAUTHORED, Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils - .dataInfo( - false, null, false, false, - OafMapperUtils - .qualifier( - ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, - ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), - "0.91"), + DATAINFO, null), OafMapperUtils .getRelation( @@ -363,14 +365,7 @@ public class ExtractPerson implements Serializable { ModelConstants.PERSON_PERSON_SUBRELTYPE, ModelConstants.PERSON_PERSON_HASCOAUTHORED, Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils - .dataInfo( - false, null, false, false, - OafMapperUtils - .qualifier( - ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, - ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), - "0.91"), + DATAINFO, null)); } @@ -424,14 +419,7 @@ public class ExtractPerson implements Serializable { ModelConstants.RESULT_PERSON_SUBRELTYPE, ModelConstants.RESULT_PERSON_HASAUTHORED, Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - OafMapperUtils - .dataInfo( - false, null, false, false, - OafMapperUtils - .qualifier( - ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - "0.91"), + DATAINFO, null); } } From 5a7ba772717c36a7f5ccbf442b427337e700a0b5 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 12 Aug 2024 18:01:15 +0200 Subject: [PATCH 107/239] [Person]fix issue in affiliation relation id construction for person (missing ::) --- .../dnetlib/dhp/actionmanager/personentity/ExtractPerson.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 7e82698f7..6f61d427d 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -313,7 +313,7 @@ public class ExtractPerson implements Serializable { } private static Relation getAffiliationRelation(Employment row) { - String source = PERSON_PREFIX + IdentifierFactory.md5(row.getOrcid()); + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(row.getOrcid()); String target = ROR_PREFIX + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAffiliationId().getValue())); List properties = new ArrayList<>(); From 89fcf4086c9562269b4020e2f7888a051b167ed6 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 12 Aug 2024 18:04:43 +0200 Subject: [PATCH 108/239] [Person]fix issue in affiliation relation id construction for person (missing ::) --- .../dnetlib/dhp/actionmanager/personentity/ExtractPerson.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index d381ed176..e63a50984 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -297,7 +297,7 @@ public class ExtractPerson implements Serializable { } private static Relation getAffiliationRelation(Employment row) { - String source = PERSON_PREFIX + IdentifierFactory.md5(row.getOrcid()); + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(row.getOrcid()); String target = ROR_PREFIX + IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAffiliationId().getValue())); List properties = new ArrayList<>(); From 468f2aa5a58ca1aee198ea96b714060a9931b313 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 12 Aug 2024 18:10:46 +0200 Subject: [PATCH 109/239] [AffiliationAffRo]align beta with new affiliation from publisher webpage introduced in production. AffRo collectedfrom OpenAIRE to discriminate against WebCrawl --- .../PrepareAffiliationRelations.java | 102 ++++++++++++------ .../PrepareAffiliationRelationsTest.java | 18 +++- 2 files changed, 83 insertions(+), 37 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 98915bdc5..70ca1576c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -44,6 +44,8 @@ public class PrepareAffiliationRelations implements Serializable { public static final String BIP_AFFILIATIONS_CLASSID = "result:organization:openaireinference"; public static final String BIP_AFFILIATIONS_CLASSNAME = "Affiliation relation inferred by OpenAIRE"; public static final String BIP_INFERENCE_PROVENANCE = "openaire:affiliation"; + public static final String OPENAIRE_DATASOURCE_ID = "10|infrastruct_::f66f1bd369679b5b077dcdf006089556"; + public static final String OPENAIRE_DATASOURCE_NAME = "OpenAIRE"; public static void main(String[] args) throws Exception { @@ -74,6 +76,9 @@ public class PrepareAffiliationRelations implements Serializable { final String webcrawlInputPath = parser.get("webCrawlInputPath"); log.info("webcrawlInputPath: {}", webcrawlInputPath); + final String publisherInputPath = parser.get("publisherInputPath"); + log.info("publisherInputPath: {}", publisherInputPath); + final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); @@ -84,43 +89,68 @@ public class PrepareAffiliationRelations implements Serializable { isSparkSessionManaged, spark -> { Constants.removeOutputDir(spark, outputPath); - - List collectedFromCrossref = OafMapperUtils - .listKeyValues(ModelConstants.CROSSREF_ID, "Crossref"); - JavaPairRDD crossrefRelations = prepareAffiliationRelations( - spark, crossrefInputPath, collectedFromCrossref); - - List collectedFromPubmed = OafMapperUtils - .listKeyValues(ModelConstants.PUBMED_CENTRAL_ID, "Pubmed"); - JavaPairRDD pubmedRelations = prepareAffiliationRelations( - spark, pubmedInputPath, collectedFromPubmed); - - List collectedFromOpenAPC = OafMapperUtils - .listKeyValues(ModelConstants.OPEN_APC_ID, "OpenAPC"); - JavaPairRDD openAPCRelations = prepareAffiliationRelations( - spark, openapcInputPath, collectedFromOpenAPC); - - List collectedFromDatacite = OafMapperUtils - .listKeyValues(ModelConstants.DATACITE_ID, "Datacite"); - JavaPairRDD dataciteRelations = prepareAffiliationRelations( - spark, dataciteInputPath, collectedFromDatacite); - - List collectedFromWebCrawl = OafMapperUtils - .listKeyValues(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME); - JavaPairRDD webCrawlRelations = prepareAffiliationRelations( - spark, webcrawlInputPath, collectedFromWebCrawl); - - crossrefRelations - .union(pubmedRelations) - .union(openAPCRelations) - .union(dataciteRelations) - .union(webCrawlRelations) - .saveAsHadoopFile( - outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); - + createActionSet( + spark, crossrefInputPath, pubmedInputPath, openapcInputPath, dataciteInputPath, webcrawlInputPath, + publisherInputPath, outputPath); }); } + private static void createActionSet(SparkSession spark, String crossrefInputPath, String pubmedInputPath, + String openapcInputPath, String dataciteInputPath, String webcrawlInputPath, String publisherlInputPath, + String outputPath) { + List collectedFromCrossref = OafMapperUtils + .listKeyValues(ModelConstants.CROSSREF_ID, "Crossref"); + JavaPairRDD crossrefRelations = prepareAffiliationRelations( + spark, crossrefInputPath, collectedFromCrossref); + + List collectedFromPubmed = OafMapperUtils + .listKeyValues(ModelConstants.PUBMED_CENTRAL_ID, "Pubmed"); + JavaPairRDD pubmedRelations = prepareAffiliationRelations( + spark, pubmedInputPath, collectedFromPubmed); + + List collectedFromOpenAPC = OafMapperUtils + .listKeyValues(ModelConstants.OPEN_APC_ID, "OpenAPC"); + JavaPairRDD openAPCRelations = prepareAffiliationRelations( + spark, openapcInputPath, collectedFromOpenAPC); + + List collectedFromDatacite = OafMapperUtils + .listKeyValues(ModelConstants.DATACITE_ID, "Datacite"); + JavaPairRDD dataciteRelations = prepareAffiliationRelations( + spark, dataciteInputPath, collectedFromDatacite); + + List collectedFromWebCrawl = OafMapperUtils + .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); + JavaPairRDD webCrawlRelations = prepareAffiliationRelations( + spark, webcrawlInputPath, collectedFromWebCrawl); + + List collectedfromPublisher = OafMapperUtils + .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); + JavaPairRDD publisherRelations = prepareAffiliationRelationFromPublisher( + spark, publisherlInputPath, collectedfromPublisher); + + crossrefRelations + .union(pubmedRelations) + .union(openAPCRelations) + .union(dataciteRelations) + .union(webCrawlRelations) + .union(publisherRelations) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); + } + + private static JavaPairRDD prepareAffiliationRelationFromPublisher(SparkSession spark, String inputPath, + List collectedfrom) { + + Dataset df = spark + .read() + .schema("`DOI` STRING, `Organizations` ARRAY>") + .json(inputPath) + .where("DOI is not null"); + + return getTextTextJavaPairRDD(collectedfrom, df.selectExpr("DOI", "Organizations as Matchings")); + + } + private static JavaPairRDD prepareAffiliationRelations(SparkSession spark, String inputPath, List collectedfrom) { @@ -132,6 +162,10 @@ public class PrepareAffiliationRelations implements Serializable { .json(inputPath) .where("DOI is not null"); + return getTextTextJavaPairRDD(collectedfrom, df); + } + + private static JavaPairRDD getTextTextJavaPairRDD(List collectedfrom, Dataset df) { // unroll nested arrays df = df .withColumn("matching", functions.explode(new Column("Matchings"))) diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index bffe41ac7..ac9977a7e 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -78,6 +78,10 @@ public class PrepareAffiliationRelationsTest { .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json") .getPath(); + String publisherAffiliationRelationPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/publishers") + .getPath(); + String outputPath = workingDir.toString() + "/actionSet"; PrepareAffiliationRelations @@ -89,6 +93,7 @@ public class PrepareAffiliationRelationsTest { "-openapcInputPath", crossrefAffiliationRelationPath, "-dataciteInputPath", crossrefAffiliationRelationPath, "-webCrawlInputPath", crossrefAffiliationRelationPath, + "-publisherInputPath", publisherAffiliationRelationPath, "-outputPath", outputPath }); @@ -105,7 +110,7 @@ public class PrepareAffiliationRelationsTest { // ); // } // count the number of relations - assertEquals(120, tmp.count()); + assertEquals(138, tmp.count()); Dataset dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); dataset.createOrReplaceTempView("result"); @@ -116,7 +121,7 @@ public class PrepareAffiliationRelationsTest { // verify that we have equal number of bi-directional relations Assertions .assertEquals( - 60, execVerification + 69, execVerification .filter( "relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'") .collectAsList() @@ -124,7 +129,7 @@ public class PrepareAffiliationRelationsTest { Assertions .assertEquals( - 60, execVerification + 69, execVerification .filter( "relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'") .collectAsList() @@ -145,5 +150,12 @@ public class PrepareAffiliationRelationsTest { .get(0) .getString(4)); + final String publisherid = ID_PREFIX + + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s00217-010-1268-9")); + final String rorId = "20|ror_________::" + IdentifierFactory.md5("https://ror.org/03265fv13"); + + Assertions + .assertEquals( + 1, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count()); } } From db03f853660767450ad1d283c1b841c849b0110a Mon Sep 17 00:00:00 2001 From: Serafeim Chatzopoulos Date: Wed, 4 Sep 2024 14:25:44 +0300 Subject: [PATCH 110/239] Remove steps for updating BIP! from the impact indicators workflow --- .../oozie_app/get_score_limits.sh | 63 ------- .../oozie_app/map_openaire_ids_to_dois.py | 60 ------- .../oozie_app/map_scores_to_dois.py | 168 ----------------- .../impact_indicators/oozie_app/workflow.xml | 169 ++---------------- 4 files changed, 16 insertions(+), 444 deletions(-) delete mode 100644 dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/get_score_limits.sh delete mode 100644 dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_openaire_ids_to_dois.py delete mode 100755 dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_scores_to_dois.py diff --git a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/get_score_limits.sh b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/get_score_limits.sh deleted file mode 100644 index 6d4161d7f..000000000 --- a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/get_score_limits.sh +++ /dev/null @@ -1,63 +0,0 @@ -#/usr/bin/bash - -# Read log files from ranking scripts and create a two-line file -# with score limits for the various measures. To be used by Kleanthis - -attrank_file=$(ls *attrank*.log); -pr_file=$(ls *pagerank*.log) -ram_file=$(ls *ram*.log); -cc_file=$(ls *cc*.log); -impulse_file=$(ls *impulse*.log); - -echo -echo "-----------------------------" -echo "Attrank file:${attrank_file}"; -echo "PageRank file:${pr_file}"; -echo "RAM file:${ram_file}"; -echo "CC file:${cc_file}"; -echo "Impulse file:${impulse_file}"; -echo "-----------------------------" -echo -echo - -# output file will be called score_limits.csv -echo -e "influence_top001\tinfluence_top01\tinfluence_top1\tinfluence_top10\tpopularity_top001\tpopularity_top01\tpopularity_top1\tpopularity_top10\timpulse_top001\timpulse_top01\timpulse_top1\timpulse_top10\tcc_top001\tcc_top01\tcc_top1\tcc_top10" > score_limits.csv -# ---------------------------------------------------- # -# Get respective score limits (we don't need RAM) -inf_001=$(grep "^0.01%" ${pr_file} | cut -f 2); -inf_01=$(grep "^0.1%" ${pr_file} | cut -f 2); -inf_1=$(grep "^1%" ${pr_file} | cut -f 2); -inf_10=$(grep "^10%" ${pr_file} | cut -f 2); -echo "Influnence limits:" -echo -e "${inf_001}\t${inf_01}\t${inf_1}\t${inf_10}"; -# ---------------------------------------------------- # -pop_001=$(grep "^0.01%" ${attrank_file} | cut -f 2); -pop_01=$(grep "^0.1%" ${attrank_file} | cut -f 2); -pop_1=$(grep "^1%" ${attrank_file} | cut -f 2); -pop_10=$(grep "^10%" ${attrank_file} | cut -f 2); -echo "Popularity limits:"; -echo -e "${pop_001}\t${pop_01}\t${pop_1}\t${pop_10}"; -# ---------------------------------------------------- # -imp_001=$(grep "^0.01%" ${impulse_file} | cut -f 2); -imp_01=$(grep "^0.1%" ${impulse_file} | cut -f 2); -imp_1=$(grep "^1%" ${impulse_file} | cut -f 2); -imp_10=$(grep "^10%" ${impulse_file} | cut -f 2); -echo "Popularity limits:"; -echo -e "${imp_001}\t${imp_01}\t${imp_1}\t${imp_10}"; -# ---------------------------------------------------- # -cc_001=$(grep "^0.01%" ${cc_file} | cut -f 2); -cc_01=$(grep "^0.1%" ${cc_file} | cut -f 2); -cc_1=$(grep "^1%" ${cc_file} | cut -f 2); -cc_10=$(grep "^10%" ${cc_file} | cut -f 2); -echo "Popularity limits:"; -echo -e "${cc_001}\t${cc_01}\t${cc_1}\t${cc_10}"; -# ---------------------------------------------------- # - -echo -e "${inf_001}\t${inf_01}\t${inf_1}\t${inf_10}\t${pop_001}\t${pop_01}\t${pop_1}\t${pop_10}\t${imp_001}\t${imp_01}\t${imp_1}\t${imp_10}\t${cc_001}\t${cc_01}\t${cc_1}\t${cc_10}" >> score_limits.csv - -echo -echo "score_limits.csv contents:" -cat score_limits.csv - -echo; -echo; diff --git a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_openaire_ids_to_dois.py b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_openaire_ids_to_dois.py deleted file mode 100644 index 7997eec82..000000000 --- a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_openaire_ids_to_dois.py +++ /dev/null @@ -1,60 +0,0 @@ -import json -import sys -from pyspark.sql import SparkSession -from pyspark import SparkConf, SparkContext - -if len(sys.argv) != 3: - print("Usage: map_openaire_ids_to_dois.py ") - sys.exit(-1) - -conf = SparkConf().setAppName('BIP!: Map OpenAIRE IDs to DOIs') -sc = SparkContext(conf = conf) -spark = SparkSession.builder.appName('BIP!: Map OpenAIRE IDs to DOIs').getOrCreate() -sc.setLogLevel('OFF') - -src_dir = sys.argv[1] -output = sys.argv[2] - -# src_dir = "/tmp/beta_provision/graph/21_graph_cleaned/" -# output = '/tmp/openaireid_to_dois/' - -def transform(doc): - - # get publication year from 'doc.dateofacceptance.value' - dateofacceptance = doc.get('dateofacceptance', {}).get('value') - - year = 0 - - if (dateofacceptance is not None): - year = dateofacceptance.split('-')[0] - - # for each pid get 'pid.value' if 'pid.qualifier.classid' equals to 'doi' - dois = [ pid['value'] for pid in doc.get('pid', []) if (pid.get('qualifier', {}).get('classid') == 'doi' and pid['value'] is not None)] - - num_dois = len(dois) - - # exlcude openaire ids that do not correspond to DOIs - if (num_dois == 0): - return None - - fields = [ doc['id'], str(num_dois), chr(0x02).join(dois), str(year) ] - - return '\t'.join([ v.encode('utf-8') for v in fields ]) - -docs = None - -for result_type in ["publication", "dataset", "software", "otherresearchproduct"]: - - tmp = sc.textFile(src_dir + result_type).map(json.loads) - - if (docs is None): - docs = tmp - else: - # append all result types in one RDD - docs = docs.union(tmp) - -docs = docs.filter(lambda d: d.get('dataInfo', {}).get('deletedbyinference') == False and d.get('dataInfo', {}).get('invisible') == False) - -docs = docs.map(transform).filter(lambda d: d is not None) - -docs.saveAsTextFile(output) diff --git a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_scores_to_dois.py b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_scores_to_dois.py deleted file mode 100755 index f6a8e9996..000000000 --- a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/map_scores_to_dois.py +++ /dev/null @@ -1,168 +0,0 @@ -#!/usr/bin/python -# This program reads the openaire to doi mapping from the ${synonymFolder} of the workflow -# and uses this mapping to create doi-based score files in the format required by BiP! DB. -# This is done by reading each openaire-id based ranking file and joining the openaire based -# score and classes to all the corresponding dois. -################################################################################################# -# Imports -import sys - -# Sparksession lib to communicate with cluster via session object -from pyspark.sql import SparkSession - -# Import sql types to define schemas -from pyspark.sql.types import * - -# Import sql functions with shorthand alias -import pyspark.sql.functions as F - -from pyspark.sql.functions import max -# from pyspark.sql.functions import udf -################################################################################################# -################################################################################################# -# Clean up directory name - no longer needed in final workflow version -''' -def clean_directory_name(dir_name): - # We have a name with the form *_bip_universe_* or *_graph_universe_* - # and we need to keep the parts in * - - - dir_name_parts = dir_name.split('_') - dir_name_parts = [part for part in dir_name_parts if ('bip' not in part and 'graph' not in part and 'universe' not in part and 'from' not in part)] - - dir_name = dir_name.replace("openaire_id_graph", "openaire_ids") - clean_name = dir_name + ".txt.gz" - - # clean_name = '_'.join(dir_name_parts) - - # if '_ids' not in clean_name: - # clean_name = clean_name.replace('id_', 'ids_') - - # clean_name = clean_name.replace('.txt', '') - # clean_name = clean_name.replace('.gz', '') - - # if 'openaire_ids_' in clean_name: - # clean_name = clean_name.replace('openaire_ids_', '') - # clean_name = clean_name + '.txt.gz' - # else: - # clean_name = clean_name + '.txt.gz' - - return clean_name -''' -################################################################################################# -if len(sys.argv) < 3: - print ("Usage: ./map_scores_to_dois.py <...etc...>") - sys.exit(-1) - -# Read arguments -synonyms_folder = sys.argv[1] -num_partitions = int(sys.argv[2]) -input_file_list = [argument.replace("_openaire_id_graph", "").replace("_openaire_id_graph_", "") + "_openaire_ids.txt.gz" for argument in sys.argv[3:]] -# input_file_list = [clean_directory_name(item) for item in input_file_list] - -# Prepare output specific variables -output_file_list = [item.replace("_openaire_ids", "") for item in input_file_list] -output_file_list = [item + ".txt.gz" if not item.endswith(".txt.gz") else item for item in output_file_list] - -# --- INFO MESSAGES --- # -print ("\n\n----------------------------") -print ("Mpping openaire ids to DOIs") -print ("Reading input from: " + synonyms_folder) -print ("Num partitions: " + str(num_partitions)) -print ("Input files:" + " -- ".join(input_file_list)) -print ("Output files: " + " -- ".join(output_file_list)) -print ("----------------------------\n\n") -####################################################################################### -# We weill define the following schemas: -# --> the schema of the openaire - doi mapping file [string - int - doi_list] (the separator of the doi-list is a non printable character) -# --> a schema for floating point ranking scores [string - float - string] (the latter string is the class) -# --> a schema for integer ranking scores [string - int - string] (the latter string is the class) - -float_schema = StructType([ - StructField('id', StringType(), False), - StructField('score', FloatType(), False), - StructField('class', StringType(), False) - ]) - -int_schema = StructType([ - StructField('id', StringType(), False), - StructField('score', IntegerType(), False), - StructField('class', StringType(), False) - ]) - -# This schema concerns the output of the file -# containing the number of references of each doi -synonyms_schema = StructType([ - StructField('id', StringType(), False), - StructField('num_synonyms', IntegerType(), False), - StructField('doi_list', StringType(), False), - ]) -####################################################################################### -# Start spark session -spark = SparkSession.builder.appName('Map openaire scores to DOIs').getOrCreate() -# Set Log Level for spark session -spark.sparkContext.setLogLevel('WARN') -####################################################################################### -# MAIN Program - -# Read and repartition the synonym folder - also cache it since we will need to perform multiple joins -synonym_df = spark.read.schema(synonyms_schema).option('delimiter', '\t').csv(synonyms_folder) -synonym_df = synonym_df.select('id', F.split(F.col('doi_list'), chr(0x02)).alias('doi_list')) -synonym_df = synonym_df.select('id', F.explode('doi_list').alias('doi')).repartition(num_partitions, 'id').cache() - -# TESTING -# print ("Synonyms: " + str(synonym_df.count())) -# print ("DF looks like this:" ) -# synonym_df.show(1000, False) - -print ("\n\n-----------------------------") -# Now we need to join the score files on the openaire-id with the synonyms and then keep -# only doi - score - class and write this to the output -for offset, input_file in enumerate(input_file_list): - - print ("Mapping scores from " + input_file) - - # Select correct schema - schema = int_schema - if "attrank" in input_file.lower() or "pr" in input_file.lower() or "ram" in input_file.lower(): - schema = float_schema - - # Load file to dataframe - ranking_df = spark.read.schema(schema).option('delimiter', '\t').csv(input_file).repartition(num_partitions, 'id') - - # Get max score - max_score = ranking_df.select(max('score').alias('max')).collect()[0]['max'] - print ("Max Score for " + str(input_file) + " is " + str(max_score)) - - # TESTING - # print ("Loaded df sample:") - # ranking_df.show(1000, False) - - # Join scores to synonyms and keep required fields - doi_score_df = synonym_df.join(ranking_df, ['id']).select('doi', 'score', 'class').repartition(num_partitions, 'doi').cache() - # Write output - output_file = output_file_list[offset] - print ("Writing to: " + output_file) - doi_score_df.write.mode('overwrite').option('delimiter','\t').option('header',False).csv(output_file, compression='gzip') - - # Creata another file for the bip update process - ranking_df = ranking_df.select('id', 'score', F.lit(F.col('score')/max_score).alias('normalized_score'), 'class', F.col('class').alias('class_dup')) - doi_score_df = synonym_df.join(ranking_df, ['id']).select('doi', 'score', 'normalized_score', 'class', 'class_dup').repartition(num_partitions, 'doi').cache() - output_file = output_file.replace(".txt.gz", "_for_bip_update.txt.gz") - print ("Writing bip update to: " + output_file) - doi_score_df.write.mode('overwrite').option('delimiter','\t').option('header',False).csv(output_file, compression='gzip') - - - # Free memory? - ranking_df.unpersist(True) - -print ("-----------------------------") -print ("\n\nFinished!\n\n") - - - - - - - - diff --git a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml index 70f5f8d2a..108cf70b1 100644 --- a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml @@ -17,10 +17,6 @@ openaireGraphInputPath ${nameNode}/${workingDir}/openaire_id_graph - - synonymFolder - ${nameNode}/${workingDir}/openaireid_to_dois/ - checkpointDir ${nameNode}/${workingDir}/check/ @@ -32,29 +28,34 @@ - + - - + + + ${wf:conf('resume') eq "start"} + + ${wf:conf('resume') eq "cc"} ${wf:conf('resume') eq "ram"} ${wf:conf('resume') eq "impulse"} ${wf:conf('resume') eq "pagerank"} ${wf:conf('resume') eq "attrank"} - - ${wf:conf('resume') eq "format-results"} - ${wf:conf('resume') eq "map-ids"} - ${wf:conf('resume') eq "map-scores"} - ${wf:conf('resume') eq "start"} - + + ${wf:conf('resume') eq "format-results"} + + ${wf:conf('resume') eq "projects-impact"} + + ${wf:conf('resume') eq "create-actionset"} + + @@ -295,18 +296,11 @@ - + - - - - - - - @@ -345,139 +339,8 @@ ${wfAppPath}/format_ranking_results.py#format_ranking_results.py - - - - - - - - - - - yarn-cluster - cluster - - - Format Ranking Results BiP! DB - - format_ranking_results.py - - - - --executor-memory=${sparkNormalExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkNormalDriverMemory} - --conf spark.executor.memoryOverhead=${sparkNormalExecutorMemory} - --conf spark.sql.shuffle.partitions=${sparkShufflePartitions} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - - - zenodo - - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['pr_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['attrank_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['cc_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['impulse_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['ram_file']} - - ${sparkShufflePartitions} - - openaire - - ${wfAppPath}/format_ranking_results.py#format_ranking_results.py - - - - - - - - - - - - - - - - - - - yarn-cluster - cluster - Openaire-DOI synonym collection - map_openaire_ids_to_dois.py - - - --executor-memory=${sparkHighExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkHighDriverMemory} - --conf spark.executor.memoryOverhead=${sparkHighExecutorMemory} - --conf spark.sql.shuffle.partitions=${sparkShufflePartitions} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - - - ${openaireDataInput}/ - - ${synonymFolder} - - ${wfAppPath}/map_openaire_ids_to_dois.py#map_openaire_ids_to_dois.py - - - - - - - - - - - - - - yarn-cluster - cluster - Mapping Openaire Scores to DOIs - map_scores_to_dois.py - - - --executor-memory=${sparkHighExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkHighDriverMemory} - --conf spark.executor.memoryOverhead=${sparkHighExecutorMemory} - --conf spark.sql.shuffle.partitions=${sparkShufflePartitions} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - - - ${synonymFolder} - - ${sparkShufflePartitions} - - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['pr_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['attrank_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['cc_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['impulse_file']} - ${nameNode}/${workingDir}/${wf:actionData('get-file-names')['ram_file']} - - ${wfAppPath}/map_scores_to_dois.py#map_scores_to_dois.py - - - - + From b043f8a96370cfdf593fb05c71b119d6175fe240 Mon Sep 17 00:00:00 2001 From: Serafeim Chatzopoulos Date: Wed, 4 Sep 2024 14:28:43 +0300 Subject: [PATCH 111/239] Remove redundant error messages from impact indicators workflow --- .../graph/impact_indicators/oozie_app/workflow.xml | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml index 108cf70b1..5d8669823 100644 --- a/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-impact-indicators/src/main/resources/eu/dnetlib/dhp/oa/graph/impact_indicators/oozie_app/workflow.xml @@ -457,18 +457,6 @@ Error formatting json files, error message[${wf:errorMessage(wf:lastErrorNode())}] - - Error formatting BIP files, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - Synonym collection failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - Mapping scores to DOIs failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - Deleting output path for actionsets failed, error message[${wf:errorMessage(wf:lastErrorNode())}] From b64c144abfad208e7c0c5137adce3f1fc555706f Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Thu, 5 Sep 2024 16:00:09 +0300 Subject: [PATCH 112/239] added new institutions --- .../scripts/step20-createMonitorDBAll.sql | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index a8392b226..ca5b0bb90 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -81,11 +81,17 @@ create table TARGET.result stored as parquet as 'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development 'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology 'openorgs____::b5ca9d4340e26454e367e2908ef3872f', -- Alma Mater Studiorum University of Bologna - 'openorgs____::a6340e6ecf60f6bba163659df985b0f2', -- TU Dresden - 'openorgs____::64badd35233ba2cd4946368ef2f4cf57', -- University of Vienna + 'openorgs____::a6340e6ecf60f6bba163659df985b0f2', -- TU Dresden + 'openorgs____::64badd35233ba2cd4946368ef2f4cf57', -- University of Vienna 'openorgs____::7501d66d2297a963ebfb075c43fff88e', -- Royal Institute of Technology - 'openorgs____::d5eb679abdd31f70fcd4c8ba711148bf', -- Sorbonne University - 'openorgs____::b316f25380d106aac402f5ae8653910d' -- Centre for Research on Ecology and Forestry Applications + 'openorgs____::d5eb679abdd31f70fcd4c8ba711148bf', -- Sorbonne University + 'openorgs____::b316f25380d106aac402f5ae8653910d', -- Centre for Research on Ecology and Forestry Applications + 'openorgs____::45a2076eee3013e0e85625ce61bcd272', -- Institut d'Investigació Sanitària Illes Balears + 'openorgs____::00b20b0a743a96169e6cf135e6e2bd7c', -- Universidad Publica De Navarra + 'openorgs____::0f398605c2459294d125ff23473a97dc', -- Aalto University + 'openorgs____::25b1fa62c7fd8e409d3a83c07e04b2d4', -- WHU-Otto Beisheim School of Management + 'openorgs____::d6eec313417f11205db4e736a34c0db6', -- KEMPELENOV INSTITUT INTELIGENTNYCH TECHNOLOGII + 'openorgs____::c2dfb90e797a2dc52f0084c549289d0c' -- National Research Institute for Agriculture, Food and Environment ) )) foo; create view if not exists TARGET.category as select * from SOURCE.category; From 37ad259296c686ce3adee1758a2dbbdab9f35ab9 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Thu, 5 Sep 2024 16:02:44 +0300 Subject: [PATCH 113/239] cleanup --- .../oozie_app/scripts/updateMonitorDB.sql | 76 ------------------- 1 file changed, 76 deletions(-) diff --git a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB.sql b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB.sql index 321fba87a..ede8a18bf 100644 --- a/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB.sql +++ b/dhp-workflows/dhp-stats-monitor-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor/oozie_app/scripts/updateMonitorDB.sql @@ -1,79 +1,3 @@ ---drop database if exists TARGET cascade; ---create database if not exists TARGET; --- ---create view if not exists TARGET.category as select * from SOURCE.category; ---create view if not exists TARGET.concept as select * from SOURCE.concept; ---create view if not exists TARGET.context as select * from SOURCE.context; ---create view if not exists TARGET.country as select * from SOURCE.country; ---create view if not exists TARGET.countrygdp as select * from SOURCE.countrygdp; ---create view if not exists TARGET.creation_date as select * from SOURCE.creation_date; ---create view if not exists TARGET.funder as select * from SOURCE.funder; ---create view if not exists TARGET.fundref as select * from SOURCE.fundref; ---create view if not exists TARGET.rndexpenditure as select * from SOURCE.rndexpediture; ---create view if not exists TARGET.rndgdpexpenditure as select * from SOURCE.rndgdpexpenditure; ---create view if not exists TARGET.doctoratestudents as select * from SOURCE.doctoratestudents; ---create view if not exists TARGET.totalresearchers as select * from SOURCE.totalresearchers; ---create view if not exists TARGET.totalresearchersft as select * from SOURCE.totalresearchersft; ---create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; --- ---create table TARGET.result stored as parquet as --- select distinct * from ( --- select * from SOURCE.result r where exists (select 1 from SOURCE.result_projects rp join SOURCE.project p on rp.project=p.id where rp.id=r.id) --- union all --- select * from SOURCE.result r where exists (select 1 from SOURCE.result_concepts rc where rc.id=r.id) --- union all --- select * from SOURCE.result r where exists (select 1 from SOURCE.result_organization ro where ro.id=r.id and ro.organization in ( --- 'openorgs____::b84450f9864182c67b8611b5593f4250', --"Athena Research and Innovation Center In Information Communication & Knowledge Technologies', --ARC" --- 'openorgs____::d41cf6bd4ab1b1362a44397e0b95c975', --National Research Council --- 'openorgs____::d2a09b9d5eabb10c95f9470e172d05d2', --??? Not exists ?? --- 'openorgs____::d169c7407dd417152596908d48c11460', --Masaryk University --- 'openorgs____::1ec924b1759bb16d0a02f2dad8689b21', --University of Belgrade --- 'openorgs____::0ae431b820e4c33db8967fbb2b919150', --University of Helsinki --- 'openorgs____::759d59f05d77188faee99b7493b46805', --University of Minho --- 'openorgs____::cad284878801b9465fa51a95b1d779db', --Universidad Politécnica de Madrid --- 'openorgs____::eadc8da90a546e98c03f896661a2e4d4', --University of Göttingen --- 'openorgs____::c0286313e36479eff8676dba9b724b40', --National and Kapodistrian University of Athens --- -- 'openorgs____::c80a8243a5e5c620d7931c88d93bf17a', --Université Paris Diderot --- 'openorgs____::c08634f0a6b0081c3dc6e6c93a4314f3', --Bielefeld University --- 'openorgs____::6fc85e4a8f7ecaf4b0c738d010e967ea', --University of Southern Denmark --- 'openorgs____::3d6122f87f9a97a99d8f6e3d73313720', --Humboldt-Universität zu Berlin --- 'openorgs____::16720ada63d0fa8ca41601feae7d1aa5', --TU Darmstadt --- 'openorgs____::ccc0a066b56d2cfaf90c2ae369df16f5', --KU Leuven --- 'openorgs____::4c6f119632adf789746f0a057ed73e90', --University of the Western Cape --- 'openorgs____::ec3665affa01aeafa28b7852c4176dbd', --Rudjer Boskovic Institute --- 'openorgs____::5f31346d444a7f06a28c880fb170b0f6', --Ghent University --- 'openorgs____::2dbe47117fd5409f9c61620813456632', --University of Luxembourg --- 'openorgs____::6445d7758d3a40c4d997953b6632a368', --National Institute of Informatics (NII) --- 'openorgs____::b77c01aa15de3675da34277d48de2ec1', -- Valencia Catholic University Saint Vincent Martyr --- 'openorgs____::7fe2f66cdc43983c6b24816bfe9cf6a0', -- Unviersity of Warsaw --- 'openorgs____::15e7921fc50d9aa1229a82a84429419e', -- University Of Thessaly --- 'openorgs____::11f7919dadc8f8a7251af54bba60c956', -- Technical University of Crete --- 'openorgs____::84f0c5f5dbb6daf42748485924efde4b', -- University of Piraeus --- 'openorgs____::4ac562f0376fce3539504567649cb373', -- University of Patras --- 'openorgs____::3e8d1f8c3f6cd7f418b09f1f58b4873b', -- Aristotle University of Thessaloniki --- 'openorgs____::3fcef6e1c469c10f2a84b281372c9814', -- World Bank --- 'openorgs____::1698a2eb1885ef8adb5a4a969e745ad3', -- École des Ponts ParisTech --- 'openorgs____::e15adb13c4dadd49de4d35c39b5da93a', -- Nanyang Technological University --- 'openorgs____::4b34103bde246228fcd837f5f1bf4212', -- Autonomous University of Barcelona --- 'openorgs____::72ec75fcfc4e0df1a76dc4c49007fceb', -- McMaster University --- 'openorgs____::51c7fc556e46381734a25a6fbc3fd398', -- University of Modena and Reggio Emilia --- 'openorgs____::235d7f9ad18ecd7e6dc62ea4990cb9db', -- Bilkent University --- 'openorgs____::31f2fa9e05b49d4cf40a19c3fed8eb06', -- Saints Cyril and Methodius University of Skopje --- 'openorgs____::db7686f30f22cbe73a4fde872ce812a6', -- University of Milan --- 'openorgs____::b8b8ca674452579f3f593d9f5e557483', -- University College Cork --- 'openorgs____::38d7097854736583dde879d12dacafca' -- Brown University --- 'openorgs____::57784c9e047e826fefdb1ef816120d92', --Arts et Métiers ParisTech --- 'openorgs____::2530baca8a15936ba2e3297f2bce2e7e', -- University of Cape Town --- 'openorgs____::d11f981828c485cd23d93f7f24f24db1', -- Technological University Dublin --- 'openorgs____::5e6bf8962665cdd040341171e5c631d8', -- Delft University of Technology --- 'openorgs____::846cb428d3f52a445f7275561a7beb5d', -- University of Manitoba --- 'openorgs____::eb391317ed0dc684aa81ac16265de041', -- Universitat Rovira i Virgili --- 'openorgs____::66aa9fc2fceb271423dfabcc38752dc0', -- Lund University --- 'openorgs____::3cff625a4370d51e08624cc586138b2f' -- IMT Atlantique --- ) )) foo; --- ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; - create view if not exists TARGET.category as select * from SOURCE.category; create view if not exists TARGET.concept as select * from SOURCE.concept; create view if not exists TARGET.context as select * from SOURCE.context; From 07e6e7b4d6e3489a55d8dca917c65e28ead21275 Mon Sep 17 00:00:00 2001 From: Alessia Date: Mon, 16 Sep 2024 13:41:56 +0200 Subject: [PATCH 114/239] #9839: include claimed affiliation relationships --- .../raw/MigrateDbEntitiesApplication.java | 22 ++++++++++++ .../raw/MigrateDbEntitiesApplicationTest.java | 35 +++++++++++++++++++ .../raw/claimsrel_resultset_affiliation.json | 27 ++++++++++++++ 3 files changed, 84 insertions(+) create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimsrel_resultset_affiliation.json diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index c9a32cde6..00505fedc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -519,6 +519,28 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i r1 = setRelationSemantic(r1, RESULT_RESULT, PUBLICATION_DATASET, IS_RELATED_TO); r2 = setRelationSemantic(r2, RESULT_RESULT, PUBLICATION_DATASET, IS_RELATED_TO); break; + case "resultOrganization_affiliation_isAuthorInstitutionOf": + if (!"organization".equals(sourceType)) { + throw new IllegalStateException( + String + .format( + "invalid claim, sourceId: %s, targetId: %s, semantics: %s", sourceId, targetId, + semantics)); + } + r1 = setRelationSemantic(r1, RESULT_ORGANIZATION, AFFILIATION, IS_AUTHOR_INSTITUTION_OF); + r2 = setRelationSemantic(r2, RESULT_ORGANIZATION, AFFILIATION, HAS_AUTHOR_INSTITUTION); + break; + case "resultOrganization_affiliation_hasAuthorInstitution": + if (!"organization".equals(targetType)) { + throw new IllegalStateException( + String + .format( + "invalid claim, sourceId: %s, targetId: %s, semantics: %s", sourceId, targetId, + semantics)); + } + r1 = setRelationSemantic(r1, RESULT_ORGANIZATION, AFFILIATION, HAS_AUTHOR_INSTITUTION); + r2 = setRelationSemantic(r2, RESULT_ORGANIZATION, AFFILIATION, IS_AUTHOR_INSTITUTION_OF); + break; default: throw new IllegalArgumentException("claim semantics not managed: " + semantics); } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java index 27304ec06..c4d1b6b58 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java @@ -16,6 +16,8 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.common.RelationInverse; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.BeforeEach; @@ -364,6 +366,39 @@ class MigrateDbEntitiesApplicationTest { assertValidId(r1.getCollectedfrom().get(0).getKey()); assertValidId(r2.getCollectedfrom().get(0).getKey()); } + @Test + void testProcessClaims_affiliation() throws Exception { + final List fields = prepareMocks("claimsrel_resultset_affiliation.json"); + + final List list = app.processClaims(rs); + + assertEquals(2, list.size()); + verifyMocks(fields); + + assertTrue(list.get(0) instanceof Relation); + assertTrue(list.get(1) instanceof Relation); + + final Relation r1 = (Relation) list.get(0); + final Relation r2 = (Relation) list.get(1); + + assertValidId(r1.getSource()); + assertValidId(r1.getTarget()); + assertValidId(r2.getSource()); + assertValidId(r2.getTarget()); + assertNotNull(r1.getDataInfo()); + assertNotNull(r2.getDataInfo()); + assertNotNull(r1.getDataInfo().getTrust()); + assertNotNull(r2.getDataInfo().getTrust()); + assertEquals(r1.getSource(), r2.getTarget()); + assertEquals(r2.getSource(), r1.getTarget()); + assertTrue(StringUtils.isNotBlank(r1.getRelClass())); + assertTrue(StringUtils.isNotBlank(r2.getRelClass())); + assertTrue(StringUtils.isNotBlank(r1.getRelType())); + assertTrue(StringUtils.isNotBlank(r2.getRelType())); + + assertValidId(r1.getCollectedfrom().get(0).getKey()); + assertValidId(r2.getCollectedfrom().get(0).getKey()); + } private List prepareMocks(final String jsonFile) throws IOException, SQLException { final String json = IOUtils.toString(getClass().getResourceAsStream(jsonFile)); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimsrel_resultset_affiliation.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimsrel_resultset_affiliation.json new file mode 100644 index 000000000..07cc025d6 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimsrel_resultset_affiliation.json @@ -0,0 +1,27 @@ +[ + { + "field": "source_type", + "type": "string", + "value": "organization" + }, + { + "field": "source_id", + "type": "string", + "value": "openorgs____::b5ca9d4340e26454e367e2908ef3872f" + }, + { + "field": "target_type", + "type": "string", + "value": "software" + }, + { + "field": "target_id", + "type": "string", + "value": "userclaim___::bde53826d07c8cf47c99222a375cd2e8" + }, + { + "field": "semantics", + "type": "string", + "value": "resultOrganization_affiliation_isAuthorInstitutionOf" + } +] \ No newline at end of file From 6df6b4583ebeecda8ff69cd370b8d39d5d8dd7b3 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 16 Sep 2024 14:04:59 +0200 Subject: [PATCH 115/239] blacklist filtering moved before the cleanup phase in order to have case sensitive regex --- .../NumAuthorsTitleSuffixPrefixChain.java | 2 +- .../java/eu/dnetlib/pace/model/FieldDef.java | 16 +++ .../eu/dnetlib/pace/model/SparkDeduper.scala | 40 +------ .../eu/dnetlib/pace/model/SparkModel.scala | 100 ++++++++++++------ .../clustering/ClusteringFunctionTest.java | 13 +++ .../pace/comparators/ComparatorTest.java | 12 +++ .../dnetlib/dhp/oa/dedup/SparkBlockStats.java | 1 - .../dnetlib/dhp/dedup/conf/pub.curr.conf.json | 2 +- 8 files changed, 111 insertions(+), 75 deletions(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/NumAuthorsTitleSuffixPrefixChain.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/NumAuthorsTitleSuffixPrefixChain.java index f1d1e17b9..4e6d8231f 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/NumAuthorsTitleSuffixPrefixChain.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/clustering/NumAuthorsTitleSuffixPrefixChain.java @@ -38,7 +38,7 @@ public class NumAuthorsTitleSuffixPrefixChain extends AbstractClusteringFunction @Override protected Collection doApply(Config conf, String s) { - return suffixPrefixChain(cleanup(s), param("mod")); + return suffixPrefixChain(cleanup(s), paramOrDefault("mod", 10)); } private Collection suffixPrefixChain(String s, int mod) { diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java index b0dc11656..2e329f690 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/FieldDef.java @@ -54,6 +54,22 @@ public class FieldDef implements Serializable { public FieldDef() { } + public FieldDef clone() { + FieldDef fieldDef = new FieldDef(); + fieldDef.setName(this.name); + fieldDef.setPath(this.path); + fieldDef.setType(this.type); + fieldDef.setOverrideMatch(this.overrideMatch); + fieldDef.setSize(this.size); + fieldDef.setLength(this.length); + fieldDef.setFilter(this.filter); + fieldDef.setSorted(this.sorted); + fieldDef.setClean(this.clean); + fieldDef.setInfer(this.infer); + fieldDef.setInferenceFrom(this.inferenceFrom); + return fieldDef; + } + public String getInferenceFrom() { return inferenceFrom; } diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkDeduper.scala b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkDeduper.scala index bc702b9e2..a3eb3cba8 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkDeduper.scala +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkDeduper.scala @@ -19,48 +19,10 @@ case class SparkDeduper(conf: DedupConfig) extends Serializable { val model: SparkModel = SparkModel(conf) val dedup: (Dataset[Row] => Dataset[Row]) = df => { - df.transform(filterAndCleanup) - .transform(generateClustersWithCollect) + df.transform(generateClustersWithCollect) .transform(processBlocks) } - - val filterAndCleanup: (Dataset[Row] => Dataset[Row]) = df => { - val df_with_filters = conf.getPace.getModel.asScala.foldLeft(df)((res, fdef) => { - if (conf.blacklists.containsKey(fdef.getName)) { - res.withColumn( - fdef.getName + "_filtered", - filterColumnUDF(fdef).apply(new Column(fdef.getName)) - ) - } else { - res - } - }) - - df_with_filters - } - - def filterColumnUDF(fdef: FieldDef): UserDefinedFunction = { - val blacklist: Predicate[String] = conf.blacklists().get(fdef.getName) - - if (blacklist == null) { - throw new IllegalArgumentException("Column: " + fdef.getName + " does not have any filter") - } else { - fdef.getType match { - case Type.List | Type.JSON => - udf[Array[String], Array[String]](values => { - values.filter((v: String) => !blacklist.test(v)) - }) - - case _ => - udf[String, String](v => { - if (blacklist.test(v)) "" - else v - }) - } - } - } - val generateClustersWithCollect: (Dataset[Row] => Dataset[Row]) = df_with_filters => { var df_with_clustering_keys: Dataset[Row] = null diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala index c6db62339..580a88b7e 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/model/SparkModel.scala @@ -5,12 +5,12 @@ import eu.dnetlib.pace.common.AbstractPaceFunctions import eu.dnetlib.pace.config.{DedupConfig, Type} import eu.dnetlib.pace.util.{MapDocumentUtil, SparkCompatUtils} import org.apache.commons.lang3.StringUtils -import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import org.apache.spark.sql.{Dataset, Row} import java.util.Locale +import java.util.function.Predicate import java.util.regex.Pattern import scala.collection.JavaConverters._ @@ -29,8 +29,20 @@ case class SparkModel(conf: DedupConfig) { identifier.setName(identifierFieldName) identifier.setType(Type.String) + // create fields for blacklist + val filtered = conf.getPace.getModel.asScala.flatMap(fdef => { + if (conf.blacklists().containsKey(fdef.getName)) { + val fdef_filtered = fdef.clone() + fdef_filtered.setName(fdef.getName + "_filtered") + Seq(fdef, fdef_filtered) + } + else { + Seq(fdef) + } + }) + // Construct a Spark StructType representing the schema of the model - (Seq(identifier) ++ conf.getPace.getModel.asScala) + (Seq(identifier) ++ filtered) .foldLeft( new StructType() )((resType, fieldDef) => { @@ -44,7 +56,6 @@ case class SparkModel(conf: DedupConfig) { }) }) - } val identityFieldPosition: Int = schema.fieldIndex(identifierFieldName) @@ -52,7 +63,8 @@ case class SparkModel(conf: DedupConfig) { val orderingFieldPosition: Int = schema.fieldIndex(orderingFieldName) val parseJsonDataset: (Dataset[String] => Dataset[Row]) = df => { - df.map(r => rowFromJson(r))(SparkCompatUtils.encoderFor(schema)) + df + .map(r => rowFromJson(r))(SparkCompatUtils.encoderFor(schema)) } def rowFromJson(json: String): Row = { @@ -64,41 +76,63 @@ case class SparkModel(conf: DedupConfig) { schema.fieldNames.zipWithIndex.foldLeft(values) { case ((res, (fname, index))) => - val fdef = conf.getPace.getModelMap.get(fname) + + val fdef = conf.getPace.getModelMap.get(fname.split("_filtered")(0)) if (fdef != null) { - res(index) = fdef.getType match { - case Type.String | Type.Int => - MapDocumentUtil.truncateValue( - MapDocumentUtil.getJPathString(fdef.getPath, documentContext), - fdef.getLength - ) + if (!fname.contains("_filtered")) { //process fields with no blacklist + res(index) = fdef.getType match { + case Type.String | Type.Int => + MapDocumentUtil.truncateValue( + MapDocumentUtil.getJPathString(fdef.getPath, documentContext), + fdef.getLength + ) - case Type.URL => - var uv = MapDocumentUtil.getJPathString(fdef.getPath, documentContext) - if (!URL_REGEX.matcher(uv).matches) - uv = "" - uv + case Type.URL => + var uv = MapDocumentUtil.getJPathString(fdef.getPath, documentContext) + if (!URL_REGEX.matcher(uv).matches) + uv = "" + uv - case Type.List | Type.JSON => - MapDocumentUtil.truncateList( - MapDocumentUtil.getJPathList(fdef.getPath, documentContext, fdef.getType), - fdef.getSize - ).asScala + case Type.List | Type.JSON => + MapDocumentUtil.truncateList( + MapDocumentUtil.getJPathList(fdef.getPath, documentContext, fdef.getType), + fdef.getSize + ).asScala - case Type.StringConcat => - val jpaths = CONCAT_REGEX.split(fdef.getPath) + case Type.StringConcat => + val jpaths = CONCAT_REGEX.split(fdef.getPath) - MapDocumentUtil.truncateValue( - jpaths - .map(jpath => MapDocumentUtil.getJPathString(jpath, documentContext)) - .mkString(" "), - fdef.getLength - ) + MapDocumentUtil.truncateValue( + jpaths + .map(jpath => MapDocumentUtil.getJPathString(jpath, documentContext)) + .mkString(" "), + fdef.getLength + ) - case Type.DoubleArray => - MapDocumentUtil.getJPathArray(fdef.getPath, json) + case Type.DoubleArray => + MapDocumentUtil.getJPathArray(fdef.getPath, json) + } } + else { //process fields with blacklist + val blacklist: Predicate[String] = conf.blacklists().get(fdef.getName) + + res(index) = fdef.getType match { + case Type.List | Type.JSON => + MapDocumentUtil.truncateList( + MapDocumentUtil.getJPathList(fdef.getPath, documentContext, fdef.getType), + fdef.getSize + ).asScala.filter((v: String) => !blacklist.test(v)) + + case _ => + val value: String = MapDocumentUtil.truncateValue( + MapDocumentUtil.getJPathString(fdef.getPath, documentContext), + fdef.getLength + ) + if (blacklist.test(value)) "" else value + } + } + val filter = fdef.getFilter @@ -125,13 +159,12 @@ case class SparkModel(conf: DedupConfig) { } if (StringUtils.isNotBlank(fdef.getInfer)) { - val inferFrom : String = if (StringUtils.isNotBlank(fdef.getInferenceFrom)) fdef.getInferenceFrom else fdef.getPath + val inferFrom: String = if (StringUtils.isNotBlank(fdef.getInferenceFrom)) fdef.getInferenceFrom else fdef.getPath res(index) = res(index) match { case x: Seq[String] => x.map(inference(_, MapDocumentUtil.getJPathString(inferFrom, documentContext), fdef.getInfer)) case _ => inference(res(index).toString, MapDocumentUtil.getJPathString(inferFrom, documentContext), fdef.getInfer) } } - } res @@ -139,6 +172,7 @@ case class SparkModel(conf: DedupConfig) { } new GenericRowWithSchema(values, schema) + } def clean(value: String, cleantype: String) : String = { diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java index e62f742f8..236f17eca 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/clustering/ClusteringFunctionTest.java @@ -227,4 +227,17 @@ public class ClusteringFunctionTest extends AbstractPaceTest { System.out.println(cf.apply(conf, Lists.newArrayList(s))); } + @Test + public void testNumAuthorsTitleSuffixPrefixChain() { + + final ClusteringFunction cf = new NumAuthorsTitleSuffixPrefixChain(params); + params.put("mod", 10); + + final String title = "PARP-2 Regulates SIRT1 Expression and Whole-Body Energy Expenditure"; + final String num_authors = "10"; + System.out.println("title = " + title); + System.out.println("num_authors = " + num_authors); + System.out.println(cf.apply(conf, Lists.newArrayList(num_authors, title))); + } + } diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index c008902c4..d2e83e695 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -327,4 +327,16 @@ public class ComparatorTest extends AbstractPaceTest { } + @Test + public void titleVersionMatchTest() { + + TitleVersionMatch titleVersionMatch = new TitleVersionMatch(params); + + double result = titleVersionMatch + .compare( + "parp 2 regulates sirt 1 expression and whole body energy expenditure", + "parp 2 regulates sirt 1 expression and whole body energy expenditure", conf); + assertEquals(1.0, result); + } + } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java index 3e5215d42..612a1cb19 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -91,7 +91,6 @@ public class SparkBlockStats extends AbstractSparkAction { .read() .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) .transform(deduper.model().parseJsonDataset()) - .transform(deduper.filterAndCleanup()) .transform(deduper.generateClustersWithCollect()) .filter(functions.size(new Column("block")).geq(1)); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json index c3a769874..c5ff1c1fa 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json @@ -96,7 +96,7 @@ "aggregation": "MAX", "positive": "layer4", "negative": "NO_MATCH", - "undefined": "MATCH", + "undefined": "layer4", "ignoreUndefined": "true" }, "layer4": { From bb9cee4f40caed22e19b65b9ee7e8ca7e5d2848b Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 16 Sep 2024 14:16:56 +0200 Subject: [PATCH 116/239] implementation of gtr2Publications plugin --- .../dhp/collection/CollectorWorker.java | 59 ++--- .../collection/plugin/CollectorPlugin.java | 2 +- .../gtr2/Gtr2PublicationsCollectorPlugin.java | 40 ++++ .../plugin/gtr2/Gtr2PublicationsIterator.java | 210 ++++++++++++++++++ .../oa/provision/utils/XmlRecordFactory.java | 2 +- 5 files changed, 283 insertions(+), 30 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java index 98caa1741..5021b7727 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java @@ -22,6 +22,7 @@ import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; import eu.dnetlib.dhp.collection.plugin.base.BaseCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.file.FileCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.file.FileGZipCollectorPlugin; +import eu.dnetlib.dhp.collection.plugin.gtr2.Gtr2PublicationsCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.mongodb.MDStoreCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.mongodb.MongoDbDumpCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.oai.OaiCollectorPlugin; @@ -58,7 +59,7 @@ public class CollectorWorker extends ReportingJob { public void collect() throws UnknownCollectorPluginException, CollectorException, IOException { - final String outputPath = mdStoreVersion.getHdfsPath() + SEQUENCE_FILE_NAME; + final String outputPath = this.mdStoreVersion.getHdfsPath() + SEQUENCE_FILE_NAME; log.info("outputPath path is {}", outputPath); final CollectorPlugin plugin = getCollectorPlugin(); @@ -68,36 +69,36 @@ public class CollectorWorker extends ReportingJob { try (SequenceFile.Writer writer = SequenceFile .createWriter( - fileSystem.getConf(), - SequenceFile.Writer.file(new Path(outputPath)), - SequenceFile.Writer.keyClass(IntWritable.class), - SequenceFile.Writer.valueClass(Text.class), + this.fileSystem.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer + .keyClass(IntWritable.class), + SequenceFile.Writer + .valueClass(Text.class), SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) { final IntWritable key = new IntWritable(counter.get()); final Text value = new Text(); plugin - .collect(api, report) - .forEach( - content -> { - key.set(counter.getAndIncrement()); - value.set(content); - try { - writer.append(key, value); - } catch (Throwable e) { - throw new RuntimeException(e); - } - }); - } catch (Throwable e) { - report.put(e.getClass().getName(), e.getMessage()); + .collect(this.api, this.report) + .forEach(content -> { + key.set(counter.getAndIncrement()); + value.set(content); + try { + writer.append(key, value); + } catch (final Throwable e) { + throw new RuntimeException(e); + } + }); + } catch (final Throwable e) { + this.report.put(e.getClass().getName(), e.getMessage()); throw new CollectorException(e); } finally { shutdown(); - report.ongoing(counter.longValue(), counter.longValue()); + this.report.ongoing(counter.longValue(), counter.longValue()); } } - private void scheduleReport(AtomicInteger counter) { + private void scheduleReport(final AtomicInteger counter) { schedule(new ReporterCallback() { + @Override public Long getCurrent() { return counter.longValue(); @@ -112,33 +113,35 @@ public class CollectorWorker extends ReportingJob { private CollectorPlugin getCollectorPlugin() throws UnknownCollectorPluginException { - switch (CollectorPlugin.NAME.valueOf(api.getProtocol())) { + switch (CollectorPlugin.NAME.valueOf(this.api.getProtocol())) { case oai: - return new OaiCollectorPlugin(clientParams); + return new OaiCollectorPlugin(this.clientParams); case rest_json2xml: - return new RestCollectorPlugin(clientParams); + return new RestCollectorPlugin(this.clientParams); case file: - return new FileCollectorPlugin(fileSystem); + return new FileCollectorPlugin(this.fileSystem); case fileGzip: - return new FileGZipCollectorPlugin(fileSystem); + return new FileGZipCollectorPlugin(this.fileSystem); case baseDump: return new BaseCollectorPlugin(this.fileSystem); + case gtr2Publications: + return new Gtr2PublicationsCollectorPlugin(this.clientParams); case other: final CollectorPlugin.NAME.OTHER_NAME plugin = Optional - .ofNullable(api.getParams().get("other_plugin_type")) + .ofNullable(this.api.getParams().get("other_plugin_type")) .map(CollectorPlugin.NAME.OTHER_NAME::valueOf) .orElseThrow(() -> new IllegalArgumentException("invalid other_plugin_type")); switch (plugin) { case mdstore_mongodb_dump: - return new MongoDbDumpCollectorPlugin(fileSystem); + return new MongoDbDumpCollectorPlugin(this.fileSystem); case mdstore_mongodb: return new MDStoreCollectorPlugin(); default: throw new UnknownCollectorPluginException("plugin is not managed: " + plugin); } default: - throw new UnknownCollectorPluginException("protocol is not managed: " + api.getProtocol()); + throw new UnknownCollectorPluginException("protocol is not managed: " + this.api.getProtocol()); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java index 97d2d2585..0bba8d764 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java @@ -11,7 +11,7 @@ public interface CollectorPlugin { enum NAME { - oai, other, rest_json2xml, file, fileGzip, baseDump; + oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications; public enum OTHER_NAME { mdstore_mongodb_dump, mdstore_mongodb diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java new file mode 100644 index 000000000..543bcbe5a --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java @@ -0,0 +1,40 @@ + +package eu.dnetlib.dhp.collection.plugin.gtr2; + +import java.util.Iterator; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +public class Gtr2PublicationsCollectorPlugin implements CollectorPlugin { + + private final HttpClientParams clientParams; + + public Gtr2PublicationsCollectorPlugin(final HttpClientParams clientParams) { + this.clientParams = clientParams; + } + + @Override + public Stream collect(final ApiDescriptor api, final AggregatorReport report) throws CollectorException { + + final String baseUrl = api.getBaseUrl(); + final String startPage = api.getParams().get("startPage"); + final String endPage = api.getParams().get("endPage"); + final String fromDate = api.getParams().get("fromDate"); + + if ((fromDate != null) && !fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { throw new CollectorException("Invalid date (YYYY-MM-DD): " + fromDate); } + + final Iterator iterator = new Gtr2PublicationsIterator(baseUrl, fromDate, startPage, endPage, this.clientParams); + final Spliterator spliterator = Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED); + + return StreamSupport.stream(spliterator, false); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java new file mode 100644 index 000000000..c3c4cbac0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java @@ -0,0 +1,210 @@ +package eu.dnetlib.dhp.collection.plugin.gtr2; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.function.Function; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.DocumentHelper; +import org.dom4j.Element; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.common.collection.HttpConnector2; + +public class Gtr2PublicationsIterator implements Iterator { + + public static final int PAGE_SIZE = 20; + + private static final Logger log = LoggerFactory.getLogger(Gtr2PublicationsIterator.class); + + private final HttpConnector2 connector; + private static final DateTimeFormatter simpleDateTimeFormatter = DateTimeFormat.forPattern("yyyy-MM-dd"); + + private static final int MAX_ATTEMPTS = 10; + + private final String baseUrl; + private int currPage; + private int endPage; + private boolean incremental = false; + private DateTime fromDate; + + private final Map cache = new HashMap<>(); + + private final Queue queue = new LinkedList<>(); + + private String nextElement; + + public Gtr2PublicationsIterator(final String baseUrl, final String fromDate, final String startPage, final String endPage, + final HttpClientParams clientParams) + throws CollectorException { + + this.baseUrl = baseUrl; + this.currPage = NumberUtils.toInt(startPage, 1); + this.endPage = NumberUtils.toInt(endPage, Integer.MAX_VALUE); + this.incremental = StringUtils.isNotBlank(fromDate); + this.connector = new HttpConnector2(clientParams); + + if (this.incremental) { + this.fromDate = parseDate(fromDate); + } + + prepareNextElement(); + } + + @Override + public boolean hasNext() { + return this.nextElement != null; + } + + @Override + public String next() { + try { + return this.nextElement; + } finally { + prepareNextElement(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + private void prepareNextElement() { + while ((this.currPage <= this.endPage) && this.queue.isEmpty()) { + log.debug("FETCHING PAGE + " + this.currPage + "/" + this.endPage); + this.queue.addAll(fetchPage(this.currPage++)); + } + this.nextElement = this.queue.poll(); + } + + private List fetchPage(final int pageNumber) { + + final List res = new ArrayList<>(); + try { + final Document doc = loadURL(cleanURL(this.baseUrl + "/outcomes/publications?p=" + pageNumber), 0); + + if (this.endPage == Integer.MAX_VALUE) { + this.endPage = NumberUtils.toInt(doc.valueOf("/*/@*[local-name() = 'totalPages']")); + } + + for (final Object po : doc.selectNodes("//*[local-name() = 'publication']")) { + final Element mainEntity = (Element) ((Element) po).detach(); + + if (filterIncremental(mainEntity)) { + res.add(expandMainEntity(mainEntity)); + } else { + log.debug("Skipped entity"); + } + + } + } catch (final Throwable e) { + log.error("Exception fetching page " + pageNumber, e); + throw new RuntimeException("Exception fetching page " + pageNumber, e); + } + + return res; + } + + private void addLinkedEntities(final Element master, final String relType, final Element newRoot, final Function mapper) { + + for (final Object o : master.selectNodes(".//*[local-name()='link']")) { + final String rel = ((Element) o).valueOf("@*[local-name()='rel']"); + final String href = ((Element) o).valueOf("@*[local-name()='href']"); + + if (relType.equals(rel) && StringUtils.isNotBlank(href)) { + final String cacheKey = relType + "#" + href; + if (this.cache.containsKey(cacheKey)) { + try { + log.debug(" * from cache (" + relType + "): " + href); + newRoot.add(DocumentHelper.parseText(this.cache.get(cacheKey)).getRootElement()); + } catch (final DocumentException e) { + log.error("Error retrieving cache element: " + cacheKey, e); + throw new RuntimeException("Error retrieving cache element: " + cacheKey, e); + } + } else { + final Document doc = loadURL(cleanURL(href), 0); + final Element elem = mapper.apply(doc); + newRoot.add(elem); + this.cache.put(cacheKey, elem.asXML()); + } + + } + } + } + + private boolean filterIncremental(final Element e) { + if (!this.incremental || isAfter(e.valueOf("@*[local-name() = 'created']"), this.fromDate) + || isAfter(e.valueOf("@*[local-name() = 'updated']"), this.fromDate)) { + return true; + } + return false; + } + + private String expandMainEntity(final Element mainEntity) { + final Element newRoot = DocumentHelper.createElement("doc"); + newRoot.add(mainEntity); + addLinkedEntities(mainEntity, "PROJECT", newRoot, this::asProjectElement); + return DocumentHelper.createDocument(newRoot).asXML(); + } + + private Element asProjectElement(final Document doc) { + final Element newOrg = DocumentHelper.createElement("project"); + newOrg.addElement("id").setText(doc.valueOf("/*/@*[local-name()='id']")); + newOrg.addElement("code").setText(doc.valueOf("//*[local-name()='identifier' and @*[local-name()='type'] = 'RCUK']")); + newOrg.addElement("title").setText(doc.valueOf("//*[local-name()='title']")); + return newOrg; + } + + private static String cleanURL(final String url) { + String cleaned = url; + if (cleaned.contains("gtr.gtr")) { + cleaned = cleaned.replace("gtr.gtr", "gtr"); + } + if (cleaned.startsWith("http://")) { + cleaned = cleaned.replaceFirst("http://", "https://"); + } + return cleaned; + } + + private Document loadURL(final String cleanUrl, final int attempt) { + try { + log.debug(" * Downloading Url: " + cleanUrl); + final byte[] bytes = this.connector.getInputSource(cleanUrl).getBytes("UTF-8"); + return DocumentHelper.parseText(new String(bytes)); + } catch (final Throwable e) { + log.error("Error dowloading url: " + cleanUrl + ", attempt = " + attempt, e); + if (attempt >= MAX_ATTEMPTS) { throw new RuntimeException("Error dowloading url: " + cleanUrl, e); } + try { + Thread.sleep(60000); // I wait for a minute + } catch (final InterruptedException e1) { + throw new RuntimeException("Error dowloading url: " + cleanUrl, e); + } + return loadURL(cleanUrl, attempt + 1); + } + } + + private DateTime parseDate(final String s) { + // I expect dates in the format 'yyyy-MM-dd'. See class + // eu.dnetlib.msro.workflows.nodes.collect.FindDateRangeForIncrementalHarvestingJobNode + return DateTime.parse(s.substring(0, s.indexOf("T")), simpleDateTimeFormatter); + } + + private boolean isAfter(final String d, final DateTime fromDate) { + return parseDate(d).isAfter(fromDate); + } +} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 44004faf3..3c8f5cef5 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -20,7 +20,6 @@ import javax.xml.transform.*; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; -import eu.dnetlib.dhp.oa.provision.model.*; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -42,6 +41,7 @@ import com.google.common.collect.Sets; import com.mycila.xmltool.XMLDoc; import com.mycila.xmltool.XMLTag; +import eu.dnetlib.dhp.oa.provision.model.*; import eu.dnetlib.dhp.schema.common.*; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Result; From 99b7adda0c352e8439d3dc09d5b72a324875f58f Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 16 Sep 2024 15:13:44 +0200 Subject: [PATCH 117/239] gtr2 unit test --- .../gtr2/Gtr2PublicationsIteratorTest.java | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java new file mode 100644 index 000000000..737ef0dca --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java @@ -0,0 +1,101 @@ +package eu.dnetlib.dhp.collection.plugin.gtr2; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +import java.util.Iterator; + +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +class Gtr2PublicationsIteratorTest { + + private static final String baseURL = "https://gtr.ukri.org/gtr/api"; + + private static final HttpClientParams clientParams = new HttpClientParams(); + + @Test + @Disabled + public void testOne() throws Exception { + System.out.println("one publication"); + + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, null, null, null, clientParams); + + if (iterator.hasNext()) { + final String res = iterator.next(); + assertNotNull(res); + System.out.println(res); + } + } + + @Test + @Disabled + public void testPaging() throws Exception { + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, null, "2", "2", clientParams); + + while (iterator.hasNext()) { + Thread.sleep(300); + final String res = iterator.next(); + assertNotNull(res); + System.out.println(res); + } + } + + @Test + @Disabled + public void testOnePage() throws Exception { + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, null, "12", "12", clientParams); + final int count = iterateAndCount(iterator); + assertEquals(21, count); + } + + @Test + @Disabled + public void testIncrementalHarvestingNoRecords() throws Exception { + System.out.println("incremental Harvesting"); + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2050-12-12", "11", "13", clientParams); + final int count = iterateAndCount(iterator); + assertEquals(1, count); + } + + @Test + @Disabled + public void testIncrementalHarvesting() throws Exception { + System.out.println("incremental Harvesting"); + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2016-11-30", "11", "11", clientParams); + final int count = iterateAndCount(iterator); + assertEquals(21, count); + } + + @Test + @Disabled + public void testCompleteHarvesting() throws Exception { + System.out.println("testing complete harvesting"); + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, null, null, null, clientParams); + // TryIndentXmlString indenter = new TryIndentXmlString(); + // it.setEndAtPage(3); + + while (iterator.hasNext()) { + final String res = iterator.next(); + assertNotNull(res); + // System.out.println(res); + // Scanner keyboard = new Scanner(System.in); + // System.out.println("press enter for next record"); + // keyboard.nextLine(); + + } + } + + private int iterateAndCount(final Iterator iterator) throws Exception { + int i = 0; + while (iterator.hasNext()) { + assertNotNull(iterator.next()); + i++; + } + System.out.println("Got " + i + " publications"); + return i; + } + +} From a2fac78dcc77ca3647765f73c4cbe86f40614dc6 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 17 Sep 2024 10:16:28 +0200 Subject: [PATCH 118/239] fixed a problem in incremental harvesting --- .../collection/plugin/gtr2/Gtr2PublicationsIterator.java | 6 ++---- .../plugin/gtr2/Gtr2PublicationsIteratorTest.java | 8 ++++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java index c3c4cbac0..9b122bbe6 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java @@ -199,12 +199,10 @@ public class Gtr2PublicationsIterator implements Iterator { } private DateTime parseDate(final String s) { - // I expect dates in the format 'yyyy-MM-dd'. See class - // eu.dnetlib.msro.workflows.nodes.collect.FindDateRangeForIncrementalHarvestingJobNode - return DateTime.parse(s.substring(0, s.indexOf("T")), simpleDateTimeFormatter); + return DateTime.parse(s.contains("T") ? s.substring(0, s.indexOf("T")) : s, simpleDateTimeFormatter); } private boolean isAfter(final String d, final DateTime fromDate) { - return parseDate(d).isAfter(fromDate); + return StringUtils.isNotBlank(d) && parseDate(d).isAfter(fromDate); } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java index 737ef0dca..f02feb0ef 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java @@ -48,16 +48,16 @@ class Gtr2PublicationsIteratorTest { public void testOnePage() throws Exception { final Iterator iterator = new Gtr2PublicationsIterator(baseURL, null, "12", "12", clientParams); final int count = iterateAndCount(iterator); - assertEquals(21, count); + assertEquals(20, count); } @Test @Disabled public void testIncrementalHarvestingNoRecords() throws Exception { System.out.println("incremental Harvesting"); - final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2050-12-12", "11", "13", clientParams); + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2050-12-12T", "11", "13", clientParams); final int count = iterateAndCount(iterator); - assertEquals(1, count); + assertEquals(0, count); } @Test @@ -66,7 +66,7 @@ class Gtr2PublicationsIteratorTest { System.out.println("incremental Harvesting"); final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2016-11-30", "11", "11", clientParams); final int count = iterateAndCount(iterator); - assertEquals(21, count); + assertEquals(20, count); } @Test From bfd05cdab26e4d2ed0cca1e5c91dfbbb37d588e7 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 16 Sep 2024 16:16:23 +0200 Subject: [PATCH 119/239] run mergeResultsOfDifferentTypes only when checkDelegatedAuthority is true --- .../dhp/schema/oaf/utils/MergeUtils.java | 3 +- .../PromoteResultWithMeasuresTest.java | 210 ++++++++++++++++++ .../measures/actionPayloads/part0000.json | 3 + .../promote/measures/graph/part00000.json | 1 + 4 files changed, 216 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java create mode 100644 dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json create mode 100644 dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index ea402ecbf..ac7694d18 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -30,6 +30,7 @@ import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; public class MergeUtils { + public static T mergeById(String s, Iterator oafEntityIterator) { return mergeGroup(s, oafEntityIterator, true); } @@ -88,7 +89,7 @@ public class MergeUtils { private static Oaf mergeEntities(Oaf left, Oaf right, boolean checkDelegatedAuthority) { if (sameClass(left, right, Result.class)) { - if (!left.getClass().equals(right.getClass()) || checkDelegatedAuthority) { + if (checkDelegatedAuthority) { return mergeResultsOfDifferentTypes((Result) left, (Result) right); } diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java new file mode 100644 index 000000000..3eafe7115 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java @@ -0,0 +1,210 @@ +/* + * Copyright (c) 2024. + * SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche + * SPDX-License-Identifier: AGPL-3.0-or-later + */ + +package eu.dnetlib.dhp.actionmanager.promote; + +import static eu.dnetlib.dhp.common.FunctionalInterfaceSupport.*; +import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; +import static org.apache.spark.sql.functions.*; +import static org.junit.jupiter.api.Assertions.*; + +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashSet; +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; + +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; + +public class PromoteResultWithMeasuresTest { + + private static final Logger log = LoggerFactory.getLogger(PromoteResultWithMeasuresTest.class); + + private static SparkSession spark; + + private static Path tempDir; + + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @BeforeAll + public static void beforeAll() throws IOException { + tempDir = Files.createTempDirectory(PromoteResultWithMeasuresTest.class.getSimpleName()); + log.info("using work dir {}", tempDir); + + SparkConf conf = new SparkConf(); + conf.setMaster("local[*]"); + conf.setAppName(PromoteResultWithMeasuresTest.class.getSimpleName()); + conf.set("spark.driver.host", "localhost"); + + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + + conf.set("spark.sql.warehouse.dir", tempDir.toString()); + conf.set("hive.metastore.warehouse.dir", tempDir.resolve("warehouse").toString()); + + spark = SparkSession.builder().config(conf).getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + spark.stop(); + FileUtils.deleteDirectory(tempDir.toFile()); + } + + @Test + void testPromoteResultWithMeasures_job() throws Exception { + + final String inputGraphTablePath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/promote/measures/graph") + .getPath(); + + final String inputActionPayloadPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads") + .getPath(); + + final String actionPayloadsPath = tempDir.resolve("actionPayloads").toString(); + + spark + .read() + .text(inputActionPayloadPath) + .withColumn("payload", col("value")) + .select("payload") + .write() + .parquet(actionPayloadsPath); + + final Path outputGraphTablePath = tempDir.resolve("outputGraphTablePath"); + + PromoteActionPayloadForGraphTableJob + .main(new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--graphTableClassName", Publication.class.getCanonicalName(), + "--inputGraphTablePath", inputGraphTablePath, + "--inputActionPayloadPath", actionPayloadsPath, + "--actionPayloadClassName", Result.class.getCanonicalName(), + "--outputGraphTablePath", outputGraphTablePath.toString(), + "--mergeAndGetStrategy", MergeAndGet.Strategy.MERGE_FROM_AND_GET.toString(), + "--promoteActionStrategy", PromoteAction.Strategy.ENRICH.toString(), + "--shouldGroupById", "true" + }); + + assertFalse(isDirEmpty(outputGraphTablePath)); + + final Encoder pubEncoder = Encoders.bean(Publication.class); + List results = spark + .read() + .schema(pubEncoder.schema()) + .json(outputGraphTablePath.toString()) + .as(pubEncoder) + .collectAsList(); + + verify(results); + } + + @Test + void testPromoteResultWithMeasures_internal() throws JsonProcessingException { + + Dataset rowDS = spark + .read() + .schema(Encoders.bean(Publication.class).schema()) + .json("src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph") + .as(Encoders.bean(Publication.class)); + + Dataset actionPayloadDS = spark + .read() + .schema(Encoders.bean(Result.class).schema()) + .json("src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads") + .as(Encoders.bean(Result.class)); + + final MergeAndGet.Strategy mergeFromAndGet = MergeAndGet.Strategy.MERGE_FROM_AND_GET; + + final SerializableSupplier> rowIdFn = ModelSupport::idFn; + final SerializableSupplier> mergeAndGetFn = MergeAndGet + .functionFor(mergeFromAndGet); + final SerializableSupplier zeroFn = () -> Publication.class + .cast(new eu.dnetlib.dhp.schema.oaf.Publication()); + final SerializableSupplier> isNotZeroFn = PromoteResultWithMeasuresTest::isNotZeroFnUsingIdOrSourceAndTarget; + + Dataset joinedResults = PromoteActionPayloadFunctions + .joinGraphTableWithActionPayloadAndMerge( + rowDS, + actionPayloadDS, + rowIdFn, + ModelSupport::idFn, + mergeAndGetFn, + PromoteAction.Strategy.ENRICH, + Publication.class, + Result.class); + + SerializableSupplier> mergeRowsAndGetFn = MergeAndGet + .functionFor(mergeFromAndGet); + + Dataset mergedResults = PromoteActionPayloadFunctions + .groupGraphTableByIdAndMerge( + joinedResults, rowIdFn, mergeRowsAndGetFn, zeroFn, isNotZeroFn, Publication.class); + + verify(mergedResults.collectAsList()); + } + + private static void verify(List results) throws JsonProcessingException { + assertNotNull(results); + assertEquals(1, results.size()); + + Result r = results.get(0); + + log.info(OBJECT_MAPPER.writeValueAsString(r)); + + assertNotNull(r.getMeasures()); + assertFalse(r.getMeasures().isEmpty()); + assertTrue( + r + .getMeasures() + .stream() + .map(Measure::getId) + .collect(Collectors.toCollection(HashSet::new)) + .containsAll( + Lists + .newArrayList( + "downloads", "views", "influence", "popularity", "influence_alt", "popularity_alt", + "impulse"))); + } + + private static Function isNotZeroFnUsingIdOrSourceAndTarget() { + return t -> { + if (isSubClass(t, Relation.class)) { + final Relation rel = (Relation) t; + return StringUtils.isNotBlank(rel.getSource()) && StringUtils.isNotBlank(rel.getTarget()); + } + return StringUtils.isNotBlank(((OafEntity) t).getId()); + }; + } + + private static boolean isDirEmpty(final Path directory) throws IOException { + try (DirectoryStream dirStream = Files.newDirectoryStream(directory)) { + return !dirStream.iterator().hasNext(); + } + } + +} diff --git a/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json new file mode 100644 index 000000000..806bcf5c8 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json @@ -0,0 +1,3 @@ +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"id":"50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":[{"id":"downloads","unit":[{"key":"opendoar____::358aee4cc897452c00244351e4d91f69||ZENODO","value":"125","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:usage_counts","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"views","unit":[{"key":"opendoar____::358aee4cc897452c00244351e4d91f69||ZENODO","value":"35","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:usage_counts","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]}],"context":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":null,"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"transformativeAgreement":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"id":"50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":[{"id":"influence","unit":[{"key":"score","value":"3.1167566E-9","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"popularity","unit":[{"key":"score","value":"7.335433E-9","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"influence_alt","unit":[{"key":"score","value":"4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"popularity_alt","unit":[{"key":"score","value":"2.96","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"impulse","unit":[{"key":"score","value":"4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]}],"context":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":null,"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"transformativeAgreement":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"id":"50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"context":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":null,"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":"hybrid","publiclyFunded":false,"transformativeAgreement":null,"isGreen":true,"isInDiamondJournal":false} \ No newline at end of file diff --git a/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json new file mode 100644 index 000000000..9f03cebe4 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json @@ -0,0 +1 @@ +{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:dedup", "classname": "sysimport:dedup", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "dedup-result-decisiontree-v4", "invisible": false, "trust": "0.8"}, "resourcetype": {"classid": "publication", "classname": "publication", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"qualifier": {"classid": "mag_id", "classname": "Microsoft Academic Graph Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "3086187510"}], "bestaccessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "relevantdate": [{"qualifier": {"classid": "created", "classname": "created", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-14"}, {"qualifier": {"classid": "published-online", "classname": "published-online", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-14"}, {"qualifier": {"classid": "published-print", "classname": "published-print", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-08-01"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "issued", "classname": "issued", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-02"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "updated", "classname": "updated", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-01"}], "contributor": [], "id": "50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b", "description": [{"value": "We present SPHINX, a system for metapath-based entity exploration in Heterogeneous Information Networks (HINs). SPHINX allows users to define different views over a HIN based on both automatically selected and user-defined meta-paths. Then, entity ranking and similarity search can be performed over these views to find and explore entities of interest, taking also into account any spatial or temporal properties of entities. A Web-based user interface is provided to facilitate users in performing the various functionalities supported by the system, including metapath-based view definition, index construction, search parameters specification, and visual comparison of the results."}], "lastupdatetimestamp": 1725554400176, "author": [{"fullname": "Thanasis Vergoulis", "pid": [], "rank": 1}, {"fullname": "Kostas Patroumpas", "pid": [], "rank": 2}, {"fullname": "Alexandros Zeakis", "pid": [], "rank": 3}, {"fullname": "Dimitrios Skoutas", "pid": [], "rank": 4}, {"fullname": "Serafeim Chatzopoulos", "pid": [], "rank": 5}], "collectedfrom": [{"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, {"value": "Crossref", "key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"}, {"value": "Microsoft Academic Graph", "key": "10|openaire____::5f532a3fc4f1ea403f37070f59a7a53a"}, {"value": "UnpayWall", "key": "10|openaire____:8ac8380272269217cb09a928c8caa993"}, {"value": "European Union Open Data Portal", "key": "10|re3data_____::c4b2081b224be6b3e79d0e5e5556f631"}], "instance": [{"refereed": {"classid": "0001", "classname": "peerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Proceedings of the VLDB Endowment", "key": "10|issn___print::8e719dcc0c83f87be79812fcf8024e2b"}, "url": ["https://doi.org/10.14778/3415478.3415507"], "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "instanceTypeMapping": [{"originalType": "journal-article", "typeLabel": "research article", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_2df8fbb1"}, {"originalType": "http://purl.org/coar/resource_type/c_2df8fbb1", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "dateofacceptance": {"value": "2020-08-01"}, "collectedfrom": {"value": "Crossref", "key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Proceedings of the VLDB Endowment", "key": "10|issn___print::8e719dcc0c83f87be79812fcf8024e2b"}, "license": {"value": "CC BY"}, "url": ["https://zenodo.org/record/4010307/files/p2913-chatzopoulos.pdf"], "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "instanceTypeMapping": [{"originalType": "journal-article", "typeLabel": "research article", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_2df8fbb1"}, {"originalType": "http://purl.org/coar/resource_type/c_2df8fbb1", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "collectedfrom": {"value": "UnpayWall", "key": "10|openaire____:8ac8380272269217cb09a928c8caa993"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes", "openAccessRoute": "green"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "Unknown Repository", "key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c"}, "url": ["http://dx.doi.org/10.14778/3415478.3415507"], "pid": [], "instanceTypeMapping": [{"originalType": "CONFERENCE_PROCEEDING", "vocabularyName": "openaire::coar_resource_types_3_1"}], "distributionlocation": "", "alternateIdentifier": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "value": "2020-01-01"}, "collectedfrom": {"value": "European Union Open Data Portal", "key": "10|re3data_____::c4b2081b224be6b3e79d0e5e5556f631"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0004", "classname": "Conference object", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "Unknown Repository", "key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c"}, "url": ["http://dx.doi.org/10.14778/3415478.3415507"], "pid": [], "instanceTypeMapping": [{"originalType": "Conference proceedings", "typeLabel": "conference proceedings", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_f744"}], "distributionlocation": "", "alternateIdentifier": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "value": "2020-01-01"}, "collectedfrom": {"value": "European Union Open Data Portal", "key": "10|re3data_____::c4b2081b224be6b3e79d0e5e5556f631"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0038", "classname": "Other literature type", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, "license": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "CC BY"}, "url": ["http://dx.doi.org/10.14778/3415478.3415507"], "pid": [], "instanceTypeMapping": [{"originalType": "ConferencePaper", "typeLabel": "conference paper", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_5794"}, {"originalType": "http://purl.org/coar/resource_type/c_5794", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "alternateIdentifier": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "oai", "classname": "Open Archives Initiative", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "oai:zenodo.org:4010307"}], "dateofacceptance": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "2020-09-02"}, "collectedfrom": {"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0004", "classname": "Conference object", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Unknown Repository", "key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c"}, "url": ["http://www.vldb.org/pvldb/vol13/p2913-chatzopoulos.pdf", "https://dblp.uni-trier.de/db/journals/pvldb/pvldb13.html#ChatzopoulosPZV20", "https://dl.acm.org/doi/10.14778/3415478.3415507", "https://doi.org/10.14778/3415478.3415507"], "pid": [{"qualifier": {"classid": "mag_id", "classname": "Microsoft Academic Graph Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "3086187510"}, {"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "instanceTypeMapping": [{"originalType": "Conference", "typeLabel": "conference output", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_c94f"}], "collectedfrom": {"value": "Microsoft Academic Graph", "key": "10|openaire____::5f532a3fc4f1ea403f37070f59a7a53a"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}], "dateofcollection": "2024-09-05T16:53:05.687", "metaResourceType": {"classid": "Research Literature", "classname": "Research Literature", "schemename": "openaire::meta_resource_types", "schemeid": "openaire::meta_resource_types"}, "fulltext": [], "dateofacceptance": {"value": "2020-08-01"}, "format": [], "journal": {"issnPrinted": "2150-8097", "vol": "13", "sp": "2913", "ep": "2916", "name": "Proceedings of the VLDB Endowment"}, "subject": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "02 engineering and technology"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "0202 electrical engineering, electronic engineering, information engineering"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5467381477355957"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "020201 artificial intelligence & image processing"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.4532618224620819"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "020204 information systems"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "02020108 Machine learning/Social Info Processing"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "02020402 Cryptography/Information governance"}], "coverage": [], "externalReference": [], "publisher": {"value": "Association for Computing Machinery (ACM)"}, "eoscifguidelines": [], "language": {"classid": "und", "classname": "Undetermined", "schemename": "dnet:languages", "schemeid": "dnet:languages"}, "resulttype": {"classid": "publication", "classname": "publication", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "country": [], "extraInfo": [], "originalId": ["10.14778/3415478.3415507", "50|doiboost____|02317b7093277ec8aa0311d5c6a25b9b", "825041_1260870_PUBLI", "50|r3c4b2081b22::0d0cc9ff8949f9091272abb7a9e083f8", "50|r3c4b2081b22::02317b7093277ec8aa0311d5c6a25b9b", "oai:zenodo.org:4010307", "50|od______2659::de3dfee8ed6f2e53c85690531ab23028", "3086187510"], "source": [{"value": "Crossref"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "value": "International Conference on Very Large Data Bases (VLDB)"}], "context": [{"dataInfo": [{"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}], "id": "https://zenodo.org/communities/smartdatalake-project"}, {"dataInfo": [{"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}], "id": "https://zenodo.org/communities/eu"}], "title": [{"qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "SPHINX"}, {"qualifier": {"classid": "subtitle", "classname": "subtitle", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "a system for metapath-based entity exploration in heterogeneous information networks"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "SPHINX: A System for Metapath-based Entity Exploration in Heterogeneous Information Networks"}, {"qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "sphinx a system for metapath based entity exploration in heterogeneous information networks"}]} \ No newline at end of file From 23e0ab3a7c301a9f1bf3e10beb1944e08cb14fcb Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 16 Sep 2024 16:16:23 +0200 Subject: [PATCH 120/239] run mergeResultsOfDifferentTypes only when checkDelegatedAuthority is true --- .../dhp/schema/oaf/utils/MergeUtils.java | 3 +- .../PromoteResultWithMeasuresTest.java | 210 ++++++++++++++++++ .../measures/actionPayloads/part0000.json | 3 + .../promote/measures/graph/part00000.json | 1 + 4 files changed, 216 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java create mode 100644 dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json create mode 100644 dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index ea402ecbf..ac7694d18 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -30,6 +30,7 @@ import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; public class MergeUtils { + public static T mergeById(String s, Iterator oafEntityIterator) { return mergeGroup(s, oafEntityIterator, true); } @@ -88,7 +89,7 @@ public class MergeUtils { private static Oaf mergeEntities(Oaf left, Oaf right, boolean checkDelegatedAuthority) { if (sameClass(left, right, Result.class)) { - if (!left.getClass().equals(right.getClass()) || checkDelegatedAuthority) { + if (checkDelegatedAuthority) { return mergeResultsOfDifferentTypes((Result) left, (Result) right); } diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java new file mode 100644 index 000000000..3eafe7115 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteResultWithMeasuresTest.java @@ -0,0 +1,210 @@ +/* + * Copyright (c) 2024. + * SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche + * SPDX-License-Identifier: AGPL-3.0-or-later + */ + +package eu.dnetlib.dhp.actionmanager.promote; + +import static eu.dnetlib.dhp.common.FunctionalInterfaceSupport.*; +import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; +import static org.apache.spark.sql.functions.*; +import static org.junit.jupiter.api.Assertions.*; + +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashSet; +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; + +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; + +public class PromoteResultWithMeasuresTest { + + private static final Logger log = LoggerFactory.getLogger(PromoteResultWithMeasuresTest.class); + + private static SparkSession spark; + + private static Path tempDir; + + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @BeforeAll + public static void beforeAll() throws IOException { + tempDir = Files.createTempDirectory(PromoteResultWithMeasuresTest.class.getSimpleName()); + log.info("using work dir {}", tempDir); + + SparkConf conf = new SparkConf(); + conf.setMaster("local[*]"); + conf.setAppName(PromoteResultWithMeasuresTest.class.getSimpleName()); + conf.set("spark.driver.host", "localhost"); + + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + + conf.set("spark.sql.warehouse.dir", tempDir.toString()); + conf.set("hive.metastore.warehouse.dir", tempDir.resolve("warehouse").toString()); + + spark = SparkSession.builder().config(conf).getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + spark.stop(); + FileUtils.deleteDirectory(tempDir.toFile()); + } + + @Test + void testPromoteResultWithMeasures_job() throws Exception { + + final String inputGraphTablePath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/promote/measures/graph") + .getPath(); + + final String inputActionPayloadPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads") + .getPath(); + + final String actionPayloadsPath = tempDir.resolve("actionPayloads").toString(); + + spark + .read() + .text(inputActionPayloadPath) + .withColumn("payload", col("value")) + .select("payload") + .write() + .parquet(actionPayloadsPath); + + final Path outputGraphTablePath = tempDir.resolve("outputGraphTablePath"); + + PromoteActionPayloadForGraphTableJob + .main(new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--graphTableClassName", Publication.class.getCanonicalName(), + "--inputGraphTablePath", inputGraphTablePath, + "--inputActionPayloadPath", actionPayloadsPath, + "--actionPayloadClassName", Result.class.getCanonicalName(), + "--outputGraphTablePath", outputGraphTablePath.toString(), + "--mergeAndGetStrategy", MergeAndGet.Strategy.MERGE_FROM_AND_GET.toString(), + "--promoteActionStrategy", PromoteAction.Strategy.ENRICH.toString(), + "--shouldGroupById", "true" + }); + + assertFalse(isDirEmpty(outputGraphTablePath)); + + final Encoder pubEncoder = Encoders.bean(Publication.class); + List results = spark + .read() + .schema(pubEncoder.schema()) + .json(outputGraphTablePath.toString()) + .as(pubEncoder) + .collectAsList(); + + verify(results); + } + + @Test + void testPromoteResultWithMeasures_internal() throws JsonProcessingException { + + Dataset rowDS = spark + .read() + .schema(Encoders.bean(Publication.class).schema()) + .json("src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph") + .as(Encoders.bean(Publication.class)); + + Dataset actionPayloadDS = spark + .read() + .schema(Encoders.bean(Result.class).schema()) + .json("src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads") + .as(Encoders.bean(Result.class)); + + final MergeAndGet.Strategy mergeFromAndGet = MergeAndGet.Strategy.MERGE_FROM_AND_GET; + + final SerializableSupplier> rowIdFn = ModelSupport::idFn; + final SerializableSupplier> mergeAndGetFn = MergeAndGet + .functionFor(mergeFromAndGet); + final SerializableSupplier zeroFn = () -> Publication.class + .cast(new eu.dnetlib.dhp.schema.oaf.Publication()); + final SerializableSupplier> isNotZeroFn = PromoteResultWithMeasuresTest::isNotZeroFnUsingIdOrSourceAndTarget; + + Dataset joinedResults = PromoteActionPayloadFunctions + .joinGraphTableWithActionPayloadAndMerge( + rowDS, + actionPayloadDS, + rowIdFn, + ModelSupport::idFn, + mergeAndGetFn, + PromoteAction.Strategy.ENRICH, + Publication.class, + Result.class); + + SerializableSupplier> mergeRowsAndGetFn = MergeAndGet + .functionFor(mergeFromAndGet); + + Dataset mergedResults = PromoteActionPayloadFunctions + .groupGraphTableByIdAndMerge( + joinedResults, rowIdFn, mergeRowsAndGetFn, zeroFn, isNotZeroFn, Publication.class); + + verify(mergedResults.collectAsList()); + } + + private static void verify(List results) throws JsonProcessingException { + assertNotNull(results); + assertEquals(1, results.size()); + + Result r = results.get(0); + + log.info(OBJECT_MAPPER.writeValueAsString(r)); + + assertNotNull(r.getMeasures()); + assertFalse(r.getMeasures().isEmpty()); + assertTrue( + r + .getMeasures() + .stream() + .map(Measure::getId) + .collect(Collectors.toCollection(HashSet::new)) + .containsAll( + Lists + .newArrayList( + "downloads", "views", "influence", "popularity", "influence_alt", "popularity_alt", + "impulse"))); + } + + private static Function isNotZeroFnUsingIdOrSourceAndTarget() { + return t -> { + if (isSubClass(t, Relation.class)) { + final Relation rel = (Relation) t; + return StringUtils.isNotBlank(rel.getSource()) && StringUtils.isNotBlank(rel.getTarget()); + } + return StringUtils.isNotBlank(((OafEntity) t).getId()); + }; + } + + private static boolean isDirEmpty(final Path directory) throws IOException { + try (DirectoryStream dirStream = Files.newDirectoryStream(directory)) { + return !dirStream.iterator().hasNext(); + } + } + +} diff --git a/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json new file mode 100644 index 000000000..806bcf5c8 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/actionPayloads/part0000.json @@ -0,0 +1,3 @@ +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"id":"50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":[{"id":"downloads","unit":[{"key":"opendoar____::358aee4cc897452c00244351e4d91f69||ZENODO","value":"125","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:usage_counts","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"views","unit":[{"key":"opendoar____::358aee4cc897452c00244351e4d91f69||ZENODO","value":"35","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:usage_counts","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]}],"context":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":null,"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"transformativeAgreement":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"id":"50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":[{"id":"influence","unit":[{"key":"score","value":"3.1167566E-9","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"popularity","unit":[{"key":"score","value":"7.335433E-9","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"influence_alt","unit":[{"key":"score","value":"4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"popularity_alt","unit":[{"key":"score","value":"2.96","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"impulse","unit":[{"key":"score","value":"4","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]}],"context":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":null,"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"transformativeAgreement":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"id":"50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"context":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":null,"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":"hybrid","publiclyFunded":false,"transformativeAgreement":null,"isGreen":true,"isInDiamondJournal":false} \ No newline at end of file diff --git a/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json new file mode 100644 index 000000000..9f03cebe4 --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/test/resources/eu/dnetlib/dhp/actionmanager/promote/measures/graph/part00000.json @@ -0,0 +1 @@ +{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:dedup", "classname": "sysimport:dedup", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "dedup-result-decisiontree-v4", "invisible": false, "trust": "0.8"}, "resourcetype": {"classid": "publication", "classname": "publication", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"qualifier": {"classid": "mag_id", "classname": "Microsoft Academic Graph Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "3086187510"}], "bestaccessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "relevantdate": [{"qualifier": {"classid": "created", "classname": "created", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-14"}, {"qualifier": {"classid": "published-online", "classname": "published-online", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-14"}, {"qualifier": {"classid": "published-print", "classname": "published-print", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-08-01"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "issued", "classname": "issued", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-02"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "updated", "classname": "updated", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2020-09-01"}], "contributor": [], "id": "50|doi_dedup___::02317b7093277ec8aa0311d5c6a25b9b", "description": [{"value": "We present SPHINX, a system for metapath-based entity exploration in Heterogeneous Information Networks (HINs). SPHINX allows users to define different views over a HIN based on both automatically selected and user-defined meta-paths. Then, entity ranking and similarity search can be performed over these views to find and explore entities of interest, taking also into account any spatial or temporal properties of entities. A Web-based user interface is provided to facilitate users in performing the various functionalities supported by the system, including metapath-based view definition, index construction, search parameters specification, and visual comparison of the results."}], "lastupdatetimestamp": 1725554400176, "author": [{"fullname": "Thanasis Vergoulis", "pid": [], "rank": 1}, {"fullname": "Kostas Patroumpas", "pid": [], "rank": 2}, {"fullname": "Alexandros Zeakis", "pid": [], "rank": 3}, {"fullname": "Dimitrios Skoutas", "pid": [], "rank": 4}, {"fullname": "Serafeim Chatzopoulos", "pid": [], "rank": 5}], "collectedfrom": [{"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, {"value": "Crossref", "key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"}, {"value": "Microsoft Academic Graph", "key": "10|openaire____::5f532a3fc4f1ea403f37070f59a7a53a"}, {"value": "UnpayWall", "key": "10|openaire____:8ac8380272269217cb09a928c8caa993"}, {"value": "European Union Open Data Portal", "key": "10|re3data_____::c4b2081b224be6b3e79d0e5e5556f631"}], "instance": [{"refereed": {"classid": "0001", "classname": "peerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Proceedings of the VLDB Endowment", "key": "10|issn___print::8e719dcc0c83f87be79812fcf8024e2b"}, "url": ["https://doi.org/10.14778/3415478.3415507"], "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "instanceTypeMapping": [{"originalType": "journal-article", "typeLabel": "research article", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_2df8fbb1"}, {"originalType": "http://purl.org/coar/resource_type/c_2df8fbb1", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "dateofacceptance": {"value": "2020-08-01"}, "collectedfrom": {"value": "Crossref", "key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Proceedings of the VLDB Endowment", "key": "10|issn___print::8e719dcc0c83f87be79812fcf8024e2b"}, "license": {"value": "CC BY"}, "url": ["https://zenodo.org/record/4010307/files/p2913-chatzopoulos.pdf"], "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "instanceTypeMapping": [{"originalType": "journal-article", "typeLabel": "research article", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_2df8fbb1"}, {"originalType": "http://purl.org/coar/resource_type/c_2df8fbb1", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "collectedfrom": {"value": "UnpayWall", "key": "10|openaire____:8ac8380272269217cb09a928c8caa993"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes", "openAccessRoute": "green"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "Unknown Repository", "key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c"}, "url": ["http://dx.doi.org/10.14778/3415478.3415507"], "pid": [], "instanceTypeMapping": [{"originalType": "CONFERENCE_PROCEEDING", "vocabularyName": "openaire::coar_resource_types_3_1"}], "distributionlocation": "", "alternateIdentifier": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "value": "2020-01-01"}, "collectedfrom": {"value": "European Union Open Data Portal", "key": "10|re3data_____::c4b2081b224be6b3e79d0e5e5556f631"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0004", "classname": "Conference object", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "Unknown Repository", "key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c"}, "url": ["http://dx.doi.org/10.14778/3415478.3415507"], "pid": [], "instanceTypeMapping": [{"originalType": "Conference proceedings", "typeLabel": "conference proceedings", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_f744"}], "distributionlocation": "", "alternateIdentifier": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "value": "2020-01-01"}, "collectedfrom": {"value": "European Union Open Data Portal", "key": "10|re3data_____::c4b2081b224be6b3e79d0e5e5556f631"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0038", "classname": "Other literature type", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, "license": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "CC BY"}, "url": ["http://dx.doi.org/10.14778/3415478.3415507"], "pid": [], "instanceTypeMapping": [{"originalType": "ConferencePaper", "typeLabel": "conference paper", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_5794"}, {"originalType": "http://purl.org/coar/resource_type/c_5794", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "alternateIdentifier": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "oai", "classname": "Open Archives Initiative", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "oai:zenodo.org:4010307"}], "dateofacceptance": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "2020-09-02"}, "collectedfrom": {"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0004", "classname": "Conference object", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}, {"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Unknown Repository", "key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c"}, "url": ["http://www.vldb.org/pvldb/vol13/p2913-chatzopoulos.pdf", "https://dblp.uni-trier.de/db/journals/pvldb/pvldb13.html#ChatzopoulosPZV20", "https://dl.acm.org/doi/10.14778/3415478.3415507", "https://doi.org/10.14778/3415478.3415507"], "pid": [{"qualifier": {"classid": "mag_id", "classname": "Microsoft Academic Graph Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "3086187510"}, {"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.14778/3415478.3415507"}], "instanceTypeMapping": [{"originalType": "Conference", "typeLabel": "conference output", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_c94f"}], "collectedfrom": {"value": "Microsoft Academic Graph", "key": "10|openaire____::5f532a3fc4f1ea403f37070f59a7a53a"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}], "dateofcollection": "2024-09-05T16:53:05.687", "metaResourceType": {"classid": "Research Literature", "classname": "Research Literature", "schemename": "openaire::meta_resource_types", "schemeid": "openaire::meta_resource_types"}, "fulltext": [], "dateofacceptance": {"value": "2020-08-01"}, "format": [], "journal": {"issnPrinted": "2150-8097", "vol": "13", "sp": "2913", "ep": "2916", "name": "Proceedings of the VLDB Endowment"}, "subject": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "02 engineering and technology"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "0202 electrical engineering, electronic engineering, information engineering"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5467381477355957"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "020201 artificial intelligence & image processing"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.4532618224620819"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "020204 information systems"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "02020108 Machine learning/Social Info Processing"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "02020402 Cryptography/Information governance"}], "coverage": [], "externalReference": [], "publisher": {"value": "Association for Computing Machinery (ACM)"}, "eoscifguidelines": [], "language": {"classid": "und", "classname": "Undetermined", "schemename": "dnet:languages", "schemeid": "dnet:languages"}, "resulttype": {"classid": "publication", "classname": "publication", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "country": [], "extraInfo": [], "originalId": ["10.14778/3415478.3415507", "50|doiboost____|02317b7093277ec8aa0311d5c6a25b9b", "825041_1260870_PUBLI", "50|r3c4b2081b22::0d0cc9ff8949f9091272abb7a9e083f8", "50|r3c4b2081b22::02317b7093277ec8aa0311d5c6a25b9b", "oai:zenodo.org:4010307", "50|od______2659::de3dfee8ed6f2e53c85690531ab23028", "3086187510"], "source": [{"value": "Crossref"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "value": "International Conference on Very Large Data Bases (VLDB)"}], "context": [{"dataInfo": [{"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}], "id": "https://zenodo.org/communities/smartdatalake-project"}, {"dataInfo": [{"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}], "id": "https://zenodo.org/communities/eu"}], "title": [{"qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "SPHINX"}, {"qualifier": {"classid": "subtitle", "classname": "subtitle", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "a system for metapath-based entity exploration in heterogeneous information networks"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.9"}, "qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "SPHINX: A System for Metapath-based Entity Exploration in Heterogeneous Information Networks"}, {"qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "sphinx a system for metapath based entity exploration in heterogeneous information networks"}]} \ No newline at end of file From dcf09811a26f93c7340f0b089aa456b88832e308 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 19 Sep 2024 12:42:45 +0200 Subject: [PATCH 121/239] partial implementation of osfPreprints plugin --- .../dhp/collection/CollectorWorker.java | 88 +++---- .../collection/plugin/CollectorPlugin.java | 2 +- .../gtr2/Gtr2PublicationsCollectorPlugin.java | 7 +- .../plugin/gtr2/Gtr2PublicationsIterator.java | 21 +- .../osf/OsfPreprintsCollectorPlugin.java | 54 ++++ .../plugin/osf/OsfPreprintsIterator.java | 232 ++++++++++++++++++ .../gtr2/Gtr2PublicationsIteratorTest.java | 4 +- 7 files changed, 353 insertions(+), 55 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java index 5021b7727..67e07ba59 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java @@ -26,6 +26,7 @@ import eu.dnetlib.dhp.collection.plugin.gtr2.Gtr2PublicationsCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.mongodb.MDStoreCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.mongodb.MongoDbDumpCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.oai.OaiCollectorPlugin; +import eu.dnetlib.dhp.collection.plugin.osf.OsfPreprintsCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.rest.RestCollectorPlugin; import eu.dnetlib.dhp.common.aggregation.AggregatorReport; import eu.dnetlib.dhp.common.collection.CollectorException; @@ -45,11 +46,11 @@ public class CollectorWorker extends ReportingJob { private final HttpClientParams clientParams; public CollectorWorker( - final ApiDescriptor api, - final FileSystem fileSystem, - final MDStoreVersion mdStoreVersion, - final HttpClientParams clientParams, - final AggregatorReport report) { + final ApiDescriptor api, + final FileSystem fileSystem, + final MDStoreVersion mdStoreVersion, + final HttpClientParams clientParams, + final AggregatorReport report) { super(report); this.api = api; this.fileSystem = fileSystem; @@ -68,25 +69,22 @@ public class CollectorWorker extends ReportingJob { scheduleReport(counter); try (SequenceFile.Writer writer = SequenceFile - .createWriter( - this.fileSystem.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer - .keyClass(IntWritable.class), - SequenceFile.Writer - .valueClass(Text.class), - SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) { + .createWriter(this.fileSystem.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer + .keyClass(IntWritable.class), SequenceFile.Writer + .valueClass(Text.class), SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) { final IntWritable key = new IntWritable(counter.get()); final Text value = new Text(); plugin - .collect(this.api, this.report) - .forEach(content -> { - key.set(counter.getAndIncrement()); - value.set(content); - try { - writer.append(key, value); - } catch (final Throwable e) { - throw new RuntimeException(e); - } - }); + .collect(this.api, this.report) + .forEach(content -> { + key.set(counter.getAndIncrement()); + value.set(content); + try { + writer.append(key, value); + } catch (final Throwable e) { + throw new RuntimeException(e); + } + }); } catch (final Throwable e) { this.report.put(e.getClass().getName(), e.getMessage()); throw new CollectorException(e); @@ -114,34 +112,36 @@ public class CollectorWorker extends ReportingJob { private CollectorPlugin getCollectorPlugin() throws UnknownCollectorPluginException { switch (CollectorPlugin.NAME.valueOf(this.api.getProtocol())) { - case oai: - return new OaiCollectorPlugin(this.clientParams); - case rest_json2xml: - return new RestCollectorPlugin(this.clientParams); - case file: - return new FileCollectorPlugin(this.fileSystem); - case fileGzip: - return new FileGZipCollectorPlugin(this.fileSystem); - case baseDump: - return new BaseCollectorPlugin(this.fileSystem); - case gtr2Publications: - return new Gtr2PublicationsCollectorPlugin(this.clientParams); - case other: - final CollectorPlugin.NAME.OTHER_NAME plugin = Optional + case oai: + return new OaiCollectorPlugin(this.clientParams); + case rest_json2xml: + return new RestCollectorPlugin(this.clientParams); + case file: + return new FileCollectorPlugin(this.fileSystem); + case fileGzip: + return new FileGZipCollectorPlugin(this.fileSystem); + case baseDump: + return new BaseCollectorPlugin(this.fileSystem); + case gtr2Publications: + return new Gtr2PublicationsCollectorPlugin(this.clientParams); + case osfPreprints: + return new OsfPreprintsCollectorPlugin(this.clientParams); + case other: + final CollectorPlugin.NAME.OTHER_NAME plugin = Optional .ofNullable(this.api.getParams().get("other_plugin_type")) .map(CollectorPlugin.NAME.OTHER_NAME::valueOf) .orElseThrow(() -> new IllegalArgumentException("invalid other_plugin_type")); - switch (plugin) { - case mdstore_mongodb_dump: - return new MongoDbDumpCollectorPlugin(this.fileSystem); - case mdstore_mongodb: - return new MDStoreCollectorPlugin(); - default: - throw new UnknownCollectorPluginException("plugin is not managed: " + plugin); - } + switch (plugin) { + case mdstore_mongodb_dump: + return new MongoDbDumpCollectorPlugin(this.fileSystem); + case mdstore_mongodb: + return new MDStoreCollectorPlugin(); default: - throw new UnknownCollectorPluginException("protocol is not managed: " + this.api.getProtocol()); + throw new UnknownCollectorPluginException("plugin is not managed: " + plugin); + } + default: + throw new UnknownCollectorPluginException("protocol is not managed: " + this.api.getProtocol()); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java index 0bba8d764..72e691579 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java @@ -11,7 +11,7 @@ public interface CollectorPlugin { enum NAME { - oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications; + oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications, osfPreprints; public enum OTHER_NAME { mdstore_mongodb_dump, mdstore_mongodb diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java index 543bcbe5a..15fdd8280 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsCollectorPlugin.java @@ -29,9 +29,12 @@ public class Gtr2PublicationsCollectorPlugin implements CollectorPlugin { final String endPage = api.getParams().get("endPage"); final String fromDate = api.getParams().get("fromDate"); - if ((fromDate != null) && !fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { throw new CollectorException("Invalid date (YYYY-MM-DD): " + fromDate); } + if ((fromDate != null) && !fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { + throw new CollectorException("Invalid date (YYYY-MM-DD): " + fromDate); + } - final Iterator iterator = new Gtr2PublicationsIterator(baseUrl, fromDate, startPage, endPage, this.clientParams); + final Iterator iterator = new Gtr2PublicationsIterator(baseUrl, fromDate, startPage, endPage, + this.clientParams); final Spliterator spliterator = Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED); return StreamSupport.stream(spliterator, false); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java index 9b122bbe6..5b8f48680 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.plugin.gtr2; import java.util.ArrayList; @@ -48,9 +49,10 @@ public class Gtr2PublicationsIterator implements Iterator { private String nextElement; - public Gtr2PublicationsIterator(final String baseUrl, final String fromDate, final String startPage, final String endPage, - final HttpClientParams clientParams) - throws CollectorException { + public Gtr2PublicationsIterator(final String baseUrl, final String fromDate, final String startPage, + final String endPage, + final HttpClientParams clientParams) + throws CollectorException { this.baseUrl = baseUrl; this.currPage = NumberUtils.toInt(startPage, 1); @@ -120,7 +122,8 @@ public class Gtr2PublicationsIterator implements Iterator { return res; } - private void addLinkedEntities(final Element master, final String relType, final Element newRoot, final Function mapper) { + private void addLinkedEntities(final Element master, final String relType, final Element newRoot, + final Function mapper) { for (final Object o : master.selectNodes(".//*[local-name()='link']")) { final String rel = ((Element) o).valueOf("@*[local-name()='rel']"); @@ -149,7 +152,7 @@ public class Gtr2PublicationsIterator implements Iterator { private boolean filterIncremental(final Element e) { if (!this.incremental || isAfter(e.valueOf("@*[local-name() = 'created']"), this.fromDate) - || isAfter(e.valueOf("@*[local-name() = 'updated']"), this.fromDate)) { + || isAfter(e.valueOf("@*[local-name() = 'updated']"), this.fromDate)) { return true; } return false; @@ -165,7 +168,9 @@ public class Gtr2PublicationsIterator implements Iterator { private Element asProjectElement(final Document doc) { final Element newOrg = DocumentHelper.createElement("project"); newOrg.addElement("id").setText(doc.valueOf("/*/@*[local-name()='id']")); - newOrg.addElement("code").setText(doc.valueOf("//*[local-name()='identifier' and @*[local-name()='type'] = 'RCUK']")); + newOrg + .addElement("code") + .setText(doc.valueOf("//*[local-name()='identifier' and @*[local-name()='type'] = 'RCUK']")); newOrg.addElement("title").setText(doc.valueOf("//*[local-name()='title']")); return newOrg; } @@ -188,7 +193,9 @@ public class Gtr2PublicationsIterator implements Iterator { return DocumentHelper.parseText(new String(bytes)); } catch (final Throwable e) { log.error("Error dowloading url: " + cleanUrl + ", attempt = " + attempt, e); - if (attempt >= MAX_ATTEMPTS) { throw new RuntimeException("Error dowloading url: " + cleanUrl, e); } + if (attempt >= MAX_ATTEMPTS) { + throw new RuntimeException("Error dowloading url: " + cleanUrl, e); + } try { Thread.sleep(60000); // I wait for a minute } catch (final InterruptedException e1) { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java new file mode 100644 index 000000000..bd628efb8 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java @@ -0,0 +1,54 @@ + +package eu.dnetlib.dhp.collection.plugin.osf; + +import java.util.Optional; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; + +import com.google.gson.Gson; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +public class OsfPreprintsCollectorPlugin implements CollectorPlugin { + + public static final int PAGE_SIZE_VALUE_DEFAULT = 100; + + private final HttpClientParams clientParams; + + public OsfPreprintsCollectorPlugin(final HttpClientParams clientParams) { + this.clientParams = clientParams; + } + + @Override + public Stream collect(final ApiDescriptor api, final AggregatorReport report) throws CollectorException { + final String baseUrl = api.getBaseUrl(); + + final Gson gson = new Gson(); + + final int pageSize = Optional + .ofNullable(api.getParams().get("pageSize")) + .filter(StringUtils::isNotBlank) + .map(s -> NumberUtils.toInt(s, PAGE_SIZE_VALUE_DEFAULT)) + .orElse(PAGE_SIZE_VALUE_DEFAULT); + + if (StringUtils.isBlank(baseUrl)) { throw new CollectorException("Param 'baseUrl' is null or empty"); } + + final OsfPreprintsIterator it = new OsfPreprintsIterator(baseUrl, pageSize, getClientParams()); + + return StreamSupport + .stream(Spliterators.spliteratorUnknownSize(it, Spliterator.ORDERED), false); + } + + public HttpClientParams getClientParams() { + return this.clientParams; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java new file mode 100644 index 000000000..72c056d32 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -0,0 +1,232 @@ + +package eu.dnetlib.dhp.collection.plugin.osf; + +import java.io.InputStream; +import java.io.StringWriter; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.PriorityBlockingQueue; + +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; +import javax.xml.xpath.XPath; +import javax.xml.xpath.XPathConstants; +import javax.xml.xpath.XPathExpression; +import javax.xml.xpath.XPathFactory; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.xml.sax.InputSource; + +import eu.dnetlib.dhp.collection.plugin.utils.JsonUtils; +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +public class OsfPreprintsIterator implements Iterator { + + private static final Logger log = LoggerFactory.getLogger(OsfPreprintsIterator.class); + public static final String UTF_8 = "UTF-8"; + private static final int MAX_ATTEMPTS = 5; + + private final HttpClientParams clientParams; + + private static final String XML_HEADER = ""; + private static final String EMPTY_XML = XML_HEADER + "<" + JsonUtils.XML_WRAP_TAG + ">"; + + private final String baseUrl; + private final int pageSize; + + private int resumptionInt = 0; // integer resumption token (first record to harvest) + private int resultTotal = -1; + private String resumptionStr = Integer.toString(this.resumptionInt); // string resumption token (first record to + // harvest + // or token scanned from results) + private InputStream resultStream; + private Transformer transformer; + private XPath xpath; + private String query; + private XPathExpression xprResultTotalPath; + private XPathExpression xprResumptionPath; + private XPathExpression xprEntity; + private final Queue recordQueue = new PriorityBlockingQueue<>(); + + public OsfPreprintsIterator( + final String baseUrl, + final int pageSize, + final HttpClientParams clientParams) { + + this.clientParams = clientParams; + this.baseUrl = baseUrl; + this.pageSize = pageSize; + + try { + final TransformerFactory factory = TransformerFactory.newInstance(); + this.transformer = factory.newTransformer(); + this.transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + this.transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "3"); + this.xpath = XPathFactory.newInstance().newXPath(); + this.xprResultTotalPath = this.xpath.compile("/*/*[local-name()='links']/*[local-name()='meta']/*[local-name()='total']"); + this.xprResumptionPath = this.xpath.compile("substring-before(substring-after(/*/*[local-name()='links']/*[local-name()='next'], 'page='), '&')"); + this.xprEntity = this.xpath.compile("/*/*[local-name()='data']"); + } catch (final Exception e) { + throw new IllegalStateException("xml transformation init failed: " + e.getMessage()); + } + + initQueue(); + } + + private void initQueue() { + this.query = this.baseUrl + "?filter:is_published:d=true&format=json&page[size]=" + this.pageSize; + log.info("REST calls starting with {}", this.query); + } + + private void disconnect() { + // TODO close inputstream + } + + /* + * (non-Javadoc) + * + * @see java.util.Iterator#hasNext() + */ + @Override + public boolean hasNext() { + synchronized (this.recordQueue) { + while (this.recordQueue.isEmpty() && !this.query.isEmpty()) { + try { + this.query = downloadPage(this.query, 0); + } catch (final CollectorException e) { + log.debug("CollectorPlugin.next()-Exception: {}", e); + throw new RuntimeException(e); + } + } + + if (!this.recordQueue.isEmpty()) { return true; } + + disconnect(); + return false; + } + } + + /* + * (non-Javadoc) + * + * @see java.util.Iterator#next() + */ + @Override + public String next() { + synchronized (this.recordQueue) { + return this.recordQueue.poll(); + } + } + + /* + * download page and return nextQuery (with number of attempt) + */ + private String downloadPage(final String query, final int attempt) throws CollectorException { + + if (attempt > MAX_ATTEMPTS) { throw new CollectorException("Max Number of attempts reached, query:" + query); } + + if (attempt > 0) { + final int delay = (attempt * 5000); + log.debug("Attempt {} with delay {}", attempt, delay); + try { + Thread.sleep(delay); + } catch (final InterruptedException e) { + new CollectorException(e); + } + } + + try { + String resultJson; + String resultXml = XML_HEADER; + String nextQuery = ""; + Node resultNode = null; + NodeList nodeList = null; + + try { + log.info("requesting URL [{}]", query); + + final URL qUrl = new URL(query); + + final HttpURLConnection conn = (HttpURLConnection) qUrl.openConnection(); + conn.setRequestMethod("GET"); + this.resultStream = conn.getInputStream(); + + resultJson = IOUtils.toString(this.resultStream, StandardCharsets.UTF_8); + resultXml = JsonUtils.convertToXML(resultJson); + this.resultStream = IOUtils.toInputStream(resultXml, UTF_8); + + if (!isEmptyXml(resultXml)) { + resultNode = (Node) this.xpath + .evaluate("/", new InputSource(this.resultStream), XPathConstants.NODE); + nodeList = (NodeList) this.xprEntity.evaluate(resultNode, XPathConstants.NODESET); + log.debug("nodeList.length: {}", nodeList.getLength()); + for (int i = 0; i < nodeList.getLength(); i++) { + final StringWriter sw = new StringWriter(); + this.transformer.transform(new DOMSource(nodeList.item(i)), new StreamResult(sw)); + final String toEnqueue = sw.toString(); + if ((toEnqueue == null) || StringUtils.isBlank(toEnqueue) || isEmptyXml(toEnqueue)) { + log + .warn("The following record resulted in empty item for the feeding queue: {}", resultXml); + } else { + this.recordQueue.add(sw.toString()); + } + } + } else { + log.warn("resultXml is equal with emptyXml"); + } + + this.resumptionInt += this.pageSize; + + this.resumptionStr = this.xprResumptionPath.evaluate(resultNode); + + } catch (final Exception e) { + log.error(e.getMessage(), e); + throw new IllegalStateException("collection failed: " + e.getMessage()); + } + + try { + if (this.resultTotal == -1) { + this.resultTotal = Integer.parseInt(this.xprResultTotalPath.evaluate(resultNode)); + log.info("resultTotal was -1 is now: " + this.resultTotal); + } + } catch (final Exception e) { + log.error(e.getMessage(), e); + throw new IllegalStateException("downloadPage resultTotal couldn't parse: " + e.getMessage()); + } + log.debug("resultTotal: " + this.resultTotal); + log.debug("resInt: " + this.resumptionInt); + if (this.resumptionInt <= this.resultTotal) { + nextQuery = this.baseUrl + "?filter:is_published:d=true&format=json&page[size]=" + this.pageSize + "&page=" + + this.resumptionStr; + } else { + nextQuery = ""; + // if (resumptionType.toLowerCase().equals("deep-cursor")) { resumptionInt -= 1; } // correct the + // resumptionInt and prevent a NullPointer Exception at mdStore + } + log.debug("nextQueryUrl: " + nextQuery); + return nextQuery; + } catch (final Throwable e) { + log.warn(e.getMessage(), e); + return downloadPage(query, attempt + 1); + } + + } + + private boolean isEmptyXml(final String s) { + return EMPTY_XML.equalsIgnoreCase(s); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java index f02feb0ef..117d7b94f 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIteratorTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.plugin.gtr2; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -55,7 +56,8 @@ class Gtr2PublicationsIteratorTest { @Disabled public void testIncrementalHarvestingNoRecords() throws Exception { System.out.println("incremental Harvesting"); - final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2050-12-12T", "11", "13", clientParams); + final Iterator iterator = new Gtr2PublicationsIterator(baseURL, "2050-12-12T", "11", "13", + clientParams); final int count = iterateAndCount(iterator); assertEquals(0, count); } From 9073b1159d51d1f939163385173e5e934812880c Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 19 Sep 2024 13:58:53 +0200 Subject: [PATCH 122/239] partial implementation of osfPreprints plugin + tests --- .../plugin/osf/OsfPreprintsIterator.java | 29 ++++--------- .../OsfPreprintsCollectorPluginTest.java} | 42 +++++-------------- 2 files changed, 18 insertions(+), 53 deletions(-) rename dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/{rest/OsfPreprintCollectorTest.java => osf/OsfPreprintsCollectorPluginTest.java} (52%) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index 72c056d32..a3b1dc237 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -3,8 +3,6 @@ package eu.dnetlib.dhp.collection.plugin.osf; import java.io.InputStream; import java.io.StringWriter; -import java.net.HttpURLConnection; -import java.net.URL; import java.nio.charset.StandardCharsets; import java.util.Iterator; import java.util.Queue; @@ -31,18 +29,18 @@ import org.xml.sax.InputSource; import eu.dnetlib.dhp.collection.plugin.utils.JsonUtils; import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.common.collection.HttpConnector2; public class OsfPreprintsIterator implements Iterator { private static final Logger log = LoggerFactory.getLogger(OsfPreprintsIterator.class); - public static final String UTF_8 = "UTF-8"; + private static final int MAX_ATTEMPTS = 5; private final HttpClientParams clientParams; private static final String XML_HEADER = ""; - private static final String EMPTY_XML = XML_HEADER + "<" + JsonUtils.XML_WRAP_TAG + ">"; + private static final String EMPTY_XML = XML_HEADER + "<" + JsonUtils.XML_WRAP_TAG + ">"; private final String baseUrl; private final int pageSize; @@ -91,10 +89,6 @@ public class OsfPreprintsIterator implements Iterator { log.info("REST calls starting with {}", this.query); } - private void disconnect() { - // TODO close inputstream - } - /* * (non-Javadoc) * @@ -114,7 +108,6 @@ public class OsfPreprintsIterator implements Iterator { if (!this.recordQueue.isEmpty()) { return true; } - disconnect(); return false; } } @@ -158,15 +151,12 @@ public class OsfPreprintsIterator implements Iterator { try { log.info("requesting URL [{}]", query); - final URL qUrl = new URL(query); + final HttpConnector2 connector = new HttpConnector2(this.clientParams); - final HttpURLConnection conn = (HttpURLConnection) qUrl.openConnection(); - conn.setRequestMethod("GET"); - this.resultStream = conn.getInputStream(); - - resultJson = IOUtils.toString(this.resultStream, StandardCharsets.UTF_8); + resultJson = connector.getInputSource(query); resultXml = JsonUtils.convertToXML(resultJson); - this.resultStream = IOUtils.toInputStream(resultXml, UTF_8); + + this.resultStream = IOUtils.toInputStream(resultXml, StandardCharsets.UTF_8); if (!isEmptyXml(resultXml)) { resultNode = (Node) this.xpath @@ -178,8 +168,7 @@ public class OsfPreprintsIterator implements Iterator { this.transformer.transform(new DOMSource(nodeList.item(i)), new StreamResult(sw)); final String toEnqueue = sw.toString(); if ((toEnqueue == null) || StringUtils.isBlank(toEnqueue) || isEmptyXml(toEnqueue)) { - log - .warn("The following record resulted in empty item for the feeding queue: {}", resultXml); + log.warn("The following record resulted in empty item for the feeding queue: {}", resultXml); } else { this.recordQueue.add(sw.toString()); } @@ -213,8 +202,6 @@ public class OsfPreprintsIterator implements Iterator { + this.resumptionStr; } else { nextQuery = ""; - // if (resumptionType.toLowerCase().equals("deep-cursor")) { resumptionInt -= 1; } // correct the - // resumptionInt and prevent a NullPointer Exception at mdStore } log.debug("nextQueryUrl: " + nextQuery); return nextQuery; diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java similarity index 52% rename from dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java rename to dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index a1b723e33..f6ddb3aeb 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/OsfPreprintCollectorTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.collection.plugin.rest; +package eu.dnetlib.dhp.collection.plugin.osf; import java.util.HashMap; import java.util.concurrent.atomic.AtomicInteger; @@ -18,9 +18,9 @@ import eu.dnetlib.dhp.common.aggregation.AggregatorReport; import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; -public class OsfPreprintCollectorTest { +public class OsfPreprintsCollectorPluginTest { - private static final Logger log = LoggerFactory.getLogger(OsfPreprintCollectorTest.class); + private static final Logger log = LoggerFactory.getLogger(OsfPreprintsCollectorPlugin.class); private final String baseUrl = "https://api.osf.io/v2/preprints/"; @@ -29,50 +29,28 @@ public class OsfPreprintCollectorTest { // private final String authToken = ""; // private final String resultOutputFormat = ""; - private final String queryParams = "filter:is_published:d=true"; - - private final String entityXpath = "/*/*[local-name()='data']"; - - private final String resultTotalXpath = "/*/*[local-name()='links']/*[local-name()='meta']/*[local-name()='total']"; - - private final String resumptionParam = "page"; - private final String resumptionType = "scan"; - private final String resumptionXpath = "substring-before(substring-after(/*/*[local-name()='links']/*[local-name()='next'], 'page='), '&')"; - - private final String resultSizeParam = "page[size]"; - private final String resultSizeValue = "100"; - - private final String resultFormatParam = "format"; - private final String resultFormatValue = "json"; + private final int pageSize = 100; private final ApiDescriptor api = new ApiDescriptor(); - private RestCollectorPlugin rcp; + + private OsfPreprintsCollectorPlugin plugin; @BeforeEach public void setUp() { final HashMap params = new HashMap<>(); - params.put("resumptionType", this.resumptionType); - params.put("resumptionParam", this.resumptionParam); - params.put("resumptionXpath", this.resumptionXpath); - params.put("resultTotalXpath", this.resultTotalXpath); - params.put("resultFormatParam", this.resultFormatParam); - params.put("resultFormatValue", this.resultFormatValue); - params.put("resultSizeParam", this.resultSizeParam); - params.put("resultSizeValue", this.resultSizeValue); - params.put("queryParams", this.queryParams); - params.put("entityXpath", this.entityXpath); + params.put("pageSize", "" + this.pageSize); this.api.setBaseUrl(this.baseUrl); this.api.setParams(params); - this.rcp = new RestCollectorPlugin(new HttpClientParams()); + this.plugin = new OsfPreprintsCollectorPlugin(new HttpClientParams()); } @Test @Disabled void test_limited() throws CollectorException { final AtomicInteger i = new AtomicInteger(0); - final Stream stream = this.rcp.collect(this.api, new AggregatorReport()); + final Stream stream = this.plugin.collect(this.api, new AggregatorReport()); stream.limit(2000).forEach(s -> { Assertions.assertTrue(s.length() > 0); @@ -88,7 +66,7 @@ public class OsfPreprintCollectorTest { @Disabled void test_all() throws CollectorException { final AtomicLong i = new AtomicLong(0); - final Stream stream = this.rcp.collect(this.api, new AggregatorReport()); + final Stream stream = this.plugin.collect(this.api, new AggregatorReport()); stream.forEach(s -> { Assertions.assertTrue(s.length() > 0); From 52bb7af03b87c8d40f4b304b7920b59720ddceba Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 19 Sep 2024 14:59:05 +0200 Subject: [PATCH 123/239] use of dom4j --- .../osf/OsfPreprintsCollectorPlugin.java | 4 - .../plugin/osf/OsfPreprintsIterator.java | 151 ++++-------------- .../osf/OsfPreprintsCollectorPluginTest.java | 5 - 3 files changed, 27 insertions(+), 133 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java index bd628efb8..fdc9df06f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java @@ -10,8 +10,6 @@ import java.util.stream.StreamSupport; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.math.NumberUtils; -import com.google.gson.Gson; - import eu.dnetlib.dhp.collection.ApiDescriptor; import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; import eu.dnetlib.dhp.common.aggregation.AggregatorReport; @@ -32,8 +30,6 @@ public class OsfPreprintsCollectorPlugin implements CollectorPlugin { public Stream collect(final ApiDescriptor api, final AggregatorReport report) throws CollectorException { final String baseUrl = api.getBaseUrl(); - final Gson gson = new Gson(); - final int pageSize = Optional .ofNullable(api.getParams().get("pageSize")) .filter(StringUtils::isNotBlank) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index a3b1dc237..311daf859 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -1,30 +1,15 @@ package eu.dnetlib.dhp.collection.plugin.osf; -import java.io.InputStream; -import java.io.StringWriter; -import java.nio.charset.StandardCharsets; import java.util.Iterator; import java.util.Queue; import java.util.concurrent.PriorityBlockingQueue; -import javax.xml.transform.OutputKeys; -import javax.xml.transform.Transformer; -import javax.xml.transform.TransformerFactory; -import javax.xml.transform.dom.DOMSource; -import javax.xml.transform.stream.StreamResult; -import javax.xml.xpath.XPath; -import javax.xml.xpath.XPathConstants; -import javax.xml.xpath.XPathExpression; -import javax.xml.xpath.XPathFactory; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.DocumentHelper; +import org.dom4j.Element; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.w3c.dom.Node; -import org.w3c.dom.NodeList; -import org.xml.sax.InputSource; import eu.dnetlib.dhp.collection.plugin.utils.JsonUtils; import eu.dnetlib.dhp.common.collection.CollectorException; @@ -39,24 +24,11 @@ public class OsfPreprintsIterator implements Iterator { private final HttpClientParams clientParams; - private static final String XML_HEADER = ""; - private static final String EMPTY_XML = XML_HEADER + "<" + JsonUtils.XML_WRAP_TAG + ">"; - private final String baseUrl; private final int pageSize; - private int resumptionInt = 0; // integer resumption token (first record to harvest) - private int resultTotal = -1; - private String resumptionStr = Integer.toString(this.resumptionInt); // string resumption token (first record to - // harvest - // or token scanned from results) - private InputStream resultStream; - private Transformer transformer; - private XPath xpath; - private String query; - private XPathExpression xprResultTotalPath; - private XPathExpression xprResumptionPath; - private XPathExpression xprEntity; + private String currentUrl; + private final Queue recordQueue = new PriorityBlockingQueue<>(); public OsfPreprintsIterator( @@ -68,38 +40,20 @@ public class OsfPreprintsIterator implements Iterator { this.baseUrl = baseUrl; this.pageSize = pageSize; - try { - final TransformerFactory factory = TransformerFactory.newInstance(); - this.transformer = factory.newTransformer(); - this.transformer.setOutputProperty(OutputKeys.INDENT, "yes"); - this.transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "3"); - this.xpath = XPathFactory.newInstance().newXPath(); - this.xprResultTotalPath = this.xpath.compile("/*/*[local-name()='links']/*[local-name()='meta']/*[local-name()='total']"); - this.xprResumptionPath = this.xpath.compile("substring-before(substring-after(/*/*[local-name()='links']/*[local-name()='next'], 'page='), '&')"); - this.xprEntity = this.xpath.compile("/*/*[local-name()='data']"); - } catch (final Exception e) { - throw new IllegalStateException("xml transformation init failed: " + e.getMessage()); - } - initQueue(); } private void initQueue() { - this.query = this.baseUrl + "?filter:is_published:d=true&format=json&page[size]=" + this.pageSize; - log.info("REST calls starting with {}", this.query); + this.currentUrl = this.baseUrl + "?filter:is_published:d=true&format=json&page[size]=" + this.pageSize; + log.info("REST calls starting with {}", this.currentUrl); } - /* - * (non-Javadoc) - * - * @see java.util.Iterator#hasNext() - */ @Override public boolean hasNext() { synchronized (this.recordQueue) { - while (this.recordQueue.isEmpty() && !this.query.isEmpty()) { + while (this.recordQueue.isEmpty() && !this.currentUrl.isEmpty()) { try { - this.query = downloadPage(this.query, 0); + this.currentUrl = downloadPage(this.currentUrl, 0); } catch (final CollectorException e) { log.debug("CollectorPlugin.next()-Exception: {}", e); throw new RuntimeException(e); @@ -112,11 +66,6 @@ public class OsfPreprintsIterator implements Iterator { } } - /* - * (non-Javadoc) - * - * @see java.util.Iterator#next() - */ @Override public String next() { synchronized (this.recordQueue) { @@ -124,12 +73,9 @@ public class OsfPreprintsIterator implements Iterator { } } - /* - * download page and return nextQuery (with number of attempt) - */ - private String downloadPage(final String query, final int attempt) throws CollectorException { + private String downloadPage(final String url, final int attempt) throws CollectorException { - if (attempt > MAX_ATTEMPTS) { throw new CollectorException("Max Number of attempts reached, query:" + query); } + if (attempt > MAX_ATTEMPTS) { throw new CollectorException("Max Number of attempts reached, url:" + url); } if (attempt > 0) { final int delay = (attempt * 5000); @@ -142,78 +88,35 @@ public class OsfPreprintsIterator implements Iterator { } try { - String resultJson; - String resultXml = XML_HEADER; - String nextQuery = ""; - Node resultNode = null; - NodeList nodeList = null; + log.info("requesting URL [{}]", url); - try { - log.info("requesting URL [{}]", query); + final HttpConnector2 connector = new HttpConnector2(this.clientParams); - final HttpConnector2 connector = new HttpConnector2(this.clientParams); + final String json = connector.getInputSource(url); + final String xml = JsonUtils.convertToXML(json); - resultJson = connector.getInputSource(query); - resultXml = JsonUtils.convertToXML(resultJson); + final Document doc = DocumentHelper.parseText(xml); - this.resultStream = IOUtils.toInputStream(resultXml, StandardCharsets.UTF_8); + for (final Object o : doc.selectNodes("/*/*[local-name()='data']")) { + final Element n = (Element) ((Element) o).detach(); - if (!isEmptyXml(resultXml)) { - resultNode = (Node) this.xpath - .evaluate("/", new InputSource(this.resultStream), XPathConstants.NODE); - nodeList = (NodeList) this.xprEntity.evaluate(resultNode, XPathConstants.NODESET); - log.debug("nodeList.length: {}", nodeList.getLength()); - for (int i = 0; i < nodeList.getLength(); i++) { - final StringWriter sw = new StringWriter(); - this.transformer.transform(new DOMSource(nodeList.item(i)), new StreamResult(sw)); - final String toEnqueue = sw.toString(); - if ((toEnqueue == null) || StringUtils.isBlank(toEnqueue) || isEmptyXml(toEnqueue)) { - log.warn("The following record resulted in empty item for the feeding queue: {}", resultXml); - } else { - this.recordQueue.add(sw.toString()); - } - } - } else { - log.warn("resultXml is equal with emptyXml"); + for (final Object o1 : n.selectNodes(".//contributors//href")) { + // TODO ADD creators + } + for (final Object o1 : n.selectNodes(".//primary_file//href")) { + // TODO ADD fulltexts } - this.resumptionInt += this.pageSize; - - this.resumptionStr = this.xprResumptionPath.evaluate(resultNode); - - } catch (final Exception e) { - log.error(e.getMessage(), e); - throw new IllegalStateException("collection failed: " + e.getMessage()); + this.recordQueue.add(DocumentHelper.createDocument(n).asXML()); } - try { - if (this.resultTotal == -1) { - this.resultTotal = Integer.parseInt(this.xprResultTotalPath.evaluate(resultNode)); - log.info("resultTotal was -1 is now: " + this.resultTotal); - } - } catch (final Exception e) { - log.error(e.getMessage(), e); - throw new IllegalStateException("downloadPage resultTotal couldn't parse: " + e.getMessage()); - } - log.debug("resultTotal: " + this.resultTotal); - log.debug("resInt: " + this.resumptionInt); - if (this.resumptionInt <= this.resultTotal) { - nextQuery = this.baseUrl + "?filter:is_published:d=true&format=json&page[size]=" + this.pageSize + "&page=" - + this.resumptionStr; - } else { - nextQuery = ""; - } - log.debug("nextQueryUrl: " + nextQuery); - return nextQuery; + return doc.valueOf("/*/*[local-name()='links']/*[local-name()='next']"); + } catch (final Throwable e) { log.warn(e.getMessage(), e); - return downloadPage(query, attempt + 1); + return downloadPage(url, attempt + 1); } } - private boolean isEmptyXml(final String s) { - return EMPTY_XML.equalsIgnoreCase(s); - } - } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index f6ddb3aeb..10b93ea76 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -24,11 +24,6 @@ public class OsfPreprintsCollectorPluginTest { private final String baseUrl = "https://api.osf.io/v2/preprints/"; - // private final String requestHeaderMap = ""; - // private final String authMethod = ""; - // private final String authToken = ""; - // private final String resultOutputFormat = ""; - private final int pageSize = 100; private final ApiDescriptor api = new ApiDescriptor(); From 339d8124f26bbf8cc6a3c6398d5d32c5066bbb91 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Fri, 20 Sep 2024 08:44:05 +0200 Subject: [PATCH 124/239] osf plugin: links to contributors and primaty_file --- .../plugin/osf/OsfPreprintsIterator.java | 55 +++++++++++-------- .../osf/OsfPreprintsCollectorPluginTest.java | 8 +++ 2 files changed, 41 insertions(+), 22 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index 311daf859..3bb2e2bdf 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -8,6 +8,7 @@ import java.util.concurrent.PriorityBlockingQueue; import org.dom4j.Document; import org.dom4j.DocumentHelper; import org.dom4j.Element; +import org.dom4j.Node; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +54,7 @@ public class OsfPreprintsIterator implements Iterator { synchronized (this.recordQueue) { while (this.recordQueue.isEmpty() && !this.currentUrl.isEmpty()) { try { - this.currentUrl = downloadPage(this.currentUrl, 0); + this.currentUrl = downloadPage(this.currentUrl); } catch (final CollectorException e) { log.debug("CollectorPlugin.next()-Exception: {}", e); throw new RuntimeException(e); @@ -73,8 +74,36 @@ public class OsfPreprintsIterator implements Iterator { } } - private String downloadPage(final String url, final int attempt) throws CollectorException { + private String downloadPage(final String url) throws CollectorException { + final Document doc = downloadUrl(url, 0); + + for (final Object o : doc.selectNodes("/*/data")) { + + final Element n = (Element) ((Element) o).detach(); + + final Element group = DocumentHelper.createElement("group"); + group.addAttribute("id", n.valueOf(".//data/id")); + + group.addElement("preprint").add(n); + + for (final Object o1 : n.selectNodes(".//contributors//href")) { + final Document doc1 = downloadUrl(((Node) o1).getText(), 0); + group.addElement("contributors").add(doc1.getRootElement().detach()); + } + for (final Object o1 : n.selectNodes(".//primary_file//href")) { + final Document doc1 = downloadUrl(((Node) o1).getText(), 0); + group.addElement("primary_file").add(doc1.getRootElement().detach()); + } + + this.recordQueue.add(DocumentHelper.createDocument(group).asXML()); + } + + return doc.valueOf("/*/links/next"); + + } + + private Document downloadUrl(final String url, final int attempt) throws CollectorException { if (attempt > MAX_ATTEMPTS) { throw new CollectorException("Max Number of attempts reached, url:" + url); } if (attempt > 0) { @@ -95,28 +124,10 @@ public class OsfPreprintsIterator implements Iterator { final String json = connector.getInputSource(url); final String xml = JsonUtils.convertToXML(json); - final Document doc = DocumentHelper.parseText(xml); - - for (final Object o : doc.selectNodes("/*/*[local-name()='data']")) { - final Element n = (Element) ((Element) o).detach(); - - for (final Object o1 : n.selectNodes(".//contributors//href")) { - // TODO ADD creators - } - for (final Object o1 : n.selectNodes(".//primary_file//href")) { - // TODO ADD fulltexts - } - - this.recordQueue.add(DocumentHelper.createDocument(n).asXML()); - } - - return doc.valueOf("/*/*[local-name()='links']/*[local-name()='next']"); - + return DocumentHelper.parseText(xml); } catch (final Throwable e) { log.warn(e.getMessage(), e); - return downloadPage(url, attempt + 1); + return downloadUrl(url, attempt + 1); } - } - } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index 10b93ea76..708a26acd 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -43,6 +43,14 @@ public class OsfPreprintsCollectorPluginTest { @Test @Disabled + void test_one() throws CollectorException { + this.plugin.collect(this.api, new AggregatorReport()) + .limit(1) + .forEach(log::info); + } + + @Test + // @Disabled void test_limited() throws CollectorException { final AtomicInteger i = new AtomicInteger(0); final Stream stream = this.plugin.collect(this.api, new AggregatorReport()); From 5f86c93be6fbe137b417213d49e0dd426b6c7898 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 20 Sep 2024 12:20:00 +0200 Subject: [PATCH 125/239] [graph provision] person serialisation --- .../CreateRelatedEntitiesJob_phase1.java | 8 +++ .../model/ProvisionModelSupport.java | 7 ++- .../dhp/oa/provision/model/RelatedEntity.java | 39 ++++++++++++- .../oa/provision/utils/XmlRecordFactory.java | 57 ++++++++++++++++++- 4 files changed, 107 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java index 63f3c2ead..3fc5893c6 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java @@ -231,6 +231,14 @@ public class CreateRelatedEntitiesJob_phase1 { if (!f.isEmpty()) { re.setFundingtree(f.stream().map(Field::getValue).collect(Collectors.toList())); } + break; + case person: + final Person person = (Person) entity; + + re.setGivenName(person.getGivenName()); + re.setFamilyName(person.getFamilyName()); + re.setAlternativeNames(person.getAlternativeNames()); + break; } return re; diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 797e84315..de7932a8a 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.solr.Person; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -38,6 +37,8 @@ import eu.dnetlib.dhp.schema.solr.Measure; import eu.dnetlib.dhp.schema.solr.OpenAccessColor; import eu.dnetlib.dhp.schema.solr.OpenAccessRoute; import eu.dnetlib.dhp.schema.solr.Organization; +import eu.dnetlib.dhp.schema.solr.Person; +import eu.dnetlib.dhp.schema.solr.PersonTopic; import eu.dnetlib.dhp.schema.solr.Pid; import eu.dnetlib.dhp.schema.solr.Project; import eu.dnetlib.dhp.schema.solr.Result; @@ -193,6 +194,10 @@ public class ProvisionModelSupport { ps.setFamilyName(p.getFamilyName()); ps.setGivenName(p.getGivenName()); ps.setAlternativeNames(p.getAlternativeNames()); + ps.setBiography(p.getBiography()); + ps.setConsent(p.getConsent()); + // ps.setSubject(...)); + return ps; } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java index ee010910c..2a6332857 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java @@ -51,6 +51,11 @@ public class RelatedEntity implements Serializable { private Qualifier contracttype; private List fundingtree; + // person + private String givenName; + private String familyName; + private List alternativeNames; + public String getId() { return id; } @@ -251,6 +256,30 @@ public class RelatedEntity implements Serializable { this.fundingtree = fundingtree; } + public String getGivenName() { + return givenName; + } + + public void setGivenName(String givenName) { + this.givenName = givenName; + } + + public String getFamilyName() { + return familyName; + } + + public void setFamilyName(String familyName) { + this.familyName = familyName; + } + + public List getAlternativeNames() { + return alternativeNames; + } + + public void setAlternativeNames(List alternativeNames) { + this.alternativeNames = alternativeNames; + } + @Override public boolean equals(Object o) { if (this == o) @@ -280,7 +309,10 @@ public class RelatedEntity implements Serializable { && Objects.equal(code, that.code) && Objects.equal(acronym, that.acronym) && Objects.equal(contracttype, that.contracttype) - && Objects.equal(fundingtree, that.fundingtree); + && Objects.equal(fundingtree, that.fundingtree) + && Objects.equal(givenName, that.givenName) + && Objects.equal(familyName, that.familyName) + && Objects.equal(alternativeNames, that.alternativeNames); } @Override @@ -309,6 +341,9 @@ public class RelatedEntity implements Serializable { code, acronym, contracttype, - fundingtree); + fundingtree, + familyName, + givenName, + alternativeNames); } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 44004faf3..b1f419a7e 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -20,7 +20,6 @@ import javax.xml.transform.*; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; -import eu.dnetlib.dhp.oa.provision.model.*; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -42,6 +41,7 @@ import com.google.common.collect.Sets; import com.mycila.xmltool.XMLDoc; import com.mycila.xmltool.XMLTag; +import eu.dnetlib.dhp.oa.provision.model.*; import eu.dnetlib.dhp.schema.common.*; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Result; @@ -1035,6 +1035,42 @@ public class XmlRecordFactory implements Serializable { .collect(Collectors.toList())); } + break; + case person: + final Person person = (Person) entity; + + if (person.getGivenName() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("givenname", person.getGivenName())); + } + if (person.getFamilyName() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("familyname", person.getFamilyName())); + } + if (person.getAlternativeNames() != null) { + metadata.addAll(person.getAlternativeNames()); + } + if (person.getBiography() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("biography", person.getBiography())); + } + if (person.getSubject() != null) { + metadata + .addAll( + person + .getSubject() + .stream() + .map(pt -> { + List> attrs = Lists.newArrayList(); + attrs.add(new Tuple2<>("schema", pt.getSchema())); + attrs.add(new Tuple2<>("value", pt.getValue())); + attrs.add(new Tuple2<>("fromYear", String.valueOf(pt.getFromYear()))); + attrs.add(new Tuple2<>("toYear", String.valueOf(pt.getToYear()))); + return XmlSerializationUtils.asXmlElement("subject", attrs); + }) + .collect(Collectors.toList())); + } + if (person.getConsent() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("consent", String.valueOf(person.getConsent()))); + } + break; default: throw new IllegalArgumentException("invalid entity type: " + type); @@ -1240,6 +1276,25 @@ public class XmlRecordFactory implements Serializable { .collect(Collectors.toList())); } break; + + case person: + + if (isNotBlank(re.getGivenName())) { + metadata.add(XmlSerializationUtils.asXmlElement("givenname", re.getGivenName())); + } + if (isNotBlank(re.getFamilyName())) { + metadata.add(XmlSerializationUtils.asXmlElement("familyname", re.getFamilyName())); + } + if (re.getAlternativeNames() != null && !re.getAlternativeNames().isEmpty()) { + metadata + .addAll( + re + .getAlternativeNames() + .stream() + .map(name -> XmlSerializationUtils.asXmlElement("alternativename", name)) + .collect(Collectors.toList())); + } + break; default: throw new IllegalArgumentException("invalid target type: " + targetType); } From 6b0f7cc8b09f93ba6435682137d691fd58bfd75e Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 23 Sep 2024 10:16:53 +0200 Subject: [PATCH 126/239] skip urls with authentication --- .../plugin/osf/OsfPreprintsIterator.java | 5 ++++ .../osf/OsfPreprintsCollectorPluginTest.java | 29 ++++++++++++++++++- 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index 3bb2e2bdf..9484297d0 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -125,8 +125,13 @@ public class OsfPreprintsIterator implements Iterator { final String xml = JsonUtils.convertToXML(json); return DocumentHelper.parseText(xml); + } catch (final Throwable e) { log.warn(e.getMessage(), e); + if ((e instanceof CollectorException) && e.getMessage().contains("401")) { + final Element root = DocumentHelper.createElement("error_401_authorization_required"); + return DocumentHelper.createDocument(root); + } return downloadUrl(url, attempt + 1); } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index 708a26acd..3d8d7a91f 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -1,11 +1,15 @@ package eu.dnetlib.dhp.collection.plugin.osf; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import java.util.HashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Stream; +import org.dom4j.DocumentHelper; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; @@ -17,6 +21,7 @@ import eu.dnetlib.dhp.collection.ApiDescriptor; import eu.dnetlib.dhp.common.aggregation.AggregatorReport; import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.common.collection.HttpConnector2; public class OsfPreprintsCollectorPluginTest { @@ -50,7 +55,7 @@ public class OsfPreprintsCollectorPluginTest { } @Test - // @Disabled + @Disabled void test_limited() throws CollectorException { final AtomicInteger i = new AtomicInteger(0); final Stream stream = this.plugin.collect(this.api, new AggregatorReport()); @@ -83,4 +88,26 @@ public class OsfPreprintsCollectorPluginTest { Assertions.assertTrue(i.get() > 0); } + @Test + // @Disabled + void test_authentication_required() { + final HttpConnector2 connector = new HttpConnector2(); + + try { + final String res = connector.getInputSource("https://api.osf.io/v2/preprints/ydtzx/contributors/?format=json"); + System.out.println(res); + fail(); + } catch (final Throwable e) { + + System.out.println("**** ERROR: " + e.getMessage()); + + if ((e instanceof CollectorException) && e.getMessage().contains("401")) { + System.out.println(" XML: " + DocumentHelper.createDocument().getRootElement().detach()); + } + + assertTrue(e.getMessage().contains("401")); + } + + } + } From 2d7a7a962da50cdc172cdc07ed0d5fb5df78a6f7 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 23 Sep 2024 10:19:36 +0200 Subject: [PATCH 127/239] unit test @Disabled --- .../collection/plugin/osf/OsfPreprintsCollectorPluginTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index 3d8d7a91f..efba0c72e 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -89,7 +89,7 @@ public class OsfPreprintsCollectorPluginTest { } @Test - // @Disabled + @Disabled void test_authentication_required() { final HttpConnector2 connector = new HttpConnector2(); From e0ff84baf0d8caf88965d13cd45a946a783f229d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 23 Sep 2024 10:29:46 +0200 Subject: [PATCH 128/239] [graph provision] person serialisation, limit the number of authorships and coauthorships before expanding the payloads --- .../dhp/schema/oaf/utils/ModelHardLimits.java | 12 +++++++ .../dhp/oa/provision/PayloadConverterJob.java | 33 +++++++++++++++++++ .../oa/provision/utils/XmlRecordFactory.java | 8 ++++- 3 files changed, 52 insertions(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java index 36d138ba1..e4b184fa1 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java @@ -1,6 +1,12 @@ package eu.dnetlib.dhp.schema.oaf.utils; +import java.util.Map; + +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.schema.common.ModelConstants; + public class ModelHardLimits { private ModelHardLimits() { @@ -19,6 +25,12 @@ public class ModelHardLimits { public static final int MAX_ABSTRACT_LENGTH = 150000; public static final int MAX_RELATED_ABSTRACT_LENGTH = 500; public static final int MAX_INSTANCES = 10; + public static final Map MAX_RELATIONS_BY_RELCLASS = Maps.newHashMap(); + + static { + MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.PERSON_PERSON_HASCOAUTHORED, 500); + MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.RESULT_PERSON_HASAUTHORED, 500); + } public static String getCollectionName(String format) { return format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION; diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java index 351526336..cb2d2e799 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java @@ -2,10 +2,12 @@ package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits.MAX_RELATIONS_BY_RELCLASS; import static eu.dnetlib.dhp.utils.DHPUtils.toSeq; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import org.apache.commons.io.IOUtils; @@ -15,11 +17,13 @@ import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.apache.spark.util.LongAccumulator; +import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -27,11 +31,13 @@ import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; +import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; import eu.dnetlib.dhp.oa.provision.model.TupleWrapper; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; import eu.dnetlib.dhp.schema.solr.SolrRecord; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -124,6 +130,9 @@ public class PayloadConverterJob { .map(Oaf::getDataInfo) .map(DataInfo::getDeletedbyinference) .orElse(false)) + .map( + (MapFunction) PayloadConverterJob::pruneRelatedEntities, + Encoders.kryo(JoinedEntity.class)) .map( (MapFunction>) je -> new Tuple2<>( recordFactory.build(je, validateXML), @@ -139,6 +148,30 @@ public class PayloadConverterJob { .json(outputPath); } + /** + This function iterates through the RelatedEntityWrapper(s) associated to the JoinedEntity and rules out + those exceeding the maximum allowed frequency defined in eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits#MAX_RELATIONS_BY_RELCLASS + */ + private static JoinedEntity pruneRelatedEntities(JoinedEntity je) { + Map freqs = Maps.newHashMap(); + List rew = Lists.newArrayList(); + + if (je.getLinks() != null) { + je.getLinks().forEach(link -> { + final String relClass = link.getRelation().getRelClass(); + Long count = freqs.putIfAbsent(relClass, 0L); + if (Objects.isNull(count) || (MAX_RELATIONS_BY_RELCLASS.containsKey(relClass) + && count <= MAX_RELATIONS_BY_RELCLASS.get(relClass))) { + rew.add(link); + freqs.put(relClass, freqs.get(relClass) + 1); + } + }); + je.setLinks(rew); + } + + return je; + } + private static void removeOutputDir(final SparkSession spark, final String path) { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index b1f419a7e..97d2d3989 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -1046,7 +1046,13 @@ public class XmlRecordFactory implements Serializable { metadata.add(XmlSerializationUtils.asXmlElement("familyname", person.getFamilyName())); } if (person.getAlternativeNames() != null) { - metadata.addAll(person.getAlternativeNames()); + metadata + .addAll( + person + .getAlternativeNames() + .stream() + .map(altName -> XmlSerializationUtils.asXmlElement("alternativename", altName)) + .collect(Collectors.toList())); } if (person.getBiography() != null) { metadata.add(XmlSerializationUtils.asXmlElement("biography", person.getBiography())); From dbea7a4072e8e0b75dd907d173a6cf067f4fa7d6 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Mon, 23 Sep 2024 14:57:11 +0300 Subject: [PATCH 129/239] removed duplicate line --- .../graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index ca5b0bb90..447fe3fb9 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -262,7 +262,6 @@ create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * f create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance orig where exists (select 1 from TARGET.result r where r.id=orig.id); -create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.indi_pub_publicly_funded stored as parquet as select * from SOURCE.indi_pub_publicly_funded orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); From fdbe629f49b96e345e9993de01450636b3e12acc Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 23 Sep 2024 13:01:45 +0200 Subject: [PATCH 130/239] removed the deletedByInference=true filter --- .../eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java index 5ff469cd0..236269ff3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java @@ -70,9 +70,8 @@ public class PrepareRelatedProjectsJob { final Dataset rels = ClusterUtils .loadRelations(graphPath, spark) - .filter((FilterFunction) r -> r.getDataInfo().getDeletedbyinference()) - .filter((FilterFunction) r -> r.getRelType().equals(ModelConstants.RESULT_PROJECT)) - .filter((FilterFunction) r -> !r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS)) + .filter((FilterFunction) r -> ModelConstants.RESULT_PROJECT.equals(r.getRelType())) + .filter((FilterFunction) r -> !BrokerConstants.IS_MERGED_IN_CLASS.equals(r.getRelClass())) .filter((FilterFunction) r -> !ClusterUtils.isDedupRoot(r.getSource())) .filter((FilterFunction) r -> !ClusterUtils.isDedupRoot(r.getTarget())); From 7f81673f3ced3fe0f373d041d5ff47ac092d09d6 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 23 Sep 2024 13:01:45 +0200 Subject: [PATCH 131/239] removed the deletedByInference=true filter --- .../eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java index 5ff469cd0..236269ff3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java @@ -70,9 +70,8 @@ public class PrepareRelatedProjectsJob { final Dataset rels = ClusterUtils .loadRelations(graphPath, spark) - .filter((FilterFunction) r -> r.getDataInfo().getDeletedbyinference()) - .filter((FilterFunction) r -> r.getRelType().equals(ModelConstants.RESULT_PROJECT)) - .filter((FilterFunction) r -> !r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS)) + .filter((FilterFunction) r -> ModelConstants.RESULT_PROJECT.equals(r.getRelType())) + .filter((FilterFunction) r -> !BrokerConstants.IS_MERGED_IN_CLASS.equals(r.getRelClass())) .filter((FilterFunction) r -> !ClusterUtils.isDedupRoot(r.getSource())) .filter((FilterFunction) r -> !ClusterUtils.isDedupRoot(r.getTarget())); From e941adbe2beb1fcb14bfba2288eb0fcd12d60d87 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 23 Sep 2024 09:47:29 +0200 Subject: [PATCH 132/239] fixed a bug with topic ENRICH/MORE/SUBJECT/ARXIV --- .../oa/matchers/simple/EnrichMoreSubject.java | 2 +- .../simple/EnrichMoreSubjectTest.java | 60 +++++++++++++++++++ 2 files changed, 61 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java index b62b509c7..390357f99 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java @@ -53,7 +53,7 @@ public class EnrichMoreSubject extends UpdateMatcher { .collect(Collectors.toSet()); return source - .getPids() + .getSubjects() .stream() .filter(s -> !existingSubjects.contains(subjectAsString(s))) .collect(Collectors.toList()); diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java new file mode 100644 index 000000000..1fb35c0c9 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java @@ -0,0 +1,60 @@ + +package eu.dnetlib.dhp.broker.oa.matchers.simple; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.List; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerTypedValue; + +public class EnrichMoreSubjectTest { + + final EnrichMoreSubject matcher = new EnrichMoreSubject(); + + @BeforeEach + void setUp() throws Exception { + } + + @Test + void testFindDifferences_1() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_2() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + final List list = this.matcher.findDifferences(source, target); + assertEquals(1, list.size()); + } + + @Test + void testFindDifferences_3() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + target.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_4() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + target.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + +} From 0e89d4a1cfc4c2fbe4db61299cda84718a62a49f Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 23 Sep 2024 09:47:29 +0200 Subject: [PATCH 133/239] fixed a bug with topic ENRICH/MORE/SUBJECT/ARXIV --- .../oa/matchers/simple/EnrichMoreSubject.java | 2 +- .../simple/EnrichMoreSubjectTest.java | 60 +++++++++++++++++++ 2 files changed, 61 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java index b62b509c7..390357f99 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java @@ -53,7 +53,7 @@ public class EnrichMoreSubject extends UpdateMatcher { .collect(Collectors.toSet()); return source - .getPids() + .getSubjects() .stream() .filter(s -> !existingSubjects.contains(subjectAsString(s))) .collect(Collectors.toList()); diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java new file mode 100644 index 000000000..1fb35c0c9 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubjectTest.java @@ -0,0 +1,60 @@ + +package eu.dnetlib.dhp.broker.oa.matchers.simple; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.List; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerTypedValue; + +public class EnrichMoreSubjectTest { + + final EnrichMoreSubject matcher = new EnrichMoreSubject(); + + @BeforeEach + void setUp() throws Exception { + } + + @Test + void testFindDifferences_1() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_2() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + final List list = this.matcher.findDifferences(source, target); + assertEquals(1, list.size()); + } + + @Test + void testFindDifferences_3() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + target.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_4() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + target.setSubjects(Arrays.asList(new OaBrokerTypedValue("arxiv", "subject_01"))); + final List list = this.matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + +} From d1cadc77c90bd3e6eca9b351b9dc4620cb915c2c Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 24 Sep 2024 10:57:20 +0200 Subject: [PATCH 134/239] [graph provision] person serialisation, limit the number of authorships and coauthorships before expanding the payloads --- .../dhp/schema/oaf/utils/ModelHardLimits.java | 6 +++--- .../dhp/oa/provision/PayloadConverterJob.java | 12 +++++++----- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java index e4b184fa1..68f60d4d9 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java @@ -25,11 +25,11 @@ public class ModelHardLimits { public static final int MAX_ABSTRACT_LENGTH = 150000; public static final int MAX_RELATED_ABSTRACT_LENGTH = 500; public static final int MAX_INSTANCES = 10; - public static final Map MAX_RELATIONS_BY_RELCLASS = Maps.newHashMap(); + public static final Map MAX_RELATIONS_BY_RELCLASS = Maps.newHashMap(); static { - MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.PERSON_PERSON_HASCOAUTHORED, 500); - MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.RESULT_PERSON_HASAUTHORED, 500); + MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.PERSON_PERSON_HASCOAUTHORED, 500L); + MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.RESULT_PERSON_HASAUTHORED, 500L); } public static String getCollectionName(String format) { diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java index cb2d2e799..58838d047 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java @@ -149,8 +149,8 @@ public class PayloadConverterJob { } /** - This function iterates through the RelatedEntityWrapper(s) associated to the JoinedEntity and rules out - those exceeding the maximum allowed frequency defined in eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits#MAX_RELATIONS_BY_RELCLASS + * This function iterates through the RelatedEntityWrapper(s) associated to the JoinedEntity and rules out + * those exceeding the maximum allowed frequency defined in eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits#MAX_RELATIONS_BY_RELCLASS */ private static JoinedEntity pruneRelatedEntities(JoinedEntity je) { Map freqs = Maps.newHashMap(); @@ -159,9 +159,11 @@ public class PayloadConverterJob { if (je.getLinks() != null) { je.getLinks().forEach(link -> { final String relClass = link.getRelation().getRelClass(); - Long count = freqs.putIfAbsent(relClass, 0L); - if (Objects.isNull(count) || (MAX_RELATIONS_BY_RELCLASS.containsKey(relClass) - && count <= MAX_RELATIONS_BY_RELCLASS.get(relClass))) { + + final Long count = freqs.getOrDefault(relClass, Long.MAX_VALUE); + final Long max = MAX_RELATIONS_BY_RELCLASS.getOrDefault(relClass, Long.MAX_VALUE); + + if (count <= max) { rew.add(link); freqs.put(relClass, freqs.get(relClass) + 1); } From 4f0463d7796ea18e32e94af551d1a951ed574503 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 24 Sep 2024 14:54:34 +0200 Subject: [PATCH 135/239] [graph provision] person serialisation, limit the number of authorships and coauthorships before expanding the payloads --- .../java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java index 58838d047..2593ef6fe 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java @@ -160,12 +160,12 @@ public class PayloadConverterJob { je.getLinks().forEach(link -> { final String relClass = link.getRelation().getRelClass(); - final Long count = freqs.getOrDefault(relClass, Long.MAX_VALUE); + final Long count = freqs.getOrDefault(relClass, 0L); final Long max = MAX_RELATIONS_BY_RELCLASS.getOrDefault(relClass, Long.MAX_VALUE); if (count <= max) { rew.add(link); - freqs.put(relClass, freqs.get(relClass) + 1); + freqs.put(relClass, freqs.getOrDefault(relClass, 0L) + 1); } }); je.setLinks(rew); From b35d046fd2e0c7e498c507031f3f926619b89449 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 24 Sep 2024 15:18:54 +0200 Subject: [PATCH 136/239] fixed a bug with 'null' string --- .../plugin/osf/OsfPreprintsIterator.java | 18 +++++++++++++----- .../osf/OsfPreprintsCollectorPluginTest.java | 7 +++++++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index 9484297d0..292e1c322 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -5,6 +5,7 @@ import java.util.Iterator; import java.util.Queue; import java.util.concurrent.PriorityBlockingQueue; +import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentHelper; import org.dom4j.Element; @@ -46,13 +47,14 @@ public class OsfPreprintsIterator implements Iterator { private void initQueue() { this.currentUrl = this.baseUrl + "?filter:is_published:d=true&format=json&page[size]=" + this.pageSize; + log.info("REST calls starting with {}", this.currentUrl); } @Override public boolean hasNext() { synchronized (this.recordQueue) { - while (this.recordQueue.isEmpty() && !this.currentUrl.isEmpty()) { + while (this.recordQueue.isEmpty() && StringUtils.isNotBlank(this.currentUrl) && this.currentUrl.startsWith("http")) { try { this.currentUrl = downloadPage(this.currentUrl); } catch (final CollectorException e) { @@ -88,12 +90,18 @@ public class OsfPreprintsIterator implements Iterator { group.addElement("preprint").add(n); for (final Object o1 : n.selectNodes(".//contributors//href")) { - final Document doc1 = downloadUrl(((Node) o1).getText(), 0); - group.addElement("contributors").add(doc1.getRootElement().detach()); + final String href = ((Node) o1).getText(); + if (StringUtils.isNotBlank(href) && href.startsWith("http")) { + final Document doc1 = downloadUrl(href, 0); + group.addElement("contributors").add(doc1.getRootElement().detach()); + } } for (final Object o1 : n.selectNodes(".//primary_file//href")) { - final Document doc1 = downloadUrl(((Node) o1).getText(), 0); - group.addElement("primary_file").add(doc1.getRootElement().detach()); + final String href = ((Node) o1).getText(); + if (StringUtils.isNotBlank(href) && href.startsWith("http")) { + final Document doc1 = downloadUrl(href, 0); + group.addElement("primary_file").add(doc1.getRootElement().detach()); + } } this.recordQueue.add(DocumentHelper.createDocument(group).asXML()); diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index efba0c72e..fe2274c89 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -18,6 +18,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.utils.JsonUtils; import eu.dnetlib.dhp.common.aggregation.AggregatorReport; import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; @@ -110,4 +111,10 @@ public class OsfPreprintsCollectorPluginTest { } + @Test + void testXML() { + final String xml = JsonUtils.convertToXML("{'next':null}"); + System.out.println(xml); + } + } From fa2532db303f66c7a1780c0720ccec51cfb10864 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 25 Sep 2024 09:38:50 +0200 Subject: [PATCH 137/239] fixed a bug with id --- .../dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index 292e1c322..de18ef37f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -85,7 +85,7 @@ public class OsfPreprintsIterator implements Iterator { final Element n = (Element) ((Element) o).detach(); final Element group = DocumentHelper.createElement("group"); - group.addAttribute("id", n.valueOf(".//data/id")); + group.addAttribute("id", n.valueOf("./id")); group.addElement("preprint").add(n); From 6a097abc89f539c4127529f2992063c31b9eed45 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Wed, 18 Sep 2024 17:16:53 +0200 Subject: [PATCH 138/239] as described on ticket #9525 1. Changed the mapping applied to Crossref records: anything that has a relationship "is-review-of" must be mapped as publication of type "Review". 2. Force the hostedby of Crossref records with DOI prefix 10.3410 and 10.12703 to the H1 Connect data source. --- .../dhp/collection/crossref/Crossref2Oaf.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index ebe72ae5b..3c1880fba 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -504,6 +504,22 @@ case object Crossref2Oaf { ) } + val is_review = json \ "relation" \ "is-review-of" \ "id" + + if (is_review != JNothing) { + instance.setInstancetype( + OafMapperUtils.qualifier( + "0015", + "peerReviewed", + ModelConstants.DNET_REVIEW_LEVELS, + ModelConstants.DNET_REVIEW_LEVELS + ) + ) + } + + if(doi.startsWith("10.3410") || doi.startsWith("10.12703")) + instance.setHostedby(OafMapperUtils.keyValue(OafMapperUtils.createOpenaireId(10, "openaire____::H1Connect", true),"H1Connect")) + instance.setAccessright( decideAccessRight(instance.getLicense, result.getDateofacceptance.getValue) ) From 535a7b99f1d12506283a67ece8dcbeac706c3fc4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 5 Aug 2024 16:55:07 +0200 Subject: [PATCH 139/239] the metadata collection plugins using the HttpConnector2 class shall now retry instead of failing in case of UnknownHostException --- .../java/eu/dnetlib/dhp/common/collection/HttpConnector2.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java index 78bb99e79..d2e53f11a 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java @@ -212,11 +212,11 @@ public class HttpConnector2 { .format( "Unexpected status code: %s errors: %s", urlConn.getResponseCode(), MAPPER.writeValueAsString(report))); - } catch (MalformedURLException | UnknownHostException e) { + } catch (MalformedURLException e) { log.error(e.getMessage(), e); report.put(e.getClass().getName(), e.getMessage()); throw new CollectorException(e.getMessage(), e); - } catch (SocketTimeoutException | SocketException e) { + } catch (SocketTimeoutException | SocketException | UnknownHostException e) { log.error(e.getMessage(), e); report.put(e.getClass().getName(), e.getMessage()); backoffAndSleep(getClientParams().getRetryDelay() * retryNumber * 1000); From e354f9853a7675982973703ed1237ad4d9da6630 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 25 Sep 2024 15:27:02 +0200 Subject: [PATCH 140/239] [OpenCitations] move the extracted contents under a backup path to avoid needing to re-download it in case of errors --- .../actionmanager/opencitations/ReadCOCI.java | 8 +++++++- .../input_readcoci_parameters.json | 19 +++++++++++++------ .../opencitations/oozie_app/workflow.xml | 1 + 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java index 479aea458..4b0bbf145 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java @@ -49,6 +49,9 @@ public class ReadCOCI implements Serializable { final String workingPath = parser.get("inputPath"); log.info("workingPath {}", workingPath); + final String backupPath = parser.get("backupPath"); + log.info("backupPath {}", backupPath); + SparkConf sconf = new SparkConf(); Configuration conf = new Configuration(); @@ -68,12 +71,14 @@ public class ReadCOCI implements Serializable { workingPath, fileSystem, outputPath, + backupPath, delimiter); }); } private static void doRead(SparkSession spark, String workingPath, FileSystem fileSystem, String outputPath, + String backupPath, String delimiter) throws IOException { RemoteIterator fileStatusListIterator = fileSystem .listFiles( @@ -107,7 +112,8 @@ public class ReadCOCI implements Serializable { .mode(SaveMode.Append) .option("compression", "gzip") .json(outputPath); - fileSystem.rename(fileStatus.getPath(), new Path("/tmp/miriam/OC/DONE")); + + fileSystem.rename(fileStatus.getPath(), new Path(backupPath)); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json index a74ceb983..d1f495d67 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json @@ -24,12 +24,19 @@ "paramLongName": "outputPath", "paramDescription": "the hdfs name node", "paramRequired": true - }, { - "paramName": "nn", - "paramLongName": "hdfsNameNode", - "paramDescription": "the hdfs name node", - "paramRequired": true -} + }, + { + "paramName": "nn", + "paramLongName": "hdfsNameNode", + "paramDescription": "the hdfs name node", + "paramRequired": true + }, + { + "paramName": "bp", + "paramLongName": "backupPath", + "paramDescription": "the hdfs path to move the OC data after the extraction", + "paramRequired": true + } ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml index 566cf7d02..f170af96f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml @@ -129,6 +129,7 @@ --inputPath${inputPath}/Extracted --outputPath${inputPath}/JSON + --backupPath${inputPath}/backup --delimiter${delimiter} --hdfsNameNode${nameNode} From 6397141e56b83f6fd6c89cba93fdda3533d1140b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 25 Sep 2024 15:27:32 +0200 Subject: [PATCH 141/239] code formatting --- .../dhp/collection/CollectorWorker.java | 89 ++++++++++--------- .../osf/OsfPreprintsCollectorPlugin.java | 14 +-- .../plugin/osf/OsfPreprintsIterator.java | 17 ++-- .../collection/crossref/Crossref2Oaf.scala | 6 +- .../osf/OsfPreprintsCollectorPluginTest.java | 10 ++- 5 files changed, 75 insertions(+), 61 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java index 67e07ba59..f63bfcb48 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java @@ -46,11 +46,11 @@ public class CollectorWorker extends ReportingJob { private final HttpClientParams clientParams; public CollectorWorker( - final ApiDescriptor api, - final FileSystem fileSystem, - final MDStoreVersion mdStoreVersion, - final HttpClientParams clientParams, - final AggregatorReport report) { + final ApiDescriptor api, + final FileSystem fileSystem, + final MDStoreVersion mdStoreVersion, + final HttpClientParams clientParams, + final AggregatorReport report) { super(report); this.api = api; this.fileSystem = fileSystem; @@ -69,22 +69,25 @@ public class CollectorWorker extends ReportingJob { scheduleReport(counter); try (SequenceFile.Writer writer = SequenceFile - .createWriter(this.fileSystem.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer - .keyClass(IntWritable.class), SequenceFile.Writer - .valueClass(Text.class), SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) { + .createWriter( + this.fileSystem.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer + .keyClass(IntWritable.class), + SequenceFile.Writer + .valueClass(Text.class), + SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) { final IntWritable key = new IntWritable(counter.get()); final Text value = new Text(); plugin - .collect(this.api, this.report) - .forEach(content -> { - key.set(counter.getAndIncrement()); - value.set(content); - try { - writer.append(key, value); - } catch (final Throwable e) { - throw new RuntimeException(e); - } - }); + .collect(this.api, this.report) + .forEach(content -> { + key.set(counter.getAndIncrement()); + value.set(content); + try { + writer.append(key, value); + } catch (final Throwable e) { + throw new RuntimeException(e); + } + }); } catch (final Throwable e) { this.report.put(e.getClass().getName(), e.getMessage()); throw new CollectorException(e); @@ -112,36 +115,36 @@ public class CollectorWorker extends ReportingJob { private CollectorPlugin getCollectorPlugin() throws UnknownCollectorPluginException { switch (CollectorPlugin.NAME.valueOf(this.api.getProtocol())) { - case oai: - return new OaiCollectorPlugin(this.clientParams); - case rest_json2xml: - return new RestCollectorPlugin(this.clientParams); - case file: - return new FileCollectorPlugin(this.fileSystem); - case fileGzip: - return new FileGZipCollectorPlugin(this.fileSystem); - case baseDump: - return new BaseCollectorPlugin(this.fileSystem); - case gtr2Publications: - return new Gtr2PublicationsCollectorPlugin(this.clientParams); - case osfPreprints: - return new OsfPreprintsCollectorPlugin(this.clientParams); - case other: - final CollectorPlugin.NAME.OTHER_NAME plugin = Optional + case oai: + return new OaiCollectorPlugin(this.clientParams); + case rest_json2xml: + return new RestCollectorPlugin(this.clientParams); + case file: + return new FileCollectorPlugin(this.fileSystem); + case fileGzip: + return new FileGZipCollectorPlugin(this.fileSystem); + case baseDump: + return new BaseCollectorPlugin(this.fileSystem); + case gtr2Publications: + return new Gtr2PublicationsCollectorPlugin(this.clientParams); + case osfPreprints: + return new OsfPreprintsCollectorPlugin(this.clientParams); + case other: + final CollectorPlugin.NAME.OTHER_NAME plugin = Optional .ofNullable(this.api.getParams().get("other_plugin_type")) .map(CollectorPlugin.NAME.OTHER_NAME::valueOf) .orElseThrow(() -> new IllegalArgumentException("invalid other_plugin_type")); - switch (plugin) { - case mdstore_mongodb_dump: - return new MongoDbDumpCollectorPlugin(this.fileSystem); - case mdstore_mongodb: - return new MDStoreCollectorPlugin(); + switch (plugin) { + case mdstore_mongodb_dump: + return new MongoDbDumpCollectorPlugin(this.fileSystem); + case mdstore_mongodb: + return new MDStoreCollectorPlugin(); + default: + throw new UnknownCollectorPluginException("plugin is not managed: " + plugin); + } default: - throw new UnknownCollectorPluginException("plugin is not managed: " + plugin); - } - default: - throw new UnknownCollectorPluginException("protocol is not managed: " + this.api.getProtocol()); + throw new UnknownCollectorPluginException("protocol is not managed: " + this.api.getProtocol()); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java index fdc9df06f..b0787eb45 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPlugin.java @@ -31,17 +31,19 @@ public class OsfPreprintsCollectorPlugin implements CollectorPlugin { final String baseUrl = api.getBaseUrl(); final int pageSize = Optional - .ofNullable(api.getParams().get("pageSize")) - .filter(StringUtils::isNotBlank) - .map(s -> NumberUtils.toInt(s, PAGE_SIZE_VALUE_DEFAULT)) - .orElse(PAGE_SIZE_VALUE_DEFAULT); + .ofNullable(api.getParams().get("pageSize")) + .filter(StringUtils::isNotBlank) + .map(s -> NumberUtils.toInt(s, PAGE_SIZE_VALUE_DEFAULT)) + .orElse(PAGE_SIZE_VALUE_DEFAULT); - if (StringUtils.isBlank(baseUrl)) { throw new CollectorException("Param 'baseUrl' is null or empty"); } + if (StringUtils.isBlank(baseUrl)) { + throw new CollectorException("Param 'baseUrl' is null or empty"); + } final OsfPreprintsIterator it = new OsfPreprintsIterator(baseUrl, pageSize, getClientParams()); return StreamSupport - .stream(Spliterators.spliteratorUnknownSize(it, Spliterator.ORDERED), false); + .stream(Spliterators.spliteratorUnknownSize(it, Spliterator.ORDERED), false); } public HttpClientParams getClientParams() { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java index de18ef37f..76adba1a8 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsIterator.java @@ -34,9 +34,9 @@ public class OsfPreprintsIterator implements Iterator { private final Queue recordQueue = new PriorityBlockingQueue<>(); public OsfPreprintsIterator( - final String baseUrl, - final int pageSize, - final HttpClientParams clientParams) { + final String baseUrl, + final int pageSize, + final HttpClientParams clientParams) { this.clientParams = clientParams; this.baseUrl = baseUrl; @@ -54,7 +54,8 @@ public class OsfPreprintsIterator implements Iterator { @Override public boolean hasNext() { synchronized (this.recordQueue) { - while (this.recordQueue.isEmpty() && StringUtils.isNotBlank(this.currentUrl) && this.currentUrl.startsWith("http")) { + while (this.recordQueue.isEmpty() && StringUtils.isNotBlank(this.currentUrl) + && this.currentUrl.startsWith("http")) { try { this.currentUrl = downloadPage(this.currentUrl); } catch (final CollectorException e) { @@ -63,7 +64,9 @@ public class OsfPreprintsIterator implements Iterator { } } - if (!this.recordQueue.isEmpty()) { return true; } + if (!this.recordQueue.isEmpty()) { + return true; + } return false; } @@ -112,7 +115,9 @@ public class OsfPreprintsIterator implements Iterator { } private Document downloadUrl(final String url, final int attempt) throws CollectorException { - if (attempt > MAX_ATTEMPTS) { throw new CollectorException("Max Number of attempts reached, url:" + url); } + if (attempt > MAX_ATTEMPTS) { + throw new CollectorException("Max Number of attempts reached, url:" + url); + } if (attempt > 0) { final int delay = (attempt * 5000); diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index 3c1880fba..4bd6bcc09 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -517,8 +517,10 @@ case object Crossref2Oaf { ) } - if(doi.startsWith("10.3410") || doi.startsWith("10.12703")) - instance.setHostedby(OafMapperUtils.keyValue(OafMapperUtils.createOpenaireId(10, "openaire____::H1Connect", true),"H1Connect")) + if (doi.startsWith("10.3410") || doi.startsWith("10.12703")) + instance.setHostedby( + OafMapperUtils.keyValue(OafMapperUtils.createOpenaireId(10, "openaire____::H1Connect", true), "H1Connect") + ) instance.setAccessright( decideAccessRight(instance.getLicense, result.getDateofacceptance.getValue) diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java index fe2274c89..664b84d5a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/osf/OsfPreprintsCollectorPluginTest.java @@ -50,9 +50,10 @@ public class OsfPreprintsCollectorPluginTest { @Test @Disabled void test_one() throws CollectorException { - this.plugin.collect(this.api, new AggregatorReport()) - .limit(1) - .forEach(log::info); + this.plugin + .collect(this.api, new AggregatorReport()) + .limit(1) + .forEach(log::info); } @Test @@ -95,7 +96,8 @@ public class OsfPreprintsCollectorPluginTest { final HttpConnector2 connector = new HttpConnector2(); try { - final String res = connector.getInputSource("https://api.osf.io/v2/preprints/ydtzx/contributors/?format=json"); + final String res = connector + .getInputSource("https://api.osf.io/v2/preprints/ydtzx/contributors/?format=json"); System.out.println(res); fail(); } catch (final Throwable e) { From c45cae447aa8aa18ec1e8a4e130d2c190cc5ff70 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Sat, 24 Feb 2024 16:54:30 +0100 Subject: [PATCH 142/239] Fix: invert the "natural" order when ordering by id lexicographically --- .../dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java index ff6c2689a..5f212c242 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeEntitiesComparator.java @@ -96,7 +96,7 @@ public class MergeEntitiesComparator implements Comparator { // id if (res == 0) { if (left instanceof OafEntity && right instanceof OafEntity) { - res = ((OafEntity) left).getId().compareTo(((OafEntity) right).getId()); + res = ((OafEntity) right).getId().compareTo(((OafEntity) left).getId()); } } From e430826e0032870ab24f579777d9bd6f52d21458 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 30 Sep 2024 15:10:10 +0200 Subject: [PATCH 143/239] [ImportOC] fix to move original folder instead of extracted ones --- .../opencitations/GetOpenCitationsRefs.java | 8 ++++++-- .../dhp/actionmanager/opencitations/ReadCOCI.java | 7 +------ .../opencitations/input_parameters.json | 6 ++++++ .../opencitations/input_readcoci_parameters.json | 6 ------ .../opencitations/oozie_app/workflow.xml | 13 +------------ 5 files changed, 14 insertions(+), 26 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java index be653aed2..6a779f6c4 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java @@ -46,6 +46,9 @@ public class GetOpenCitationsRefs implements Serializable { final String outputPath = parser.get("outputPath"); log.info("outputPath {}", outputPath); + final String backupPath = parser.get("backupPath"); + log.info("backupPath {}", backupPath); + Configuration conf = new Configuration(); conf.set("fs.defaultFS", hdfsNameNode); @@ -53,11 +56,11 @@ public class GetOpenCitationsRefs implements Serializable { GetOpenCitationsRefs ocr = new GetOpenCitationsRefs(); - ocr.doExtract(inputPath, outputPath, fileSystem); + ocr.doExtract(inputPath, outputPath, backupPath, fileSystem); } - private void doExtract(String inputPath, String outputPath, FileSystem fileSystem) + private void doExtract(String inputPath, String outputPath, String backupPath, FileSystem fileSystem) throws IOException { RemoteIterator fileStatusListIterator = fileSystem @@ -89,6 +92,7 @@ public class GetOpenCitationsRefs implements Serializable { } } + fileSystem.rename(fileStatus.getPath(), new Path(backupPath)); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java index 4b0bbf145..de45d50b2 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java @@ -49,9 +49,6 @@ public class ReadCOCI implements Serializable { final String workingPath = parser.get("inputPath"); log.info("workingPath {}", workingPath); - final String backupPath = parser.get("backupPath"); - log.info("backupPath {}", backupPath); - SparkConf sconf = new SparkConf(); Configuration conf = new Configuration(); @@ -71,14 +68,12 @@ public class ReadCOCI implements Serializable { workingPath, fileSystem, outputPath, - backupPath, delimiter); }); } private static void doRead(SparkSession spark, String workingPath, FileSystem fileSystem, String outputPath, - String backupPath, String delimiter) throws IOException { RemoteIterator fileStatusListIterator = fileSystem .listFiles( @@ -113,7 +108,7 @@ public class ReadCOCI implements Serializable { .option("compression", "gzip") .json(outputPath); - fileSystem.rename(fileStatus.getPath(), new Path(backupPath)); + fileSystem.delete(fileStatus.getPath()); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json index f4b6e2d68..10225f367 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json @@ -16,5 +16,11 @@ "paramLongName": "hdfsNameNode", "paramDescription": "the hdfs name node", "paramRequired": true + }, + { + "paramName": "bp", + "paramLongName": "backupPath", + "paramDescription": "the hdfs path to move the OC data after the extraction", + "paramRequired": true } ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json index d1f495d67..f3d72e063 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json @@ -30,12 +30,6 @@ "paramLongName": "hdfsNameNode", "paramDescription": "the hdfs name node", "paramRequired": true - }, - { - "paramName": "bp", - "paramLongName": "backupPath", - "paramDescription": "the hdfs path to move the OC data after the extraction", - "paramRequired": true } ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml index f170af96f..bb6a0eb21 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml @@ -94,17 +94,7 @@ --hdfsNameNode${nameNode} --inputPath${inputPath}/Original --outputPath${inputPath}/Extracted - - - - - - - - eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs - --hdfsNameNode${nameNode} - --inputPath${inputPath}/correspondence - --outputPath${inputPath}/correspondence_extracted + --backupPath${inputPath}/backup @@ -129,7 +119,6 @@ --inputPath${inputPath}/Extracted --outputPath${inputPath}/JSON - --backupPath${inputPath}/backup --delimiter${delimiter} --hdfsNameNode${nameNode} From f3c179658a8d978796e31d6095b947e44665b2ef Mon Sep 17 00:00:00 2001 From: antleb Date: Mon, 30 Sep 2024 17:12:21 +0300 Subject: [PATCH 144/239] datasource table creation split in steps --- .../oozie_app/scripts/buildIrishMonitorDB.sql | 4 ++-- .../graph/stats/oozie_app/scripts/step8.sql | 24 ++++++++++++------- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/scripts/buildIrishMonitorDB.sql b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/scripts/buildIrishMonitorDB.sql index 3f0922020..acad90f63 100644 --- a/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/scripts/buildIrishMonitorDB.sql +++ b/dhp-workflows/dhp-stats-monitor-irish/src/main/resources/eu/dnetlib/dhp/oa/graph/stats-monitor-irish/oozie_app/scripts/buildIrishMonitorDB.sql @@ -32,7 +32,7 @@ select distinct * from ( from SOURCE.result r join SOURCE.result_projects rp on rp.id=r.id join SOURCE.project p on p.id=rp.project - join openaire_prod_stats_monitor_ie_20231226b.irish_funders irf on irf.funder=p.funder + join TARGET.irish_funders irf on irf.funder=p.funder union all select r.* from SOURCE.result r @@ -238,4 +238,4 @@ create table TARGET.indi_pub_publicly_funded stored as parquet as select * from create table TARGET.indi_result_oa_with_license stored as parquet as select * from SOURCE.indi_result_oa_with_license orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.indi_result_oa_without_license stored as parquet as select * from SOURCE.indi_result_oa_without_license orig where exists (select 1 from TARGET.result r where r.id=orig.id); -create table TARGET.indi_result_under_transformative stored as parquet as select * from SOURCE.indi_result_under_transformative orig where exists (select 1 from TARGET.result r where r.id=orig.id); \ No newline at end of file +create table TARGET.indi_result_under_transformative stored as parquet as select * from SOURCE.indi_result_under_transformative orig where exists (select 1 from TARGET.result r where r.id=orig.id); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 98225af14..7504eba18 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -8,14 +8,20 @@ set mapred.job.queue.name=analytics; /*EOS*/ ------------------------------------------------------------ ------------------------------------------------------------ DROP TABLE IF EXISTS ${stats_db_name}.datasource purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.harested_datasources purge; /*EOS*/ +DROP TABLE IF EXISTS ${stats_db_name}.piwik_datasource purge; /*EOS*/ + +create table ${stats_db_name}.harested_datasources stored as parquet as +select distinct inst.hostedby.key as d_id +from ${openaire_db_name}.result lateral view outer explode (instance) insts as inst; /*EOS*/ + +create table ${stats_db_name}.piwik_datasource stored as parquet as +select id, split(originalidd, '\\:')[1] as piwik_id +from ${openaire_db_name}.datasource + lateral view explode(originalid) temp as originalidd +where originalidd like "piwik:%"; /*EOS*/ CREATE TABLE ${stats_db_name}.datasource stored as parquet as -with piwik_datasource as ( - select id, split(originalidd, '\\:')[1] as piwik_id - from ${openaire_db_name}.datasource - lateral view explode(originalid) temp as originalidd - where originalidd like "piwik:%" -) select /*+ COALESCE(100) */ substr(dtrce.id, 4) as id, case when dtrce.officialname.value='Unknown Repository' then 'Other' else dtrce.officialname.value end as name, @@ -31,10 +37,12 @@ select /*+ COALESCE(100) */ dtrce.journal.issnprinted as issn_printed, dtrce.journal.issnonline as issn_online from ${openaire_db_name}.datasource dtrce - left outer join (select inst.hostedby.key as d_id from ${openaire_db_name}.result lateral view outer explode (instance) insts as inst) res on res.d_id=dtrce.id - left outer join piwik_datasource piwik_d on piwik_d.id=dtrce.id +left outer join ${stats_db_name}.harested_datasources res on res.d_id=dtrce.id +left outer join ${stats_db_name}.piwik_datasource piwik_d on piwik_d.id=dtrce.id where dtrce.datainfo.deletedbyinference = false and dtrce.datainfo.invisible = false; /*EOS*/ +drop table ${stats_db_name}.harested_datasources; /*EOS*/ +drop table ${stats_db_name}.piwik_datasource; /*EOS*/ DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; /*EOS*/ From 10696f2a442d8dd62ac32464b2d6853ea5fd31f8 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 1 Oct 2024 15:54:13 +0200 Subject: [PATCH 145/239] reverted procedure for creating the UsageCounts actionset --- .../actionmanager/usagestats/SparkAtomicActionUsageJob.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java index 897153762..d6b52ad9b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java @@ -112,10 +112,7 @@ public class SparkAtomicActionUsageJob implements Serializable { .joinWith(datasource, resultModel.col("datasourceId").equalTo(datasource.col("id")), "left") .map((MapFunction, UsageStatsResultModel>) t2 -> { UsageStatsResultModel usrm = t2._1(); - if (Optional.ofNullable(t2._2()).isPresent()) - usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue()); - else - usrm.setDatasourceId(usrm.getDatasourceId() + "||NO_MATCH_FOUND"); + usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue()); return usrm; }, Encoders.bean(UsageStatsResultModel.class)) .write() From 191fc3a4618c85bccc4a20e42f5745ed9a8dc2c9 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 2 Apr 2024 16:28:27 +0200 Subject: [PATCH 146/239] [UsageCount] add check in case the datasource is not matched against those present in the graph (cherry picked from commit b42bdd5fb3f33f0d90770602ad4fea87af550751) --- .../usagestats/SparkAtomicActionUsageJob.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java index d6b52ad9b..d74b55475 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java @@ -111,8 +111,11 @@ public class SparkAtomicActionUsageJob implements Serializable { resultModel .joinWith(datasource, resultModel.col("datasourceId").equalTo(datasource.col("id")), "left") .map((MapFunction, UsageStatsResultModel>) t2 -> { - UsageStatsResultModel usrm = t2._1(); - usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue()); + if(Optional.ofNullable(t2._2()).isPresent()) + usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue()); + else + usrm.setDatasourceId(usrm.getDatasourceId() + "||NO_MATCH_FOUND"); + return usrm; return usrm; }, Encoders.bean(UsageStatsResultModel.class)) .write() From 7e6d12fa77d336acbb66b60ff327950a39616169 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 2 Apr 2024 16:56:37 +0200 Subject: [PATCH 147/239] [UsageCount] fixed error (cherry picked from commit 9c9a9562ae31f21cfcc20fafd06455f86bd2492f) --- .../dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java index d74b55475..1d5b35cff 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java @@ -111,12 +111,12 @@ public class SparkAtomicActionUsageJob implements Serializable { resultModel .joinWith(datasource, resultModel.col("datasourceId").equalTo(datasource.col("id")), "left") .map((MapFunction, UsageStatsResultModel>) t2 -> { + UsageStatsResultModel usrm = t2._1(); if(Optional.ofNullable(t2._2()).isPresent()) usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue()); else usrm.setDatasourceId(usrm.getDatasourceId() + "||NO_MATCH_FOUND"); return usrm; - return usrm; }, Encoders.bean(UsageStatsResultModel.class)) .write() .mode(SaveMode.Overwrite) From e5df68772d4f9a766224557069c74a16544c2f6b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 2 Oct 2024 09:35:21 +0200 Subject: [PATCH 148/239] [graph provision] fixed serialisation of the usage counts as measures in the XML records --- .../utils/XmlSerializationUtils.java | 3 ++- .../dnetlib/dhp/oa/provision/publication.json | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java index b4517002c..85cebcdb6 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java @@ -175,7 +175,8 @@ public class XmlSerializationUtils { .append("<") .append(name) .append(" ") - .append(attr(measure.getId(), kv.getValue())) + .append(attr("id", measure.getId())) + .append(attr("score", kv.getValue())) .append(attr("datasource", kv.getKey())) .append(" />"); } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json index a073fbebd..a8cd6dcf6 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/publication.json @@ -8,6 +8,26 @@ } ], "measures": [ + { + "id": "views", + "unit": [ + { + "key": "opendoar____::358aee4cc897452c00244351e4d91f69||ZENODO", + "value": "5", + "dataInfo": null + } + ] + }, + { + "id": "downloads", + "unit": [ + { + "key": "opendoar____::358aee4cc897452c00244351e4d91f69||ZENODO", + "value": "2", + "dataInfo": null + } + ] + }, { "id": "influence", "unit": [ From d5867a19920d8acd232b263b5d4bb9a6a4d87de0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 8 Oct 2024 14:58:28 +0200 Subject: [PATCH 149/239] merged #490 --- .../oaf/HashableStructuredProperty.java | 70 ++++++++++ .../schema/oaf/utils/CleaningFunctions.java | 30 ---- .../dhp/schema/oaf/utils/DoiCleaningRule.java | 14 +- .../oaf/utils/GraphCleaningFunctions.java | 18 ++- .../schema/oaf/utils/IdentifierFactory.java | 4 +- .../dhp/schema/oaf/utils/MergeUtils.java | 2 +- .../schema/oaf/utils/PidValueComparator.java | 4 +- .../java/eu/dnetlib/dhp/utils/DHPUtils.java | 3 +- .../oaf/utils/IdentifierFactoryTest.java | 4 +- .../schema/oaf/utils/publication_doi3.json | 2 +- .../schema/oaf/utils/publication_pmc2.json | 2 +- .../PrepareAffiliationRelations.java | 129 +++++++++++++++--- .../input_actionset_parameter.json | 12 +- .../collection/crossref/Crossref2Oaf.scala | 4 +- .../PrepareAffiliationRelationsTest.java | 67 ++++++--- .../CreateOpenCitationsASTest.java | 13 +- .../CreateTAActionSetTest.java | 13 +- .../bipaffiliations/doi_to_ror.json | 19 +-- .../bipaffiliations/doi_to_ror_old.json | 9 ++ .../bipaffiliations/publichers_old/publisher | 6 + .../bipaffiliations/publishers/publisher | 6 + .../raw/AbstractMdRecordToOafMapper.java | 35 ++--- .../dhp/oa/graph/raw/OafToOafMapper.java | 3 +- .../dhp/oa/graph/raw/OdfToOafMapper.java | 6 +- .../graph/clean/CleanGraphSparkJobTest.java | 16 +-- .../clean/GraphCleaningFunctionsTest.java | 12 +- .../raw/GenerateEntitiesApplicationTest.java | 8 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 97 ++++++++----- .../clean/graph/publication/publication.json | 2 +- .../eu/dnetlib/dhp/oa/graph/clean/result.json | 32 +---- .../dhp/oa/graph/raw/odf_record_base.xml | 129 ++++++++++++++++++ .../raw/organizations_resultset_entry.json | 5 + .../oa/graph/sample/person/person_10.json.gz | Bin 0 -> 1233 bytes .../graph/scholix/ScholixGenerationTest.scala | 3 +- 34 files changed, 540 insertions(+), 239 deletions(-) create mode 100644 dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/HashableStructuredProperty.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror_old.json create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publichers_old/publisher create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publishers/publisher create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record_base.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/person/person_10.json.gz diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/HashableStructuredProperty.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/HashableStructuredProperty.java new file mode 100644 index 000000000..aa60b097a --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/HashableStructuredProperty.java @@ -0,0 +1,70 @@ +/* + * Copyright (c) 2024. + * SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche + * SPDX-License-Identifier: AGPL-3.0-or-later + */ + +package eu.dnetlib.dhp.schema.oaf; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + +public class HashableStructuredProperty extends StructuredProperty { + + private static final long serialVersionUID = 8371670185221126045L; + + public static HashableStructuredProperty newInstance(String value, Qualifier qualifier, DataInfo dataInfo) { + if (value == null) { + return null; + } + final HashableStructuredProperty sp = new HashableStructuredProperty(); + sp.setValue(value); + sp.setQualifier(qualifier); + sp.setDataInfo(dataInfo); + return sp; + } + + public static HashableStructuredProperty newInstance(StructuredProperty sp) { + HashableStructuredProperty hsp = new HashableStructuredProperty(); + hsp.setQualifier(sp.getQualifier()); + hsp.setValue(sp.getValue()); + hsp.setQualifier(sp.getQualifier()); + return hsp; + } + + public static StructuredProperty toStructuredProperty(HashableStructuredProperty hsp) { + StructuredProperty sp = new StructuredProperty(); + sp.setQualifier(hsp.getQualifier()); + sp.setValue(hsp.getValue()); + sp.setQualifier(hsp.getQualifier()); + return sp; + } + + @Override + public int hashCode() { + return new HashCodeBuilder(11, 91) + .append(getQualifier().getClassid()) + .append(getQualifier().getSchemeid()) + .append(getValue()) + .hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + if (obj.getClass() != getClass()) { + return false; + } + final HashableStructuredProperty rhs = (HashableStructuredProperty) obj; + return new EqualsBuilder() + .append(getQualifier().getClassid(), rhs.getQualifier().getClassid()) + .append(getQualifier().getSchemeid(), rhs.getQualifier().getSchemeid()) + .append(getValue(), rhs.getValue()) + .isEquals(); + } +} diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/CleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/CleaningFunctions.java index c0ef339bd..49b30897c 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/CleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/CleaningFunctions.java @@ -43,34 +43,4 @@ public class CleaningFunctions { return !PidBlacklistProvider.getBlacklist(s.getQualifier().getClassid()).contains(pidValue); } - /** - * Utility method that normalises PID values on a per-type basis. - * @param pid the PID whose value will be normalised. - * @return the PID containing the normalised value. - */ - public static StructuredProperty normalizePidValue(StructuredProperty pid) { - pid - .setValue( - normalizePidValue( - pid.getQualifier().getClassid(), - pid.getValue())); - - return pid; - } - - public static String normalizePidValue(String pidType, String pidValue) { - String value = Optional - .ofNullable(pidValue) - .map(String::trim) - .orElseThrow(() -> new IllegalArgumentException("PID value cannot be empty")); - - switch (pidType) { - - // TODO add cleaning for more PID types as needed - case "doi": - return value.toLowerCase().replaceFirst(DOI_PREFIX_REGEX, DOI_PREFIX); - } - return value; - } - } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/DoiCleaningRule.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/DoiCleaningRule.java index 43fb0feda..cd8957764 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/DoiCleaningRule.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/DoiCleaningRule.java @@ -6,18 +6,11 @@ import org.apache.commons.lang3.StringUtils; public class DoiCleaningRule { public static String clean(final String doi) { - return doi - .toLowerCase() - .replaceAll("\\s", "") - .replaceAll("^doi:", "") - .replaceFirst(CleaningFunctions.DOI_PREFIX_REGEX, CleaningFunctions.DOI_PREFIX); - } - - public static String normalizeDoi(final String input) { - if (input == null) + if (doi == null) return null; - final String replaced = input + final String replaced = doi .replaceAll("\\n|\\r|\\t|\\s", "") + .replaceAll("^doi:", "") .toLowerCase() .replaceFirst(CleaningFunctions.DOI_PREFIX_REGEX, CleaningFunctions.DOI_PREFIX); if (StringUtils.isEmpty(replaced)) @@ -32,7 +25,6 @@ public class DoiCleaningRule { return null; return ret; - } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index 2be4e8e0c..dfa9c5ad0 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -563,12 +563,24 @@ public class GraphCleaningFunctions extends CleaningFunctions { Optional .ofNullable(i.getPid()) .ifPresent(pid -> { - final Set pids = Sets.newHashSet(pid); + final Set pids = pid + .stream() + .map(HashableStructuredProperty::newInstance) + .collect(Collectors.toCollection(HashSet::new)); Optional .ofNullable(i.getAlternateIdentifier()) .ifPresent(altId -> { - final Set altIds = Sets.newHashSet(altId); - i.setAlternateIdentifier(Lists.newArrayList(Sets.difference(altIds, pids))); + final Set altIds = altId + .stream() + .map(HashableStructuredProperty::newInstance) + .collect(Collectors.toCollection(HashSet::new)); + i + .setAlternateIdentifier( + Sets + .difference(altIds, pids) + .stream() + .map(HashableStructuredProperty::toStructuredProperty) + .collect(Collectors.toList())); }); }); diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java index 1d61c87df..2c77c3b37 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java @@ -175,7 +175,7 @@ public class IdentifierFactory implements Serializable { return entity .getPid() .stream() - .map(CleaningFunctions::normalizePidValue) + .map(PidCleaner::normalizePidValue) .filter(CleaningFunctions::pidFilter) .collect( Collectors @@ -207,7 +207,7 @@ public class IdentifierFactory implements Serializable { // filter away PIDs provided by a DS that is not considered an authority for the // given PID Type .filter(p -> shouldFilterPidByCriteria(collectedFrom, p, mapHandles)) - .map(CleaningFunctions::normalizePidValue) + .map(PidCleaner::normalizePidValue) .filter(p -> isNotFromDelegatedAuthority(collectedFrom, p)) .filter(CleaningFunctions::pidFilter)) .orElse(Stream.empty()); diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index ac7694d18..e01813110 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -972,7 +972,7 @@ public class MergeUtils { private static String extractKeyFromPid(final StructuredProperty pid) { if (pid == null) return null; - final StructuredProperty normalizedPid = CleaningFunctions.normalizePidValue(pid); + final StructuredProperty normalizedPid = PidCleaner.normalizePidValue(pid); return String.format("%s::%s", normalizedPid.getQualifier().getClassid(), normalizedPid.getValue()); } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidValueComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidValueComparator.java index 0e2083590..1e591ce72 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidValueComparator.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidValueComparator.java @@ -18,8 +18,8 @@ public class PidValueComparator implements Comparator { if (right == null) return -1; - StructuredProperty l = CleaningFunctions.normalizePidValue(left); - StructuredProperty r = CleaningFunctions.normalizePidValue(right); + StructuredProperty l = PidCleaner.normalizePidValue(left); + StructuredProperty r = PidCleaner.normalizePidValue(right); return Optional .ofNullable(l.getValue()) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java index e10d0c500..77453e5f0 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java @@ -28,6 +28,7 @@ import com.jayway.jsonpath.JsonPath; import eu.dnetlib.dhp.schema.mdstore.MDStoreWithInfo; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; import net.minidev.json.JSONArray; import scala.collection.JavaConverters; import scala.collection.Seq; @@ -104,7 +105,7 @@ public class DHPUtils { public static String generateUnresolvedIdentifier(final String pid, final String pidType) { - final String cleanedPid = CleaningFunctions.normalizePidValue(pidType, pid); + final String cleanedPid = PidCleaner.normalizePidValue(pidType, pid); return String.format("unresolved::%s::%s", cleanedPid, pidType.toLowerCase().trim()); } diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactoryTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactoryTest.java index bce4b76b5..9cc0698a4 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactoryTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactoryTest.java @@ -29,7 +29,7 @@ class IdentifierFactoryTest { "publication_doi2.json", "50|doi_________::79dbc7a2a56dc1532659f9038843256e", true); verifyIdentifier( - "publication_doi3.json", "50|pmc_________::94e4cb08c93f8733b48e2445d04002ac", true); + "publication_doi3.json", "50|pmc_________::e2a339e0e11bfbf55462e14a07f1b304", true); verifyIdentifier( "publication_doi4.json", "50|od______2852::38861c44e6052a8d49f59a4c39ba5e66", true); @@ -41,7 +41,7 @@ class IdentifierFactoryTest { "publication_pmc1.json", "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f", true); verifyIdentifier( - "publication_pmc2.json", "50|pmc_________::94e4cb08c93f8733b48e2445d04002ac", true); + "publication_pmc2.json", "50|pmc_________::e2a339e0e11bfbf55462e14a07f1b304", true); verifyIdentifier( "publication_openapc.json", "50|doi_________::79dbc7a2a56dc1532659f9038843256e", true); diff --git a/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_doi3.json b/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_doi3.json index b1ea01f60..303c023be 100644 --- a/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_doi3.json +++ b/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_doi3.json @@ -29,7 +29,7 @@ }, { "qualifier": {"classid": "pmc"}, - "value": "21459329" + "value": "PMC21459329" } ] } diff --git a/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_pmc2.json b/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_pmc2.json index e7d49eebb..3b7b6590e 100644 --- a/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_pmc2.json +++ b/dhp-common/src/test/resources/eu/dnetlib/dhp/schema/oaf/utils/publication_pmc2.json @@ -13,7 +13,7 @@ }, { "qualifier":{"classid":"pmc"}, - "value":"21459329" + "value":"PMC21459329" } ] } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 70ca1576c..028fa47dc 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -10,7 +10,6 @@ import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.BZip2Codec; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; @@ -29,6 +28,7 @@ import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; +import eu.dnetlib.dhp.schema.oaf.utils.DoiCleaningRule; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import scala.Tuple2; @@ -46,6 +46,8 @@ public class PrepareAffiliationRelations implements Serializable { public static final String BIP_INFERENCE_PROVENANCE = "openaire:affiliation"; public static final String OPENAIRE_DATASOURCE_ID = "10|infrastruct_::f66f1bd369679b5b077dcdf006089556"; public static final String OPENAIRE_DATASOURCE_NAME = "OpenAIRE"; + public static final String DOI_URL_PREFIX = "https://doi.org/"; + public static final int DOI_URL_PREFIX_LENGTH = 16; public static void main(String[] args) throws Exception { @@ -98,35 +100,26 @@ public class PrepareAffiliationRelations implements Serializable { private static void createActionSet(SparkSession spark, String crossrefInputPath, String pubmedInputPath, String openapcInputPath, String dataciteInputPath, String webcrawlInputPath, String publisherlInputPath, String outputPath) { - List collectedFromCrossref = OafMapperUtils - .listKeyValues(ModelConstants.CROSSREF_ID, "Crossref"); - JavaPairRDD crossrefRelations = prepareAffiliationRelations( - spark, crossrefInputPath, collectedFromCrossref); + List collectedfromOpenAIRE = OafMapperUtils + .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); + + JavaPairRDD crossrefRelations = prepareAffiliationRelationsNewModel( + spark, crossrefInputPath, collectedfromOpenAIRE); - List collectedFromPubmed = OafMapperUtils - .listKeyValues(ModelConstants.PUBMED_CENTRAL_ID, "Pubmed"); JavaPairRDD pubmedRelations = prepareAffiliationRelations( - spark, pubmedInputPath, collectedFromPubmed); + spark, pubmedInputPath, collectedfromOpenAIRE); - List collectedFromOpenAPC = OafMapperUtils - .listKeyValues(ModelConstants.OPEN_APC_ID, "OpenAPC"); - JavaPairRDD openAPCRelations = prepareAffiliationRelations( - spark, openapcInputPath, collectedFromOpenAPC); + JavaPairRDD openAPCRelations = prepareAffiliationRelationsNewModel( + spark, openapcInputPath, collectedfromOpenAIRE); - List collectedFromDatacite = OafMapperUtils - .listKeyValues(ModelConstants.DATACITE_ID, "Datacite"); JavaPairRDD dataciteRelations = prepareAffiliationRelations( - spark, dataciteInputPath, collectedFromDatacite); + spark, dataciteInputPath, collectedfromOpenAIRE); - List collectedFromWebCrawl = OafMapperUtils - .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); JavaPairRDD webCrawlRelations = prepareAffiliationRelations( - spark, webcrawlInputPath, collectedFromWebCrawl); + spark, webcrawlInputPath, collectedfromOpenAIRE); - List collectedfromPublisher = OafMapperUtils - .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); JavaPairRDD publisherRelations = prepareAffiliationRelationFromPublisher( - spark, publisherlInputPath, collectedfromPublisher); + spark, publisherlInputPath, collectedfromOpenAIRE); crossrefRelations .union(pubmedRelations) @@ -138,6 +131,21 @@ public class PrepareAffiliationRelations implements Serializable { outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); } + private static JavaPairRDD prepareAffiliationRelationFromPublisherNewModel(SparkSession spark, + String inputPath, + List collectedfrom) { + + Dataset df = spark + .read() + .schema( + "`DOI` STRING, `Organizations` ARRAY>") + .json(inputPath) + .where("DOI is not null"); + + return getTextTextJavaPairRDD(collectedfrom, df.selectExpr("DOI", "Organizations as Matchings")); + + } + private static JavaPairRDD prepareAffiliationRelationFromPublisher(SparkSession spark, String inputPath, List collectedfrom) { @@ -165,6 +173,20 @@ public class PrepareAffiliationRelations implements Serializable { return getTextTextJavaPairRDD(collectedfrom, df); } + private static JavaPairRDD prepareAffiliationRelationsNewModel(SparkSession spark, + String inputPath, + List collectedfrom) { + // load and parse affiliation relations from HDFS + Dataset df = spark + .read() + .schema( + "`DOI` STRING, `Matchings` ARRAY>") + .json(inputPath) + .where("DOI is not null"); + + return getTextTextJavaPairRDDNew(collectedfrom, df); + } + private static JavaPairRDD getTextTextJavaPairRDD(List collectedfrom, Dataset df) { // unroll nested arrays df = df @@ -181,7 +203,7 @@ public class PrepareAffiliationRelations implements Serializable { // DOI to OpenAIRE id final String paperId = ID_PREFIX - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", row.getAs("doi"))); + + IdentifierFactory.md5(DoiCleaningRule.clean(removePrefix(row.getAs("doi")))); // ROR id to OpenAIRE id final String affId = GenerateRorActionSetJob.calculateOpenaireId(row.getAs("rorid")); @@ -213,6 +235,69 @@ public class PrepareAffiliationRelations implements Serializable { new Text(OBJECT_MAPPER.writeValueAsString(aa)))); } + private static JavaPairRDD getTextTextJavaPairRDDNew(List collectedfrom, Dataset df) { + // unroll nested arrays + df = df + .withColumn("matching", functions.explode(new Column("Matchings"))) + .select( + new Column("DOI").as("doi"), + new Column("matching.PID").as("pidtype"), + new Column("matching.Value").as("pidvalue"), + new Column("matching.Confidence").as("confidence"), + new Column("matching.Status").as("status")) + .where("status = 'active'"); + + // prepare action sets for affiliation relations + return df + .toJavaRDD() + .flatMap((FlatMapFunction) row -> { + + // DOI to OpenAIRE id + final String paperId = ID_PREFIX + + IdentifierFactory.md5(DoiCleaningRule.clean(removePrefix(row.getAs("doi")))); + + // Organization to OpenAIRE identifier + String affId = null; + if (row.getAs("pidtype").equals("ROR")) + // ROR id to OpenIARE id + affId = GenerateRorActionSetJob.calculateOpenaireId(row.getAs("pidvalue")); + else + // getting the OpenOrgs identifier for the organization + affId = row.getAs("pidvalue"); + + Qualifier qualifier = OafMapperUtils + .qualifier( + BIP_AFFILIATIONS_CLASSID, + BIP_AFFILIATIONS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS); + + // format data info; setting `confidence` into relation's `trust` + DataInfo dataInfo = OafMapperUtils + .dataInfo( + false, + BIP_INFERENCE_PROVENANCE, + true, + false, + qualifier, + Double.toString(row.getAs("confidence"))); + + // return bi-directional relations + return getAffiliationRelationPair(paperId, affId, collectedfrom, dataInfo).iterator(); + + }) + .map(p -> new AtomicAction(Relation.class, p)) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))); + } + + private static String removePrefix(String doi) { + if (doi.startsWith(DOI_URL_PREFIX)) + return doi.substring(DOI_URL_PREFIX_LENGTH); + return doi; + } + private static List getAffiliationRelationPair(String paperId, String affId, List collectedfrom, DataInfo dataInfo) { return Arrays diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json index 4d85cf26b..b3d1d742b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/input_actionset_parameter.json @@ -28,13 +28,19 @@ "paramLongName": "dataciteInputPath", "paramDescription": "the path to get the input data from Datacite", "paramRequired": true - },{ + }, + { "paramName": "wip", "paramLongName": "webCrawlInputPath", "paramDescription": "the path to get the input data from Web Crawl", "paramRequired": true -} -, + }, + { + "paramName": "pub", + "paramLongName": "publisherInputPath", + "paramDescription": "the path to get the input data from publishers", + "paramRequired": true + }, { "paramName": "o", "paramLongName": "outputPath", diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index 4bd6bcc09..7c45234f6 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -332,7 +332,7 @@ case object Crossref2Oaf { implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats //MAPPING Crossref DOI into PID - val doi: String = DoiCleaningRule.normalizeDoi((json \ "DOI").extract[String]) + val doi: String = DoiCleaningRule.clean((json \ "DOI").extract[String]) result.setPid( List( structuredProperty( @@ -673,7 +673,7 @@ case object Crossref2Oaf { val doi = input.getString(0) val rorId = input.getString(1) - val pubId = s"50|${PidType.doi.toString.padTo(12, "_")}::${DoiCleaningRule.normalizeDoi(doi)}" + val pubId = s"50|${PidType.doi.toString.padTo(12, "_")}::${DoiCleaningRule.clean(doi)}" val affId = GenerateRorActionSetJob.calculateOpenaireId(rorId) val r: Relation = new Relation diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index ac9977a7e..179cbecb5 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -28,8 +28,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; public class PrepareAffiliationRelationsTest { @@ -39,8 +39,7 @@ public class PrepareAffiliationRelationsTest { private static Path workingDir; private static final String ID_PREFIX = "50|doi_________::"; - private static final Logger log = LoggerFactory - .getLogger(PrepareAffiliationRelationsTest.class); + private static final Logger log = LoggerFactory.getLogger(PrepareAffiliationRelationsTest.class); @BeforeAll public static void beforeAll() throws IOException { @@ -74,26 +73,34 @@ public class PrepareAffiliationRelationsTest { @Test void testMatch() throws Exception { - String crossrefAffiliationRelationPath = getClass() + String crossrefAffiliationRelationPathNew = getClass() .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json") .getPath(); + String crossrefAffiliationRelationPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror_old.json") + .getPath(); + String publisherAffiliationRelationPath = getClass() .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/publishers") .getPath(); + String publisherAffiliationRelationOldPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/publichers_old") + .getPath(); + String outputPath = workingDir.toString() + "/actionSet"; PrepareAffiliationRelations .main( new String[] { "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-crossrefInputPath", crossrefAffiliationRelationPath, + "-crossrefInputPath", crossrefAffiliationRelationPathNew, "-pubmedInputPath", crossrefAffiliationRelationPath, - "-openapcInputPath", crossrefAffiliationRelationPath, + "-openapcInputPath", crossrefAffiliationRelationPathNew, "-dataciteInputPath", crossrefAffiliationRelationPath, "-webCrawlInputPath", crossrefAffiliationRelationPath, - "-publisherInputPath", publisherAffiliationRelationPath, + "-publisherInputPath", publisherAffiliationRelationOldPath, "-outputPath", outputPath }); @@ -104,13 +111,8 @@ public class PrepareAffiliationRelationsTest { .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) .map(aa -> ((Relation) aa.getPayload())); -// for (Relation r : tmp.collect()) { -// System.out.println( -// r.getSource() + "\t" + r.getTarget() + "\t" + r.getRelType() + "\t" + r.getRelClass() + "\t" + r.getSubRelType() + "\t" + r.getValidationDate() + "\t" + r.getDataInfo().getTrust() + "\t" + r.getDataInfo().getInferred() -// ); -// } // count the number of relations - assertEquals(138, tmp.count()); + assertEquals(150, tmp.count());// 18 + 24 *3 + 30 * 2 = Dataset dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); dataset.createOrReplaceTempView("result"); @@ -121,7 +123,7 @@ public class PrepareAffiliationRelationsTest { // verify that we have equal number of bi-directional relations Assertions .assertEquals( - 69, execVerification + 75, execVerification .filter( "relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'") .collectAsList() @@ -129,21 +131,21 @@ public class PrepareAffiliationRelationsTest { Assertions .assertEquals( - 69, execVerification + 75, execVerification .filter( "relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'") .collectAsList() .size()); // check confidence value of a specific relation - String sourceDOI = "10.1061/(asce)0733-9399(2002)128:7(759)"; + String sourceDOI = "10.1089/10872910260066679"; final String sourceOpenaireId = ID_PREFIX - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", sourceDOI)); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", sourceDOI)); Assertions .assertEquals( - "0.7071067812", execVerification + "1.0", execVerification .filter( "source='" + sourceOpenaireId + "'") .collectAsList() @@ -151,11 +153,34 @@ public class PrepareAffiliationRelationsTest { .getString(4)); final String publisherid = ID_PREFIX - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s00217-010-1268-9")); - final String rorId = "20|ror_________::" + IdentifierFactory.md5("https://ror.org/03265fv13"); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1089/10872910260066679")); + final String rorId = "20|ror_________::" + IdentifierFactory.md5("https://ror.org/05cf8a891"); Assertions .assertEquals( - 1, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count()); + 2, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count()); + + Assertions + .assertEquals( + 1, execVerification + .filter( + "source = '" + ID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue("doi", "10.1007/s00217-010-1268-9")) + + "' and target = '" + "20|ror_________::" + + IdentifierFactory.md5("https://ror.org/03265fv13") + "'") + .count()); + + Assertions + .assertEquals( + 3, execVerification + .filter( + "source = '" + ID_PREFIX + + IdentifierFactory + .md5(PidCleaner.normalizePidValue("doi", "10.1007/3-540-47984-8_14")) + + "' and target = '" + "20|ror_________::" + + IdentifierFactory.md5("https://ror.org/00a0n9e72") + "'") + .count()); + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateOpenCitationsASTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateOpenCitationsASTest.java index ed80ed5c5..b690b6228 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateOpenCitationsASTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateOpenCitationsASTest.java @@ -31,6 +31,7 @@ import eu.dnetlib.dhp.schema.oaf.Publication; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; public class CreateOpenCitationsASTest { @@ -280,17 +281,17 @@ public class CreateOpenCitationsASTest { @Test void testRelationsSourceTargetCouple() throws Exception { final String doi1 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-015-3684-x")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1007/s10854-015-3684-x")); final String doi2 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1111/j.1551-2916.2008.02408.x")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1111/j.1551-2916.2008.02408.x")); final String doi3 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-014-2114-9")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1007/s10854-014-2114-9")); final String doi4 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/j.ceramint.2013.09.069")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1016/j.ceramint.2013.09.069")); final String doi5 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-009-9913-4")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1007/s10854-009-9913-4")); final String doi6 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/0038-1098(72)90370-5")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1016/0038-1098(72)90370-5")); String inputPath = getClass() .getResource( diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/transformativeagreement/CreateTAActionSetTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/transformativeagreement/CreateTAActionSetTest.java index 0ec34311f..592c6283a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/transformativeagreement/CreateTAActionSetTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/transformativeagreement/CreateTAActionSetTest.java @@ -28,6 +28,7 @@ import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; /** * @author miriam.baglioni @@ -270,17 +271,17 @@ public class CreateTAActionSetTest { @Test void testRelationsSourceTargetCouple() throws Exception { final String doi1 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-015-3684-x")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1007/s10854-015-3684-x")); final String doi2 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1111/j.1551-2916.2008.02408.x")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1111/j.1551-2916.2008.02408.x")); final String doi3 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-014-2114-9")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1007/s10854-014-2114-9")); final String doi4 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/j.ceramint.2013.09.069")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1016/j.ceramint.2013.09.069")); final String doi5 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-009-9913-4")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1007/s10854-009-9913-4")); final String doi6 = "50|doi_________::" - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/0038-1098(72)90370-5")); + + IdentifierFactory.md5(PidCleaner.normalizePidValue("doi", "10.1016/0038-1098(72)90370-5")); String inputPath = getClass() .getResource( diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json index 08dc3f7eb..b5a711694 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json @@ -1,9 +1,10 @@ -{"DOI":"10.1061\/(asce)0733-9399(2002)128:7(759)","Matchings":[{"RORid":"https:\/\/ror.org\/03yxnpp24","Confidence":0.7071067812},{"RORid":"https:\/\/ror.org\/01teme464","Confidence":0.89}]} -{"DOI":"10.1105\/tpc.8.3.343","Matchings":[{"RORid":"https:\/\/ror.org\/02k40bc56","Confidence":0.7071067812}]} -{"DOI":"10.1161\/01.cir.0000013305.01850.37","Matchings":[{"RORid":"https:\/\/ror.org\/00qjgza05","Confidence":1}]} -{"DOI":"10.1142\/s021821650200186x","Matchings":[{"RORid":"https:\/\/ror.org\/035xkbk20","Confidence":1},{"RORid":"https:\/\/ror.org\/05apxxy63","Confidence":1}]} -{"DOI":"10.1061\/(asce)0733-9372(2002)128:7(575)","Matchings":[{"RORid":"https:\/\/ror.org\/04j198w64","Confidence":0.82}]} -{"DOI":"10.1061\/(asce)0733-9372(2002)128:7(588)","Matchings":[{"RORid":"https:\/\/ror.org\/03m8km719","Confidence":0.8660254038},{"RORid":"https:\/\/ror.org\/02aze4h65","Confidence":0.87}]} -{"DOI":"10.1161\/hy0202.103001","Matchings":[{"RORid":"https:\/\/ror.org\/057xtrt18","Confidence":0.7071067812}]} -{"DOI": "10.1080/13669877.2015.1042504", "Matchings": [{"Confidence": 1.0, "RORid": "https://ror.org/03265fv13"}]} -{"DOI": "10.1007/3-540-47984-8_14", "Matchings": [{"Confidence": 1.0, "RORid": "https://ror.org/00a0n9e72"}]} \ No newline at end of file +{"DOI":"10.1021\/ac020069k","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/01f5ytq51","Status":"active","Confidence":1}]} +{"DOI":"10.1161\/01.cir.0000013846.72805.7e","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/02pttbw34","Status":"active","Confidence":1}]} +{"DOI":"10.1161\/hy02t2.102992","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/00qqv6244","Status":"active","Confidence":1},{"PID":"ROR","Value":"https:\/\/ror.org\/00p991c53","Status":"active","Confidence":1}]} +{"DOI":"10.1126\/science.1073633","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/03xez1567","Status":"active","Confidence":1},{"PID":"ROR","Value":"https:\/\/ror.org\/006w34k90","Status":"active","Confidence":1}]} +{"DOI":"10.1089\/10872910260066679","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/05cf8a891","Status":"active","Confidence":1}]} +{"DOI":"10.1108\/02656719610116117","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/03mnm0t94","Status":"active","Confidence":1},{"PID":"ROR","Value":"https:\/\/ror.org\/007tn5k56","Status":"active","Confidence":1}]} +{"DOI":"10.1080\/01443610050111986","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/001x4vz59","Status":"active","Confidence":1},{"PID":"ROR","Value":"https:\/\/ror.org\/01tmqtf75","Status":"active","Confidence":1}]} +{"DOI":"10.1021\/cm020118+","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/02cf1je33","Confidence":1,"Status":"inactive"},{"PID":"ROR","Value":"https:\/\/ror.org\/01hvx5h04","Confidence":1,"Status":"active"}]} +{"DOI":"10.1161\/hc1202.104524","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/040r8fr65","Status":"active","Confidence":1},{"PID":"ROR","Value":"https:\/\/ror.org\/04fctr677","Status":"active","Confidence":1}]} +{"DOI":"10.1021\/ma011134f","Matchings":[{"PID":"ROR","Value":"https:\/\/ror.org\/04tj63d06","Status":"active","Confidence":1}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror_old.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror_old.json new file mode 100644 index 000000000..d7f004deb --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror_old.json @@ -0,0 +1,9 @@ +{"DOI":"10.1061\/(asce)0733-9399(2002)128:7(759)","Matchings":[{"RORid":"https:\/\/ror.org\/03yxnpp24","Confidence":0.7071067812},{"RORid":"https:\/\/ror.org\/01teme464","Confidence":0.89}]} +{"DOI":"10.1105\/tpc.8.3.343","Matchings":[{"RORid":"https:\/\/ror.org\/02k40bc56","Confidence":0.7071067812}]} +{"DOI":"10.1161\/01.cir.0000013305.01850.37","Matchings":[{"RORid":"https:\/\/ror.org\/00qjgza05","Confidence":1}]} +{"DOI":"10.1142\/s021821650200186x","Matchings":[{"RORid":"https:\/\/ror.org\/035xkbk20","Confidence":1},{"RORid":"https:\/\/ror.org\/05apxxy63","Confidence":1}]} +{"DOI":"10.1061\/(asce)0733-9372(2002)128:7(575)","Matchings":[{"RORid":"https:\/\/ror.org\/04j198w64","Confidence":0.82}]} +{"DOI":"10.1061\/(asce)0733-9372(2002)128:7(588)","Matchings":[{"RORid":"https:\/\/ror.org\/03m8km719","Confidence":0.8660254038},{"RORid":"https:\/\/ror.org\/02aze4h65","Confidence":0.87}]} +{"DOI":"10.1161\/hy0202.103001","Matchings":[{"RORid":"https:\/\/ror.org\/057xtrt18","Confidence":0.7071067812}]} +{"DOI": "10.1080/13669877.2015.1042504", "Matchings": [{"Confidence": 1.0, "RORid": "https://ror.org/03265fv13"}]} +{"DOI": "https://doi.org/10.1007/3-540-47984-8_14", "Matchings": [{"Confidence": 1.0, "RORid": "https://ror.org/00a0n9e72"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publichers_old/publisher b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publichers_old/publisher new file mode 100644 index 000000000..851263933 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publichers_old/publisher @@ -0,0 +1,6 @@ +{"DOI": "10.1007/s00217-010-1268-9", "Authors": [{"Name": {"Full": "Martin Zarnkow", "First": null, "Last": null}, "Raw_affiliations": ["TU M\u00fcnchen, Lehrstuhl f\u00fcr Brau- und Getr\u00e4nketechnologie"], "Organization_PIDs": []}, {"Name": {"Full": "Andrea Faltermaier", "First": null, "Last": null}, "Raw_affiliations": ["Lehrstuhl f\u00fcr Brau- und Getr\u00e4nketechnologie"], "Organization_PIDs": []}, {"Name": {"Full": "Werner Back", "First": null, "Last": null}, "Raw_affiliations": ["Lehrstuhl f\u00fcr Technologie der Brauerei I"], "Organization_PIDs": []}, {"Name": {"Full": "Martina Gastl", "First": null, "Last": null}, "Raw_affiliations": ["Lehrstuhl f\u00fcr Brau- und Getr\u00e4nketechnologie"], "Organization_PIDs": []}, {"Name": {"Full": "Elkek K. Arendt", "First": null, "Last": null}, "Raw_affiliations": ["University College Cork"], "Organization_PIDs": [{"RORid": "https://ror.org/03265fv13", "Confidence": 1}]}], "Organizations": [{"RORid": "https://ror.org/03265fv13", "Confidence": 1}]} +{"DOI": "10.1007/BF01154707", "Authors": [{"Name": {"Full": "Buggy, M.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Materials Science and Technology, University of Limerick, Limerick, Ireland"], "Organization_PIDs": [{"RORid": "https://ror.org/00a0n9e72", "Confidence": 1}]}, {"Name": {"Full": "Carew, A.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Materials Science and Technology, University of Limerick, Limerick, Ireland"], "Organization_PIDs": [{"RORid": "https://ror.org/00a0n9e72", "Confidence": 1}]}], "Organizations": [{"RORid": "https://ror.org/00a0n9e72", "Confidence": 1}]} +{"DOI": "10.1007/s10237-017-0974-7", "Authors": [{"Name": {"Full": "Donnacha J. McGrath", "First": null, "Last": null}, "Raw_affiliations": ["Biomechanics Research Centre (BMEC), Biomedical Engineering, College of Engineering and Informatics, NUI Galway, Galway, Ireland"], "Organization_PIDs": [{"RORid": "https://ror.org/03bea9k73", "Confidence": 1}]}, {"Name": {"Full": "Anja Lena Thiebes", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biohybrid and Medical Textiles (BioTex), AME-Helmholtz Institute for Biomedical Engineering, ITA-Institut f\u00fcr Textiltechnik, RWTH Aachen University and at AMIBM Maastricht University, Maastricht, The Netherlands, Aachen, Germany"], "Organization_PIDs": [{"RORid": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"RORid": "https://ror.org/04xfq0f34", "Confidence": 0.87}]}, {"Name": {"Full": "Christian G. Cornelissen", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biohybrid and Medical Textiles (BioTex), AME-Helmholtz Institute for Biomedical Engineering, ITA-Institut f\u00fcr Textiltechnik, RWTH Aachen University and at AMIBM Maastricht University, Maastricht, The Netherlands, Aachen, Germany"], "Organization_PIDs": [{"RORid": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"RORid": "https://ror.org/04xfq0f34", "Confidence": 0.87}]}, {"Name": {"Full": "Barry O\u2019Brien", "First": null, "Last": null}, "Raw_affiliations": ["Department for Internal Medicine \u2013 Section for Pneumology, Medical Faculty, RWTH Aachen University, Aachen, Germany"], "Organization_PIDs": [{"RORid": "https://ror.org/04xfq0f34", "Confidence": 1}]}, {"Name": {"Full": "Stefan Jockenhoevel", "First": null, "Last": null}, "Raw_affiliations": ["Biomechanics Research Centre (BMEC), Biomedical Engineering, College of Engineering and Informatics, NUI Galway, Galway, Ireland"], "Organization_PIDs": [{"RORid": "https://ror.org/03bea9k73", "Confidence": 1}]}, {"Name": {"Full": "Mark Bruzzi", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biohybrid and Medical Textiles (BioTex), AME-Helmholtz Institute for Biomedical Engineering, ITA-Institut f\u00fcr Textiltechnik, RWTH Aachen University and at AMIBM Maastricht University, Maastricht, The Netherlands, Aachen, Germany"], "Organization_PIDs": [{"RORid": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"RORid": "https://ror.org/04xfq0f34", "Confidence": 0.87}]}, {"Name": {"Full": "Peter E. McHugh", "First": null, "Last": null}, "Raw_affiliations": ["Biomechanics Research Centre (BMEC), Biomedical Engineering, College of Engineering and Informatics, NUI Galway, Galway, Ireland"], "Organization_PIDs": [{"RORid": "https://ror.org/03bea9k73", "Confidence": 1}]}], "Organizations": [{"RORid": "https://ror.org/03bea9k73", "Confidence": 1}, {"RORid": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"RORid": "https://ror.org/04xfq0f34", "Confidence": 0.87}, {"RORid": "https://ror.org/04xfq0f34", "Confidence": 1}]} +{"DOI": "10.1007/BF03168973", "Authors": [{"Name": {"Full": "Sheehan, G.", "First": null, "Last": null}, "Raw_affiliations": ["Dept of Infectious Diseases, Mater Misercordiae Hospital, Dublin 7"], "Organization_PIDs": []}, {"Name": {"Full": "Chew, N.", "First": null, "Last": null}, "Raw_affiliations": ["Dept of Infectious Diseases, Mater Misercordiae Hospital, Dublin 7"], "Organization_PIDs": []}], "Organizations": []} +{"DOI": "10.1007/s00338-009-0480-1", "Authors": [{"Name": {"Full": "Gleason, D. F.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biology, Georgia Southern University, Statesboro, USA"], "Organization_PIDs": [{"RORid": "https://ror.org/04agmb972", "Confidence": 1}]}, {"Name": {"Full": "Danilowicz, B. S.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biology, Georgia Southern University, Statesboro, USA"], "Organization_PIDs": [{"RORid": "https://ror.org/04agmb972", "Confidence": 1}]}, {"Name": {"Full": "Nolan, C. J.", "First": null, "Last": null}, "Raw_affiliations": ["School of Biology and Environmental Science, University College Dublin, Dublin 4, Ireland"], "Organization_PIDs": [{"RORid": "https://ror.org/05m7pjf47", "Confidence": 1}]}], "Organizations": [{"RORid": "https://ror.org/04agmb972", "Confidence": 1}, {"RORid": "https://ror.org/05m7pjf47", "Confidence": 1}]} +{"DOI": "10.1007/s10993-010-9187-y", "Authors": [{"Name": {"Full": "Martin Howard", "First": null, "Last": null}, "Raw_affiliations": ["University College Cork"], "Organization_PIDs": [{"RORid": "https://ror.org/03265fv13", "Confidence": 1}]}], "Organizations": [{"RORid": "https://ror.org/03265fv13", "Confidence": 1}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publishers/publisher b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publishers/publisher new file mode 100644 index 000000000..426500e73 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/publishers/publisher @@ -0,0 +1,6 @@ +{"DOI": "10.1007/s00217-010-1268-9", "Authors": [{"Name": {"Full": "Martin Zarnkow", "First": null, "Last": null}, "Raw_affiliations": ["TU M\u00fcnchen, Lehrstuhl f\u00fcr Brau- und Getr\u00e4nketechnologie"], "Organization_PIDs": []}, {"Name": {"Full": "Andrea Faltermaier", "First": null, "Last": null}, "Raw_affiliations": ["Lehrstuhl f\u00fcr Brau- und Getr\u00e4nketechnologie"], "Organization_PIDs": []}, {"Name": {"Full": "Werner Back", "First": null, "Last": null}, "Raw_affiliations": ["Lehrstuhl f\u00fcr Technologie der Brauerei I"], "Organization_PIDs": []}, {"Name": {"Full": "Martina Gastl", "First": null, "Last": null}, "Raw_affiliations": ["Lehrstuhl f\u00fcr Brau- und Getr\u00e4nketechnologie"], "Organization_PIDs": []}, {"Name": {"Full": "Elkek K. Arendt", "First": null, "Last": null}, "Raw_affiliations": ["University College Cork"], "Organization_PIDs": [{"Value": "https://ror.org/03265fv13", "Confidence": 1}]}], "Organizations": [{"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/03265fv13", "Confidence": 1}]} +{"DOI": "10.1007/BF01154707", "Authors": [{"Name": {"Full": "Buggy, M.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Materials Science and Technology, University of Limerick, Limerick, Ireland"], "Organization_PIDs": [{"Value": "https://ror.org/00a0n9e72", "Confidence": 1}]}, {"Name": {"Full": "Carew, A.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Materials Science and Technology, University of Limerick, Limerick, Ireland"], "Organization_PIDs": [{"Value": "https://ror.org/00a0n9e72", "Confidence": 1}]}], "Organizations": [{"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/00a0n9e72", "Confidence": 1}]} +{"DOI": "10.1007/s10237-017-0974-7", "Authors": [{"Name": {"Full": "Donnacha J. McGrath", "First": null, "Last": null}, "Raw_affiliations": ["Biomechanics Research Centre (BMEC), Biomedical Engineering, College of Engineering and Informatics, NUI Galway, Galway, Ireland"], "Organization_PIDs": [{"Value": "https://ror.org/03bea9k73", "Confidence": 1}]}, {"Name": {"Full": "Anja Lena Thiebes", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biohybrid and Medical Textiles (BioTex), AME-Helmholtz Institute for Biomedical Engineering, ITA-Institut f\u00fcr Textiltechnik, RWTH Aachen University and at AMIBM Maastricht University, Maastricht, The Netherlands, Aachen, Germany"], "Organization_PIDs": [{"Value": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"Value": "https://ror.org/04xfq0f34", "Confidence": 0.87}]}, {"Name": {"Full": "Christian G. Cornelissen", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biohybrid and Medical Textiles (BioTex), AME-Helmholtz Institute for Biomedical Engineering, ITA-Institut f\u00fcr Textiltechnik, RWTH Aachen University and at AMIBM Maastricht University, Maastricht, The Netherlands, Aachen, Germany"], "Organization_PIDs": [{"Value": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"Value": "https://ror.org/04xfq0f34", "Confidence": 0.87}]}, {"Name": {"Full": "Barry O\u2019Brien", "First": null, "Last": null}, "Raw_affiliations": ["Department for Internal Medicine \u2013 Section for Pneumology, Medical Faculty, RWTH Aachen University, Aachen, Germany"], "Organization_PIDs": [{"Value": "https://ror.org/04xfq0f34", "Confidence": 1}]}, {"Name": {"Full": "Stefan Jockenhoevel", "First": null, "Last": null}, "Raw_affiliations": ["Biomechanics Research Centre (BMEC), Biomedical Engineering, College of Engineering and Informatics, NUI Galway, Galway, Ireland"], "Organization_PIDs": [{"Value": "https://ror.org/03bea9k73", "Confidence": 1}]}, {"Name": {"Full": "Mark Bruzzi", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biohybrid and Medical Textiles (BioTex), AME-Helmholtz Institute for Biomedical Engineering, ITA-Institut f\u00fcr Textiltechnik, RWTH Aachen University and at AMIBM Maastricht University, Maastricht, The Netherlands, Aachen, Germany"], "Organization_PIDs": [{"Value": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"Value": "https://ror.org/04xfq0f34", "Confidence": 0.87}]}, {"Name": {"Full": "Peter E. McHugh", "First": null, "Last": null}, "Raw_affiliations": ["Biomechanics Research Centre (BMEC), Biomedical Engineering, College of Engineering and Informatics, NUI Galway, Galway, Ireland"], "Organization_PIDs": [{"Value": "https://ror.org/03bea9k73", "Confidence": 1}]}], "Organizations": [{"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/03bea9k73", "Confidence": 1}, {"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/02jz4aj89", "Confidence": 0.82}, {"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/04xfq0f34", "Confidence": 0.87}, {"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/04xfq0f34", "Confidence": 1}]} +{"DOI": "10.1007/BF03168973", "Authors": [{"Name": {"Full": "Sheehan, G.", "First": null, "Last": null}, "Raw_affiliations": ["Dept of Infectious Diseases, Mater Misercordiae Hospital, Dublin 7"], "Organization_PIDs": []}, {"Name": {"Full": "Chew, N.", "First": null, "Last": null}, "Raw_affiliations": ["Dept of Infectious Diseases, Mater Misercordiae Hospital, Dublin 7"], "Organization_PIDs": []}], "Organizations": []} +{"DOI": "10.1007/s00338-009-0480-1", "Authors": [{"Name": {"Full": "Gleason, D. F.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biology, Georgia Southern University, Statesboro, USA"], "Organization_PIDs": [{"Value": "https://ror.org/04agmb972", "Confidence": 1}]}, {"Name": {"Full": "Danilowicz, B. S.", "First": null, "Last": null}, "Raw_affiliations": ["Department of Biology, Georgia Southern University, Statesboro, USA"], "Organization_PIDs": [{"Value": "https://ror.org/04agmb972", "Confidence": 1}]}, {"Name": {"Full": "Nolan, C. J.", "First": null, "Last": null}, "Raw_affiliations": ["School of Biology and Environmental Science, University College Dublin, Dublin 4, Ireland"], "Organization_PIDs": [{"Value": "https://ror.org/05m7pjf47", "Confidence": 1}]}], "Organizations": [{"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/04agmb972", "Confidence": 1}, {"Provenance":"AffRo","PID":"ROR","Status":"active","Value": "https://ror.org/05m7pjf47", "Confidence": 1}]} +{"DOI": "10.1007/s10993-010-9187-y", "Authors": [{"Name": {"Full": "Martin Howard", "First": null, "Last": null}, "Raw_affiliations": ["University College Cork"], "Organization_PIDs": [{"Value": "https://ror.org/03265fv13", "Confidence": 1}]}], "Organizations": [{"PID":"ROR","Status":"active","Value": "https://ror.org/03265fv13", "Confidence": 1}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index c95d5442a..a85f47d99 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -55,29 +55,7 @@ import eu.dnetlib.dhp.common.Constants; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.AccessRight; -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.Context; -import eu.dnetlib.dhp.schema.oaf.Country; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.EoscIfGuidelines; -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.InstanceTypeMapping; -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.OafEntity; -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.Result; -import eu.dnetlib.dhp.schema.oaf.Software; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import eu.dnetlib.dhp.schema.oaf.Subject; +import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; @@ -667,22 +645,25 @@ public abstract class AbstractMdRecordToOafMapper { return this.vocs.getTermAsQualifier(schemeId, classId); } - protected List prepareListStructPropsWithValidQualifier( + protected List prepareListStructPropsWithValidQualifier( final Node node, final String xpath, final String xpathClassId, final String schemeId, final DataInfo info) { - final List res = new ArrayList<>(); + final Set res = new HashSet<>(); for (final Object o : node.selectNodes(xpath)) { final Node n = (Node) o; final String classId = n.valueOf(xpathClassId).trim(); if (this.vocs.termExists(schemeId, classId)) { - res.add(structuredProperty(n.getText(), this.vocs.getTermAsQualifier(schemeId, classId), info)); + res + .add( + HashableStructuredProperty + .newInstance(n.getText(), this.vocs.getTermAsQualifier(schemeId, classId), info)); } } - return res; + return Lists.newArrayList(res); } protected List prepareListStructProps( diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index eee518353..98da48f9e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -25,6 +25,7 @@ import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; public class OafToOafMapper extends AbstractMdRecordToOafMapper { @@ -380,7 +381,7 @@ public class OafToOafMapper extends AbstractMdRecordToOafMapper { return prepareListStructPropsWithValidQualifier( doc, "//oaf:identifier", "@identifierType", DNET_PID_TYPES, info) .stream() - .map(CleaningFunctions::normalizePidValue) + .map(PidCleaner::normalizePidValue) .collect(Collectors.toList()); } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index 57e0d2955..41fc37cb1 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -24,6 +24,7 @@ import eu.dnetlib.dhp.schema.common.RelationInverse; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; public class OdfToOafMapper extends AbstractMdRecordToOafMapper { @@ -504,7 +505,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { @Override protected List prepareResultPids(final Document doc, final DataInfo info) { - final Set res = new HashSet<>(); + final Set res = new HashSet<>(); res .addAll( prepareListStructPropsWithValidQualifier( @@ -524,7 +525,8 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { return res .stream() - .map(CleaningFunctions::normalizePidValue) + .map(PidCleaner::normalizePidValue) + .filter(CleaningFunctions::pidFilter) .collect(Collectors.toList()); } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java index 4ae3f82c2..23de57204 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java @@ -388,7 +388,7 @@ public class CleanGraphSparkJobTest { .collect(Collectors.toList()); assertNotNull(fos_subjects); - assertEquals(2, fos_subjects.size()); + assertEquals(3, fos_subjects.size()); assertTrue( fos_subjects @@ -396,18 +396,10 @@ public class CleanGraphSparkJobTest { .anyMatch( s -> "0101 mathematics".equals(s.getValue()) & ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()) & - "sysimport:crosswalk:datasetarchive" - .equals(s.getDataInfo().getProvenanceaction().getClassid()))); + "subject:fos".equals(s.getDataInfo().getProvenanceaction().getClassid()))); - assertTrue( - fos_subjects - .stream() - .anyMatch( - s -> "0102 computer and information sciences".equals(s.getValue()) & - ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()))); - - verify_keyword(p, "In Situ Hybridization"); - verify_keyword(p, "Avicennia"); + verify_keyword(p, "FOS: Mathematics"); + verify_keyword(p, "FOS: Computer and information sciences"); } @Test diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/GraphCleaningFunctionsTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/GraphCleaningFunctionsTest.java index 8d10508a9..cdc052dfa 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/GraphCleaningFunctionsTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/GraphCleaningFunctionsTest.java @@ -266,7 +266,7 @@ public class GraphCleaningFunctionsTest { .collect(Collectors.toList()); assertNotNull(fos_subjects); - assertEquals(2, fos_subjects.size()); + assertEquals(3, fos_subjects.size()); assertTrue( fos_subjects @@ -274,18 +274,18 @@ public class GraphCleaningFunctionsTest { .anyMatch( s -> "0101 mathematics".equals(s.getValue()) & ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()) & - "sysimport:crosswalk:datasetarchive" - .equals(s.getDataInfo().getProvenanceaction().getClassid()))); + "subject:fos".equals(s.getDataInfo().getProvenanceaction().getClassid()))); assertTrue( fos_subjects .stream() .anyMatch( s -> "0102 computer and information sciences".equals(s.getValue()) & - ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()))); + ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()) & + "subject:fos".equals(s.getDataInfo().getProvenanceaction().getClassid()))); - verify_keyword(p_cleaned, "In Situ Hybridization"); - verify_keyword(p_cleaned, "Avicennia"); + verify_keyword(p_cleaned, "FOS: Computer and information sciences"); + verify_keyword(p_cleaned, "FOS: Mathematics"); // TODO add more assertions to verity the cleaned values System.out.println(MAPPER.writeValueAsString(p_cleaned)); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java index 6ec2f1d51..612e1d0b6 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java @@ -44,7 +44,7 @@ class GenerateEntitiesApplicationTest { } @Test - void testMergeResult() throws IOException, DocumentException { + void testMergeResult() throws IOException { Result publication = getResult("oaf_record.xml", Publication.class); Result dataset = getResult("odf_dataset.xml", Dataset.class); Result software = getResult("odf_software.xml", Software.class); @@ -69,15 +69,15 @@ class GenerateEntitiesApplicationTest { verifyMerge(orp, software, Software.class, ModelConstants.SOFTWARE_RESULTTYPE_CLASSID); } - protected void verifyMerge(Result publication, Result dataset, Class clazz, + protected void verifyMerge(Result r1, Result r2, Class clazz, String resultType) { - final Result merge = (Result) MergeUtils.merge(publication, dataset); + final Result merge = MergeUtils.checkedMerge(r1, r2, true); assertTrue(clazz.isAssignableFrom(merge.getClass())); assertEquals(resultType, merge.getResulttype().getClassid()); } protected Result getResult(String xmlFileName, Class clazz) - throws IOException, DocumentException { + throws IOException { final String xml = IOUtils.toString(getClass().getResourceAsStream(xmlFileName)); return new OdfToOafMapper(vocs, false, true) .processMdRecord(xml) diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 4fb8f86f7..228c6308c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -216,7 +216,7 @@ class MappersTest { } @Test - void testPublication_PubMed() throws IOException, DocumentException { + void testPublication_PubMed() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf_record_pubmed.xml"))); @@ -264,8 +264,17 @@ class MappersTest { assertFalse(p.getSubject().isEmpty()); assertFalse(p.getPid().isEmpty()); - assertEquals("PMC1517292", p.getPid().get(0).getValue()); - assertEquals("pmc", p.getPid().get(0).getQualifier().getClassid()); + + assertTrue(p.getPid().stream().anyMatch(pi -> "pmc".equals(pi.getQualifier().getClassid()))); + assertEquals( + "PMC1517292", + p + .getPid() + .stream() + .filter(pi -> "pmc".equals(pi.getQualifier().getClassid())) + .findFirst() + .get() + .getValue()); assertNotNull(p.getInstance()); assertFalse(p.getInstance().isEmpty()); @@ -292,7 +301,7 @@ class MappersTest { } @Test - void testPublicationInvisible() throws IOException, DocumentException { + void testPublicationInvisible() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf_record.xml"))); @@ -307,6 +316,25 @@ class MappersTest { } + @Test + void testPublicationInvisible_BASE() throws IOException { + + final String xml = IOUtils + .toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_record_base.xml"))); + + final List list = new OdfToOafMapper(vocs, true, true).processMdRecord(xml); + + assertFalse(list.isEmpty()); + assertTrue(list.get(0) instanceof Publication); + + final Publication p = (Publication) list.get(0); + + assertTrue(p.getDataInfo().getInvisible()); + + System.out.println(new ObjectMapper().writeValueAsString(p)); + + } + @Test void testOdfFwfEBookLibrary() throws IOException { final String xml = IOUtils @@ -318,7 +346,7 @@ class MappersTest { } @Test - void testDataset() throws IOException, DocumentException { + void testDataset() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_dataset.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -332,19 +360,19 @@ class MappersTest { final Relation r1 = (Relation) list.get(1); final Relation r2 = (Relation) list.get(2); - assertEquals(d.getId(), r1.getSource()); - assertEquals("40|corda_______::e06332dee33bec6c2ba4c98601053229", r1.getTarget()); + assertEquals(d.getId(), r1.getTarget()); + assertEquals("40|corda_______::e06332dee33bec6c2ba4c98601053229", r1.getSource()); assertEquals(ModelConstants.RESULT_PROJECT, r1.getRelType()); assertEquals(ModelConstants.OUTCOME, r1.getSubRelType()); - assertEquals(ModelConstants.IS_PRODUCED_BY, r1.getRelClass()); + assertEquals(ModelConstants.PRODUCES, r1.getRelClass()); assertTrue(r1.getValidated()); assertEquals("2020-01-01", r1.getValidationDate()); - assertEquals(d.getId(), r2.getTarget()); - assertEquals("40|corda_______::e06332dee33bec6c2ba4c98601053229", r2.getSource()); + assertEquals(d.getId(), r2.getSource()); + assertEquals("40|corda_______::e06332dee33bec6c2ba4c98601053229", r2.getTarget()); assertEquals(ModelConstants.RESULT_PROJECT, r2.getRelType()); assertEquals(ModelConstants.OUTCOME, r2.getSubRelType()); - assertEquals(ModelConstants.PRODUCES, r2.getRelClass()); + assertEquals(ModelConstants.IS_PRODUCED_BY, r2.getRelClass()); assertTrue(r2.getValidated()); assertEquals("2020-01-01", r2.getValidationDate()); @@ -450,7 +478,7 @@ class MappersTest { } @Test - void testOdfBielefeld() throws IOException, DocumentException { + void testOdfBielefeld() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_bielefeld.xml"))); @@ -501,7 +529,7 @@ class MappersTest { } @Test - void testOpentrial() throws IOException, DocumentException { + void testOpentrial() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_opentrial.xml"))); @@ -741,7 +769,7 @@ class MappersTest { } @Test - void testSoftware() throws IOException, DocumentException { + void testSoftware() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_software.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -763,22 +791,21 @@ class MappersTest { final Relation r1 = (Relation) list.get(1); final Relation r2 = (Relation) list.get(2); - assertEquals(s.getId(), r1.getSource()); - assertEquals("50|doi_________::b453e7b4b2130ace57ff0c3db470a982", r1.getTarget()); + assertEquals(s.getId(), r1.getTarget()); + assertEquals("50|doi_________::b453e7b4b2130ace57ff0c3db470a982", r1.getSource()); assertEquals(ModelConstants.RESULT_RESULT, r1.getRelType()); assertEquals(ModelConstants.RELATIONSHIP, r1.getSubRelType()); - assertEquals(ModelConstants.IS_REFERENCED_BY, r1.getRelClass()); + assertEquals(ModelConstants.REFERENCES, r1.getRelClass()); - assertEquals(s.getId(), r2.getTarget()); - assertEquals("50|doi_________::b453e7b4b2130ace57ff0c3db470a982", r2.getSource()); + assertEquals(s.getId(), r2.getSource()); + assertEquals("50|doi_________::b453e7b4b2130ace57ff0c3db470a982", r2.getTarget()); assertEquals(ModelConstants.RESULT_RESULT, r2.getRelType()); assertEquals(ModelConstants.RELATIONSHIP, r2.getSubRelType()); - assertEquals(ModelConstants.REFERENCES, r2.getRelClass()); - + assertEquals(ModelConstants.IS_REFERENCED_BY, r2.getRelClass()); } @Test - void testClaimDedup() throws IOException, DocumentException { + void testClaimDedup() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf_claim_dedup.xml"))); final List list = new OafToOafMapper(vocs, false, true).processMdRecord(xml); @@ -792,7 +819,7 @@ class MappersTest { } @Test - void testNakala() throws IOException, DocumentException { + void testNakala() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_nakala.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -820,7 +847,7 @@ class MappersTest { } @Test - void testEnermaps() throws IOException, DocumentException { + void testEnermaps() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("enermaps.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -845,7 +872,7 @@ class MappersTest { } @Test - void testClaimFromCrossref() throws IOException, DocumentException { + void testClaimFromCrossref() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf_claim_crossref.xml"))); final List list = new OafToOafMapper(vocs, false, true).processMdRecord(xml); @@ -862,7 +889,7 @@ class MappersTest { } @Test - void testODFRecord() throws IOException, DocumentException { + void testODFRecord() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_record.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); System.out.println("***************"); @@ -882,7 +909,7 @@ class MappersTest { } @Test - void testTextGrid() throws IOException, DocumentException { + void testTextGrid() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("textgrid.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -916,7 +943,7 @@ class MappersTest { } @Test - void testBologna() throws IOException, DocumentException { + void testBologna() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf-bologna.xml"))); final List list = new OafToOafMapper(vocs, false, true).processMdRecord(xml); @@ -933,7 +960,7 @@ class MappersTest { } @Test - void testJairo() throws IOException, DocumentException { + void testJairo() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf_jairo.xml"))); final List list = new OafToOafMapper(vocs, false, true).processMdRecord(xml); @@ -971,7 +998,7 @@ class MappersTest { } @Test - void testZenodo() throws IOException, DocumentException { + void testZenodo() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_zenodo.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -1016,7 +1043,7 @@ class MappersTest { } @Test - void testOdfFromHdfs() throws IOException, DocumentException { + void testOdfFromHdfs() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_from_hdfs.xml"))); @@ -1065,7 +1092,7 @@ class MappersTest { } @Test - void testXMLEncodedURL() throws IOException, DocumentException { + void testXMLEncodedURL() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("encoded-url.xml"))); final List list = new OafToOafMapper(vocs, false, true).processMdRecord(xml); @@ -1081,7 +1108,7 @@ class MappersTest { } @Test - void testXMLEncodedURL_ODF() throws IOException, DocumentException { + void testXMLEncodedURL_ODF() throws IOException { final String xml = IOUtils .toString(Objects.requireNonNull(getClass().getResourceAsStream("encoded-url_odf.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); @@ -1245,7 +1272,7 @@ class MappersTest { } @Test - void testRiunet() throws IOException, DocumentException { + void testRiunet() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("riunet.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); System.out.println("***************"); @@ -1291,7 +1318,7 @@ class MappersTest { } @Test - void testIRISPub() throws IOException, DocumentException { + void testIRISPub() throws IOException { final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("iris-odf.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); System.out.println("***************"); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/publication/publication.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/publication/publication.json index e2a2b9449..2536934c5 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/publication/publication.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/publication/publication.json @@ -1,4 +1,4 @@ -{"id":"50|CSC_________::2250a70c903c6ac6e4c01438259e9375","author":[{"affiliation":[],"fullname":"Brien, Tom","name":"Tom","pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"ORCID12","classname":"ORCID12","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"0000-0001-9613-6639"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"ORCID12","classname":"ORCID12","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"https://orcid.org/0000-0001-9613-6639"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"orcid","classname":"ORCID12","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"0000-0001-9613-6639"}],"rank":1,"surname":"Brien"},{"affiliation":[],"fullname":"Ade, Peter","name":"Peter","pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"xyz","classname":"XYZ","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"qwerty"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"","schemename":""},"value":"asdasd"}],"rank":2,"surname":"Ade"},{"affiliation":[],"fullname":"Barry, Peter S.","name":"Peter S.","pid":null,"rank":3,"surname":"Barry"},{"affiliation":[],"fullname":"Dunscombe, Chris J.","name":"Chris J.","pid":[],"rank":4,"surname":"Dunscombe"},{"affiliation":[],"fullname":"Leadley, David R.","name":"David R.","pid":[],"rank":5,"surname":"Leadley"},{"affiliation":[],"fullname":"Morozov, Dmitry V.","name":"Dmitry V.","pid":[],"rank":6,"surname":"Morozov"},{"affiliation":[],"fullname":"Myronov, Maksym","name":"Maksym","pid":[],"rank":7,"surname":"Myronov"},{"affiliation":[],"fullname":"Parker, Evan","name":"Evan","pid":[],"rank":8,"surname":"Parker"},{"affiliation":[],"fullname":"Prest, Martin J.","name":"Martin J.","pid":[],"rank":9,"surname":"Prest"},{"affiliation":[],"fullname":"Prunnila, Mika","name":"Mika","pid":[],"rank":10,"surname":"Prunnila"},{"affiliation":[],"fullname":"Sudiwala, Rashmi V.","name":"Rashmi V.","pid":[],"rank":11,"surname":"Sudiwala"},{"affiliation":[],"fullname":"Whall, Terry E.","name":"Terry E.","pid":[],"rank":12,"surname":"Whall"},{"affiliation":[],"fullname":" - ","name":"","pid":[],"rank":13,"surname":""},{"affiliation":[],"fullname":" :none","name":"","pid":[],"rank":14,"surname":""}],"bestaccessright":null,"publisher":{"value":null},"collectedfrom":[{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"}],"context":[],"contributor":[],"country":[{"classid":"DE","classname":"DE","schemeid":"dnet:countries","schemename":"dnet:countries"}],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"7 oct 1970"},"dateofcollection":"","dateoftransformation":"2020-04-22T12:34:08.009Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"instance":[{"pid":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1008/abcd"}],"alternateIdentifier":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1009/qwerty"}],"accessright":{"classid":"CLOSED","classname":"CLOSED","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-01-01"},"distributionlocation":"","hostedby":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"instancetype":{"classid":"Comment/debate","classname":"Comment/debate","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"url":["http://juuli.fi/Record/0275158616","http://dx.doi.org/10.1007/s109090161569x","http://academia.edu/abcd","http://repo.scoap3.org/api","http://hdl.handle.net/"]},{"pid":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1002/s21010127267xy"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1008/abcd"}],"alternateIdentifier":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1009/qwerty"}],"accessright":{"classid":"CLOSED","classname":"CLOSED","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-01-01"},"distributionlocation":"","hostedby":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"instancetype":{"classid":"Model","classname":"Model","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"url":["http://dx.doi.org/10.1002/s21010127267xy"]},{"pid":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1002/s21010127267xy"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1008/abcd"}],"alternateIdentifier":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1009/qwerty"}],"accessright":{"classid":"CLOSED","classname":"CLOSED","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-01-01"},"distributionlocation":"","hostedby":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"instancetype":{"classid":"xyz","classname":"xyz","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"url":["http://dx.doi.org/10.1002/t32121238378t"]}],"journal":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"edition":"","ep":" 7","iss":"9 March","issnLinking":"","issnOnline":"","issnPrinted":"0022-2291","name":"Journal of Low Temperature Physics - Early Acces","sp":"1 ","vol":""},"language":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1591283286319,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fvirta-jtp.csc.fi%2Fapi%2Fcerif","datestamp":"2019-07-30","harvestDate":"2020-04-22T11:04:38.685Z","identifier":"oai:virta-jtp.csc.fi:Publications/0275158616","metadataNamespace":""}},"originalId":["CSC_________::2250a70c903c6ac6e4c01438259e9375"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":""}],"relevantdate":[],"resourcetype":{"classid":"0001","classname":"0001","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"deletedbyinference":false,"inferred":false,"inferenceprovenance":"","invisible":false,"trust":"0.9"},"qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"In Situ Hybridization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"ta213"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"FOS: Mathematics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"FOS: Computer and information sciences"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"0101 mathematics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"subject:fos","classname":"subject:fos","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"0101 mathematics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"slot antennas"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"strained silicon"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"cold electron bolometers"},{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:actionset","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Avicennia"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"measure noise"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"noise equivalent power"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"optical characterisation"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"optical response"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"photon noise"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"silicon absorbers"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Optical response of strained- and unstrained-silicon cold-electron bolometers test"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"test test 123 test"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"omic"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"「マキャベリ的知性と心の理論の進化論」 リチャード・バーン, アンドリュー・ホワイトゥン 編/藤田和生, 山下博志, 友永雅巳 監訳"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"-"}]} +{"id":"50|CSC_________::2250a70c903c6ac6e4c01438259e9375","author":[{"affiliation":[],"fullname":"Brien, Tom","name":"Tom","pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"ORCID12","classname":"ORCID12","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"0000-0001-9613-6639"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"ORCID12","classname":"ORCID12","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"https://orcid.org/0000-0001-9613-6639"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"orcid","classname":"ORCID12","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"0000-0001-9613-6639"}],"rank":1,"surname":"Brien"},{"affiliation":[],"fullname":"Ade, Peter","name":"Peter","pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"xyz","classname":"XYZ","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"qwerty"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"","schemename":""},"value":"asdasd"}],"rank":2,"surname":"Ade"},{"affiliation":[],"fullname":"Barry, Peter S.","name":"Peter S.","pid":null,"rank":3,"surname":"Barry"},{"affiliation":[],"fullname":"Dunscombe, Chris J.","name":"Chris J.","pid":[],"rank":4,"surname":"Dunscombe"},{"affiliation":[],"fullname":"Leadley, David R.","name":"David R.","pid":[],"rank":5,"surname":"Leadley"},{"affiliation":[],"fullname":"Morozov, Dmitry V.","name":"Dmitry V.","pid":[],"rank":6,"surname":"Morozov"},{"affiliation":[],"fullname":"Myronov, Maksym","name":"Maksym","pid":[],"rank":7,"surname":"Myronov"},{"affiliation":[],"fullname":"Parker, Evan","name":"Evan","pid":[],"rank":8,"surname":"Parker"},{"affiliation":[],"fullname":"Prest, Martin J.","name":"Martin J.","pid":[],"rank":9,"surname":"Prest"},{"affiliation":[],"fullname":"Prunnila, Mika","name":"Mika","pid":[],"rank":10,"surname":"Prunnila"},{"affiliation":[],"fullname":"Sudiwala, Rashmi V.","name":"Rashmi V.","pid":[],"rank":11,"surname":"Sudiwala"},{"affiliation":[],"fullname":"Whall, Terry E.","name":"Terry E.","pid":[],"rank":12,"surname":"Whall"},{"affiliation":[],"fullname":" - ","name":"","pid":[],"rank":13,"surname":""},{"affiliation":[],"fullname":" :none","name":"","pid":[],"rank":14,"surname":""}],"bestaccessright":null,"publisher":{"value":null},"collectedfrom":[{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"}],"context":[],"contributor":[],"country":[{"classid":"DE","classname":"DE","schemeid":"dnet:countries","schemename":"dnet:countries"}],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"7 oct 1970"},"dateofcollection":"","dateoftransformation":"2020-04-22T12:34:08.009Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"instance":[{"pid":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1008/abcd"}],"alternateIdentifier":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1009/qwerty"}],"accessright":{"classid":"CLOSED","classname":"CLOSED","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-01-01"},"distributionlocation":"","hostedby":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"instancetype":{"classid":"Comment/debate","classname":"Comment/debate","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"url":["http://juuli.fi/Record/0275158616","http://dx.doi.org/10.1007/s109090161569x","http://academia.edu/abcd","http://repo.scoap3.org/api","http://hdl.handle.net/"]},{"pid":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1002/s21010127267xy"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1008/abcd"}],"alternateIdentifier":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1009/qwerty"}],"accessright":{"classid":"CLOSED","classname":"CLOSED","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-01-01"},"distributionlocation":"","hostedby":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"instancetype":{"classid":"Model","classname":"Model","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"url":["http://dx.doi.org/10.1002/s21010127267xy"]},{"pid":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1002/s21010127267xy"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1008/abcd"}],"alternateIdentifier":[{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":null,"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1009/qwerty"}],"accessright":{"classid":"CLOSED","classname":"CLOSED","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-01-01"},"distributionlocation":"","hostedby":{"key":"10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747","value":"VIRTA"},"instancetype":{"classid":"xyz","classname":"xyz","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"url":["http://dx.doi.org/10.1002/t32121238378t"]}],"journal":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"edition":"","ep":" 7","iss":"9 March","issnLinking":"","issnOnline":"","issnPrinted":"0022-2291","name":"Journal of Low Temperature Physics - Early Acces","sp":"1 ","vol":""},"language":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1591283286319,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fvirta-jtp.csc.fi%2Fapi%2Fcerif","datestamp":"2019-07-30","harvestDate":"2020-04-22T11:04:38.685Z","identifier":"oai:virta-jtp.csc.fi:Publications/0275158616","metadataNamespace":""}},"originalId":["CSC_________::2250a70c903c6ac6e4c01438259e9375"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s109090161569x"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":""}],"relevantdate":[],"resourcetype":{"classid":"0001","classname":"0001","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"deletedbyinference":false,"inferred":false,"inferenceprovenance":"","invisible":false,"trust":"0.9"},"qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"In Situ Hybridization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"ta213"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"FOS: Mathematics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"FOS: Computer and information sciences"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"0101 mathematics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"subject:fos","classname":"subject:fos","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"0101 mathematics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"slot antennas"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"strained silicon"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"cold electron bolometers"},{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:actionset","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Avicennia"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"measure noise"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"noise equivalent power"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"optical characterisation"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"optical response"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"photon noise"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"silicon absorbers"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Optical response of strained- and unstrained-silicon cold-electron bolometers test"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"test test 123 test"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"omic"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"「マキャベリ的知性と心の理論の進化論」 リチャード・バーン, アンドリュー・ホワイトゥン 編/藤田和生, 山下博志, 友永雅巳 監訳"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"-"}]} {"id":"50|doi_________::b0baa0eb88a5788f0b8815560d2a32f2","context": [], "dataInfo": {"invisible": false, "trust": "0.9", "provenanceaction": {"classid": "sysimport:actionset", "classname": "sysimport:actionset", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "inferred": false, "deletedbyinference": false}, "resourcetype": {"classid": "0001", "classname": "Article", "schemeid": "dnet:publication_resource", "schemename": "dnet:publication_resource"}, "pid": [{"qualifier": {"classid": "doi", "classname": "doi", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.1097/00132586-197308000-00003"}], "contributor": [], "bestaccessright": {"classid": "UNKNOWN", "classname": "not available", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "relevantdate": [{"qualifier": {"classid": "created", "classname": "created", "schemeid": "dnet:dataCite_date", "schemename": "dnet:dataCite_date"}, "value": "2006-11-06T11:36:37Z"}], "collectedfrom": [{"key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "value": "Crossref"}], "subject": [], "lastupdatetimestamp": 1620353302565, "author": [{"fullname": "N. S. AGRUSS", "surname": "AGRUSS", "name": "N. S.", "rank": 1}, {"fullname": "E. Y. ROSIN", "surname": "ROSIN", "name": "E. Y.", "rank": 2}, {"fullname": "R. J. ADOLPH", "surname": "ADOLPH", "name": "R. J.", "rank": 3}, {"fullname": "N. O. FOWLER", "surname": "FOWLER", "name": "N. O.", "rank": 4}], "instance": [{"hostedby": {"key": "10|issn___print::b8cee613d4f898f8c03956d57ea69be2", "value": "Survey of Anesthesiology"}, "url": ["https://doi.org/10.1097/00132586-197308000-00003"], "pid": [{"qualifier": {"classid": "doi", "classname": "doi", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.1097/00132586-197308000-00003"}], "dateofacceptance": {"value": "2006-11-06T11:36:37Z"}, "collectedfrom": {"key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "value": "Crossref"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemeid": "dnet:publication_resource", "schemename": "dnet:publication_resource"}}], "dateofcollection": "2021-05-07T02:08:22Z", "fulltext": [], "description": [], "format": [], "journal": {"issnPrinted": "0039-6206", "vol": "17", "sp": "304", "name": "Survey of Anesthesiology"}, "measures": [], "coverage": [], "externalReference": [], "publisher": {"value": "Ovid Technologies (Wolters Kluwer Health)"}, "resulttype": {"classid": "publication", "classname": "publication", "schemeid": "dnet:result_typologies", "schemename": "dnet:result_typologies"}, "country": [], "extraInfo": [], "originalId": ["10.1097/00132586-197308000-00003", "50|doiboost____::b0baa0eb88a5788f0b8815560d2a32f2"], "source": [{"value": "Crossref"}], "dateofacceptance": {"value": "2006-11-06T11:36:37Z"}, "title": [{"qualifier": {"classid": "main title", "classname": "main title", "schemeid": "dnet:dataCite_title", "schemename": "dnet:dataCite_title"}, "value": "SIGNIFICANCE OF CHRONIC SINUS BRADYCARDIA IN ELDERLY PEOPLE"}]} {"id":"50|doi_________::4972b0ca81b96b225aed8038bb965656","context": [], "dataInfo": {"invisible": false, "trust": "0.9", "provenanceaction": {"classid": "sysimport:actionset", "classname": "sysimport:actionset", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions"}, "inferred": false, "deletedbyinference": false}, "resourcetype": {"classid": "0001", "classname": "Article", "schemeid": "dnet:publication_resource", "schemename": "dnet:publication_resource"}, "pid": [{"qualifier": {"classid": "doi", "classname": "doi", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.2143/tvg.62.1.5002364"}], "contributor": [], "bestaccessright": {"classid": "UNKNOWN", "classname": "not available", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "relevantdate": [{"qualifier": {"classid": "created", "classname": "created", "schemeid": "dnet:dataCite_date", "schemename": "dnet:dataCite_date"}, "value": "2007-08-20T08:35:04Z"}, {"qualifier": {"classid": "published-online", "classname": "published-online", "schemeid": "dnet:dataCite_date", "schemename": "dnet:dataCite_date"}, "value": "2006-01-01"}], "collectedfrom": [{"key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "value": "Crossref"}], "subject": [{"qualifier": {"classid": "keywords", "classname": "keywords", "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies"}, "value": "General Medicine"}], "lastupdatetimestamp": 1620381522840, "author": [{"fullname": "null VERHAMME P", "surname": "VERHAMME P", "rank": 1}], "instance": [{"hostedby": {"key": "10|issn__online::7ec728ad1ac65c60cd563a5137111125", "value": "Tijdschrift voor Geneeskunde"}, "url": ["https://doi.org/10.2143/tvg.62.1.5002364"], "pid": [{"qualifier": {"classid": "doi", "classname": "doi", "schemeid": "dnet:pid_types", "schemename": "dnet:pid_types"}, "value": "10.2143/tvg.62.1.5002364"}], "dateofacceptance": {"value": "2006-01-01"}, "collectedfrom": {"key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "value": "Crossref"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemeid": "dnet:access_modes", "schemename": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemeid": "dnet:publication_resource", "schemename": "dnet:publication_resource"}}], "dateofcollection": "2021-05-07T09:58:42Z", "fulltext": [], "description": [], "format": [], "journal": {"vol": "62", "sp": "55", "issnOnline": "0371-683X", "ep": "61", "name": "Tijdschrift voor Geneeskunde"}, "measures": [], "coverage": [], "externalReference": [], "publisher": {"value": "Peeters Publishers"}, "resulttype": {"classid": "publication", "classname": "publication", "schemeid": "dnet:result_typologies", "schemename": "dnet:result_typologies"}, "country": [], "extraInfo": [], "originalId": ["10.2143/tvg.62.1.5002364", "50|doiboost____::4972b0ca81b96b225aed8038bb965656"], "source": [{"value": "Crossref"}], "dateofacceptance": {"value": "2006-01-01"}, "title": [{"qualifier": {"classid": "main title", "classname": "main title", "schemeid": "dnet:dataCite_title", "schemename": "dnet:dataCite_title"}, "value": "Antitrombotica: nieuwe moleculen"}]} {"id":"50|DansKnawCris::0224aae28af558f21768dbc6439a_ctx","author":[],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"dataInfo":null,"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"bulktagging","inferred":true,"invisible":false,"provenanceaction":{"classid":"community:subject","classname":"Bulktagging for Community - Subject","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":null}],"id":"sobigdata::projects::2"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"dateofcollection":"","dateoftransformation":"2020-05-25T16:14:18.452Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Lit.opg., bijl."}],"embargoenddate":null,"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":{"dataInfo":null,"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"distributionlocation":"","hostedby":{"dataInfo":null,"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0017","classname":"Report","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"license":null,"processingchargeamount":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":true,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"8250"},"processingchargecurrency":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":true,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"EUR"},"refereed":null,"url":null}],"language":{"classid":"nl","classname":"nl","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1591282676557,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2019-12-01T07:51:24Z","harvestDate":"2020-05-25T11:33:13.427Z","identifier":"oai:services.nod.dans.knaw.nl:Publications/rce:document:550013110","metadataNamespace":""}},"originalId":["DansKnawCris::0224aae28af558f21768dbc6439c7a95"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"http://cultureelerfgoed.adlibsoft.com/dispatcher.aspx?action=search&database=ChoiceRapporten&search=priref=550013110"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"http://cultureelerfgoed.adlibsoft.com/dispatcher.aspx?action=search&database=ChoiceRapporten&search=priref=550013110"}],"publisher":null,"relevantdate":[],"resourcetype":{"classid":"0017","classname":"0017","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"archeologie"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"prospectie"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"Archaeology"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Gcube veldonderzoek d.m.v. boringen (karterende fase) : Raadhuisstraat te Dirkshorn, gemeente Harenkarspel"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"sysimport:crosswalk:datasetarchive","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Synthegra Archeologie Rapportenreeks P0502381"}],"journal":null} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json index 8ef642dd3..bf43016ba 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json @@ -794,28 +794,6 @@ }, "value": "FOS: Computer and information sciences" }, - { - "dataInfo": { - "deletedbyinference": false, - "inferenceprovenance": "", - "inferred": false, - "invisible": false, - "provenanceaction": { - "classid": "sysimport:crosswalk:datasetarchive", - "classname": "sysimport:crosswalk:datasetarchive", - "schemeid": "dnet:provenanceActions", - "schemename": "dnet:provenanceActions" - }, - "trust": "0.9" - }, - "qualifier": { - "classid": "keyword", - "classname": "keyword", - "schemeid": "dnet:subject_classification_typologies", - "schemename": "dnet:subject_classification_typologies" - }, - "value": "0101 mathematics" - }, { "dataInfo": { "deletedbyinference": false, @@ -831,8 +809,8 @@ "trust": "0.9" }, "qualifier": { - "classid": "keyword", - "classname": "keyword", + "classid": "FOS", + "classname": "Fields of Science and Technology classification", "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies" }, @@ -910,8 +888,8 @@ "inferred": false, "invisible": false, "provenanceaction": { - "classid": "sysimport:actionset", - "classname": "Harvested", + "classid": "subject:fos", + "classname": "subject:fos", "schemeid": "dnet:provenanceActions", "schemename": "dnet:provenanceActions" }, @@ -923,7 +901,7 @@ "schemeid": "dnet:subject_classification_typologies", "schemename": "dnet:subject_classification_typologies" }, - "value": "Avicennia" + "value": "0102 computer and information sciences" }, { "dataInfo": { diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record_base.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record_base.xml new file mode 100644 index 000000000..6ae4a7f80 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record_base.xml @@ -0,0 +1,129 @@ + + +
+ base_oa_____::7ecf1ef502253efffe203ca9a22bb9f1 + ftunivqespace:oai:espace.library.uq.edu.au:UQ:336902 + 2020-12-22T10:30:27Z + 2024-09-10T17:21:36.972Z +
+ + + https://doi.org/10.1016/j.envint.2014.07.004 + + https://espace.library.uq.edu.au/view/UQ:336902 + ftunivqespace:oai:espace.library.uq.edu.au:UQ:336902 + + + Article contribution + + The role of environmental factors in the spatial distribution of Japanese encephalitis in mainland China + + + + Wang, Liya + + + Hu, Wenbiao + + + Soares Magalhaes, Ricardo J. + + + Bi, Peng + + + Ding, Fan + + + Sun, Hailong + + + Li, Shenlong + + + Yin, Wenwu + + + Wei, Lan + + + Liu, Qiyong + + + Haque, Ubydul + + + Sun, Yansong + + + Huang, Liuyu + + + Tong, Shilu + + + Clements, Archie C.A. + + + Zhang, Wenyi + + + Li, Chengyi + + + + + Japanese encephalitis (JE) is the most common cause of viral encephalitis and an important public health concern in the Asia-Pacific region, particularly in China where 50% of global cases are notified. To explore the association between environmental factors and human JE cases and identify the high risk areas for JE transmission in China, we used annual notified data on JE cases at the center of administrative township and environmental variables with a pixel resolution of 1. km. ×. 1. km from 2005 to 2011 to construct models using ecological niche modeling (ENM) approaches based on maximum entropy. These models were then validated by overlaying reported human JE case localities from 2006 to 2012 onto each prediction map. ENMs had good discriminatory ability with the area under the curve (AUC) of the receiver operating curve (ROC) of 0.82-0.91, and low extrinsic omission rate of 5.44-7.42%. Resulting maps showed JE being presented extensively throughout southwestern and central China, with local spatial variations in probability influenced by minimum temperatures, human population density, mean temperatures, and elevation, with contribution of 17.94%-38.37%, 15.47%-21.82%, 3.86%-21.22%, and 12.05%-16.02%, respectively. Approximately 60% of JE cases occurred in predicted high risk areas, which covered less than 6% of areas in mainland China. Our findings will help inform optimal geographical allocation of the limited resources available for JE prevention and control in China, find hidden high-risk areas, and increase the effectiveness of public health interventions against JE transmission. + + + Japanese encephalitis + Ecological niche model + MaxEnt + China + 2300 Environmental Science + 950 + + Pergamon Press + 2014 + + eng + + + 0001 + UNKNOWN + 10.1163/qwerty + 0.1163/18763308-90001038 + https://doi.org/10.1016/j.envint.2014.07.004 + https://doi.org/10.1080/09672567.2013.792375 + http://doi.org/10.1080/08673487.2012.812376 + http://dx.doi.org/10.1090/08673487.2012.812376 + https://espace.library.uq.edu.au/view/UQ:336902 + ftunivqespace:oai:espace.library.uq.edu.au:UQ:336902 + + + 2014-12-01 + ror_________::https://ror.org/00rqy9422 + + false + false + 0.89 + + + + +
\ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/organizations_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/organizations_resultset_entry.json index cac6e5a32..204ba6aeb 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/organizations_resultset_entry.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/organizations_resultset_entry.json @@ -130,5 +130,10 @@ "value": [ "Pippo", "Foo" ] + }, + { + "field": "typology", + "type": "string", + "value": "Government" } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/person/person_10.json.gz b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/person/person_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..5bd798a42687776f4ba0bbec72b4bae1f35326e6 GIT binary patch literal 1233 zcmV;?1TOm@iwFqaCk1B!18`+>b8l{6F)%J_b8l_{?O4rj+eQ$+_bCXTTEG9q)7E42P+Uk9XLC?`)THz z*_pk}P2GpKF?~D_&&)h@dHL$~QEp>8ZOvkKk(YUXl)F^d!S%scZJL9|WquaMA&mY^ zi_nwqy&h27bw62cw zz8$(AG|)uj@f*w#Qgv(UCHR~m!U{wSjHgJHM3odx6cI0Blgi9veevvOlkZX!#+^^W zi_NX<2h%zbEum8I)A$G8+&pE8FsBs!L$UKrhsrV`DvMOWf}x6z6H^;cV`J(ku)wPv z!BfDG6pU(FD1=xJu3L=D108-HOdYEj+7B)Mqf3jS%N`B=EVUW9`$I#2WO&2A0J^_e zge`V`KYFzy7MECQiJ9UE#9OgFO=*#VW2y)Q`ViOJ&?ms8F-2JKf^*{^2qqt!(=|NL zL7cq^T>zKPLYtY!XUD1O+h~XW6I$+LPQH45^~L2OOFZx_0Vx7J2U!A@%Xb1^+{h9q zLz|kI*7C%+5aiqQ1f{rAC=gv?#EJKgg&?(J*t#HXwKY%RLUY6l#u2|qo_I*)zl;1p z{?CT|kB74}(}ZS?{ZkX)?w~&-L>Y~}r$`vjwL?-=j5F^MQ;cqk zi~Kvv&$WC~6kV$2dGEAA9E0sLf%kvLc><&M@(3l5orakJJ(0w@$ z_1UAR*;kW83_bX^<{0=F{d!M-V$S3CAd(akDZ#!{m2<){j+szSI_T?FaNC<20%))R zD)Rf>{XMK8S|L9iAp1c6<+V4FQ>;p?NAhb;HxH1RS?}jh+H7j(rinY6ddUcO(BOJ* z8Js2>8s1>3a~ygCM#7HPMj@pOCF%a%X|0eS_`eVQzhCm7wXy3%(`{U9JqCO3D5E5= zIMa+`jj*f~1cPVNFvqn;o|;`{6nM74oQpk+fV;>K5%AeW!1rn7|Ji!|^Ue9`PWl^b zxw64nY^Y_7i=eoxLIusVWoifgg`yN%8z%RT{ww6bye#hq`xdO7ml^zlTkQ Date: Tue, 8 Oct 2024 16:22:54 +0200 Subject: [PATCH 150/239] adopting dhp-schemas:8.0.1 to support Auhtor's rawAffiliationString(s). Improved graph2hive implementation --- .../usagestats/SparkAtomicActionUsageJob.java | 2 +- .../DataciteToOAFTransformation.scala | 3 +-- .../dnetlib/doiboost/mag/MagDataModel.scala | 4 ++-- .../graph/hive/GraphHiveTableImporterJob.java | 15 +++++++-------- .../dhp/oa/graph/raw/OdfToOafMapper.java | 2 +- .../oa/graph/GraphHiveImporterJobTest.java | 12 ++++-------- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 10 +++++----- .../graph/sample/dataset/dataset_10.json.gz | Bin 6736 -> 6744 bytes .../otherresearchproduct_10.json.gz | Bin 8073 -> 8086 bytes .../oa/graph/sample/person/person_10.json.gz | Bin 1233 -> 1235 bytes .../sample/publication/publication_10.json.gz | Bin 5143 -> 5155 bytes .../graph/sample/software/software_10.json.gz | Bin 6410 -> 6421 bytes pom.xml | 2 +- 13 files changed, 22 insertions(+), 28 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java index 1d5b35cff..897153762 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/usagestats/SparkAtomicActionUsageJob.java @@ -112,7 +112,7 @@ public class SparkAtomicActionUsageJob implements Serializable { .joinWith(datasource, resultModel.col("datasourceId").equalTo(datasource.col("id")), "left") .map((MapFunction, UsageStatsResultModel>) t2 -> { UsageStatsResultModel usrm = t2._1(); - if(Optional.ofNullable(t2._2()).isPresent()) + if (Optional.ofNullable(t2._2()).isPresent()) usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue()); else usrm.setDatasourceId(usrm.getDatasourceId() + "||NO_MATCH_FOUND"); diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala index 7603715b9..9f448d48b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala @@ -407,10 +407,9 @@ object DataciteToOAFTransformation { ) } if (c.affiliation.isDefined) - a.setAffiliation( + a.setRawAffiliationString( c.affiliation.get .filter(af => af.nonEmpty) - .map(af => OafMapperUtils.field(af, dataInfo)) .asJava ) a.setRank(idx + 1) diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala index 185381f8f..ba3c92794 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala @@ -313,7 +313,7 @@ case object ConversionUtil { if (f.author.DisplayName.isDefined) a.setFullname(f.author.DisplayName.get) if (f.affiliation != null) - a.setAffiliation(List(asField(f.affiliation)).asJava) + a.setRawAffiliationString(List(f.affiliation).asJava) a.setPid( List( createSP( @@ -386,7 +386,7 @@ case object ConversionUtil { a.setFullname(f.author.DisplayName.get) if (f.affiliation != null) - a.setAffiliation(List(asField(f.affiliation)).asJava) + a.setRawAffiliationString(List(f.affiliation).asJava) a.setPid( List( diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java index 76e1d57a1..73243dbc5 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java @@ -9,10 +9,7 @@ import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -25,8 +22,6 @@ public class GraphHiveTableImporterJob { private static final Logger log = LoggerFactory.getLogger(GraphHiveTableImporterJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -74,7 +69,12 @@ public class GraphHiveTableImporterJob { private static void loadGraphTable(SparkSession spark, String inputPath, String hiveDbName, Class clazz, int numPartitions) { - Dataset dataset = spark.read().textFile(inputPath); + final Encoder clazzEncoder = Encoders.bean(clazz); + + Dataset dataset = spark + .read() + .schema(clazzEncoder.schema()) + .json(inputPath); if (numPartitions > 0) { log.info("repartitioning {} to {} partitions", clazz.getSimpleName(), numPartitions); @@ -82,7 +82,6 @@ public class GraphHiveTableImporterJob { } dataset - .map((MapFunction) s -> OBJECT_MAPPER.readValue(s, clazz), Encoders.bean(clazz)) .write() .mode(SaveMode.Overwrite) .saveAsTable(tableIdentifier(hiveDbName, clazz)); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index 41fc37cb1..ad61304a0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -94,7 +94,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { author.setFullname(String.format("%s, %s", author.getSurname(), author.getName())); } - author.setAffiliation(prepareListFields(n, "./*[local-name()='affiliation']", info)); + author.setRawAffiliationString(prepareListString(n, "./*[local-name()='affiliation']")); author.setPid(preparePids(n, info)); author.setRank(pos++); res.add(author); 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 afaac04ea..e059cbc86 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 @@ -73,14 +73,10 @@ public class GraphHiveImporterJobTest { GraphHiveImporterJob .main( new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-inputPath", - getClass().getResource("/eu/dnetlib/dhp/oa/graph/sample").getPath(), - "-hiveMetastoreUris", - "", - "-hiveDbName", - dbName + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--inputPath", getClass().getResource("/eu/dnetlib/dhp/oa/graph/sample").getPath(), + "--hiveMetastoreUris", "", + "--hiveDbName", dbName }); ModelSupport.oafTypes diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 228c6308c..2cf3ea0c0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -406,15 +406,15 @@ class MappersTest { assertEquals("Baracchini", author.get().getSurname()); assertEquals("Theo", author.get().getName()); - assertEquals(1, author.get().getAffiliation().size()); - final Optional> opAff = author + assertEquals(1, author.get().getRawAffiliationString().size()); + final Optional opAff = author .get() - .getAffiliation() + .getRawAffiliationString() .stream() .findFirst(); assertTrue(opAff.isPresent()); - final Field affiliation = opAff.get(); - assertEquals("ISTI-CNR", affiliation.getValue()); + final String affiliation = opAff.get(); + assertEquals("ISTI-CNR", affiliation); assertFalse(d.getSubject().isEmpty()); assertFalse(d.getInstance().isEmpty()); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/dataset/dataset_10.json.gz b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/dataset/dataset_10.json.gz index 0da3c4071ab705e39e22e33967e39de56f788855..e112002d698a006ee75cfa83e72253d58e8db456 100644 GIT binary patch literal 6744 zcmV-e8mHwSiwFpYJ_Tn017u-zVRL14UokK)YIARH0PS7Pa^p6(-p^B@R5evIRkBP{ zqW*S{?PQ$lPP)1~Q$jff><;Q8CORV0pj3%2fydT!OxwD=JX<+iO%%r4o|Om#)Fu3rZXC6tP8))n3OEq z{1CC2IqB3Ds3){EbmM^PH)mNIq~^YiYt~M=8svRoN%1>}-?9yzz!q zZ{Bn}P;Ooev2O6* zGPCRS2JYBDe*Wb2#gXT@<0ID_^dajH1&#TPGr0|HB=`<``j=}E@#wpJwPZIyvw8Za zYfbks(?#^qY{kr2iITkIIi7Rm^x^NN+n+lA)EoAkiThujGIcI#n$3hv=>8nTso(Fp zLzRR52&pv||A68I<#pZtZ0!4tWufHDQgsU|ub?5FL20+kL*tw&BmpNtE1((s=J^xI zv&i%1pZh0&_CANdjrzFg72##rV^IM=gDmRd4+WHk=teBfje(+|(|!4-p40Okr{|(n z$NtbWS~1POqYU10k3108M^!i@6f|KuMXUS@Ajy_g*$v7#JwAT@`gIRmBbDKTL$!NC z&X2!7xj6QE&M}DbT@)UxeuftLLP*utX3+Kwx?j*ZrZN&or!=AALUhTO@B#*9ngr$7 ztdKWkfro(T_t0sX){IRXe|vp0oAH<{m|x_Qr*qTnyIUf>l*J{FV?jP=GF`t-BlZ(Y zk*(j6=QLUWn%0HM*zdpK&*JxA(k-dB-a_Fiyk;Yr_hFK$x|5R_>>U(ylF~{N{iQ0t zo^(@e|IlK=v?a@mILAI{r+)+3GnVi6ahqRv+p~TTc4i_LbACgAcRiIFiPCvN=Q{$h zEItOC+Y#W!lBPUc>=B@R%l?9^r*Cj{DbgHzCDpdGBDjQ3WNR=u<Mi9Gz!D|eZ*{2 zXcPl@zNB-SM%3caBn~j8mN4hFLql6|urM?U!=;%hSrnR|C*>=PLbE8?SyICg_!Fm3 zE&v8t_OIW=%KDjz*Y9&!RA&~LCV?6DU8m>v$xAzZYIH|C=Q*mTwh46}6P`vKIxj>6 zfFYrPe^{1FnzK2-WxbJ5o)Z1W?MiJu*sTDfb;@)0-FAYy^gBr`#2?b+F6yWf=nH8| zKn}>oGZqnp?=5o%3vcV-RXgc1yP|0xDgO^{Ek||#P_1RZU;sHF`x7-yIws$i`h$oAud^%y=;;>Ni)oJ(O9CqAf~ApeEBe<=6f!g| z2c2O#`!P4)l+#v!4n?Y(M{azIB{ol4#OV_nkRx)<=WhYJv+L5z*KR-!M+ix!^2(vw z19TNbqsTJJ=Zm~blvXgo2lRo)&)G(S_%ly4(~SKsO2eZxIv(_#V~g!Q8s8RZb(z0@cK+3lWM3~~SUFJ{_K2b1^j#vN8+@!= z3Eot|Uw1A$jyKpT`*R*A`|6H;N2om4-<4Lnzxw+Uf)ERlDdbUr2jviTJB5sq-DK={ z5yia)O{19gK%V1kgbN&7%+QWP#v&|vD4lD3rw13+j%p7-s!4(7&eM6gfYFT61X%@T z$GH16EF@NcwIOAwK9WTh5hDX3p}Vb*h!MNuVKqMTpEjS>Y!8*Czap2f63mm4oAK_7<#iw5R4-)aE23q7|nvv z4@W`g4*KL>+5dy|$$K#IMw5Fo@x1=zF`0N;o~D_2o;T`TW8#gDT=&TJU%JDoH<|kW z<1+D{<8t!Z`kh4j+fyp@8r!V=TJgE!_!dJq`CZg)Rx<<$)wxLUtc z^P{X#F3=*(3kfr(mHsUw3$PCm58gwwSDh{xCrg?3ugq{Qr794L6xQVu>vc~ddgKKo zlA+gT{VOR_6rzYY*3qc`N{|JT8a56Z&;%?ENLEvV6!X4AhQlV-lv# zr8E-v3^MqzC=!|?G9YwS^D-G!kmsNpcmvEa84;kWtY>#}9?<9GA34J#&#^jfhtJ3Q zD?Vn!vHpq{_$zQ*s?Gii+ap_lh4oiFpub|`Ohz~QD`w-#U^oZ|G#rHR(~LRc$Q|_i zfjjc4I!b=4zv8a!?etd+@9VGd2aoBl7`DP)F?RL_4nTLs00Rf~o$26_0|(r?9PN^O z@1&d8Zt^|Z%EW*FMdkXo2U+UbDv!v1fiPTvA>UaaiS?0u0z*uFLwExYt9DcDL63%6 z=Lz(~Fj&r1bFrs98U^E-*&)rYZ;@JHngqrhxGv1LVC0A8hd!&OhL!V6CPN-_+iu$8 z)N(rk}c8wW)No9e9JLaN-zNiu*d(q1_2SNnh#4H7&&hIWtf z0X=C$+}H3d&%7m`v&vvh%!l-4dai$y zFVh`)6Ldn3-N_`F_>mj=!{OL>>1-7BL*Ku{mw9LQ7Wy*%-q7#g)0a6KKAtmkBTw_r zO!ro2raPJT$JUu?otf5|`N@qj+rAnkhFjlE<3Rk4M-sGVk!TzV+b^{S1X$KuW<|p% zf^qsSMa=C%)K)YJW}H|Vv<)@QvY=Q>g@xx%O2w0HH?3ZpENYW3^p}PrS+blP)7B<7Ft94H zF+9{uZa-Lz+=&G>Cv;VLw=*W7{AIPU!{4EKCaNADFJ0-OZ zsB(Mj7`+L{PXBnLnqKd&;IP)WYv~YwAf5h)=n(%-k7d!c><5djWk0lq{oq>m!$;Pz zEc@XT#{W+C!(e>lNp6qzXZ=CQCXVkpGs^nDLr2VY2jgfIff_ouvmfrv-aht2f6$w_ z-o4llgP!M(AB+7^$pa2YPPy~9VujPFFrpkh0(NXq6^e?|<8tUQb! zni*Mj@c#B$vSl;0(I^gQGd#PUR**3aO0a%gkIIl979XAb1^2Q3N-`11ZPQw*pykY1 zWmu>;W@U(RS{hwXIr_8xp8oN1Opk&fJj#@#|5d7J8D|?>M#S~Lh%)adG!T-iEg`Ju zjXjuY19xoBq^i-F;1=QfSA8+{`Yp*gZuF^;57#wW=?^2ZZ;>L9Zm~1_Q`8Ggm=%c< zt5lwjI^X%KPt_3RYZ_c&Ox$wDQ=Z$ z3rpwj-E`{ux zT>3dh69e6wfr@IoF`Y%uWWwR)SU|nLuR2Qo&(JVXwB7cRLO-+qH7?6g+uR@pm`)fM zndbHJ6nCdHtAwjMj&VqbA%T%RD3H@>Td(N>9bVt0TpAn z(a_955sFxKw?+dG+4o|+9!gH;HIf;8haoIyVRfAaB`;J&9X*4o0_1#*wX)9R2jrxU zj$G%+8CspTjg$6oddn=DmXr3WV{18SE#ag&t>&cJA<%NtEGO+jIcbyejhwXEVAc=E zo)ZpyCy1PJL__$;8HMm;6wW$#aMJF|-bzl|*cslJljc3zZRnLeO>@#llRdYgZ#Ze* z6#l&a<8snY3$kB=1#Tg0x1pC`D^6K0b;lCO9*017Dv|;>Lv7fP#=W0Usr()~ z_;gMp-6vW&8ij+YIN6zi))HHWM&oTGw1HR}=I|wWSlpt}0u<7SMMBz6saYs4F31x; zSD`%4SU~a8lZ(@n^E)p+vDiF5Hbd7P^xQt#OsXs#?VygSWgan0yQ2?lK$a2KMywBK zgw<`JyNPE*??d@ok@IZHaB8k-2#~W{K9yTaD`MvGFU%ZXd%hKU3CRzvK9u$d(h|;7 zP3jPk0$0RlmtrGps@Oi7j>BS-@p;Pe?jG$E1WRDmiUBE*kbFS{yn#ni)C><&(ROr{ z8)Re@SwS%6J|Nq32=_r@3JmH0Cl8N(^9aguYZIL z5$Ztze33{MF$Ri<8eOh`1=H0mCCQ3Z6Iyh+TpEyamTb016mcb_X{4w@&>{;{=M`Ck zz7nWkKov1ebMo?9#3LoQRD3y-Ln5$#^d3`X5||+p2g{CbCI4wJc-(WJ(W+4a8! zf~ag13Pp|=k8~=iGW5OBOKYxKs_yZlv9h)^ns4Y{7a!c+SpVGv(t^h56ZQI5ryt(liidID+i@NlXh&3%V(uB{t4Wqd+`o z#>sWXB|5w;8jbc>)ieq{V^I-WOxi7im?KQ2zG~rg-E1j+=v^j59u(^MMlY&1+J#I zSUOu5Qql6#r~RRns^~R3@(NTAgu5)}ia$fEA;08M?6wn(1oM>d>IuUZVay%v5ihkQ z6(J#CzY7&bLaDdj8ZkJE=3eAAee38#iV6XhEH|JwJO&jMWeMpG!GWg>9vBu&bqN(! z0d6vp(pt!(^d{{ht02{{HQ{;C|0u8qSyA@#ssa>5`jMABf(6AFnFvEQ;qlXzjxw_o z)X5ZRqOO$mrfqF@322-lHARsS?ha37%oEuP93w8JXdqDTMG7y?i`?wdY$5g;7s>iv zq;ZMK-$b@l8+@*ZiNA(H4ZFkDZt2hPVz3&ZPiZ81)Gcdz%FX~{v=OLUsv=fFL;0cL z%YCg&^HDUhqf2PS6m=rf0b7_bK%nZno0v{2{tq?85$ImD!EafF0nb>5j@ZId9rh(R zl9a}Q@th*%$fTs!zDgR<^-+R@`F4>%wf-&Lw52a-AUTNrya+hRddX&rH3Cxws5;GM z86;-EmNANuvfN0u7{DhVbJtaMb1*B z8XB9RSdB!m#Is z&;l#o6AFgZz6B)-3vq)hDc5*agRLe|-Ky~-kx$nR$v#-pPnwMua%t*k+RwLGRe&yl zMz~nNH<#u?8y1S1G^0@KvPLn<@oGmye^RWhnb1+kOrcmKrNdX9t%4Po2Q0bZsyo2= z?yow8;`^@F?=;g(in*kQr>59bDcyt)tMkh?Y(%j4j7bwMz6+uzxS5He?V$KC3+U=p z)GLAfH0;gRZvn3CS|8d)FlYdcD%DK_Dwe>gr@A4Hv}#oF14e{fsdqtz==wdrIRaF8 zTC5nx(Xzh22ahb|Tb)`8(I<(fr4W6**jfrvODII+R#S-V5NIhxmO}K96rz#u+(;ps zg#qhRr|*rO;jkY@5esHfKMW?G(|1Q0YUWl7(OucwM`m{(iOSgkn ziv^<{VBB=d*rL*GXN4PE@W*f>Yq4m{z=8f#E~54OEQe9u;?g88pVRLdN!e>V#cdP@ zf8shl$1CIcsAx6y7$u4rVnY+Qh86jxkaI@9pvcLJ#0_D-ZU)N|9NP%<;RMHR_$%(9 zwgAbHC2}~Mu}jW`x@QrxY&ZLp0=fghtkA)}K0CoAWth_XQw;C*q6lg?HmdMpmx4T7 z|ArnnCC>?lZ=$46z7oi5qc_F`+cEEYgfC5Z*(Tm-F4cmjq}nppmpCRA??*Gb&`hIj z6DcSWWe72|l7SzzL?0ftr`18iawbEi4jUxQ1_`r4!fcSRPi%m-5mNq03_nXa`G>!&vgb7W5Iw8CPTM# zJK^lk?Cm3*VT`Ep{jRbb^+ulim@%uyt+>i=I4~^VZN`~1IC8z0&S>gSrryXh&Mf2X zzKpY%s)y`GWB>ktn`{3KJwK6Piq0gb%jAn;khqA~6Jfj4&ggS#&M zxHEez9Vip`ekb-vz0qLwhz=AZPty*R(PXa!1&{0xU%LKu(4V@Fb)Z-WiaoObM;lFp>rwj!&1gNoaqUuVWTZ02TUW%BCd{&>t9Q!QD{v5K+k6wQcb zj_iilr`kR860cTSzr)LZsvCA9q3csdFEIR&E_p$d0-ijDgzNX|(PvZ!ObW{zxmR;I up^}3YxLI6wc!n%b0#}B3Lo6+dk24=esiYiPw~R z{kq?Sc8hWi>S6`YSYFU<-JAR2H1xu7I-btF8FU(33cdf9$v67{hO_BxKKFto36mfW zg212okr&OPWV9F#<0+es!vM7<6<=~e(=&L$KYI79{!h?~Oj8yEFbgF!c*IY=Z`lT9 zk!vOrNfkbAKF8-lV;3|nv9piZdix9NiadhW3z`yk^wsI-r%ub8rMeYW7ch_>L2Eb5L(_t3BmpNtE2263>iH9> zvnq=9pNA)Z4nBo{JN0onDC6^Zz>*Ta23hov-<8l7qMI;LSOY~t=U3I62i_p`yn&BW z9f#wqeqbV@TCuVkNm4i8|^rY%tY znV0ICD)1v9`YkvuJDPE5Lwkz_Pr1cdkM{mv%P&=V&eK$qPniIwm>r(E4|{p)L-LP}A9>(K)^FQaqt6nh^3l!H&?G$#ooDJ7>eD+P~f%#96P zG8?|43C}a>U}zJD3$8%-2~8Xft-`Q;mjM7Gk^hJ`?v4l z&H9;4x9iRZ2~hM`rg1Fl9z7z)atf%!3$JPV-xCeO1MZkI4?v4fFZGj|FFDJ zv|vkq!}~@?xgh3=`<>eR;jjaU)+sO8H~R&e(jOFY5dWSbch*FefG?zifE-YZ=PV%> z-y7Bp4&Khet9Q~x6Jz60o1*JYG%!hi;j7>X(#l|!{h7!QU)k>`pp zS4CY&)G)yZjCiKc*-nA@H5a)ZNdQ;MWAJEuqsm5&W5s9zZVWtP29oKZu`Qh+N1G!N zABp66H1Licw)Yr(d!RLC{_@%5FAgO8at+hUiLP*k80JaeWHPxX#=2MFbq)OG;Iiih zql30T0m`glClBF zb9{+afpd$U+EK_<#ueG5cZt;W;G()&?crxNCDGiuSjH=u%^1y)RZuy`!(Zc4VfQ!H z?@IefmefRy9E1dS+nf;-cERI%e&j#yKI_$9MdL7py^4@yhpE{`vY(KRNs#iX$10uH{IE$uypX{%9O5W>GXvg2)@s!f~>Q;xL{>y<0evw^jcS zawP9Tx0}rFNw*7zv-e52GxD@ew+n(v?-Jc^a^(9*e)!TK&x6@K4Bs!^?m50rKHI)k zNOpTlRnZ`sRe#nrGgF%V_Ooh`+HPanIhxs>XlCeMF7!M6r|_B&iy5+kUs3`C6Py}z z`?76r+C7F}3yDx++x$YhxJ9dYJZFW2pj`-@U+Pr6=!~37vBY;@@|uLkfde!u;uGu=g|CBuMft_1T$fk1-?S*4#ZNx>$t*x-BXAGdBKQc7_8a;LP~)` zlnKW^TGd}jvO+GyE-nL_fHwk?H%BB+y>4AhI+yyZ+sMZPN&@n^|&y_hio`5OwoZb z1@1<5IZWX^vI|qVFvSCgDQ4bma&4GmF`bRZqj*H)Q4C)#m={m{(Qp{~laT6T;x~pV zZmZtSFva-3VTy3{KBE-lPDClD-jzuKFiJ7PqyR&2K6=kd0d9O9?UQ@&q}#{t>OI-3 z#DDu)*ZO(@Rhrf6K*-esVNL+cxpScr7aI9+XO`wZ@D3bSZ=$$k9W8UrGjP8!M=tbm zac4SO1>>3B7|k#5j5=W21SS~yJ}k0e!iTpGb23d&E052aih0c4xamr4s~$8`Fug&4 zzZC*GJw5sKQB4kafv-Q1J={I;^$Dyp0s7}MV@mdGUz zOeliqOZ(iMq}z9b(JMTE6(m|wj6~&BDR?=x@i&Iypf!@ObjZjWFFXRRcaA^v{rSCW zE#Vq`cu^zt%2gS33&Tg*_H7}plo)YZGi9a9wr^l~Eil^^RK%zfHpeQ5ST#6!C9JMN z#)xm<>P{3Cz5qQL3}cT| ze>RI|Vd5v@csvb#x|k%xI1FzIVcuH3gCWdtFb;?J3}H^j?=ObAlc)U{rhj7$)1S?U zQy0T@F-#Z3{8*-z{SXb(z+FhDbs+x669rmxNVJZG^Glrp0p4aEp`zvUzqsmFGUe{r zX(xsRv$3lj#)e*IIZzyv!ohPZlj6yKbk@9@tQw0h43ow(TeE^&vPEN%^$AoZpO-pj zosv_T-vF#5Q*_ZJ?oOuIgL$7!0k5v``~}P5qY3bTR-GNbXHXvZy}+u##_$L)x%n6| zQYKc^p2yYg-At5#_Sf}`9exjeW#5%N<&ols{y^c49%N#T`MsiMBve#Krjx5D9Rh7r zUA!)&7i&hw7;5KbO~t_4rJy9Ef1L^cX#De&DJeEV#kschjj`YwK(u40v4o2lpa9j0S-} zeP5)9TAubv4>noGm86H^d^DT~eygO1zk>&YAs?f6(rLuVnGo_~7yq+QIVe_@xpeos zV>CEM!$&-CxbSVqKscofwtXjDaH`$d?Lwcvb|AExB_f5TAM`EZI$Ng;2)65(nh`z+ zMVnB3!UUeT<<3Dl3WI~^4itv7{h;s>vGY68GUx=(8zR_}n$L(~nO8^8W4j`&kJ;b6 zN_HfME*iz%Nes_!W)tKLQxI(5G;=VByCp#Hz=rIHgBX6d&a}zW+*SHpZhJ8zbRnJ4BoJ6B^I8vP$G|On zGC~g;o5~{I{$ehj-o7C@$NfE>;Nh|+YyDxm^*t^GvMdf(f0}WD1@kh~VpYmBspWfL zl=`0Q!-fzpkx><6$3&q_;`x|@>Dj(54*FhauFx&Y=$z@C?I>~qGjFDoP&87to3s=b zJ6%d=ls|AC2Xqf)uy7IcHBB>}VI-#!Q=l+>HZnfwrc9usT7qP61#XoX3(M#RFhm9m zt5UmHRFghthL?4IjVg)#cO@VNKb@C)(LmS_nvG(pC=?`454k9spF&Ala5O(#62w(F z+6w40D5iyjk~e`U0> zn+i~{0p3?&Ohj*-%7%St=aAcBfyly8H(a{DwJH1X6-BUg&lJhnxm0~y%luB!#6b6U zqN2ubtg|SX$~Zh68))|TRZpw`85#zPb~ru+_?hi5Y1M|_-3BSZbmFwk4V{k*+?39Z zN;hQtD<}*;X*C>Ljv;wLJ#$;ZtKPmIcJt?zNUmj~hu7}*1^6887Q09xt3f%KoV6w^ z!?&7(5e=W4I1#ioRw5hCXuA|UfnY!^5niZJ)2w8rza8Y(XKC*N9b-4K&+J4I%T&8t ztAU5~y_lVcl9OeFTLyo_6jrdfzO#ao7dn5AS;5rla6ZIdxtQ?-Qqd+yzIWt}old(( zMf*2{WDZS7Mf=#XbyT#DP|>_jQ_y z)LSN}RLDNO;FhX!(|Pr0O(koT?m6DrgYw2sWme)Is7vzEdeHMJRo{YN7fX^DSLonq z6%MA8VdoM?OX`Rjt&fe^rCn*6x|iY6aEC$%P!I`Aq;gKFT`11Z$P>QQi8~&#h~mvA zXQwBRZ@t;XVe|g68TC=>ruYTL3yCncnK0bF9-jacSrbr=Wan=5bQSnm0rKqGkOn79>(Fsi@dPNcj-Rx=Q%2L9OVe!UDF# ziRmx}eQw`oc(oCEoD#EVr>Q=;J(w5RC%`$~zRmjp0rZh7!$!fTzWf0yB&Y`o@Fh}c zJ`6MuJv!h10;X%oNs5)i@K{W{Tv?C`mhHAjG+!mABGHT>7?Gvz^M|!Ex*LlL*nM1km26}xC8P`U*u(#Bop|o z1w#)5K!ajtgdPz0!0vY0^n$yWCS?wHX&^0EQQ4_mEK|CsyJ* z1((jz7V`+6R%eg0dE9EH_L~~FR=;kN4~@ZAG(_+?Och2(TN?LcxBrp|qP9_J6scW2 zF{Pl(;CsPK8;Y6Gm-W$9TiYXA>{woBzuVoou-yZ)fu=IKsg3%obM$hn0a&tb( z#a!CW@)FCHbS$E_Q-9&+rMnfSZ6K=T1Nr51mkyW?xGA1zE~`tcKs;wQwspZ3#LRV-C?m@%6oK%VYZ7Z`U@{{m}Y|slV_$esOl6oNxd@A0ZDq>a?P<` zx@a3+N`Hib^Mb{q@*7kWRlpFz?vSUDNKs{pxW@XB1@m0ylzd*|OY|N&XYbZ2THWk) zb>gHtSB=TD0-Xcl&da5y&Cq(vuc#A;;{@}*JmrUa!X!nQVaEi*D=jI_LnyXyW6g}v z>TQljOod`77ezzgCYO+AKtLy}eWx9LL8m}jLwU<<;9|uiOJ1pOnxZhq;D2Vg}ujm6SiVw2+gc`!*rwfx)<{*WWZP4aesaQ?> z+Uyh1I73E?<{sRhjLMoPk`y@RR}yF-(C(RlhZbdFw`TSb`-IDE`!+GSq~vchU+aB6 zKfuCY!lZ`oaC6xDGraw)0q9ejD4z7Io}RKt05RGKbWP~|R4`D!EBX4W(Pd~ThR@L_ zG+~-Ck(l%>Oj;mNb^ToiC!OtwTD}Ol7h~{y7Gc4&mSOU4OTg2nRkoE5rf@CW7r(j*u(J)t>SnoC1dn<5JaIy{WU zOeceq=y~v0Fhrzs?NWX%aBI0Df#t+B$8`FyS3RAf>GBXnO(&S-yp%>@O?N`UkkPlI zBx5mdaAnmyUOiyz1ysKtyvW}(eM629w)CSRp~YO;{@L;KHFg!Ci(nA0w(soCcrb>g zW+W{r^t$R%N(#Kc(XyU28EYX;jxpOP_DJjSRd26hP2mAcF1dCGn9coFk5GKx#rCZs zcqzG5)Y8*5X-d#tqOc~dY)3)_d(WA&x#9aDYKGgGn8*&A|GWfOr*m9Mq@|%Z+r9y~ z@=HFnk6_RU29+>F0y>t#tf$?OL0S(g_yY64ZS=FCLv;HNpPT?HT$CGzS+iV-@4@p3 zg-)lAIrLGY>6k+wF1C(2)Dh;;wA0KXHw8N8kYf%#By(sIde<_C7IDOe)Efp^_rr4p< zuCv1ZEcnN8{pzsj%EWJzx5RA~hH&P4123qu_vqX- z&5R_P58@ILc7_c3xl~I=KBGv*O5`<3y{-q#@fy1b^WAxk`|#J4L1O`mAvxskB*s2@ zB=u#AP-TDEpOoMZ0JGAh_WI-mi&RNTo4;Z*uNP(1xRy~T3pcS9=@?Bn_OjZfQ?3)aH zVI=zfIeI#s60Be<*6MI6!CXo(mlDjS1pCM)SeFmwk3{TqJhKnka2(I96FjqVr+H@1 zBRig%c%5W?%gMvd7~pgccjfr+)&}UrT6(4QmaLP+q8&N!;!1P=p-66`BCyPwR3e&qg zAX)|DF}xy~H;}{}S#)r;J1#T`Ia!q%x+{lHhXZ&b`u zJeaQ|){R$1Lh<;bKKoeISHs-#1;()_x@Zn}XHT3;(PY8yTTN=MrcW?TfQRh4Iq{4q zWuH-%(oEB3jUnO9DduJtnypLiB2LnbE|t6wQ6i$}5+72)I~yJ-f7Sav28(yC)MxBh zc>57kdUiMGl+ArSsJN@^o66XS-P(+7Y+ZA>9}iVysWnSEb}`jX(au=*fNp$wsXZVs z@&1(UTfEVyzD6gJrax7#0!s_&lNU5A;g_dSaQjXieL_{llyr2Fdku%X9P09Yr&C84 mX+%97UBuBvK2*90Mg_Y~7jYih(M4YW>Hh(z`8FgaPyqlr-ru7D diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/otherresearchproduct/otherresearchproduct_10.json.gz b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/otherresearchproduct/otherresearchproduct_10.json.gz index 20b6a4dba3f02516796283f817134026a083e7a3..b32268d6aea1d4dcf83691483c04d7531f234831 100644 GIT binary patch delta 7788 zcmV-y9+TmTKbAiTABzYG^F0M;kqA9EB+mjn9vE3wNiywXC7oHt7UdOS?>xXB=DK?j z_F#Ab0}FWfGRNUnPvK|S2iSqY9 zQ!x|Na2ShZRQ7#BeVd|8*5U=FogDt1eXI9dHPmVsiSVs~Sc z$Rte^J5>>s`7wy;jbOi*Mx@Y6sW@>OMbCkVIS&61f23DJjWe1^#0Svzpy_s-?%)ZG zsWljN`_7=#{+ENibfS4|#!An4S!3Aryk^^d;&w*faMT$%gJBsE~`rR zZN|;p!hZx>&T}1`a=iet-G<&?O%IO}K`IXl4QlOkBR+q059&;lWIj4RE;LLkN6r_{ zJUIpmj*-<|YJs{R0x#uQ3XLM*a&*4R$a6cc>v+TCPPgC6xR5Co64xiFn&xAp&$Tl`u?4>3_-b!;`zmEyqRbz6^t7(qf(_Q>95m z#vry#?LSqajHjx>?sF|U%Kz5tM-+1Q-aft!kHyRH@wrL$dR2G^0{j|gM5YI*6!`4y zWIUFUB&j}5w2UX2!Zg+>xTRq54R%XQ(7TwklNr~lB;efhk^rh%blj@oPb%RIb%})E0~<52)R8t1qGCW3^_LTP0FK8g{$O&SvgZW+#s=h zdI5j(?q0#1yvXxLy(V35O~NTu$WE(7tbcx3{g+}gPQ*lmrKqL27gB`T2um%prZN{F zvee*0Y}mY*l}yZ-H3mF#*zZ!o0zO0IYkqyPsApWp%(lNi#I+Zj!$H~Il8N|oy+Cy= z8%1h?Uq#>=#=hWj64Gvj_Dt2?N<|S)7@uevz}!kTXP^tK7cj%cLMD76BIqp<41b29 z0tqvoahA?lpyIE<)YEQl0W0_F#X?BKV#P!NV}q4c%wVlmF}%kH$z}vmy7~$A4fJfT z`2^Js^iRaAAK=dz{+H2!&*w1P=?!+F(gdH*VX;Zz-)~Sa)oc#3oq_U3ih*7tov^)F zy@2Hi>PP7_hur}>pt}hU!7zhlK!3AyU{XdfhyxnXsQEFzi$mKEcF}-GuvfGUXfPII zb!r3lMG_mdU1z*g8v0|a9~;!D1b#J$P$Qt-K!IK|zBboJP#A|9NSKCmG_Dh&S(OX0 zNdkSxMv!Z;CpN%EIg=A*oG;=heCDryoM9^q90KqOi}E#T!b^CN$c3Z=A%8M;p(6Zg z$^T5vfQeWUW-E5WIMDJ`&baN=iW0?x?MW03JZPXoNcabIVa}l;766wtkDwfmB~0x? zfnCC8Q5&LPFXZY6MdMmi6Obj0l=)KG!j@JT#_A{7;dA682-pq)L;xEcpn?S5K<(nb z%-#l|2CEmqU(|pZkJAKIh<^wHvVeL}g_*-1YShhb<_{J&B>ou_V-PXCi^T-Fr^qrU z3MmJobGD3NfV2+In5C-$vWDtVmRE>ICZN{Ig3z#_Kj5i>>ac4|r6pBZjl?oj7+S&; zr;bnB-H9q#Sn zmMYrgt!*lRZZ(}av6F)a5&|5wla~fATR7gX6?k5fb^b{2x-ECJZ#>bnhmTGf72hID z)a!se2Vdsg+%C>-1%Vvcn3EZBlOP8me-?f zZ7r%dgFCc)E$<@S!Slmm-{)fN4MRU{g?_u`wtdfQ`Tn5o2i+aG!_Mk^f;$YGZmZeq zB8t#>+wr>RaR*CW`-$5|f83$fzFFMiV_<#= zN1q0}&0{GdUV8k?2c@f@iu0F$dTq4fcFHxn4Ic1h^#Y2Q-NF<8@_YNJ?AVsSI>@lz z>xt&8zkUUSy;tq=-bP z+FB!I&8uJ6PX zJcr#dNc1~&oFq%8#w?O!QDY4Iu#2c7MYqsn8Y7?^*ErvPNN+1bRF&i3{>M5r%X!IQ zrDR}>m{yU%hf@Ljizz*hbUp*2#4fo`3~_p)E-tr|AsZj#NBBSZnF9I}6;&28a&%DowCWsn2Qn z;qTPFBgQpie?ej)G?a?q(AognWfVb~8K_w3u-kG>N|2z&qYDAsc}Al!*cdd^AhOXr zhP<-_%vh9R3<$Cz(Nu$n3zSQU;8iivrl65pO++j*BB{wJBz8w363UF42}GY#7+Nyr zNKp!}fFBV}sVRaHW8xf`N?89HJ(Y;c85kR)T@HIle+Cq{+bC7Ug-BRj&J0R`!+CD` zOi*iZ9uEYR3~`j$7-t6d`qm^=Bx`6Gc}+>gBPOOZVZ!kYMM-?m>;^k56=s2i+JhUg zjz>pUuGZuns+?YWn%RH+Nz{?v9^tsNsfyUgeF4XX5XN_c090rsFlu(lb-V2#pJC|0ox(SZIf85s`j3MNG?G&}6N z&fr%}+c71GLH_dHe@v7LfBEh|8?1*f?K7&Lf4v|=+fFBcC!PiZj>T~T?~XoVtpQdm z2`SpmoNQoy^F}<^Y2~f>9NTu-1GT0WaDGhi`>cRRRXJLwjA2I^!vAY)O$xpeG>sjh zyl*K4pwUPG$EqtEGlHaJZvpS!vG{2GSurB0AZl*M1*GAyc8*B>u0<=Yh zs8osp$6+){Zkmu3YmX%vmCQI5onB<*f5EWAf)p*+HWG8Q45oBE39=Crm!ma#*4X9S z$1Xcf*Q?#&*TycbC!ogQY7FjeV{qh&-E$1CHa;~5_o0evCk8jT`0kG_0^W1@L#hNHi?waBkOUO;*6z+5Gj&BLkZNV)PAYL zP$LsiuL>wwmGkix* zQG_L+8L$aacqZ`%#@)EMH`ib?4)MYbC>LEm3faK@M+Th9{L5XZ$9|Q2NE4NSmR4v( zQd&KyTNc5WO_seThc`9a>rz=%@C;Y*9QGNvBf$GQgp=Uy4_MZL3AyjLf9&cEL97V^ z0alLaXh*kdwo7JoziS~PHMg)Hh^5Ks4KzB@d}cA7Y(Hv!xLgZvV6fk$Qd@q(l;lZ* zMO2K)Qj=xi%vNX#rEjmX6oij2p6GZFiAE4J-}aB^sB-X?>0UxURw?kv3sXSQweP@! zZom$EizZGgk~@6B{)u(lf2~H#8y;l>h&f1PX`n<7H{f8=O|6=tm?3xOQyHwW^Kc^2 zY2IKnt|dwxL5OV3rWey2t)&Ky#@5lmpYlEc7ghdqXHEaPw_HRY|9`v zA}h$=IE@3#iv|PbGjK-8TloUZ!v7My(#ON&GaykMf$p7w<_ZFeE&N7j zPz#ojX}3;29E@c$e>)uZoHo!|FuNFVMAHUmf?R+>g8ot?rl68Q?_l5zvf%<3J8h>0 z#98M`5-szOBQjjXiua7bQ24#H3g#;flVy#?hyAwWA}wfRH9}7G>YT`7d5i?k0u_sj z{Btoc*)QMynfV-yK|DDQwRHditzf>J44>gGI-k@zffFa9f89b*5l{S?tJ6Wi9Q#-~$BUE-*ReROM+CVMUKJ;3aX|tVT)Sl07H;Q~u}Wm6 zR~a>7^cmbXS765SF9>pzm&?1!Xr_YAu&8TT`C5S<(fvil9P!(QbdI$*EvmylNi{i~ z(Ws$3=6p%^e>gXQS?^LagwibaX#&SCQ+yI&kcqY@P5>O5QUZKy$m^Y4*Q>qGHRM%8 zUi$}mwLXMvXSMOEA+HZoRJ$Orj@xft1bMYQw>9vBt{-?ozdhuh7r+z}BH#nJ*BLk8?^>E3wfQu$(8HoBV~AleXP^C5@{~~SwV!#(pSg4-dA|$9fp~EqT$R*bL%&# z#kIeGFII#qPIDKGqvSSd&20?Xq!T##*K*u9Io=l-Gzm>bExf%G-fbD$Xj--C_Cj=9 z@zTw;e{jT3Ve=5Iews^V*h7uGR30iVUa?Bnfav=u>&pY8w^fi9aSbUPVT7=#VX*6` zhG31lg z&v=sqs$m#C`4l4n3MLjRN{e3R2qNH3A3>!;e`)X2K?~>ui|COxMz*#r_XkvArf7VI zL8+f(q3Dha2CG~j4+fd6v|WVg!v;V`6nT;ZgQ9Hff#?P>Y8w+6F%Yjh^0*ph5;OpP zW*As7CO8xQ4T>FxbUn_fdAAF8N!LLy$TeIX>$>heK*(DHk{ z9s2#=*yqE3&u(;E-lyD|XntiCspNSSe`hqrE8UGGYei062%8_EDwrROa}BJI}D zJr9XcR8y~$X8^k4NPPDtMu(8;tUyAte?3nQgXBm@jecAQ{n+aKp{?BNFEN~z4fDb3 zN;T1|7Z_9nPOKEnUPg+YoSxLqpZ!@UF5~>Ezig|&Y^#rPALKEP7V|wn#?{8BKE{2h zqS|?kYYm4N9^*P**d4ZeEgy5Tx~>=u!&bj1`o7=uc^K?E#_g)UC&##9(}n-`e<*mu zaMT*YvFVDZxL(_Pqp=*ek@L2{YxA}p;w)|U5a*6M-VGn(L@b_R(C>Tc>L+th#yCjj z%kOE56d$!^MGxLo5Qh`cxu0WM`H&8tsF&Z{M{hiq!+ngAhgj!H%-JbVFd11jszd%5 z#vN<~c*v?Ayk>ldM=AvEOp)8Se-Qet26*5vzu$~_UKBsLa)gJ*MylQ!_W2o@@3_)u z@w=LtcUI}{ni19?s0B#(G0L_YMxG?Q-W`@KdfWT(>hJLT$opVIX7#5%`?IB33t_>* zqGw+%;8KWDMY*iMaorC=Zgtb0;>~QzgPG#i3A0EV6_K&4opO65*~@>Ce{~0TaLk7z z+Z403;NOTu*3VkjtF_s z^F`2Zcf?Nb_pa*uUb(;a^OSjyM*e*Z?z=a&o^jYVJA>HV6)lrQc3eqX)O?<$z`om8uN zo#S&ami9Ww=Uz*1tLu3!6Pk?KuK8Fyp5I5gUL|8^874Or0S|@EOc|#$U+8l=8!5zb z6XUiD%Eu|=00aDDf6l^BVApE%a#9pbipmNnu2(cyM8(9GKr#x`QmPoYfINzpjHeU2 z4-1WdN-=$HIzLQM_cY6-^~LcQ>r@Hy;EpsI_p7e zsJ(`zF~*Li5UM84z>>G4DWYiG~WQj)-um^ z*%B1MV0_qW{?$1~TGck|Y7t7te*oH|^hfLy8N+reY_QLNOWocL zL`1R5xA&WI?c$D!)m7D;X7R%Fn8B8EBXFMK1S%SE@H-exL`w5rlmV*Ce3tBRJ+g5b zKUe9EpvPG%R~t?Is8F*;p<{TeNv5fNm@^v&evc;$RL67Ur}Lp+*yxPe3@2(d6aY&$GgDi8bb^a&wP=biOkepV1tVY$7}sQp}_H<*+|`Y zo}&8#II8L0r?*d$e&u7#Vl=0fK}BY7S4_?;k%fBo7&U0)5U!Lj`V$H+yx=ipdvd}?s) zLlxCdaID{UF9gSUzu)(}f#;3gz6%@TZl@RayI!y5cKTj_xCd$MURgY7>kP8ZRRwEwo{c2mqtJXY*wfOcz ze|%d3t9%MFK86Lh>Ioj(WQ}n(RbX%_n6d`7-bdMA9@IkI!6G@7%I#eC%}Sz7g9Qn- z8pBQ;o{$^fLLWvD{#4Do8D-vcYye%rG6RupZ?ACLBE+R(j~j1bT$P-m7Eqds7_kkG z4tR2$0y!DVhE%>1NNb=I7yY@JQy$T)e@KKd3vxH^T6($$TTr~)BrgwSaBO-CI;Pb` z#DZWpbfx1-vB@l|^^E-M`KTTCnL5K%j27vGgFHx|pc4>A9As@vZF8$Klh4-@WTQ-g zIDpVXM!DO5Jrih=Ry6Mc6j24RNfA~QWy|L*K@?IVL}oZ*hd9CuOoGEssIJXXe<(34 zA*Pn3$|c{L7NfJ|sdeu+J3M^?J^%9EpC8(Qs1-n+J;oYG4Ti841|&{oAm`|DM`Q_x z1BY^p$#Is5sL8~9ib)%7%wkMhjBH-A9-G18drJ5`EJLWYDyz;wEXM@fSs(y$8)@+x zi_bPeO|iE;Vl$@Zw8#dYKiMHPe`y;(Dxe4$4zxtsMni6qLeLXN&v>fkON37ILY#XOrP7(2Jo4VmR+k=diRYp)*T$iwXCib<_fARoRkcMC? zK*`u4F_)S-!o-`=xJj_j+@dFAuou8r9Bv_LDAlDH>Yyg z3|NTZUpx_%A{4X&e>5@6X;`(W4nmTc-P2GQ@ktEEVM{vAym^AT@{wjLnrFSeXmCw- zet58G3#hn+$$S`t?ni-|e<~_uQp|;mFmf)W^GP_(RHYyzFvO9{fWvIah{n+(lG%Jy z*fYv3d?uoZ!ke3AFjXWs{5^+#TqOuPvONR-h=X8G!1mJj4cHa!IkFVFs6M_}DuSQ| zn}RK@Q+vKElxt8snroD+M!EJ82Xh-rKH701ToCJsP1PBtkDQ8S;Rc2s zGPyvlTX{Qx`|BE}-7mek%H6|6E4;o3FGhr~yaGI(Xu*?ce_7r*ERlV>Q?hxDkuVS7 zEgXxoD~$yR1i20OAorROC*R-+$0*ADk z;{hH7_N=3xe$6431ws9BqQ8Hm7f5zgox#RNP9nbJy@$ zlOep0x4!UG2a_YM6`Yk2&N)dER71dg_Bu{C$FrfAS0Job3oIJV<5*IhO1*U0+MV^Y zsj}2tfA#=#^6v?77{ygUt$dy-49E&+%(?aFY%1dN3LBALu4!b6LNTXX&dJ6{1g9CF zFXAr&zp+X#H-$F8jT1rzPuN`>l+bMwU-UddYoO6%~8?f;_=2;A}khmsrGY? zzq4WfYC_+F>ltwq7_Se;YW*+tfKKlwBI8k4&I=P96*s+c1H& zj-vnrml~_~)()=?rv_Kw3Aoy+@r@c>t-;lO1XsP@zJsf^@u|Vp4^>pVz}3FnxiGs_ zhkJeyhQnUhAM}S~p9g~;?}u%F*y;0Luq(di&gy#tu2z%sc1B%H7S+EZxZ1y|B*9gB zf8H{CagiD{JHgMOZ!HV yw^r36x`K#m2iXle$h=Jl**#dqQ~h`A8N>6@o;$|sWnk|dWB(6-6scE0^8f()PST|S delta 7760 zcmV-W9>s`5}nujbOi*Mx@Y6sW@>OMbChUIS&61f23DJjWe1^#0Svzpy_s-?%)i@ z)EbPseP_^V|I0yMI?+5fW2I-jtTAkQUbF3*~yG+RrY(~dD$=3EP8G2=oPd?3zJ4kvYd-`i=&b$ z$bV_wu;kA+?T2mSOK3zzY9hsU`*E=pnU8pq^2tW$V7xZEJ_U1@PEujSmj3fbw)b1U zeTEhxjT10}G($}4e+5&K#GcRoQ&2Y<$dF@W-=sXs6t+qp7?E?O!wnM4r#8M1M>)*o9h(dm%-bjj*sHYbtZ`AxjM|yN1n+ zS;@qVS!2K>hy5-UEZ{RVwdU8Ci+aXo%xwGXLtJ~YIUJPTEt!Zf*9%m~vQeZK_;m!X zVeAVYCn4=UXopnYn^Y9xgz<@%0nDvba|XJwdI2+BEM&qLB7)u$!C)9FkTByJXMgF8 z1uFguOg-(*7O+mQUMz$(ELKbeFg92~#SE5M6~lXMkZeW}rK=xN-$2jinom&OK>tL% z`X2s_;eQzo_6N|1Uo~^fCghRR;M;#7bLMk z+jYh}rJ+Bz`k_IMO5j(62sHxQ4HW1l<7;zm1ch;!frM!|N8>sXnpL>~TO!bRYy`Om zJ75D`lruR|#`z*X<1>Ht!wg$l;1GaMSd_0x6JElDL@p#12$87^72!`y{(om`228|? zFk7(`#(|b6a>i|^R+K0nY)_(S;6VcwLc%|w3v&()u>iQFc?9KfEMaOF3hWZLhT0JQ zdLdWeD;n3Dnt&`}q|BGf7Php)Fjhapex4&ALBMtZAOhIn02L(Y25J}gW%f1zHCVj> z{-OrVc$_AvLPQ9V1=NEo%zqsAP@`^cGk>tKA@NU{7=wu6T`VTZJw=u=QAjxuowH>G z1Eh6u#w=Y8kTq0?vb;hxG6A(t7KDZc{Q*x6REJ$#DlMtPY9yAK!q5_)An&3Dsxi$N zvB4>aoqUr@)NF}tV_?U0`cy_!oUT?L`NADm)C*Jy%$^#Fw25Zu*l56j-~!Lc0iSCg~3hB2q!_ayz)-NRucBAb;lXHJJku zmcBH!8o+`%?k3~XB*CMJR_WY6Nt8mJ{NuBK__h|+n?V`cy_R_!yWU!qKO}Zu3}*h?kE3@}P3=It2S@I6Z_* zDif4?#^!3CMzlJzDL2d)B4%(>J4ba2Q-Kw!SviSqqd+GUPv)SaSSQy_cmfsxhPEVq zoTf8qJ5uok0jj0L?ksRm8X(T&t2DtjrM{ryhrd(zju_X7>VJrZ&`>IZ<7WeCmr(>| zW}srB!*0tlDM5l7kIn#W=P8ZCU}MlsgUCid7xK;yFk?}M79d=PL{kkOE>JEZ!cfIT zn}SAaH4(AMh@>W?kk}oCNGLOECJ;wTVQ9&eBSk5^0)9j^rKSigjEQq#Dq;O+^i(3g zW?*cHTRH3@8GlgRIHOb%;UQshIWs5$4kxqaQ$el4$vO~FGQ?40W1Jb-wOf-=k*uL% zN6eaOJvm5NNRG0-4YL8&RIvyQaxmuHRsB(JgX=eZNCs9XwdxYc8 zrYfQs_XTYHLm1x)mM=-}4q5qO%lZJ$r;5#C=Pd&{hkvONvW%gpXP)bR`4A^4es@$V z^JC;1B2_WY5nBPyGHrH_hzckND4`|=%I8-$0ywlZ4QL}W2svgz1n0=EqA3``BUm9Axy!)%=8br+)5=@%IkxSv2Wm|% z;3Syf2UY=(s&ceU8N-e;g#XvpniPB`kQqBddEZh9C8Lo5j#XDSW(1*h8F-wqshmt1 zSU?fcR9sW=1}K3HrVNMz+mr*`$hUKeYaiGu0e_Q@z(s53;DkC)>>r4|p|&S!W*TUM zsvt;_a;;M6*UBrRGb?)l6CItw*?a)2Qq}t0T;%l27^d~VrO23aX3ldBk@+%zF6)*eeTDw%OAI=#%u zgMVR#1u0stZ6xMq8BFPT5@aJLE=OzftWn6fk3x2uu2(y>UmJzA4uBd>tI@Q#ji!-L zbx3hcQ_pM+ufk!@oqPu8yl}g({@$g zUo@?EEi`TDbi2J9Mbmok1WjvgxwEr@rhnb=ot?LgroBE=^Bw|er#y`s3=Xfj=UIJU z=_q>^y!?K>L#%OSOs&SA*0HCDg*UARF7{#J+<=`-S_4k|A-Ao7(?3tica}NitA5zh z%l&J1!X}aOd1SrmQk=8Y5ZXlYXDERhaoR6cxH{tWF%Q1Rvql_#5uau7hpwVV(tj|S z7!i#bhh5sFVPOZ@w+MnHdAyFTNW_=|HN$u0@I=S~ngN>-h369QN!*Q#+ieXd;}EaK zfO63lp^y#Se`LU!%y-;%dhA!Z_cKumXlaEuB&F4Jx@8f3*<{&ka%NMby)KnS1(4Gi|1RBFpFn36n6u!xEgHEOa9c-aaYq4e!#lY;Q^Vo@4CcEMRJD^*gvsuyMNVadBdYj z05J!NEDe;%p#&T(x}a4v6f@+`d@6%Ab{Z}n#ew#dC_2?d;ZD&YNkd%C4MC!cc1(2EHD~Fw)0t{d|l(J)Kdo-os&={+P zeH}wPQ}|zk7w>p@d=4avBhbBb&|Ed})CYGVRu3hl8<9W`Bpnp3??83uYGs zj%eE8OppsONYGzu#1vE#=p784K{j0AVyEr2fH>6Fw zVX~~T_^{u0T%-kUtVTGAUR@A5ERT`ES)gK3k$*1cCHwiiKQo_$F^DI}p|;K)pcTw_ zli@SG>*kX>CvciXB!627DgtVu>5my`xLzI;kd2q~tnL!r(_{qrV0AhOm}4I+=U9+Z z;X2M{^@!j!!lL2?EiOo)f?1ag+rsT!GFFMK^eUq!{5^xx<_gSM{&hfZ@^X1M8O>C% z85U6uD_K26}j zWeT|8AQNp*oWM0Sr3B{IaMnAyR9AbHYdEWhv-S_pYJCWo!)oJG!&x7ssCL0w9k<`Q z49;qKZfoELT|e-GetXD0FMuf|M8F4buRZMTfU|a1-xD}%fZ$YXi11OXHF5_|tJl3U zoYitiU?_X;8-KhUb{Rcy8P4+D!SEuSWr^#ZdEHUl8?^>E3um3c$(8HoBV~AleXP^C z5(X~+S;2qG(pSc_-d9xS9maq=({R+JDfJsv-rCK-7YjiZp1BJ)QF0qm<~GJt(g_^+ zYdP+l9PbN^XoRMs7T(?o@3stWsH<9Zdm*~5FzM!6IDcZNTE@ZZ$GKF7J=C~4<)PBz z6{}>8gT9ZlzB&$iTLozmbdX{NMu>(Q2Ah2fmp2g+aimt6-B+2#hUXJA4@41Crs<>8@f@#pL&|27m*MOIqG%tWIQ({=f#EUWph^MeWu(~2=}GPL*`MX% zDn6h3JGS~ew)*t;L7v`d4d3(ATWx&m)7ytCs-35|)^K>~>8<01-C?`e@-Zu_>x#iJ zZ1sDh@B2NUhrzDX+pg++a(Wv!UHEU0!hawPN39{8macht>$SZ%8kJ!iIdA(bHgDV6 z&C+JiZtke#-SF8>#NsK&=)RY(el!PV{DM@zd`?rO_=GJhdhiRwF`R&^{S?c}*K+Vg zy?kyTy{V`S_c68|0-7f=XQw>DbYRu^4EbXiWv~&dA**`udXX9)sSs2!MV8(|Sbwt` zs)4_Jz8Q|ZD1PvI@y|b1GwkX!FwJqLtKxSx@9nJ8%k`q3KTr#h++&nmHIO?=ZM_#P zd9d#x&)?w(j`zVB%<6A*_Gfdj7J7n%ZO%SBz(oxseR5fUq`Du1-0E;U#T(U>WirK0 z5awt!Dk2+KyVv$evRD74>JIG1n12t4uPNq0#gBVnbKW$R95qVgECp$7@Y_3o^}^uR z_|S9y_E#t{q<^5yQG?+fc4gC(Vq9uW1d8{MV-Zj^2iou2uP%)8@jkL)!G`@XnE|1X zyYS{aG;wO18 z-NCE)MYh$8yZc(L>hF}^i|>@~Jihb$nv3;UNVOmPJ@#XhKV#4R*tNy3{n#I>sCN3X zd#y{u3%3Si(e1X~Zg1dvVTbp79TD=N=Zm1-?uea!>|NFO$B(@k&UbJvM|Ruk4~I7z zUU=|M!V9-|hZpYLtS39(Lf+o2`I;{7*Ng;wJ;(Ozhc&*QANx0IKEGn5 zW-R)e?(1I|*Lar~`+apVzpJ>$cT%n9bt=uhSgq?+ntLt1t={Cdgkv&hyXIrG8vPOBt0 z81Mp{;d;3i2{(^LOc~)|Lrzha>G%e%q4pY<#u&zxqM7n+CSZ0>j0o%4xWRKir?hvo zIR_FL>|&vzEm3{fn16_;2i9(tE#j)!WQmrx2@xW3HcEUJQaKBF3zRJck0@Ot5cP>h z(PL<6%Nbf*EekN1DNsgXQ8x)_urbLBn(>$HZo(s*eDqT|^_y0xYzYcrFjQ+b&FLJ2o@$%iw210!C0I?#sN%B6 zA`C7D=tg~MvP@&nDX4Trnf*q^)zb>=C{}7I3Th0gw@fKBHCAFR(|d~N4HCsvveB5H zVQBd*7Hm=aBY*aZjA1(!HrVICrEc#ABBHS2+xyM9c6rCd>Z)o^v(Vjn_+Cr75jf9q z0u>E7_}L2P`=t3U%2?55+DLY|9%Q%-Wvlc?{NXHPstxRYRH#{_&@nu*B-7MB%$W@X zzsHjWs^f+6)BLC=%EaL7MQq=TbMg{WKPp(fZIW5;tbd0LzDldWJj!i~<6Yo$jo1E( zXTC_zMP}+QvB5>K;kACxD2DrIHfZ*pC+Lv?j%s@M>FrabU-=k57|m%V=K0*lIyaYu zUoVp!WU-^GA<;_%wq{*y=22;KyBKD3y~%_I8ZQ{(OA{-Tp}h0?2@UzgNM6SuekUMd zzjis-_kT8OylemPF7kTrIo?$ppBnG_P(`&9@9MYROYtt=@Av&~;CW-W@4|+-+v$b< zuGee1oxax}?!vovRsVazyE;y5(ER}MuGU}|-qm(%yz33%UB3_^y5{AL*9O<>44crzmz8{!1`84*HHKX_JRujmg*A*IY^j>wGRi#T*Z{hBWd;!0 z-d^F5Mfgg?{x#mfuPUiIEm$-aG2#{+o$KTj1#&V34XJ!3NY+3HE&6IRhdUxukqBWH zhy_7t=qATgV3S$I>M8lo^HDqOGj)y`4=suZ zhjfrW!6G1xILO+T+NLmNCZDe*fJO=TZ~&o&gL0Srdcw^jduQGQD545rdm^kT%9bx! zf+(bXgUoQm1#yIzm;{HNP+gk}P-5~w%zxlWl}o-gE&gW7Q|q2>c6fRQJ^%dOpC8%) zq!qB7J;oYG4Tg{u1^`ZEAm`{!N7M+00|#!4w{ez;sL8~9in$SOj9|+kEJLWYDyz;wEXM@NStmbYGSXr(7JF@inqqHxgk;QsX;BM2Te3rF(l&lnz<&%d z9B7G>dWPJhfS@PvNa+$6uF{(w`yE!=W{U$E>Uu6Tp0=#ZAEk;A$7{+J`oL7CxIYXs zm<<}Hzw_n@!YhaCFt6jH$eOGISyQgmTmaaL=XKfPpyhbKI%1_}V%s>~a*Bv&+tdxm z*&bx9tTKYi;<|+8GqJCYmIs*WGk*kA0ZPUWiMiCw5$3my#!Z5B<`x+lgS`O0;&2N| zTLEbqf1RbqOj5Ik=Ab&oAzc8wFsE`z3|NT3UOW+$i4(K}e>5?fXjrwV4#JI?MAJ~% z?nw;BVN2G^ym^AK@{wjLnkSdNY;a9>ad@z33#hn+$$S`t?ni-|Dk@}BOn*6x5OFS~ zi@7e%RHYyzFvO9{K*4Oth_ulnlG%Jy*fUDtdoH4g!ik$@FjXWs{5^+#TqWo@vONR! zh=X8G!1mI&3)mIyIkHT&s6M_}DuR~W7h_*_5rWDopOzde;)8;^&+Zv1W)+O=g}q@ zN9A7=ulgACG*PCcD_zv0;{4^Gt`1vmFz~5>Q!Mm$pT11~je@Sr9)I(PSqBx4+G+={ z7kljWF}ZyWKlZLLxm)OkNbKRbMs?o%%|R8EtKrmrmE0$3OwZ9%N8z86Wi7aOlEbmz0~j<%}M33`vOiSp&7yEIpKJnM%X(s>yY&tW=RkwJAb?sp(IY_aE4jCj5Fc5 zf27NWTBVaIjzKgN)x>{Gq|#ixWSQSdk{X%!1c8_^C9njJY;DJO%|rypPhS$RLDdj$ z<|PnY@_>*&L~lyW%>qN!%T(C_fkRr&@hA=>JOQ1%fJ}-E0xY4)DztiCzQ>mL#eLIF7&mJngUADM%zgTo^wJvb1%YpH zn#OMzjyB;>n+-Ry-yh@yD(;Tjxoh~W$q*>VyIJ_rg2^${3SvrF<(wo5sv%%LdmX2n zBh%2!E3i_l1r}H4aV#l7q+U8~?PmG;R9WgRdw@Clmwy2`jN-zcRzA-Z24saZ=EC}O zJ{56!(Tqqh*EF(3p_tQMfYXf6m+_Y%+*l=7o5CB=^(}>ZjYb3*U*2Bc^|gno z+RFy!3(hXG=J;lDJ$U1L4;Bl%RQtI`;BC9DUtQq!-ykiT5f*XXGKxldIWQLITdxI@ z4V>eh=YJgE$*%U&=OR!%Cl7{+ZJ0n>7f^r^Lyc8?Ylqi{Q^Tb11Saj&NJR~k)-dTl z!lYhr-(k|)_|!1zhbpRFFlpcIT$-S%!#zI;!(p%M5BkHg&x1ja_rtb7?DTmr*cI4v zXZ1aSNvr8>JEJbX4u88BQPK`&*X^FU9%g82-!Mx05YIt` zl48yx0@)7M(ir8BN+2w+a%G(KeMQ3FVIm%Ka2Fn@56df#aB-4D$UX7 z049>xvfMLSZiQKLscKdcDC4m5ELK0@2l|FRMl%n8j2QfZmlWXX>U+%TLaAJ4uw_g2 zD|wP^)yT^Gs0zP4O}2}xcYP50pggX#k@_w*gaUpQ~h`A5W{29 Wp8LbiwFoQJOyU}18`+>b8l{6F)%J_b8l_{?O9818%Ge`=M;lkC4h$L|7UiN z6vR#(8*msPv0)hf8?mk7G&3J1ilUoj6J+5VZ;~tZt6a?jp^fgcxL9I%ga}`E-lqg8&E5s~!+`wArrXB*!eg5M3$w__%n|aflCry>| z@-p|K4zQ2Ch)osR&^S=7Ox=Z}d?{&zzau!0U9|N|WznW{fCICP>tkxh6A(Gn^JKaD@&&COGm2y;roKNLI9bf_#7qOwQ@EEuZjI5D;HG&ZJw0t>v# z5j+KaNWrLa+ zGPJ3QX)RBD3qig;Pf&_0g#ytPMx1!>SO`)phOGxy#i=D)%HRMS;Z>%aZDn>4`vOBPy@|g7{gl z$FBc9yM7n8N2cJ!A;v60vf@rr6TnZF7>R?GQtoKowBV2_C|7%SCf!ATVE>-5pA-a5 z5ia2+WJ1v9ne=^}uW|n*HK$?NaViPUtumgmAeqpHQ!AYno&{Y=%k&QJOQBUEiKKhS z{T1^2a{p`NW~L2w7Uo$q0JQ6O*Z-t!5$cp32f&)LpBk9ncCo+yyNxNE#@2Wk3Eh|D zP@g?|nte4n#L$CpYmR|`(XaRPC+0kE4(_WU?l8lZ4^?vP?GN7oz@EZf&cr!|NABXSsS}PG~LFv)?={ejxtK} ziZjh9)(Fc=K`?kG4Rc&uXJ2)K*<5CNY}1bm-H{-3SaKi{06?xer5 zmMa^K#fDnOxCn~7Dpb%+Tc&op?(@X} xVUcw%E-5%_jysug^!JNDM!^^gzX0%2M8>}eLDr|01mB!7{{i-c9c5b}008t=Y<&O# literal 1233 zcmV;?1TOm@iwFqaCk1B!18`+>b8l{6F)%J_b8l_{?O4rj+eQ$+_bCXTTEG9q)7E42P+Uk9XLC?`)THz z*_pk}P2GpKF?~D_&&)h@dHL$~QEp>8ZOvkKk(YUXl)F^d!S%scZJL9|WquaMA&mY^ zi_nwqy&h27bw62cw zz8$(AG|)uj@f*w#Qgv(UCHR~m!U{wSjHgJHM3odx6cI0Blgi9veevvOlkZX!#+^^W zi_NX<2h%zbEum8I)A$G8+&pE8FsBs!L$UKrhsrV`DvMOWf}x6z6H^;cV`J(ku)wPv z!BfDG6pU(FD1=xJu3L=D108-HOdYEj+7B)Mqf3jS%N`B=EVUW9`$I#2WO&2A0J^_e zge`V`KYFzy7MECQiJ9UE#9OgFO=*#VW2y)Q`ViOJ&?ms8F-2JKf^*{^2qqt!(=|NL zL7cq^T>zKPLYtY!XUD1O+h~XW6I$+LPQH45^~L2OOFZx_0Vx7J2U!A@%Xb1^+{h9q zLz|kI*7C%+5aiqQ1f{rAC=gv?#EJKgg&?(J*t#HXwKY%RLUY6l#u2|qo_I*)zl;1p z{?CT|kB74}(}ZS?{ZkX)?w~&-L>Y~}r$`vjwL?-=j5F^MQ;cqk zi~Kvv&$WC~6kV$2dGEAA9E0sLf%kvLc><&M@(3l5orakJJ(0w@$ z_1UAR*;kW83_bX^<{0=F{d!M-V$S3CAd(akDZ#!{m2<){j+szSI_T?FaNC<20%))R zD)Rf>{XMK8S|L9iAp1c6<+V4FQ>;p?NAhb;HxH1RS?}jh+H7j(rinY6ddUcO(BOJ* z8Js2>8s1>3a~ygCM#7HPMj@pOCF%a%X|0eS_`eVQzhCm7wXy3%(`{U9JqCO3D5E5= zIMa+`jj*f~1cPVNFvqn;o|;`{6nM74oQpk+fV;>K5%AeW!1rn7|Ji!|^Ue9`PWl^b zxw64nY^Y_7i=eoxLIusVWoifgg`yN%8z%RT{ww6bye#hq`xdO7ml^zlTkQY-wX*bZKvHUokK)YIARH0PS7dZri+;zMrQcd@NqK)J_#F*NB3XkQ#WnF_#zU#t3rG|(&{B&YA~v0I5g&Mh zN6|eAZVv>V@rcD-Jiz0LXh^1X8{x>C&?!w`?1wZyD0Fud-BB*Hc;gdNKYr9}pf{e+ zpt@+xrZkGlbXGgSmSbs_g|X2wbttsdWNH6Ny)WZgdqPPNPYw<=_;W1J>5NUcWBZ=2 z+a1?8+E9&#WoR~Jf(v%dLJ|z%1%K6ceEdJ4R(KFl4;p4H_!M68y>?3 zLvyaMePHSb*lb#DET5oXK{{=Vzd#))tXy~B7dC%HBTukdR&^61?%?AB%cGm~113z9 zmV|bK){MGxsci|S}dNf=VpZ^9e$J4I477iD=!f$F>k zFS7WW|s1xxDqG8QbnUVL`-C8dXig?03P?z4D;PD?t=Nrg@+o~zVfrgF&d zDU`D}5iE+qVHG;dNu_d1`CO^-GNteMeI1=q7NBnPqEyU961rF}Cy3`7&6jEHi+gqr z%}GSm7=eQjrS>eR6-tvQa<$gBYT>&OG@1vobYRP_V^(l2{UN)oc^HFROPoVAcW=N8Ep;eQ z_ZutOSQu$w~$Zcou;wXhkYAi+!S1r{JGZs(!f!mTs(;RyJ*q#q`UvK)I+ZBFY|iu?GQV zzawGnCq1(KR{e61{Z~GaARZJu&N}zZc;q;+p<(z}Q`Nq0wbxS@A5GFvB&*A=M|?VE z9vXr7j#Pc_P@i<_5^>FG=%;p^cFYl<3vb!k9nZsS5}1ch$uj(U7Lg+esB zxNa?^qn7*7GMZXT8Nd!--coaw^L#$)pRMHVd`3gmO&)k88|jmHKpQ$-Z6R)?e<6=6l3l$nVmjf4UY*zW+psJEI?-DF>D6wmOa)?yWx)U zxM7m9*Kl03-S9ORn{8}5E_O!PMIiB{g>Cy^n?7fWNk0wN z$o?EAa><7YWeB*JP4#PQFw<+JOloNH8bb@gRoRxWbq&J zxoqN9sb9=rZ@8QU2eSMN?Hl?~S))+y5ln$&`H~veoOS~{rnR_p9tJF2RI$7svhX&U zF3SWe!5Q1p(tGe*wosHLw&l+|4%w(BJv8-M*=+Mx3=$|O?=}|6)Yhbi6K|_OK z18WVR!jMmPaOmu~aZ6@1f)0_Oo|v5!2|MXLLA*uGy^k-_eD<|gJOfKXh!?9L7IYWehwRS+&}7Hbgu^I zXQ+GHzd)DgJ=Fg^LWhI%p{m{2;D)Zi^Bs3D`-kX#ig~4skZ!8#{(Tss^!X=Qi{9}3 zvVXKvfxUo7(CI1W;#V;jt4PtCA}v%8SVhue&AG2r6_%kjY{%BNrYf+}wsNk*%D4(_ z*j7$fB%OyMD->B#nYwzOtgttc6`q4T7WIA0)Tv2rle(^<+;Sr)e7YuB3(YDxAeqcB;i$9CP8~!`*Ib3&^W%QqK^84d6Pn4c3J3KNbbv6 zq+2HH=!y#%VV7#NNGKuSiSWLx#Rb2n0;=?s2(N^*r&nc4MvCprTU^fLn1^-rfy9%C zLR)rOs7nR+Wi3vLe@*M)%I}z8Xv;23)*|KhWi382n1jk4BGI>i*GjuX7T1NkL~>uY z;yss!a>j*AMO>&W5(<4%ba%#LS=|X2chE|8^iLkZ6sAy^Ul;lovc<=)y|npaaObzs zf*%4hFMaRXWw`}Y(L(17JBxqdHV8JhaxWi4WHE4X83p7YKDRJ?1>npOU zigvR<*<>iF?dOn9!@+1YIP0BXU7TJy;Di)8nu!@4EqG0_dApTP@i30I|^KbIk>?zsUt*+zunx;28rn$o& z6KoX#zL^2|7@KB};HTZTg76iDUzxgk9>U)g5es9@H^v=bvmC4wVmpQp&$K(X>H41k zJcPgb@#;YMSa({dy~8Fr1p_y*uAuwsS@O2fy^fofxpPD;$9OI5zQ|9t!25TChl2B{ zzI2Ml%JCVQ(7vGk2e=U>dlUvu^uDAoyDU^@qWgmP7i3CcL?Kt0gW7Vmqm@ucWx?IS z`&Ff(PXb98EC*P+SPQVUkjxdnaG%L1_DvRVRMItCR+LEOYF@a*em7hD9-%`XQgq72Y{IY8 zxV&;zeN0){NVyRoCGX055EM@YpI=WF3+z*F14yDkIHZzc?hS|LGR*R#&9}+_ou4X*C0DiISkX2rIZO%DopKHJYDaAq~Ep-SML~Y=o2)(GsYd;H=YOIH$Pq>@EzmUO%t1Tr=i&$ zEV*Ia)C{|0H@_+%zIs-@!y0-`+xbXq$9}~{^o#sd3yAMWGx-&71aTKK`@-;-R2~l- z*U?Ey)ycA|h0MP2`w_djjV7Fnu#Sf1W4f2M6=M5B@1L05h*0Jh-me1TrOgK~4(sF* z$zhZUjAd9K1a=486#}jh@M<~ubp-r^fGsEEdzhTfgF({lJRg3P-cC2esapN%7l(FyamQ)2P2=;{)B?tuQBVswqZVY)samnH zUr`H+TBuT8Jx?vPH-+Z0F|~B+Ic?4Jpl7g%Moi7Ke8-{I7NL1IKVBVbLDO-Ic)Fyz z+c%0!sGb>bePTqX*=g%La|y3^WJI2yYHB;qx0tk{N%7>6elUtorkOUp`^T zK;NHBs{UNn0y6t@34ah7^5=@M2`{TE5=vozNGa^k3z>u=tE2aDb6Hy{rChk(83hIQ zE3m&>9(Wz>A4yd*1R6d)qoKF8?IX(=j$T5bwG4qa99$sKDzc11psQ6^ z&m+*CO_60>!=sjNleTSGMu*ytNi5HCsfTrai^c7mAFmJsZ8)t^OeKM3Xm(p+(A6{K zt=F@g&34<`8H4V;+IsdpKh?sZdoq?vAkX(bWzg5spukXzgz|a2srbHFv`qn*(@9DgV%k&}i5E5G3^l)vR%CZLT-_Cr4P(a~A=AX#gF^6>;w z@O9zb3#bgvAI)b}jLDP*4+=qlTehI0^@?>!K4!yfL|2R?= z`x-d<8HwkDq7<-uDL`HIf>i|6H+cCqTphPft${6l z`=BwF;pDKolcDOCZsnjlRwu41P`y%h^(?4v==LU1-ESMNO)TOQ-6poCW8c;spW2>H z%q>9mjgMCdRL6GfQp4a6?6zV54eNimbzRdNj)K>#XVP28ZSOQ4+};&l@8DO9+n(pA zTJZV_R0|F2=t|H#F3?p0ie@P*5O5;DFS`DoPr@{S`z3`6h#3jduuxq}T-KgFw=c#n zcXmb-a$iRyDkDg|FLW;wmi1=xyF=~@4_0_^wY=v#9-I!87e}B@<+-dRKuoT(rU=Yv z2G7#uXjS+qCsU$8-?!D^3iPcE=&O!K{gp?f4ioro5gFW3=Bp51h4B93BRU_SS1YXd zxAq>t>Wt3Rzpp{Nxyfw}?KXb(+)(zy9(Z|OkF(>?^^l>c-O}^ z!5EKSkC+`t!)?#;jgG5nw&RYUhj%wWUM+a{rb)lfLOK^bSj3fbnwmP8vx-K&Z43`f z1_#?2-_>6$gh!s8YQejw*I zW_bEUZO`@7Jt6`o*Tah&QZ8zdP+AW!WotXHn^v&3f~~97C#=KPCou1COt}jN96w}H zzxme&MvyNUU_@w2r;rtiTu+%t^RqEs9#2pVy-Ootw-Vx(Ge0hnnE>Dy*`*8~()I6T zR($j>cv!?G?}N-paDVQ_>g3EfW<)B2$p14Go_0K2qqzOAGb&(a6UHP!XUX_IU$U#K z!yXl)s}S8^jp#l#m3~v0uJYnl#B?9`_UrhrhRufQINRg9@Ga!{uAYIr*fer*S68P! R{vvSq{{Tx(6id47000V^{i6T? literal 5143 zcmV+y6zJ<8iwFo|V|-o!18{X>Y-wX*bZKvHUokK)YIARH0PS7da@)9;eV?yD>0#=e zDqRTPMEN|)vK%XsZI$h$yHD3t3M3&5Hbv-wq#dUx5A!+m9rHRLH57hJ?g zp5RgRNP_z#K^Ht?F&9tpdMX-{Io(A#@@8~S(+~S0jgLy*{Zx0fl*zyNjMTSp%?9X= zlLb^4joF+=F_|wKN7!;K&9X2ydZrGAmYOW>KdJY5y6Da*3F6t&kp}-9%X4~D)9u;5 zr|Wjl^^Go6qhT4E4VmDA-LjAbBly7YjXhug52zI$1k{6unF>CKPyErir%#XZ!QB_v=|M#XAl`<$({XqK|kY|Q^0KP~7p z&9(gXTT_|yZssf-P?|EI=9ZJ@yaBCdw=}(l@eOv4uy&;BZKr4avr$OLf`rkOi#b_& z!)R&F4YrR={Ro?FtBd6u^b1I5ZSgy(mhbY~|v=&$vh}e+oSzujny-WKk21DG5W0hHd!5@|Q9xN<>v2QlL8T z!HeX*Np>yJFohzpyn@r-)?CqbOjy`N?-QTJGjv{2RZS{YN%5s7eJ@R8{z#!zgPCAa z4DP8^RZXgtO3E*F8GGq^#~+*Mg0cYh+gFuB^sVfFMMBV`Rg}13MF9-o+|C{QR3c#V_6v^o*xR$11mqQmsU3IfW`Flsz9}_l{J3 z?opp~?h`s&LmIUY>g^w{WO8wU{lVjx5KhP&0AhV|`ejooK zCsQ%Z?lV`Btp>8cJr3hPbCWooA!jq8v_Z1DP}W2qd{v}V5-H)zSvU9@n3uELSzHQ* zr3(0i!elJCqRYCL$t;X==LD?{(8RsA zb&R&wQ3kL_&s%A(YMw7A!;7_?T`p*d`soYb$wv02&vWiSGre39_*ORg=GA2XbpfDf9`;b3|=&IUqYq(3Zk%&M-M7eGIU)nxrsYySNzsT_%R%>a; zgfax&D^~A~4dm#xfg?4wc#Wxr;If;r@Yb8bm`KT7F2o8~y8f$|2wD86;&+jHRqB`P z*E=rPzJV-%u6@fGDr*$VJrN8%wx0rlPPc^})5?Fj3K_Ew-a& zzro+Kg`zaEEB}1pkc~z0MjvwXkVfT|-d;^|Rz% zsQ`@I4#s;^0ln9v0?PctqT_lE?=PedME#GU>qHF7aZ=H7vc{BTGCL#v zYvo`BR+v!R+nb~mTk5g*q8hVO_Q7p`x8e=zEn2%u!g~3_~)>CLSZegwEQ<$O2 z@d#aG2Ki&5MoP0|69LX>kUtSMS)+%ri z@CXJyMKt^E;ZO=iI|w3hAI+Rn@ZHoDf5TCfUg0UNfpL8c{h+P6 z8aAkb9o^|V8g&)NP(O>_iDS@uN2b#@4SQdX!Fd&mVI@2Dq8LuO@S8A7&cYHe&7Vrd zR0j>g2E0)@gpog&T2jda$%Xe^w*3Vc zE){X9rA#Q*NYR}MhE*+RTs%MvG|@kK04t7C&+@rclgpNUQ`@s?VsN_m&^jLil2p29 z@mOu0Ote(BpC9-{gy2oo4}C!@ik8on7Lt;snmsv20)860%|{U>LFJIq^10G=QqfY; zOP)Z{K#B&c*EDF820BaTBt%0wEs_UrAUmnacqB--*-c=QGAG_LZ305zxxl0lpQ2xf z7p)TwYxoHD!OK0+0QFd$@SM^%d_^N4xp+ypN$! zufK-(8ILBD(Z%5M<_7&T8jnZ)^K10)Fqi*_eEcxFxK>sC8stxo3QDgxAi`(-fkYlm z5Jc5Q=0|#LW#)T%7sUuEM(EEnLeIc)Z;BaGV`Lqfq1`-xW1TM2u&uRp)3h7A(nTiz z4qIeuX;(?fqI6tQWRW6^YExG)l0~{^K2H{5@X)qFv4g2Wd)Rec+xKi2QoL^O<%1|M zyj~-+C_NZS?iM7T$+MA0TeI65?mfQ>R%B7F$s&0G#BSTp-dvIXYUfdk?9_`Zny{~5 zdAgCmAKua`yt;fY;nhO8aTss(3yh|cnYfE8YKq4Sox{Qs&EF3yKI6gMC$pf5uGl>h z6>ehrTm{-Q;l{4KkwS|VT3jzbxQQ0`(Yqk=U~Sv7TN79YE@t2!&?w?_Sb;|rO<-XI z9t4d-KY@uiy2;LJqX7+q*2(g8!C)qM$UM{+@l3#UR)&B=1FL&HhSeN8hn08br$+{! zC5d5qNHRqE^2^gL4bk<2dJ>108U}p62xO^Zg)F}<8&k+~ZII>R_{+2y^Pau3mAp5<{Q(VuUQV(39%i+hj+R?+jM=;e;KFW{(60I zdaOI0^TBZ&oPvQ{SXcOb{VaJ`d|t@Sl;NKqO89Iexae-X$pv6}82p#i~qH``5Gk%)|pp~28W6Hu- z#&Y;5{Z=s+pm-*Day!fS!)GiBlqmo=?W5~C@o~p$$^RQUZSjSwI7n~Ip4254wP7um z!H>>IgR4s!{pj-DZ>m~fgSI&5FyT&jKBku1$_Y{s0Lp691tkoquM$X>h^vdv`<1k|soy#6$9qO*#IvsDeb z%z;4q3A?+GW}J(#iN@6nrB{`dVh4igpO`!xP~`;PuVdSl0|EO6MDmEF9IAZ7DlQKK zJHzG*)K;K&y?pj2sC`AigwrWIthW*&_vRO_Ei*#78|IdVTweMb<@@L5WFd{}-}hkB z9&;D!fePdj85t&I4E?R|{vF*4o-C02gwFfplTqvD@)Vt?9FfAU-;l{D+`2Zn^)akq z?;qve=9BE_>|C0v)xR9J9t12*gIAq1$d{X87^#Zs z*oMU@qTna*m#@c=&umu?RoL1K~e%+ZZz*sX%wg4Awfeic9D0kf}wm`83b*ihE z*@Eu2;5as>mQFpVt9c%b3>MLdsd<*~IMmu9IL`Lh>%$gkI_?lpm+W=-PSFGPGvl4F zPw2IKU43tQ;PviLSmvi*^uVu46J5`Ea>q!z;xF*}N>T|?|C$^~5sVqA`E5nRZ|ffi znFHy8Ux;}@GQ*xJ3$sR_yPid4WEXW(#<||BJSa@6KI)YDzD)FH)L3U9`46# z)%_Q4Qy-iS&qss)SngL>Rr(rOe;*b^PjQM!q<%1h;VuLBRV99*KyDx`}-m+MtmFQd!7Z6RG; z!=sjNldf%8MvvN#Ni5HCsfTrahrQ_AU#}6mY&e}z%q2NxXm(fO%k?wlowuKx?QYlF z8(;3d+V=Bhe(HrU4`f)AKwfQo&S0#iK?$9f3DrAiGw}m)W}l*fpn*ufHqoh^zlVP1 zI@L-_1*^^E4}_jio_qqT1g(T$&z~y$(n07z(D{sc_wzJ*3@qY#h>oj4WGYG3*eK~6 zI|I;bH>&awCNxZHd`Dcts_j)KRIXJmx`%P{V}ODRrXNoS;{q3*cSz? zzs+G~6IeeMl*s9SihAF17RtZnswAMTNRB8z_0h>i|0vxNfA;YVQE*>j;ssR3w~vwq z6;m>2!IQ$m-9{0P3YZHt3M7N2p>Dae5$EUVu6LSZ6eCz8qg2%Dl zxz;fF1G{V3f5ZCUZC%&&mZO03`kD03q0f752Y2@c%6s_LLZ2`5Q!k+W4620&O>`sZ z0~hF~#6pXdHJmq&u3v41O1vpxno8`G%ht(5?7TL?;VJi%hQ(8j662cgvzK6 z?@N_=!m7GLerNDpfxZg#t(S}31bwsFaDG+jT;5+w0>tE|Xo|p$7Vs`juONjkIGYoN z?7poASIBN{kX?0!=#RWYbev*rdE9MBnXiIz6^#3Xud@99;#mQ@KehMxQFmEp{(S?# zU7Fm^_^s0|H#qkkyH`NCj*XWPZZ?1@3|C>ewW+I@Fk+f(Xt?V+ zzR`0v&34@B%NXwV*XxDh-nHqMMMx9DgFM8O)7I1lmUT4hUBhWuGEUdtAg=ye(KMFX zsTYQOPQDU(EWU~Qp)daDzn@BcQzp>={jaALe6x^05XC(wP!Qfk=Om69DRnKMtD2Ds@Tmoe|r$0NnKJR}!Z62Ps5UOI1y(MC8uiXdJfPc|b(K z|UbQaxupRGE+?Zs(6&)JNo{E%!-fR1y6a@?;*%c z1osm!R##2FF(XnjJpP~2?QFc+8JHb@Sx^Csl`tj&x=5$w%dNJ$x*bu0vxvegT`7MsQruhrG{h(8Fg{XdhsTv?~; F002Dh3zYx> diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/software/software_10.json.gz b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/software/software_10.json.gz index a5b8c8774c5650641feb3a3c257b4bc182777b38..e59ae640208b554b8899c8c55d6766a27402ed7d 100644 GIT binary patch literal 6421 zcmV+w8S3UAiwFo-J_Tn019NX?ba!ELWnVEcE^2dcZUF6FUvu0xl7GKXft#wmNL8Z| zIh_C5+bda;ZDl1}B{`eRC8-pK1ZNPF#4HF(p5*edZ}(AFPx~SCnh$l|0RPQM%2<}| z8AV7rk_ZB5pu5pPH+}?uX#`Z!qa+lK(Ju|2T=0~SBi0y&G)h?$QbHyr3(ACmMNF|^ zJlAhn;#YZ<%u>}DHJsM4k(b2xGbt`uLUqmtyr`b3ueeB%upiMh^jfUe4B zz%HhE%u+?;S!3jO2VJMz?F|Rspbwo!N?qrFMew8k4_>!NqmJvf!!GlOonbF@f*uY0 z{@8J77yk9aumg<{l23R-qa&!`yT(0>{{UPOQN(-@OejSRRs7O;%jVD)F=I&}sKnYv z*xEJ_yP#2qojsKzI7%diq)H2tEQ)@G8fVy-?|#s)m^8wUar|G?M z2P3aD^17`~cZhUpq?p!G&_K|`F-_UmXU9-xs?=<>y9?x6Kri6X*OvYf%H2?*k_IeM zJmgFwuP7faXB%}o?QXx9cPy6x;t+BjpK~;KWshnulJ62+bsv%_haDiYAvIgnhL2U7=k2nwS2~vVwr4`^LcbZdBT8N z{;HOeW^yqf6@olW`QGe(#qVC5LOWMguafc__%t26=?iO~XPF*%I+F(;s`IOzYL(J0sSdf#Hu z9b)J$c5&gwaxYQK_ksM6iy7)&UCQ!n)6O*SH`AEG5U~rIs6g93ZH4Q$u`4xDCtp_hmaYJR9Sv#vx)X7K_JJP6SM$Mi3?yY@GCSv7oV3FuUbXde$?4 z`vL|xA7@Hib7SCiL1W@fRX(&sYI}n@#yG)%K+_ba!>Ot=B^3$y1IK!F#!Bg6GEdT6 z3&3bZ+-FI;pbz`Q1~d)_h;jAU_s*dOHhl z%Lombid0J<8}Xo(XS0#X2-m>*!s*7kO@i*qU^icvp&{g{8EDX^Ncgo?=hCA1!7Z~U z_v4#fDNv(fCFsKnXToTVi=x8vEx+|MiQS)tSZOWzT5A%dmIl!n z=761*9nyJ9YOubFV<2~2t^;+&|MrU&>Bn%B`>xyXbnkMLyN)|yt4R{bCZXoZU<<#dcFRqc9Yv6T*tf3O?JGUw)2_XT5QZRdFw6SV-BrEAUPCC zQ$}bMU0+2QC$Iupr$8px|4bNxF*}cioK3AX)+!Bf4!D)TIwcUmoGyXJ^%VgJ76!&F zi)c}umCI)jGu{$sHc=}ZFMeEtToo^_1pH>oVc;207k^)H9Ep^~jKboW5fKtKWu%N= zLU~LBj!bLgl_&xK6pctRq~~NSuOcAhIeFRC)v$Ln?QW4Nh(?N^=M&_>yh((_TxTX1 zELRw~=mgg<2`pf?TSPZ=p%L($&LZ5`3J7Z2rGWRq;sjiibVT4c2{NV|231zg$~#oG z-Ov>9Llz~tN})uk)k2vSKS2*s=adXp8vBU_N0k`#QVh>?{44BVfJlqJX^Yyjv!xKw;XL3OqXxQv*aDeT!o zi;fiWX;QWUhyw$E$`V86lB=oVMBeBrokifqT{H=Lf&oFP;%ho%S~#arDftg#93^)+ z>^8}py%%KYG)bq|-s$w=kIN}^3#yKyLYD{zBm*h~0;5yNRJV6gBJlnOw`UZZggTOQ zmM61J5%9kl3Z4=SfSbpyl%ZZ}ZjF$S=x?^vJGG6?fjTv{kjqq&h$mQ8fDud3a(t0w z@fi9Cywm+h7cM1Rx*T*9Xt{&!=2FfYmlv{jASZT4Ye`L@T83MYC!LBc3P>UpN+rS& z7BYhxPNy*{2Q;%bu4-EFB`JGzqjGGgQxPv~sw)xXX*+r-fhvU}io~U10oqk9mgPac zh5SM`7OGOQKxZvEhPrj5a3GQ|6sjJKm`HCjz!+lSgCssUtlJ|3tR)?2H*tyi-_gHa z(rUG~;GHj`EKbsGo!ZuK;4cq;Kn9(8pnrO#m#1NdVV>59Cvf`FPfm+*0IHH`V}S!c zQ>9966>0;Cn#*6}h#lkD;E6U@Lh_%%`T$$!z(VqZB}{^~%S{|;BuHigT}+IojWo}I zQjH-s%lI-yom#e+FIBETFF9Y8E)Jr z>k*o0w0n!=wfPEQbGmILOm%zvQ&NqxFFl;6Sl%Gd@EO{2U$V+jFN~S~FP|YO@ zOX(4gqEFd{RxtEY{)-$4Z8H>oYnTNM2Z8IAgjN=8rg_@0_geRBuIrP?PiPvdnN}>Cx5)FttXJ!oEj5c`xIc;^r543tQ4Eg&#gK2C zZ4ipVnp}%wc=94$g<=@=2JW3GhJo8_yKS#cyFGW@XM=HP*lmy9!FVusShu$d#jvXQ z(?v1#z0ZPT=vx$nMKRdTD2rmS0r$aJ3l|W zTR;|$wTYs=2OR6=AvryJ{o-uzgq$9poqYX0Wl`_h;obqz zHEk;r7eHh@y#+v&lC1^&5CSL^_frHC^_s-(bVhF5>+iVjPSWX#khqDyKhA_OX3P##ej7de0dF*#6?hDDg-i?AotyrQvX7fiMaSAjqR z&=@FJZs zrTLPV;fWt*L5}Lum=8o+l&d?Dj0$adUSYn9v{xm7$)BsdU$1V(b=7iKtwRn1wT!1z zB#KSAppwJb0R|%QpxC)u0E^{*GH6mL=>+5WsRleQz0@EQp)_U=46Q*uR^yhYZ!7Rz zt$ICJX^C_IrpGj3g=h+RO%9MFTi<_wX#O-)%J<)EWF2x@Gx^4A1Cv5vR5i?QAjqob zAh_;=2cRh_N*B3Ab6`}#ak6EAUE7HEDPGorV^Ul~-=AYdJer$$mW+56&%4}v} zM+L-C11@lsEr<(|OlSyl=ub_R3t*teKxzH(g+vgENCM471do>4O$&5h({K$Sspz!G zD^a|uG@r#Y#B?Qs#9Z(7R26Pl6)tzXR)GeyYBL&=y*K3eaPRyL@#+lz_nJHw;A|y~ zw@?nKMNcr=7`{rC;XJkqas%B8UZt}Y#tk<;NcT_rrT*LflhMgKAm}Cyw0j7CHIl6j zUF=nalTHrGMd*?txHX<@y8|l$J#nJ$3u7+1AtT0!3Iigq%0fVA8A7QcWtYf2G6REaPB8 zhAbK`cgfQNPl~L=HoQ#`D+X1c?&vymI5g&X%MJFLX~TrfLZ<$w?t)Q|TuJhI+bkYN zpKZddPjbG-iB`4(7F@u(YBtiKtUKq;^ixfowRZl_{FuwDm>fa8dd4)Xg2jmvb*q-| zmU;7!K4WSnD`9dUI)#z*J0~x##w+J8vwZysQ&$Ca{i#3Ivz;2(w&K~y! zcD8Z?(=OVBX0Wr5JcFHm{u#`40UT0eJtrdn(gjp|2Tg14O>OZO!+Zs+q1azE6OlBw z4-)>2Dccr_d#{*143aV-*e6V%2cao2^!?Qaqi*P2|5#C`H2U4g`HyPoIViNvMM zcilnH>2^HF9rt>^<97q!>kj*E+8rRbeiw=4cFfJ@huKyX~H>UUPjtuQtN+UVPs!`U^s1(=b%SSa{p~bww_cGH3dqtoev{> zPRC;uA)eYX;c26GY=?QQ2o_mOl3lw~@R4rHJf;3yC`V2XPL&YRSiB?l`?*InHQ)C* zbB6Zit+*fu%;!3cbZVAk_>mJR!8R$>0H`5AH6>wwCU$C0DmLeJ=LhDBs66u=R_x%1 zDz<|wguAJXNXQesiZwqpxLKe*6_a^@3kDAN=k*i>@lr`@YdJ!r2~?n_acT}=PDNb% z+z8=~8l+nGLchPYD`!lDy?x}iBF8`ok1w})$b0Fa^`dOxYE|IMl(&JEH3U}jrMwNS zJaUs@11o}24R~56Z@XWI@DPeSXov4DF#-Y zQMcC`v_JXDnH^^^@a{S}^Rwago-t6;h)M9k1u(7Vf#;8v2VP`-Xl&&Kh8ZsCzhX4m zIpt^tDzb}-HKd>}^w8+a9#oeFJ1?n7f+jf^v*`jrv3Rb?)(bo|#_|%32ve#_p3^aB zHCC6;72Q1VL*pwtWTX^>zN+i1*_0*={Z;I@f0T)JUYk<)549`>l`v4oVhrz?*wr!3 zEyWMTS)3D!Z{88Zq`To@=!3nI%wQz!` zF47fn0&mdwdUwJJSf38ZZI`w~#zH(Q*BcC-j@PE4*J*b;jTLZ$RmGnooWL7(`>jFW z{q%4G&+T^_YlagXFx+Wm`R4w&7p#WquP4=aKchSr{UA`m8D4=HvaCt|#z9tzs_0e; z71bhp%jb%s&e?b%gy4(@Khf)dA%s9qsov@=F$6_It&^CW z|Mq{4d!q>K9`9{?yeA^S+ci^jm@`xMh@}mQRf1qxq(2Bq0&;GJvJpZt2Yus3K(M9* zt3tWGCV0&u{o->OD~I)tjM75k-3r%N(w;xIZjpph_6`qCoY0@GAT~h|c!@c;_iS#; zj^p!4T#m)NyTxJm88=ao_xmas<8tKr@#}B!(_fy#Fbpu#Tz%|oF2bV4*S=w-uPl)w zo8UD!@Y{TJo3HmIiSg1D4!mxCpVaCt@4mzvW}qoC{~4aW19D82;yxqKM5KO>X_8c5 z_?DLlV%R1}9!4+axLdq5BGpvs+GqHiO@ON3_lc{0PkO!;JrS6DGcA!|TljuI}`z-fLR z&pcHO<4ryZX*z{I=*xCeb|X99KMGC3wn{AmX!5kUrwdjMcp?LhY91)Ez+B=bVvM5B zrgQuG^1TMjip_2jW{QiFK%HmIOKiBJyJh#;ueDZ_$I-Pd6ck6zx z>+ZPi&-HmYSBhoQT;@U-&fcq~)V>in`5O=1%{_*pMWgq+LdIlm+cAf zfzq2dvnd9s&4wqWT(U7aIXeC;IZE`(8CeUi==mHkv*lTAgG7&KcKJ9*(s%4S&AP_- zca4+4y2jQuehjX$S&MCwYiv!fb&a38NLRVWU9Wq$YYhJEa7_DN*bY6`A9fh{wgYGE z^t+ztuyB=YysG#Ubd5Xi$m_SfPWRJ8o%^nP`^jnDo%Y}}xyH}wg~WrNM3kSEm8Vw; zHSjOyeW+9XLgLv@ds!gaydqaV*Urz9>;oL)Wny?N7&!ii7c83B{r)PUPEqG>JP_Wz z4+aCbJ9kdNBr>6zw0c}fkZ zZ}OL9-}{8L%?`ALlJ15#TJRia$xjU@55T@ETHwj=j+!jBZLw9$?Xac{X*EHz-i(Iz(Ismu4 zB7IA|*jbv9be@Mt;ut3p|vUzc?1WXAje#!V;!ul9glk96qE8V{_1Cq(C(>3WO8q@7229+R)3PT|KQfhg z|1ZL8>JS#{K#LKsuUn>t7MZiC@hk761k^-A8T|gcNw^|(HjBW5=?}5>B_!0J7B?~n zWIQK@WyW9|jap<2SMuBB&j+uMMlNWWGxT=pPn7Ladg=80f5GprVTR-QaXEfceM>WS z>lE{+iT+5MCk$*mh5`LKfj_~PK@XRB<%kI5*&CVfBpaf*-c3? literal 6410 zcmV+l8TIBLiwFqDh>%_Y19NX?ba!ELWnVEcE^2dcZUF6FUvuL&l7GKXfm7Aq+Eubl ziu#|uz4F+e8E5Somp!|=T(XsdNKnEINoWbs9wnEDeY=mce%cS=*Zok}4e+02JF-2Q zksKp6u}u&_1L#Hr-Tfo*QzM{?9wnh@jDBkHDgc`ihGL3HuS1k^_~7oJ$_hM5;!9 zDx|!m(T7pWWl*LR}93)JQD%yLRwk-4HHQtfy=bxJmpsTVO z3>Q;8W>V33))=|nLD%Vad&7Y@=tHNGQrG!k5&WS4gV*iRsN;I=u*>{mXV?p!phpA0 zKXx42g@3&;>_8(#$|pRb(GgVeUE_hp{|cibqKNsxm@pMFRPj^e1Diuz#Ed0@pefci z!q&Ed*aeL;?CfbOf}MRw*d2OT|xl_Yau#Q$8z%VoK8s80Q11wNBG} z=MF|*XXJHTo$e6n(nv8aqacAGhhr+)pU;k=%v7n_Xm=OLwSZp0p|36dBTRQgg+dyz zMDdWb6vvA4(Q>v?r_=8CdwJ(_rej+MJFdIqw%<92vY=f8Y7o%OpX$e_C${-jX)q`)OT4-4r6}I35X_6@xNs>%H_J=w-_Xa7&pGs6*Czkc0m&rXicYeZe0^rN|^f4ZB;1ELRygJ zs2Lr%VTb*`!wwD4#yG2SiWo)3;xSDp0v1fe2$Kq`E%kG@ppI0qmgP@+jWd7y0wy;f zXG-gCqu_KwqvA|eKC~m%`UZ21aXA5oN(qbKR8^UhiUjz){9P_UG9$OJ4vwX!TMvx<^f`}z_NOq-<}@6S!&r^ zSnoA1I<5proJ8nh`Aer=Drv<3d;mRXhi>#N*U zphWXz;=>MS!f1?}qC)d6zxA^eyFV}9r&-9gkQQQXRI^JzYvE%KuoiAh*pMuOOa@P8~)g)&u zo>x|C{#2Q$c~kRB0{33jFe7GD356{ zBGcM@B}%|QMI%y7={ecTs|d(=PF^>4HSFU|TUlfZtVzX>^9hc?yh%igxz0>3*sd_J z%n9ya64=0Ow}@`$LL=Z=&LRw61q3DSQowr{;sh)bIU?|z1R2u}gDR_L?3zL>^8}} zy;o%DG)bq|-s$w=kIN}^3#yKyLY4^nAp@EQ1ZJm5rEc#cN8tU9(VkIg63R$EvOJk( zih$+CknxnD-`hNHNrrNzM{9)Rh-PL>eN@}n9H>)M3wbIPiFkrl1?Z>*DaRK{7LTES zFm}5CXu?UdrOQD#ftEYiZZ74ljq*a)4&=nnXep^NRP%5P^2n*kqJShqAy*>wT;X6) zGtx3f;eck==2cA!z9MCBZd8u#bSmOyO?4rHJZ(o$B@m@hM3J~OLx6f!vt@ZwZ{c|1 zFczXx4S~*Dju^_;jlzLQey>pUV8%o`wg7X8fen)QV6bkF2n;RhK)Z=c%>R!5?UGij zwFU2d5oK{Aw{>b;zk$Cz_yHMo=7Ij{kzSsL8HRaUAD+P0M>|;-^8iF8(aHiRe5Oi~ z+A72b67?v5g)?@HbAu;ZU5S+c2-*jRbq+&FUa^FwpzU%M2NDUKnLrm4BWWW&W&w?Fm!DU_xw67$&MTuwVHM#ZHx=P>t}ZK zs%X@Ei}SU44~5mN=JN!X(MnYsrMfU(4qnS9Ie4w$2XwQEEUBp(7%|v!NS0ohR=wcp z-mSxjETIL>#zVLX9pGa(@SJmTY&XK{iy_u{nK~4(lfQ=7A?jeR>e!= zBNBtG@>z5nH-W`a6BN-EZ2=2$TLA(Hxse}>xnY!R%YTZ`H+C4NueS_`)PfOYK1_{; z7VFuCJB~~hN*Tx%!-U%rtVOnL)aM}^*I$mLIXxRmv*3k?gBK*V-~|g_cmm*sJX*Fv z;03F4EqLMCvvd`BVbB}6cY+rNZm;dOy*BOk+;N``#+_leJ$47!Tr-x@Je||^y-W-s6@;*ucd2zV^_T-SfeRKRR{3zeH zz8j~zec$z{HYLNRPj)!Tte$G9*QW9nGrUT8d9dpqTaK^y#t_Y z+EyekV8^(;1vezg)>@aTALB67fU5CUHBRk=yqAJ8rwvH8I$&7g;)nc7n{)sD=54 zO>!b4a_Ga40hH+uy&bpbw71JnBhbkNVyl=w24ZiA82NQTR}tE;At5HFjT#oyG;#SG za5(hNBfrU*sY?Wv)WAcCPr9HyqA)LV+yX*zpdyWiFvSY!#dW@dF?+ zP_D|Orc{xjh7aHg#_ysfcYoHb1yKJbYLOv5DVIT)&(XkOwwR5|VlnlP~v zgQCen1!IPoiM{jvqoW*MgRl~uFGY?iyGt{q4dBw_ zl9%C$A7w!f=F(6P1Xz^2JCTeEX?R&dy^6FqC2q-|tGr)tZiREza#pRw5d>-(ms2E) zO}L;bhq(g-he7*E_b_DfyS_EH5!t=cjWkR@BAI{>I3}mHGV9h*h=(n zAskSPUSPB`d?S?^d2AKr2D%lhN@pvS8^$_F_jmfG{@eYXk;ys^=q3%+dx&{8oUKh= z>{Wz|PL9Pz#F8PnHJ)p|13Lk|aH8xBqb|82BgO~@10rwALO^F3IjNCkmpFK+^rqqx zCPOm6rHu%MS}t5C(IQ7?Vbm%L#^BE0P7rc%Yc=IqzQI}fAx zXDb&lZK8eB3U>A@uV80ieg!jK0E5(M&xy!CZ~@WYLDiaDQ(L^nFkdm$Q0$L-5Ro*t z4-)=>Dccr+`yii%J<7%ca4i6L6U4>>aMuOE?QaqQ*Q#6#z-#tyPoIV3BaYy zcilnH>2^HF9rt>^<97q!>kj*E+8y9%{UQK&Rq@ZeKmX}r9awJqcby*Q z4!;lp_axJwBywR6)#gh_YV*YdwaeTMi^2UA3@+?OuKy9?7p3^sbM9`i7=?90MGBV8 zra)f{jzv|iAL|CYnJKgQ&rRWoPtkZ{)N)~+TnOnTlSJZ;7YZy(%Z$yyww)flA_sM2 zs!4S8P9((MMe)SA`{{R4^sQ^ddXud;*?NofgF}--1lzA zC3-2IFL**z&UU^QlGVs9Ce{c-k&>GCAZo8*^f_&kmmn-G<^G3?T0Nnr(9v2>NMWY`%D?(~&-!=W>zK}dsP(02x&GYH!h2C?sXtV5kuzLHhNpQEqD z8FhQDLHn~Gec5pa1MjY*FTWV_?gaxOjaUkXw}3&b8Q%G0Wq21^AL%YRf!T%o_^%mF zc1}4?{)*aSVh!r23q8_dvIn(f!Om+clAuY>#ca9&EG(WYYV`t-bep^eO~I6El9zPM zS#3Pa=ZajO_mM6W?ekHJLC@6n)oeGmE(_b(I#PXN(57DiEnuCY=zLa482Nt1-Ml4C{BAP$fykB|dlGf0Aza9a7^&?Pk5my$; zw^j@p%Y2&7g=NEf*^pwN0`q}2F;l~GWHUg=V>-XSlCX0~EJxPLk?SkptY+lm`f750 zrR*G8`!T@g6i3U2wJzaFASN{Cb~3DY!N`>02_6jtoBuJ`0ALeZj%)@be;;;Khf?tq zsu*!w+TU%1>raTK_YbbuS=}U*yr6R?IZa5+)Kr+eOo8?28k}PUJJknK;n7~XK{{VP z{cFM!eKu4;sKQGxO=w*zHj@p@#9uEuZEs<-(P{gog#A+Dw;KJ=!%dqM{kJODM*p8X zOIJkyy+PmW-5LF7eL5VsUD^&A3-OFvZ!mN^UYmwqr`_o^Rz&|-6@QM=e{a<7w+4Op z^GE+Zx8G^3Ir@LVFu=(2p!~NNtcJX=C)Hz}QSOU=3< zSdqQ)xcL7|EYu|bO)t1cE#%4Nb45t!Y&zx^;hQbRAuCG#iy4Si{5=PlaJd$uif3%F)1V-S^;M`tbxhc`a zmyx&}i;s7+!^RLdk&zGkKo@;(2T!sCdU&x@ZsZbu#Sh-hGs7>18TMt$CsVu^iYW?N0}^xOaIpuL`5iZN zsTlf_d=gSQg+AyTY$Usp9WV8SreIrB%>rohw7C2Vb`5wG0*z`GC=P+S8B4_IA)QU< z_DkXiO_mjbgRTu6*jTNN)!JC?Z#Gt|g9DostFq`CIu?_)ao-rN4QQ@7j%^==Ug=QH zY>Mt^6W)ZBOExAaN5_96M~U7im(bH$OMac>-8*(yW)0zo8^TFo z4Pk2tKLtbB?7TL~5Vk7U8p6+=rK=3#uGhWW5C-dYIHr9sY=<7}4?7Gj*?}{5`d!a+ zSh&g%URC@V8p0iS!#r~COq%YE0q{m8QJPJ8f$4B?mbBEF$=UVQWWkYe#{Tv9$&Ewum&!FBcV5{G3hEu~U;{R9Ea1!+9T$1V#^a7C zJ3c&n`)!lFp=t`YaIJH-l$!H`rDEQA+@WL#*-T_?o}=fFGk>vDc}fjCZ}K6z`yTydn~nWu~p0Mu+EDIZ0YKkaiX&-(5I_M((+=R zyqH{HiS+s^yuQ*MjLr8nmLY3n$OULF*v?r!wSHFMCtkhmf((}m%Zbf^ z6CW(>rSw9smvqFB`uf~KNk#;={uzr1>|aw7QL>=dG)DaQIdAiQ0LH8$ePz1XSz3{F zo|q>Y(s041B#z_gIw;9E6Tc^iO4}RnMS8cK*r3)XN|?Q~*ao#My7P&MjSXt81>LE= zsGAhjvMSdGwVpdmR|T~OgMoKvP>c4c&%B^dgHD_J1J`4nVYeIhdbHE&c&qL#URnGZ z2DQ5FQM=vhIiDZh>GXU3USsVi@yPcq``9Gk()tY``P0H(Bbcx8NK_|<0SrUdBun?U z+`O@+$l9>D&QA$doGSqffW*%XU#74d2088ccsBt#ALJ{ge?I=LG%)T?Rdvg4rh$25)DX!f(d)iT^ASnY3RuKS-vm5oY~f*6+0mVq^VY>+yS` Yp=JGEPkZh>5x>{}19s*i?oIsw0IUmUV*mgE diff --git a/pom.xml b/pom.xml index 666ba2350..e1d99f25b 100644 --- a/pom.xml +++ b/pom.xml @@ -937,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [7.0.1] + [8.0.1] cdh5.9.2 3.5 11.0.2 From 62c4c3ed29dcf395b2f18c5c6495f6502f011c65 Mon Sep 17 00:00:00 2001 From: miconis Date: Wed, 9 Oct 2024 12:26:03 +0200 Subject: [PATCH 151/239] implementation of new comparators for organization and dataset disambiguation --- .../java/eu/dnetlib/pace/tree/CodeMatch.java | 2 +- .../eu/dnetlib/pace/tree/JsonListMatch.java | 43 +++++++++++++------ .../pace/comparators/ComparatorTest.java | 34 ++++++++++++--- 3 files changed, 60 insertions(+), 19 deletions(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java index 25a12bcdf..5cd21adcd 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/CodeMatch.java @@ -21,7 +21,7 @@ public class CodeMatch extends AbstractStringComparator { public CodeMatch(Map params) { super(params); this.params = params; - this.CODE_REGEX = Pattern.compile(params.getOrDefault("codeRegex", "[a-zA-Z]::\\d+")); + this.CODE_REGEX = Pattern.compile(params.getOrDefault("codeRegex", "[a-zA-Z]+::\\d+")); } public Set getRegexList(String input) { diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java index 3897e37f8..e95d9206e 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java @@ -41,21 +41,38 @@ public class JsonListMatch extends AbstractListComparator { return -1; } - final Set ca = sa.stream().map(this::toComparableString).collect(Collectors.toSet()); - final Set cb = sb.stream().map(this::toComparableString).collect(Collectors.toSet()); + Set ca = sa.stream().map(this::toComparableString).collect(Collectors.toSet()); + Set cb = sb.stream().map(this::toComparableString).collect(Collectors.toSet()); - int incommon = Sets.intersection(ca, cb).size(); - int simDiff = Sets.symmetricDifference(ca, cb).size(); + switch (MODE) { + case "count": + return Sets.intersection(ca, cb).size(); - if (incommon + simDiff == 0) { - return 0.0; + case "percentage": + int incommon = Sets.intersection(ca, cb).size(); + int simDiff = Sets.symmetricDifference(ca, cb).size(); + if (incommon + simDiff == 0) { + return 0.0; + } + return (double) incommon / (incommon + simDiff); + + case "type": + Set typesA = ca.stream().map(s -> s.split("::")[0]).collect(Collectors.toSet()); + Set typesB = cb.stream().map(s -> s.split("::")[0]).collect(Collectors.toSet()); + + Set types = Sets.intersection(typesA, typesB); + + if (types.isEmpty()) // if no common type, it is impossible to compare + return -1; + + ca = ca.stream().filter(s -> types.contains(s.split("::")[0])).collect(Collectors.toSet()); + cb = cb.stream().filter(s -> types.contains(s.split("::")[0])).collect(Collectors.toSet()); + + return (double) Sets.intersection(ca, cb).size() / types.size(); + + default: + return -1; } - - if (MODE.equals("percentage")) - return (double) incommon / (incommon + simDiff); - else - return incommon; - } // converts every json into a comparable string basing on parameters @@ -69,7 +86,7 @@ public class JsonListMatch extends AbstractListComparator { // for each path in the param list for (String key : params.keySet().stream().filter(k -> k.contains("jpath")).collect(Collectors.toList())) { String path = params.get(key); - String value = MapDocumentUtil.getJPathString(path, documentContext); + String value = MapDocumentUtil.getJPathString(path, documentContext).toLowerCase(); if (value == null || value.isEmpty()) value = ""; st.append(value); diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index c008902c4..635f1fa44 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -65,6 +65,23 @@ public class ComparatorTest extends AbstractPaceTest { } + @Test + public void datasetVersionCodeMatchTest() { + + params.put("codeRegex", "(?=[\\w-]*[a-zA-Z])(?=[\\w-]*\\d)[\\w-]+"); + CodeMatch codeMatch = new CodeMatch(params); + + // names have different codes + assertEquals(0.0, codeMatch.distance("physical oceanography at ctd station june 1998 ev02a", "physical oceanography at ctd station june 1998 ir02", conf)); + + // names have same code + assertEquals(1.0, codeMatch.distance("physical oceanography at ctd station june 1998 ev02a", "physical oceanography at ctd station june 1998 ev02a", conf)); + + // code is not in both names + assertEquals(-1, codeMatch.distance("physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998 ev02a", conf)); + assertEquals(1.0, codeMatch.distance("physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998", conf)); + } + @Test public void listContainsMatchTest() { @@ -257,15 +274,15 @@ public class ComparatorTest extends AbstractPaceTest { List a = createFieldList( Arrays .asList( - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"doi\",\"classname\":\"Digital Object Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"10.1111/pbi.12655\"}"), + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"grid\",\"classname\":\"GRID Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"grid_1\"}", + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_1\"}"), "authors"); List b = createFieldList( Arrays .asList( - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"pmc\",\"classname\":\"PubMed Central ID\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"PMC5399005\"}", - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"pmid\",\"classname\":\"PubMed ID\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"27775869\"}", - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"user:claim\",\"classname\":\"Linked by user\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"doi\",\"classname\":\"Digital Object Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"10.1111/pbi.12655\"}", - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"handle\",\"classname\":\"Handle\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"1854/LU-8523529\"}"), + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"grid\",\"classname\":\"GRID Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"grid_1\"}", + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_2\"}", + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"user:claim\",\"classname\":\"Linked by user\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"isni\",\"classname\":\"ISNI Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"isni_1\"}"), "authors"); double result = jsonListMatch.compare(a, b, conf); @@ -277,6 +294,13 @@ public class ComparatorTest extends AbstractPaceTest { result = jsonListMatch.compare(a, b, conf); assertEquals(1.0, result); + + params.put("mode", "type"); + jsonListMatch = new JsonListMatch(params); + result = jsonListMatch.compare(a, b, conf); + + assertEquals(0.5, result); + } @Test From 56b05cde0b621d91e9b974453d6fe28f7061561b Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Wed, 9 Oct 2024 23:00:13 +0200 Subject: [PATCH 152/239] Revert the changes for IgnoreUndefined management in tree evaluation --- .../pace/tree/support/TreeNodeDef.java | 2 +- .../pace/tree/support/TreeNodeStats.java | 33 ++++--------------- .../pace/tree/support/TreeProcessor.java | 8 +++-- 3 files changed, 12 insertions(+), 31 deletions(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java index 0ff03f5e1..0973fdf1e 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeDef.java @@ -48,7 +48,7 @@ public class TreeNodeDef implements Serializable { // function for the evaluation of the node public TreeNodeStats evaluate(Row doc1, Row doc2, Config conf) { - TreeNodeStats stats = new TreeNodeStats(ignoreUndefined); + TreeNodeStats stats = new TreeNodeStats(); // for each field in the node, it computes the for (FieldConf fieldConf : fields) { diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java index f6b210a8c..2b96048b4 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeNodeStats.java @@ -9,11 +9,8 @@ public class TreeNodeStats implements Serializable { private Map results; // this is an accumulator for the results of the node - private final boolean ignoreUndefined; - - public TreeNodeStats(boolean ignoreUndefined) { + public TreeNodeStats() { this.results = new HashMap<>(); - this.ignoreUndefined = ignoreUndefined; } public Map getResults() { @@ -25,10 +22,7 @@ public class TreeNodeStats implements Serializable { } public int fieldsCount() { - if (ignoreUndefined) - return this.results.size(); - else - return this.results.size() - undefinedCount(); // do not count undefined + return this.results.size(); } public int undefinedCount() { @@ -84,22 +78,11 @@ public class TreeNodeStats implements Serializable { double min = 100.0; // random high value for (FieldStats fs : this.results.values()) { if (fs.getResult() < min) { - if (fs.getResult() == -1) { - if (fs.isCountIfUndefined()) { - min = 0.0; - } else { - min = -1; - } - } else { + if (fs.getResult() >= 0.0 || (fs.getResult() == -1 && fs.isCountIfUndefined())) min = fs.getResult(); - } } } - if (ignoreUndefined) { - return min == -1.0 ? 0.0 : min; - } else { - return min; - } + return min; } // if at least one is true, return 1.0 @@ -108,11 +91,7 @@ public class TreeNodeStats implements Serializable { if (fieldStats.getResult() >= fieldStats.getThreshold()) return 1.0; } - if (!ignoreUndefined && undefinedCount() > 0) { - return -1.0; - } else { - return 0.0; - } + return 0.0; } // if at least one is false, return 0.0 @@ -121,7 +100,7 @@ public class TreeNodeStats implements Serializable { if (fieldStats.getResult() == -1) { if (fieldStats.isCountIfUndefined()) - return ignoreUndefined ? 0.0 : -1.0; + return 0.0; } else { if (fieldStats.getResult() < fieldStats.getThreshold()) return 0.0; diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java index 28b3a82af..263504dbb 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/support/TreeProcessor.java @@ -44,10 +44,12 @@ public class TreeProcessor { TreeNodeStats stats = currentNode.evaluate(doc1, doc2, config); treeStats.addNodeStats(nextNodeName, stats); - double finalScore = stats.getFinalScore(currentNode.getAggregation()); - if (finalScore == -1.0) + // if ignoreUndefined=false the miss is considered as undefined + if (!currentNode.isIgnoreUndefined() && stats.undefinedCount() > 0) { nextNodeName = currentNode.getUndefined(); - else if (finalScore >= currentNode.getThreshold()) { + } + // if ignoreUndefined=true the miss is ignored and the score computed anyway + else if (stats.getFinalScore(currentNode.getAggregation()) >= currentNode.getThreshold()) { nextNodeName = currentNode.getPositive(); } else { nextNodeName = currentNode.getNegative(); From 5015ba10eba66ba12750ff348cea559062f68e7e Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 14 Oct 2024 10:23:42 +0200 Subject: [PATCH 153/239] addition of date comparator --- dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java new file mode 100644 index 000000000..91b8d1c41 --- /dev/null +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java @@ -0,0 +1,2 @@ +package eu.dnetlib.pace.tree;public class DateRange { +} From eab623ddfa0b01137df729981ef99abe0c129215 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 14 Oct 2024 10:24:19 +0200 Subject: [PATCH 154/239] implementation of date matcher --- .../java/eu/dnetlib/pace/tree/DateRange.java | 67 ++++++++++++++++++- .../pace/comparators/ComparatorTest.java | 19 ++++++ 2 files changed, 85 insertions(+), 1 deletion(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java index 91b8d1c41..c913109a4 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java @@ -1,2 +1,67 @@ -package eu.dnetlib.pace.tree;public class DateRange { +package eu.dnetlib.pace.tree; + +import com.wcohen.ss.AbstractStringDistance; +import eu.dnetlib.pace.config.Config; +import eu.dnetlib.pace.tree.support.AbstractStringComparator; +import eu.dnetlib.pace.tree.support.ComparatorClass; +import org.joda.time.DateTime; + +import java.time.DateTimeException; +import java.time.LocalDate; +import java.time.Period; +import java.time.format.DateTimeFormatter; +import java.util.Locale; +import java.util.Map; + +@ComparatorClass("dateRange") +public class DateRange extends AbstractStringComparator { + + int YEAR_RANGE; + + public DateRange(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + YEAR_RANGE = Integer.parseInt(params.getOrDefault("year_range", "3")); + } + + public DateRange(final double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } + + protected DateRange(final double weight, final AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } + + public static boolean isNumeric(String str) { + return str.matches("\\d+"); //match a number with optional '-' and decimal. + } + + @Override + public double distance(final String a, final String b, final Config conf) { + if (a.isEmpty() || b.isEmpty()) { + return -1.0; // return -1 if a field is missing + } + + try { + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ENGLISH); + LocalDate d1 = LocalDate.parse(a, formatter); + LocalDate d2 = LocalDate.parse(b, formatter); + Period period = Period.between(d1, d2); + + return period.getYears() <= YEAR_RANGE? 1.0 : 0.0; + } + catch (DateTimeException e) { + return -1.0; + } + + } + + @Override + public double getWeight() { + return super.weight; + } + + @Override + protected double normalize(final double d) { + return d; + } } diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index 635f1fa44..5e1484254 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -351,4 +351,23 @@ public class ComparatorTest extends AbstractPaceTest { } + @Test + public void dateMatch() { + + DateRange dateRange = new DateRange(params); + + double result = dateRange.distance("2021-05-13", "2023-05-13", conf); + assertEquals(1.0, result); + + result = dateRange.distance("2021-05-13", "2025-05-13", conf); + assertEquals(0.0, result); + + result = dateRange.distance("", "2020-05-05", conf); + assertEquals(-1.0, result); + + result = dateRange.distance("invalid date", "2021-05-02", conf); + assertEquals(-1.0, result); + + } + } From 0e5dd14538fc8b2ba2bc08f3af93793e3b9e19b7 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 18 Oct 2024 16:22:21 +0200 Subject: [PATCH 155/239] [createASfromAffRo] adding the provenance datasource used to get the relation (no datasource can be webcrawl = publisher, rawaff means oalex) --- .../PrepareAffiliationRelations.java | 47 ++++++++++--------- .../PrepareAffiliationRelationsTest.java | 16 +++---- 2 files changed, 34 insertions(+), 29 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 028fa47dc..61a018a41 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -104,22 +104,22 @@ public class PrepareAffiliationRelations implements Serializable { .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); JavaPairRDD crossrefRelations = prepareAffiliationRelationsNewModel( - spark, crossrefInputPath, collectedfromOpenAIRE); + spark, crossrefInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::crossref"); JavaPairRDD pubmedRelations = prepareAffiliationRelations( - spark, pubmedInputPath, collectedfromOpenAIRE); + spark, pubmedInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::pubmed"); JavaPairRDD openAPCRelations = prepareAffiliationRelationsNewModel( - spark, openapcInputPath, collectedfromOpenAIRE); + spark, openapcInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::openapc"); - JavaPairRDD dataciteRelations = prepareAffiliationRelations( - spark, dataciteInputPath, collectedfromOpenAIRE); + JavaPairRDD dataciteRelations = prepareAffiliationRelationsNewModel( + spark, dataciteInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::datacite"); - JavaPairRDD webCrawlRelations = prepareAffiliationRelations( - spark, webcrawlInputPath, collectedfromOpenAIRE); + JavaPairRDD webCrawlRelations = prepareAffiliationRelationsNewModel( + spark, webcrawlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::rawaff"); - JavaPairRDD publisherRelations = prepareAffiliationRelationFromPublisher( - spark, publisherlInputPath, collectedfromOpenAIRE); + JavaPairRDD publisherRelations = prepareAffiliationRelationFromPublisherNewModel( + spark, publisherlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::webcrawl"); crossrefRelations .union(pubmedRelations) @@ -133,7 +133,8 @@ public class PrepareAffiliationRelations implements Serializable { private static JavaPairRDD prepareAffiliationRelationFromPublisherNewModel(SparkSession spark, String inputPath, - List collectedfrom) { + List collectedfrom, + String dataprovenance) { Dataset df = spark .read() @@ -142,12 +143,13 @@ public class PrepareAffiliationRelations implements Serializable { .json(inputPath) .where("DOI is not null"); - return getTextTextJavaPairRDD(collectedfrom, df.selectExpr("DOI", "Organizations as Matchings")); + return getTextTextJavaPairRDDNew( + collectedfrom, df.selectExpr("DOI", "Organizations as Matchings"), dataprovenance); } private static JavaPairRDD prepareAffiliationRelationFromPublisher(SparkSession spark, String inputPath, - List collectedfrom) { + List collectedfrom, String dataprovenance) { Dataset df = spark .read() @@ -155,13 +157,14 @@ public class PrepareAffiliationRelations implements Serializable { .json(inputPath) .where("DOI is not null"); - return getTextTextJavaPairRDD(collectedfrom, df.selectExpr("DOI", "Organizations as Matchings")); + return getTextTextJavaPairRDD( + collectedfrom, df.selectExpr("DOI", "Organizations as Matchings"), dataprovenance); } private static JavaPairRDD prepareAffiliationRelations(SparkSession spark, String inputPath, - List collectedfrom) { + List collectedfrom, String dataprovenance) { // load and parse affiliation relations from HDFS Dataset df = spark @@ -170,12 +173,12 @@ public class PrepareAffiliationRelations implements Serializable { .json(inputPath) .where("DOI is not null"); - return getTextTextJavaPairRDD(collectedfrom, df); + return getTextTextJavaPairRDD(collectedfrom, df, dataprovenance); } private static JavaPairRDD prepareAffiliationRelationsNewModel(SparkSession spark, String inputPath, - List collectedfrom) { + List collectedfrom, String dataprovenance) { // load and parse affiliation relations from HDFS Dataset df = spark .read() @@ -184,10 +187,11 @@ public class PrepareAffiliationRelations implements Serializable { .json(inputPath) .where("DOI is not null"); - return getTextTextJavaPairRDDNew(collectedfrom, df); + return getTextTextJavaPairRDDNew(collectedfrom, df, dataprovenance); } - private static JavaPairRDD getTextTextJavaPairRDD(List collectedfrom, Dataset df) { + private static JavaPairRDD getTextTextJavaPairRDD(List collectedfrom, Dataset df, + String dataprovenance) { // unroll nested arrays df = df .withColumn("matching", functions.explode(new Column("Matchings"))) @@ -219,7 +223,7 @@ public class PrepareAffiliationRelations implements Serializable { DataInfo dataInfo = OafMapperUtils .dataInfo( false, - BIP_INFERENCE_PROVENANCE, + dataprovenance, true, false, qualifier, @@ -235,7 +239,8 @@ public class PrepareAffiliationRelations implements Serializable { new Text(OBJECT_MAPPER.writeValueAsString(aa)))); } - private static JavaPairRDD getTextTextJavaPairRDDNew(List collectedfrom, Dataset df) { + private static JavaPairRDD getTextTextJavaPairRDDNew(List collectedfrom, Dataset df, + String dataprovenance) { // unroll nested arrays df = df .withColumn("matching", functions.explode(new Column("Matchings"))) @@ -276,7 +281,7 @@ public class PrepareAffiliationRelations implements Serializable { DataInfo dataInfo = OafMapperUtils .dataInfo( false, - BIP_INFERENCE_PROVENANCE, + dataprovenance, true, false, qualifier, diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index 179cbecb5..c704bb99b 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -98,9 +98,9 @@ public class PrepareAffiliationRelationsTest { "-crossrefInputPath", crossrefAffiliationRelationPathNew, "-pubmedInputPath", crossrefAffiliationRelationPath, "-openapcInputPath", crossrefAffiliationRelationPathNew, - "-dataciteInputPath", crossrefAffiliationRelationPath, - "-webCrawlInputPath", crossrefAffiliationRelationPath, - "-publisherInputPath", publisherAffiliationRelationOldPath, + "-dataciteInputPath", crossrefAffiliationRelationPathNew, + "-webCrawlInputPath", crossrefAffiliationRelationPathNew, + "-publisherInputPath", publisherAffiliationRelationPath, "-outputPath", outputPath }); @@ -112,7 +112,7 @@ public class PrepareAffiliationRelationsTest { .map(aa -> ((Relation) aa.getPayload())); // count the number of relations - assertEquals(150, tmp.count());// 18 + 24 *3 + 30 * 2 = + assertEquals(162, tmp.count());// 18 + 24 + 30 * 4 = Dataset dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); dataset.createOrReplaceTempView("result"); @@ -123,7 +123,7 @@ public class PrepareAffiliationRelationsTest { // verify that we have equal number of bi-directional relations Assertions .assertEquals( - 75, execVerification + 81, execVerification .filter( "relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'") .collectAsList() @@ -131,7 +131,7 @@ public class PrepareAffiliationRelationsTest { Assertions .assertEquals( - 75, execVerification + 81, execVerification .filter( "relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'") .collectAsList() @@ -158,7 +158,7 @@ public class PrepareAffiliationRelationsTest { Assertions .assertEquals( - 2, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count()); + 4, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count()); Assertions .assertEquals( @@ -173,7 +173,7 @@ public class PrepareAffiliationRelationsTest { Assertions .assertEquals( - 3, execVerification + 1, execVerification .filter( "source = '" + ID_PREFIX + IdentifierFactory From 2b27afaec8d3a6bd7bdaf2ff83040f88c68660e2 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 18 Oct 2024 16:22:51 +0200 Subject: [PATCH 156/239] [createASfromAffRo] refactoring after compilation --- .../dhp/oa/graph/hive/GraphHiveTableImporterJob.java | 6 +++--- .../dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java index 73243dbc5..d4fec3f52 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java @@ -72,9 +72,9 @@ public class GraphHiveTableImporterJob { final Encoder clazzEncoder = Encoders.bean(clazz); Dataset dataset = spark - .read() - .schema(clazzEncoder.schema()) - .json(inputPath); + .read() + .schema(clazzEncoder.schema()) + .json(inputPath); if (numPartitions > 0) { log.info("repartitioning {} to {} partitions", clazz.getSimpleName(), numPartitions); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala index 4e5ad5365..eece56b74 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala @@ -31,6 +31,7 @@ class ORCIDAuthorMatchersTest { assertTrue(matchOrderedTokenAndAbbreviations("孙林 Sun Lin", "Sun Lin")) // assertTrue(AuthorsMatchRevised.compare("孙林 Sun Lin", "孙林")); // not yet implemented } + @Test def testDocumentationNames(): Unit = { assertTrue(matchOrderedTokenAndAbbreviations("James C. A. Miller-Jones", "James Antony Miller-Jones")) } From ce4ee1189f4d83ff114c63ddb782621db7a939b2 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 21 Oct 2024 14:38:15 +0200 Subject: [PATCH 157/239] [personEntity] create entity for each profile in orcid even without works. Added validated true to each relation coming from orcid data --- .../personentity/CoAuthorshipIterator.java | 5 ++- .../personentity/ExtractPerson.java | 31 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java index 76e4c4851..94ac7ab28 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java @@ -61,7 +61,8 @@ public class CoAuthorshipIterator implements Iterator { private Relation getRelation(String orcid1, String orcid2) { String source = PERSON_PREFIX + IdentifierFactory.md5(orcid1); String target = PERSON_PREFIX + IdentifierFactory.md5(orcid2); - return OafMapperUtils + Relation relation = + OafMapperUtils .getRelation( source, target, ModelConstants.PERSON_PERSON_RELTYPE, ModelConstants.PERSON_PERSON_SUBRELTYPE, @@ -76,5 +77,7 @@ public class CoAuthorshipIterator implements Iterator { ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"), null); + relation.setValidated(true); + return relation; } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 6f61d427d..c29c04699 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -144,18 +144,13 @@ public class ExtractPerson implements Serializable { .parquet(inputPath + "Employments") .as(Encoders.bean(Employment.class)); - Dataset peopleToMap = authors - .joinWith(works, authors.col("orcid").equalTo(works.col("orcid"))) - .map((MapFunction, Author>) t2 -> t2._1(), Encoders.bean(Author.class)) - .groupByKey((MapFunction) a -> a.getOrcid(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Author.class)); - Dataset employment = employmentDataset - .joinWith(peopleToMap, employmentDataset.col("orcid").equalTo(peopleToMap.col("orcid"))) + .joinWith(authors, employmentDataset.col("orcid").equalTo(authors.col("orcid"))) .map((MapFunction, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class)); + //Mapping all the orcid profiles even if the profile has no visible works Dataset people; - peopleToMap.map((MapFunction) op -> { + authors.map((MapFunction) op -> { Person person = new Person(); person.setId(DHPUtils.generateIdentifier(op.getOrcid(), PERSON_PREFIX)); person @@ -325,6 +320,7 @@ public class ExtractPerson implements Serializable { Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), DATAINFO, null); + relation.setValidated(true); if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())) { KeyValue kv = new KeyValue(); @@ -412,14 +408,15 @@ public class ExtractPerson implements Serializable { default: return null; } - - return OafMapperUtils - .getRelation( - source, target, ModelConstants.RESULT_PERSON_RELTYPE, - ModelConstants.RESULT_PERSON_SUBRELTYPE, - ModelConstants.RESULT_PERSON_HASAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - DATAINFO, - null); + Relation relation = OafMapperUtils + .getRelation( + source, target, ModelConstants.RESULT_PERSON_RELTYPE, + ModelConstants.RESULT_PERSON_SUBRELTYPE, + ModelConstants.RESULT_PERSON_HASAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + DATAINFO, + null); + relation.setValidated(true); + return relation; } } From 09a2c93fc7197e48a8f769a131908e22ed02a78c Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 21 Oct 2024 16:21:15 +0200 Subject: [PATCH 158/239] [personEntity] added relations with projects extracting the info from the database --- .../personentity/ExtractPerson.java | 166 ++++++++++++++---- 1 file changed, 133 insertions(+), 33 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index c29c04699..960dfbe44 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -2,13 +2,18 @@ package eu.dnetlib.dhp.actionmanager.personentity; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import static org.apache.spark.sql.functions.*; +import java.io.BufferedWriter; import java.io.IOException; +import java.io.OutputStreamWriter; import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.sql.ResultSet; +import java.sql.SQLException; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.common.DbClient; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; @@ -22,6 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.spark_project.jetty.util.StringUtil; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -43,9 +49,14 @@ import eu.dnetlib.dhp.schema.oaf.utils.PidType; import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + public class ExtractPerson implements Serializable { private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class); - + private static final String QUERY = "SELECT * FROM project_person WHERE pid_type = 'ORCID'"; private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String OPENAIRE_PREFIX = "openaire____"; private static final String SEPARATOR = "::"; @@ -62,6 +73,12 @@ public class ExtractPerson implements Serializable { private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + public static final String OPENAIRE_DATASOURCE_ID = "10|infrastruct_::f66f1bd369679b5b077dcdf006089556"; + public static final String OPENAIRE_DATASOURCE_NAME = "OpenAIRE"; + + public static List collectedfromOpenAIRE = OafMapperUtils + .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); + public static final DataInfo DATAINFO = OafMapperUtils .dataInfo( @@ -106,19 +123,130 @@ public class ExtractPerson implements Serializable { final String workingDir = parser.get("workingDir"); log.info("workingDir {}", workingDir); + final String dbUrl = parser.get("postgresUrl"); + final String dbUser = parser.get("postgresUser"); + final String dbPassword = parser.get("postgresPassword"); + + final String hdfsNameNode = parser.get("hdfsNameNode"); + SparkConf conf = new SparkConf(); runWithSparkSession( conf, isSparkSessionManaged, spark -> { HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration()); - createActionSet(spark, inputPath, outputPath, workingDir); + extractInfoForActionSetFromORCID(spark, inputPath, workingDir); + extractInfoForActionSetFromProjects(spark, inputPath, workingDir, dbUrl, dbUser, dbPassword, workingDir + "/project", hdfsNameNode); + createActionSet(spark, outputPath, workingDir); }); } - private static void createActionSet(SparkSession spark, String inputPath, String outputPath, String workingDir) { + private static void extractInfoForActionSetFromProjects(SparkSession spark, String inputPath, String workingDir, + String dbUrl, String dbUser, String dbPassword, String hdfsPath, String hdfsNameNode) throws IOException { + Configuration conf = new Configuration(); + conf.set("fs.defaultFS", hdfsNameNode); + + FileSystem fileSystem = FileSystem.get(conf); + Path hdfsWritePath = new Path(hdfsPath); + FSDataOutputStream fos = fileSystem.create(hdfsWritePath); + try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) { + try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos, StandardCharsets.UTF_8))) { + dbClient.processResults(QUERY, rs -> writeRelation(getRelationWithProject(rs), writer)); + } + + } catch (IOException e) { + e.printStackTrace(); + } + + } + + public static Relation getRelationWithProject(ResultSet rs) { + try { + return getProjectRelation(rs.getString("project"), rs.getString("pid"), + rs.getString("role")); + } catch (final SQLException e) { + throw new RuntimeException(e); + } + } + + private static Relation getProjectRelation(String project, String orcid, String role) { + + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); + String target = project.substring(0,14) + + IdentifierFactory.md5(project.substring(15)); + List properties = new ArrayList<>(); + + Relation relation = OafMapperUtils + .getRelation( + source, target, ModelConstants.PROJECT_PERSON_RELTYPE, ModelConstants.PROJECT_PERSON_SUBRELTYPE, + ModelConstants.PROJECT_PERSON_PARTICIPATES, + collectedfromOpenAIRE, + DATAINFO, + null); + relation.setValidated(true); + + if (StringUtil.isNotBlank(role)) { + KeyValue kv = new KeyValue(); + kv.setKey("role"); + kv.setValue(role); + properties.add(kv); + } + + + if (!properties.isEmpty()) + relation.setProperties(properties); + return relation; + + + } + + protected static void writeRelation(final Relation relation, BufferedWriter writer) { + try { + writer.write(OBJECT_MAPPER.writeValueAsString(relation)); + writer.newLine(); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } + + private static void createActionSet(SparkSession spark,String outputPath, String workingDir) { + + Dataset people; + people = spark + .read() + .textFile(workingDir + "/people") + .map( + (MapFunction) value -> OBJECT_MAPPER + .readValue(value, Person.class), + Encoders.bean(Person.class)); + + people + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .union( + getRelations(spark, workingDir + "/authorship").toJavaRDD().map(r -> new AtomicAction(r.getClass(), r))) + .union( + getRelations(spark, workingDir + "/coauthorship") + .toJavaRDD() + .map(r -> new AtomicAction(r.getClass(), r))) + .union( + getRelations(spark, workingDir + "/affiliation") + .toJavaRDD() + .map(r -> new AtomicAction(r.getClass(), r))) + .union( + getRelations(spark, workingDir + "/project") + .toJavaRDD() + .map(r -> new AtomicAction(r.getClass(), r))) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); + } + + private static void extractInfoForActionSetFromORCID(SparkSession spark, String inputPath, String workingDir) { Dataset authors = spark .read() .parquet(inputPath + "Authors") @@ -149,7 +277,7 @@ public class ExtractPerson implements Serializable { .map((MapFunction, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class)); //Mapping all the orcid profiles even if the profile has no visible works - Dataset people; + authors.map((MapFunction) op -> { Person person = new Person(); person.setId(DHPUtils.generateIdentifier(op.getOrcid(), PERSON_PREFIX)); @@ -257,34 +385,6 @@ public class ExtractPerson implements Serializable { .option("compression", "gzip") .mode(SaveMode.Overwrite) .json(workingDir + "/affiliation"); - - people = spark - .read() - .textFile(workingDir + "/people") - .map( - (MapFunction) value -> OBJECT_MAPPER - .readValue(value, Person.class), - Encoders.bean(Person.class)); - - people.show(false); - people - .toJavaRDD() - .map(p -> new AtomicAction(p.getClass(), p)) - .union( - getRelations(spark, workingDir + "/authorship").toJavaRDD().map(r -> new AtomicAction(r.getClass(), r))) - .union( - getRelations(spark, workingDir + "/coauthorship") - .toJavaRDD() - .map(r -> new AtomicAction(r.getClass(), r))) - .union( - getRelations(spark, workingDir + "/affiliation") - .toJavaRDD() - .map(r -> new AtomicAction(r.getClass(), r))) - .mapToPair( - aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), - new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile( - outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class); } private static Dataset getRelations(SparkSession spark, String path) { From 821540f94a0ae055c74cd642ec6921465465e8ae Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 22 Oct 2024 10:13:30 +0200 Subject: [PATCH 159/239] [personEntity] updated the property file to include also the db parameters. The same for the wf definition. Refactoring for compilation --- .../personentity/CoAuthorshipIterator.java | 3 +- .../personentity/ExtractPerson.java | 99 +++++++++---------- .../personentity/as_parameters.json | 25 +++++ .../actionmanager/personentity/job.properties | 5 +- .../personentity/oozie_app/workflow.xml | 16 +++ .../orcid/ORCIDAuthorMatchersTest.scala | 1 + 6 files changed, 95 insertions(+), 54 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java index 94ac7ab28..131f3f466 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java @@ -61,8 +61,7 @@ public class CoAuthorshipIterator implements Iterator { private Relation getRelation(String orcid1, String orcid2) { String source = PERSON_PREFIX + IdentifierFactory.md5(orcid1); String target = PERSON_PREFIX + IdentifierFactory.md5(orcid2); - Relation relation = - OafMapperUtils + Relation relation = OafMapperUtils .getRelation( source, target, ModelConstants.PERSON_PERSON_RELTYPE, ModelConstants.PERSON_PERSON_SUBRELTYPE, diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 960dfbe44..fb0621b6e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -13,9 +13,12 @@ import java.sql.SQLException; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.common.DbClient; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; @@ -27,13 +30,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.spark_project.jetty.util.StringUtil; - import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.collection.orcid.model.Author; import eu.dnetlib.dhp.collection.orcid.model.Employment; import eu.dnetlib.dhp.collection.orcid.model.Work; +import eu.dnetlib.dhp.common.DbClient; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; @@ -49,11 +52,6 @@ import eu.dnetlib.dhp.schema.oaf.utils.PidType; import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - public class ExtractPerson implements Serializable { private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class); private static final String QUERY = "SELECT * FROM project_person WHERE pid_type = 'ORCID'"; @@ -77,8 +75,7 @@ public class ExtractPerson implements Serializable { public static final String OPENAIRE_DATASOURCE_NAME = "OpenAIRE"; public static List collectedfromOpenAIRE = OafMapperUtils - .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); - + .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); public static final DataInfo DATAINFO = OafMapperUtils .dataInfo( @@ -136,14 +133,15 @@ public class ExtractPerson implements Serializable { spark -> { HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration()); extractInfoForActionSetFromORCID(spark, inputPath, workingDir); - extractInfoForActionSetFromProjects(spark, inputPath, workingDir, dbUrl, dbUser, dbPassword, workingDir + "/project", hdfsNameNode); + extractInfoForActionSetFromProjects( + spark, inputPath, workingDir, dbUrl, dbUser, dbPassword, workingDir + "/project", hdfsNameNode); createActionSet(spark, outputPath, workingDir); }); } private static void extractInfoForActionSetFromProjects(SparkSession spark, String inputPath, String workingDir, - String dbUrl, String dbUser, String dbPassword, String hdfsPath, String hdfsNameNode) throws IOException { + String dbUrl, String dbUser, String dbPassword, String hdfsPath, String hdfsNameNode) throws IOException { Configuration conf = new Configuration(); conf.set("fs.defaultFS", hdfsNameNode); @@ -164,41 +162,40 @@ public class ExtractPerson implements Serializable { public static Relation getRelationWithProject(ResultSet rs) { try { - return getProjectRelation(rs.getString("project"), rs.getString("pid"), - rs.getString("role")); + return getProjectRelation( + rs.getString("project"), rs.getString("pid"), + rs.getString("role")); } catch (final SQLException e) { throw new RuntimeException(e); } - } + } private static Relation getProjectRelation(String project, String orcid, String role) { - String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); - String target = project.substring(0,14) - + IdentifierFactory.md5(project.substring(15)); - List properties = new ArrayList<>(); + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); + String target = project.substring(0, 14) + + IdentifierFactory.md5(project.substring(15)); + List properties = new ArrayList<>(); - Relation relation = OafMapperUtils - .getRelation( - source, target, ModelConstants.PROJECT_PERSON_RELTYPE, ModelConstants.PROJECT_PERSON_SUBRELTYPE, - ModelConstants.PROJECT_PERSON_PARTICIPATES, - collectedfromOpenAIRE, - DATAINFO, - null); - relation.setValidated(true); + Relation relation = OafMapperUtils + .getRelation( + source, target, ModelConstants.PROJECT_PERSON_RELTYPE, ModelConstants.PROJECT_PERSON_SUBRELTYPE, + ModelConstants.PROJECT_PERSON_PARTICIPATES, + collectedfromOpenAIRE, + DATAINFO, + null); + relation.setValidated(true); - if (StringUtil.isNotBlank(role)) { - KeyValue kv = new KeyValue(); - kv.setKey("role"); - kv.setValue(role); - properties.add(kv); - } - - - if (!properties.isEmpty()) - relation.setProperties(properties); - return relation; + if (StringUtil.isNotBlank(role)) { + KeyValue kv = new KeyValue(); + kv.setKey("role"); + kv.setValue(role); + properties.add(kv); + } + if (!properties.isEmpty()) + relation.setProperties(properties); + return relation; } @@ -211,7 +208,7 @@ public class ExtractPerson implements Serializable { } } - private static void createActionSet(SparkSession spark,String outputPath, String workingDir) { + private static void createActionSet(SparkSession spark, String outputPath, String workingDir) { Dataset people; people = spark @@ -221,7 +218,7 @@ public class ExtractPerson implements Serializable { (MapFunction) value -> OBJECT_MAPPER .readValue(value, Person.class), Encoders.bean(Person.class)); - + people .toJavaRDD() .map(p -> new AtomicAction(p.getClass(), p)) @@ -235,10 +232,10 @@ public class ExtractPerson implements Serializable { getRelations(spark, workingDir + "/affiliation") .toJavaRDD() .map(r -> new AtomicAction(r.getClass(), r))) - .union( - getRelations(spark, workingDir + "/project") - .toJavaRDD() - .map(r -> new AtomicAction(r.getClass(), r))) + .union( + getRelations(spark, workingDir + "/project") + .toJavaRDD() + .map(r -> new AtomicAction(r.getClass(), r))) .mapToPair( aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), new Text(OBJECT_MAPPER.writeValueAsString(aa)))) @@ -276,7 +273,7 @@ public class ExtractPerson implements Serializable { .joinWith(authors, employmentDataset.col("orcid").equalTo(authors.col("orcid"))) .map((MapFunction, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class)); - //Mapping all the orcid profiles even if the profile has no visible works + // Mapping all the orcid profiles even if the profile has no visible works authors.map((MapFunction) op -> { Person person = new Person(); @@ -509,13 +506,13 @@ public class ExtractPerson implements Serializable { return null; } Relation relation = OafMapperUtils - .getRelation( - source, target, ModelConstants.RESULT_PERSON_RELTYPE, - ModelConstants.RESULT_PERSON_SUBRELTYPE, - ModelConstants.RESULT_PERSON_HASAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - DATAINFO, - null); + .getRelation( + source, target, ModelConstants.RESULT_PERSON_RELTYPE, + ModelConstants.RESULT_PERSON_SUBRELTYPE, + ModelConstants.RESULT_PERSON_HASAUTHORED, + Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), + DATAINFO, + null); relation.setValidated(true); return relation; } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json index 5175552e7..1894a6beb 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/as_parameters.json @@ -21,5 +21,30 @@ "paramLongName": "workingDir", "paramDescription": "the hdfs name node", "paramRequired": false +}, + { + "paramName": "pu", + "paramLongName": "postgresUrl", + "paramDescription": "the hdfs name node", + "paramRequired": false + }, + + { + "paramName": "ps", + "paramLongName": "postgresUser", + "paramDescription": "the hdfs name node", + "paramRequired": false + }, + { + "paramName": "pp", + "paramLongName": "postgresPassword", + "paramDescription": "the hdfs name node", + "paramRequired": false +},{ + "paramName": "nn", + "paramLongName": "hdfsNameNode", + "paramDescription": "the hdfs name node", + "paramRequired": false } + ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties index d2269718c..ac63d8a68 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties @@ -1,2 +1,5 @@ inputPath=/data/orcid_2023/tables/ -outputPath=/user/miriam.baglioni/peopleAS \ No newline at end of file +outputPath=/user/miriam.baglioni/peopleAS +postgresUrl=jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus +postgresUser=dnet' +postgresPassword=dnetPwd \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml index 166e7bb9c..5b613a76a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/oozie_app/workflow.xml @@ -9,6 +9,18 @@ outputPath the path where to store the actionset + + postgresUrl + the path where to store the actionset + + + postgresUser + the path where to store the actionset + + + postgresPassword + the path where to store the actionset + sparkDriverMemory memory for driver process @@ -102,6 +114,10 @@ --inputPath${inputPath} --outputPath${outputPath} --workingDir${workingDir} + --hdfsNameNode${nameNode} + --postgresUrl${postgresUrl} + --postgresUser${postgresUser} + --postgresPassword${postgresPassword} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala index 4e5ad5365..eece56b74 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala @@ -31,6 +31,7 @@ class ORCIDAuthorMatchersTest { assertTrue(matchOrderedTokenAndAbbreviations("孙林 Sun Lin", "Sun Lin")) // assertTrue(AuthorsMatchRevised.compare("孙林 Sun Lin", "孙林")); // not yet implemented } + @Test def testDocumentationNames(): Unit = { assertTrue(matchOrderedTokenAndAbbreviations("James C. A. Miller-Jones", "James Antony Miller-Jones")) } From aac5eb34995b6cf736c5a8dbdc67d66891a89992 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 22 Oct 2024 11:54:16 +0200 Subject: [PATCH 160/239] [personEntity] changed the data info for the relations with projects. added missing parameters to the job.properties file --- .../personentity/ExtractPerson.java | 51 ++++++++----------- .../actionmanager/personentity/job.properties | 2 +- 2 files changed, 22 insertions(+), 31 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index fb0621b6e..7b04d4d52 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -71,13 +71,15 @@ public class ExtractPerson implements Serializable { private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; + public static final String FUNDER_AUTHORS_CLASSID = "sysimport:crosswalk:funderdatabase"; + public static final String FUNDER_AUTHORS_CLASSNAME = "Imported from Funder Database"; public static final String OPENAIRE_DATASOURCE_ID = "10|infrastruct_::f66f1bd369679b5b077dcdf006089556"; public static final String OPENAIRE_DATASOURCE_NAME = "OpenAIRE"; public static List collectedfromOpenAIRE = OafMapperUtils .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); - public static final DataInfo DATAINFO = OafMapperUtils + public static final DataInfo ORCIDDATAINFO = OafMapperUtils .dataInfo( false, null, @@ -91,6 +93,20 @@ public class ExtractPerson implements Serializable { ModelConstants.DNET_PROVENANCE_ACTIONS), "0.91"); + public static final DataInfo FUNDERDATAINFO = OafMapperUtils + .dataInfo( + false, + null, + false, + false, + OafMapperUtils + .qualifier( + FUNDER_AUTHORS_CLASSID, + FUNDER_AUTHORS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.91"); + public static void main(final String[] args) throws IOException, ParseException { final ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -182,7 +198,7 @@ public class ExtractPerson implements Serializable { source, target, ModelConstants.PROJECT_PERSON_RELTYPE, ModelConstants.PROJECT_PERSON_SUBRELTYPE, ModelConstants.PROJECT_PERSON_PARTICIPATES, collectedfromOpenAIRE, - DATAINFO, + FUNDERDATAINFO, null); relation.setValidated(true); @@ -328,7 +344,7 @@ public class ExtractPerson implements Serializable { ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, null)); person.setDateofcollection(op.getLastModifiedDate()); person.setOriginalId(Arrays.asList(op.getOrcid())); - person.setDataInfo(DATAINFO); + person.setDataInfo(ORCIDDATAINFO); return person; }, Encoders.bean(Person.class)) .write() @@ -415,7 +431,7 @@ public class ExtractPerson implements Serializable { source, target, ModelConstants.ORG_PERSON_RELTYPE, ModelConstants.ORG_PERSON_SUBRELTYPE, ModelConstants.ORG_PERSON_PARTICIPATES, Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - DATAINFO, + ORCIDDATAINFO, null); relation.setValidated(true); @@ -438,31 +454,6 @@ public class ExtractPerson implements Serializable { } - private static Collection getCoAuthorshipRelations(String orcid1, String orcid2) { - String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid1); - String target = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid2); - - return Arrays - .asList( - OafMapperUtils - .getRelation( - source, target, ModelConstants.PERSON_PERSON_RELTYPE, - ModelConstants.PERSON_PERSON_SUBRELTYPE, - ModelConstants.PERSON_PERSON_HASCOAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - DATAINFO, - null), - OafMapperUtils - .getRelation( - target, source, ModelConstants.PERSON_PERSON_RELTYPE, - ModelConstants.PERSON_PERSON_SUBRELTYPE, - ModelConstants.PERSON_PERSON_HASCOAUTHORED, - Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - DATAINFO, - null)); - - } - private static @NotNull Iterator getAuthorshipRelationIterator(Work w) { if (Optional.ofNullable(w.getPids()).isPresent()) @@ -511,7 +502,7 @@ public class ExtractPerson implements Serializable { ModelConstants.RESULT_PERSON_SUBRELTYPE, ModelConstants.RESULT_PERSON_HASAUTHORED, Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)), - DATAINFO, + ORCIDDATAINFO, null); relation.setValidated(true); return relation; diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties index ac63d8a68..b9325bcb7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/personentity/job.properties @@ -1,5 +1,5 @@ inputPath=/data/orcid_2023/tables/ outputPath=/user/miriam.baglioni/peopleAS postgresUrl=jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus -postgresUser=dnet' +postgresUser=dnet postgresPassword=dnetPwd \ No newline at end of file From 0e34b0ece13f4d6dfeb5a4f0cb274168327c954a Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Mon, 21 Oct 2024 09:05:13 +0200 Subject: [PATCH 161/239] Fix imports: point them from the main distribution packages --- .../src/test/java/eu/dnetlib/pace/util/UtilTest.java | 1 - .../dhp/actionmanager/personentity/ExtractPerson.java | 6 +++--- .../eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala | 2 +- .../scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala | 2 +- .../main/scala/eu/dnetlib/doiboost/orcid/ORCIDToOAF.scala | 2 +- .../src/main/java/eu/dnetlib/dhp/api/Utils.java | 4 ++-- .../java/eu/dnetlib/dhp/bulktag/community/Constraint.java | 2 +- .../dhp/oa/graph/hive/GraphHiveTableImporterJob.java | 6 +++--- .../main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java | 2 +- 9 files changed, 13 insertions(+), 14 deletions(-) diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/util/UtilTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/util/UtilTest.java index be5c1ebb9..93db552c1 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/util/UtilTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/util/UtilTest.java @@ -11,7 +11,6 @@ import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import eu.dnetlib.pace.model.Person; -import jdk.nashorn.internal.ir.annotations.Ignore; public class UtilTest { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index e63a50984..debf7e38e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -11,6 +11,7 @@ import java.util.stream.Collectors; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; @@ -20,7 +21,6 @@ import org.apache.spark.sql.*; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.spark_project.jetty.util.StringUtil; import com.fasterxml.jackson.databind.ObjectMapper; @@ -317,13 +317,13 @@ public class ExtractPerson implements Serializable { "0.91"), null); - if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())) { + if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtils.isNotBlank(row.getStartDate())) { KeyValue kv = new KeyValue(); kv.setKey("startDate"); kv.setValue(row.getStartDate()); properties.add(kv); } - if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtil.isNotBlank(row.getEndDate())) { + if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtils.isNotBlank(row.getEndDate())) { KeyValue kv = new KeyValue(); kv.setKey("endDate"); kv.setValue(row.getEndDate()); diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index 7c45234f6..e7d68920b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -14,7 +14,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.{ PidType } import eu.dnetlib.dhp.utils.DHPUtils -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.apache.spark.sql.Row import org.json4s import org.json4s.DefaultFormats diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index d8292a631..a2c36041d 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -7,7 +7,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, IdentifierFactor import eu.dnetlib.dhp.utils.DHPUtils import eu.dnetlib.doiboost.DoiBoostMappingUtil import eu.dnetlib.doiboost.DoiBoostMappingUtil._ -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.json4s import org.json4s.DefaultFormats import org.json4s.JsonAST._ diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/orcid/ORCIDToOAF.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/orcid/ORCIDToOAF.scala index 7c58afc09..6ec75f5c3 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/orcid/ORCIDToOAF.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/orcid/ORCIDToOAF.scala @@ -6,7 +6,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Publication} import eu.dnetlib.doiboost.DoiBoostMappingUtil import eu.dnetlib.doiboost.DoiBoostMappingUtil.{createSP, generateDataInfo} -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.json4s import org.json4s.DefaultFormats import org.json4s.JsonAST._ diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java index 27fb37e5b..6079da365 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -6,11 +6,11 @@ import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.amazonaws.util.StringUtils; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; @@ -81,7 +81,7 @@ public class Utils implements Serializable { Community c = new Community(); c.setId(cm.getId()); c.setZenodoCommunities(cm.getOtherZenodoCommunities()); - if (!StringUtils.isNullOrEmpty(cm.getZenodoCommunity())) + if (StringUtils.isNotBlank(cm.getZenodoCommunity())) c.getZenodoCommunities().add(cm.getZenodoCommunity()); c.setSubjects(cm.getSubjects()); c.getSubjects().addAll(cm.getFos()); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java index 82a6a3b85..51525e4d3 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java @@ -4,7 +4,7 @@ package eu.dnetlib.dhp.bulktag.community; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; -import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonIgnore; import eu.dnetlib.dhp.bulktag.criteria.Selection; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java index 73243dbc5..d4fec3f52 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java @@ -72,9 +72,9 @@ public class GraphHiveTableImporterJob { final Encoder clazzEncoder = Encoders.bean(clazz); Dataset dataset = spark - .read() - .schema(clazzEncoder.schema()) - .json(inputPath); + .read() + .schema(clazzEncoder.schema()) + .json(inputPath); if (numPartitions > 0) { log.info("repartitioning {} to {} partitions", clazz.getSimpleName(), numPartitions); diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java index 5e705716c..0461e2f94 100644 --- a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java @@ -3,8 +3,8 @@ package eu.dnetlib.dhp.swh.models; import java.io.Serializable; -import com.cloudera.com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; @JsonIgnoreProperties(ignoreUnknown = true) public class LastVisitData implements Serializable { From aa7b8fd014f8b1a3855330806efa40cec1fc11d6 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Mon, 21 Oct 2024 18:05:01 +0200 Subject: [PATCH 162/239] Use workingDir parameter for temporary data of ORCID enrichment --- .../enrich/orcid/enrich_graph_orcid_parameters.json | 6 ++++++ .../orcid/SparkEnrichGraphWithOrcidAuthors.scala | 12 +++++++----- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json index 765c0e8ff..772e1381f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json @@ -22,5 +22,11 @@ "paramLongName": "targetPath", "paramDescription": "the output path of the graph enriched", "paramRequired": true + }, + { + "paramName": "wp", + "paramLongName": "workingDir", + "paramDescription": "the working dir", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala index 0824c2a71..847a5f090 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -47,13 +47,15 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] log.info(s"orcidPath is '$orcidPath'") val targetPath = parser.get("targetPath") log.info(s"targetPath is '$targetPath'") + val workingDir = parser.get("workingDir") + log.info(s"targetPath is '$workingDir'") - createTemporaryData(graphPath, orcidPath, targetPath) - analisys(targetPath) - generateGraph(graphPath, targetPath) + createTemporaryData(graphPath, orcidPath, workingDir) + analisys(workingDir) + generateGraph(graphPath, workingDir, targetPath) } - private def generateGraph(graphPath: String, targetPath: String): Unit = { + private def generateGraph(graphPath: String, workingDir: String, targetPath: String): Unit = { ModelSupport.entityTypes.asScala .filter(e => ModelSupport.isResult(e._1)) @@ -63,7 +65,7 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] val matched = spark.read .schema(Encoders.bean(classOf[ORCIDAuthorEnricherResult]).schema) - .parquet(s"${targetPath}/${resultType}_matched") + .parquet(s"${workingDir}/${resultType}_matched") .selectExpr("id", "enriched_author") spark.read From 6bc741715c08a4b71e3b737b3d8befdbfa743aab Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Wed, 23 Oct 2024 14:01:12 +0200 Subject: [PATCH 163/239] Fix OafMapperUtilsTest.testMergePubs --- .../eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java index 9317c0ce4..1ee8e52de 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java @@ -179,7 +179,7 @@ class OafMapperUtilsTest { assertEquals( ModelConstants.DATASET_RESULTTYPE_CLASSID, ((Result) MergeUtils - .merge(p2, d1)) + .merge(p2, d1, true)) .getResulttype() .getClassid()); } From c921cf7ee033053eaf00ccf876e3bb2edc8bfa3e Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 09:57:20 +0200 Subject: [PATCH 164/239] [personEntity] removed the deletedbyinference results (not indexed, but still in the graph). Changed the writing mode: append instead of overwrite --- .../common/person}/CoAuthorshipIterator.java | 2 +- .../dnetlib/dhp/common/person}/Coauthors.java | 5 +- .../personentity/ExtractPerson.java | 2 + dhp-workflows/dhp-enrichment/pom.xml | 7 +- .../input_personpropagation_parameters.json | 21 ++++++ .../dhp/wf/subworkflows/person/job.properties | 1 + .../person/oozie_app/config-default.xml | 58 ++++++++++++++++ .../person/oozie_app/workflow.xml | 68 +++++++++++++++++++ 8 files changed, 153 insertions(+), 11 deletions(-) rename {dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity => dhp-common/src/main/java/eu/dnetlib/dhp/common/person}/CoAuthorshipIterator.java (98%) rename {dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity => dhp-common/src/main/java/eu/dnetlib/dhp/common/person}/Coauthors.java (70%) create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/input_personpropagation_parameters.json create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/job.properties create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/person/CoAuthorshipIterator.java similarity index 98% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java rename to dhp-common/src/main/java/eu/dnetlib/dhp/common/person/CoAuthorshipIterator.java index 131f3f466..853f223d0 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/CoAuthorshipIterator.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/person/CoAuthorshipIterator.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.actionmanager.personentity; +package eu.dnetlib.dhp.common.person; import java.util.Arrays; import java.util.Iterator; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/person/Coauthors.java similarity index 70% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java rename to dhp-common/src/main/java/eu/dnetlib/dhp/common/person/Coauthors.java index 17f46d5c7..ff9324d2e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/Coauthors.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/person/Coauthors.java @@ -1,12 +1,9 @@ -package eu.dnetlib.dhp.actionmanager.personentity; +package eu.dnetlib.dhp.common.person; import java.io.Serializable; -import java.util.ArrayList; import java.util.List; -import eu.dnetlib.dhp.schema.oaf.Relation; - public class Coauthors implements Serializable { private List coauthors; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 7b04d4d52..3ee89e772 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -13,6 +13,8 @@ import java.sql.SQLException; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.common.person.CoAuthorshipIterator; +import eu.dnetlib.dhp.common.person.Coauthors; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; diff --git a/dhp-workflows/dhp-enrichment/pom.xml b/dhp-workflows/dhp-enrichment/pom.xml index 9698dee03..41f57e6df 100644 --- a/dhp-workflows/dhp-enrichment/pom.xml +++ b/dhp-workflows/dhp-enrichment/pom.xml @@ -48,12 +48,7 @@ io.github.classgraph classgraph
- - eu.dnetlib.dhp - dhp-aggregation - 1.2.5-SNAPSHOT - compile - + diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/input_personpropagation_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/input_personpropagation_parameters.json new file mode 100644 index 000000000..df65d5320 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/input_personpropagation_parameters.json @@ -0,0 +1,21 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + } +] diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/job.properties b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/job.properties new file mode 100644 index 000000000..61bd3d121 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/job.properties @@ -0,0 +1 @@ +sourcePath=/tmp/miriam/13_graph_copy \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/config-default.xml new file mode 100644 index 000000000..1cb0b8a5e --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/config-default.xml @@ -0,0 +1,58 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 5G + + + sparkExecutorCores + 4 + + + spark2MaxExecutors + 50 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/workflow.xml new file mode 100644 index 000000000..c9b914384 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/subworkflows/person/oozie_app/workflow.xml @@ -0,0 +1,68 @@ + + + + sourcePath + the source path + + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + yarn + cluster + personPropagation + eu.dnetlib.dhp.person.SparkExtractPersonRelations + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --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.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=7680 + + --sourcePath${sourcePath}/ + --outputPath${workingDir}/relation + + + + + + + + + + \ No newline at end of file From cf07ed90584b8d792f7868f325404eb35fbd3fbf Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 14:35:14 +0200 Subject: [PATCH 165/239] [person] refactoring --- .../person/SparkExtractPersonRelations.java | 271 ++++++++++++++++++ 1 file changed, 271 insertions(+) create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java new file mode 100644 index 000000000..34bd976ea --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java @@ -0,0 +1,271 @@ + +package eu.dnetlib.dhp.person; + +import static com.ibm.icu.text.PluralRules.Operand.w; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +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.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; +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 eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.person.CoAuthorshipIterator; +import eu.dnetlib.dhp.common.person.Coauthors; +import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import scala.Tuple2; + +public class SparkExtractPersonRelations { + + private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob.class); + private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; + + public static final DataInfo DATAINFO = OafMapperUtils + .dataInfo( + false, + "openaire", + true, + false, + OafMapperUtils + .qualifier( + ModelConstants.SYSIMPORT_CROSSWALK_REPOSITORY, + ModelConstants.SYSIMPORT_CROSSWALK_REPOSITORY, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.85"); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkCountryPropagationJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/wf/subworkflows/person/input_personpropagation_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String sourcePath = parser.get("sourcePath"); + log.info("sourcePath: {}", sourcePath); + + final String workingPath = parser.get("outputPath"); + log.info("workingPath: {}", workingPath); + + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + + extractRelations( + spark, + sourcePath, + workingPath); + }); + } + + private static void extractRelations(SparkSession spark, String sourcePath, String workingPath) { + + Dataset> relationDataset = spark + .read() + .schema(Encoders.bean(Relation.class).schema()) + .json(sourcePath + "relation") + .as(Encoders.bean(Relation.class)) + .map( + (MapFunction>) r -> new Tuple2<>( + r.getSource() + r.getRelClass() + r.getTarget(), r), + Encoders.tuple(Encoders.STRING(), Encoders.bean(Relation.class))); + + ModelSupport.entityTypes + .keySet() + .stream() + .filter(ModelSupport::isResult) + .forEach( + e -> { + // 1. search for results having orcid_pending and orcid in the set of pids for the authors + Dataset resultWithOrcids = spark + .read() + .schema(Encoders.bean(Result.class).schema()) + .json(sourcePath + e.name()) + .as(Encoders.bean(Result.class)) + .filter( + (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && + !r.getDataInfo().getInvisible() && + Optional + .ofNullable(r.getAuthor()) + .isPresent()) + .filter( + (FilterFunction) r -> r + .getAuthor() + .stream() + .anyMatch( + a -> Optional + .ofNullable( + a + .getPid()) + .isPresent() && + a + .getPid() + .stream() + .anyMatch( + p -> Arrays + .asList("orcid", "orcid_pending") + .contains(p.getQualifier().getClassid().toLowerCase())))); + // 2. create authorship relations between the result identifier and the person entity with + // orcid_pending. + Dataset> newRelations = resultWithOrcids + .flatMap( + (FlatMapFunction) r -> getAuthorshipRelations(r), + Encoders.bean(Relation.class)) + .map( + (MapFunction>) r -> new Tuple2<>( + r.getSource() + r.getRelClass() + r.getTarget(), r), + Encoders.tuple(Encoders.STRING(), Encoders.bean(Relation.class))); + newRelations + .joinWith(relationDataset, newRelations.col("_1").equalTo(relationDataset.col("_1")), "left") + .map((MapFunction, Tuple2>, Relation>) t2 -> { + if (t2._2() == null) + return t2._1()._2(); + return null; + }, Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r != null) + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(workingPath); + + // 2.1 store in a separate location the relation between the person and the pids for the result? + + // 3. create co_authorship relations between the pairs of authors with orcid/orcid_pending pids + newRelations = resultWithOrcids + .map((MapFunction) r -> getAuthorsPidList(r), Encoders.bean(Coauthors.class)) + .flatMap( + (FlatMapFunction) c -> new CoAuthorshipIterator(c.getCoauthors()), + Encoders.bean(Relation.class)) + .groupByKey( + (MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction) (k, it) -> it.next(), + Encoders.bean(Relation.class)) + .map( + (MapFunction>) r -> new Tuple2<>( + r.getSource() + r.getRelClass() + r.getTarget(), r), + Encoders.tuple(Encoders.STRING(), Encoders.bean(Relation.class))); + newRelations + .joinWith(relationDataset, newRelations.col("_1").equalTo(relationDataset.col("_1")), "left") + .map((MapFunction, Tuple2>, Relation>) t2 -> { + if (t2._2() == null) + return t2._1()._2(); + return null; + }, Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r != null) + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(workingPath); + + }); + spark + .read() + .schema(Encoders.bean(Relation.class).schema()) + .json(workingPath) + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(sourcePath + "relation"); + + } + + private static Coauthors getAuthorsPidList(Result r) { + Coauthors coauth = new Coauthors(); + coauth + .setCoauthors( + r + .getAuthor() + .stream() + .filter( + a -> a + .getPid() + .stream() + .anyMatch( + p -> Arrays.asList("orcid", "orcid_pending").contains(p.getQualifier().getClassid()))) + .map(a -> { + Optional tmp = a + .getPid() + .stream() + .filter(p -> p.getQualifier().getClassid().equalsIgnoreCase("orcid")) + .findFirst(); + if (tmp.isPresent()) + return tmp.get().getValue(); + tmp = a + .getPid() + .stream() + .filter(p -> p.getQualifier().getClassid().equalsIgnoreCase("orcid_pending")) + .findFirst(); + if (tmp.isPresent()) + return tmp.get().getValue(); + + return null; + }) + .filter(Objects::nonNull) + .collect(Collectors.toList())); + return coauth; + + } + + private static Iterator getAuthorshipRelations(Result r) { + List relationList = new ArrayList<>(); + for (Author a : r.getAuthor()) + + relationList.addAll(a.getPid().stream().map(p -> { + + if (p.getQualifier().getClassid().equalsIgnoreCase("orcid_pending")) + return getRelation(p.getValue(), r.getId()); + return null; + }) + .filter(Objects::nonNull) + .collect(Collectors.toList())); + + return relationList.iterator(); + } + + private static Relation getRelation(String orcid, String resultId) { + + String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); + + Relation relation = OafMapperUtils + .getRelation( + source, resultId, ModelConstants.RESULT_PERSON_RELTYPE, + ModelConstants.RESULT_PERSON_SUBRELTYPE, + ModelConstants.RESULT_PERSON_HASAUTHORED, + null, // collectedfrom = null + DATAINFO, + null); + + return relation; + } + +} From c773421cc73531dce34ee869f5dce44296ea425c Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 14:44:13 +0200 Subject: [PATCH 166/239] [person] added new substep in propagation worflow main --- .../dnetlib/dhp/wf/main/oozie_app/import.txt | 3 ++- .../dhp/wf/main/oozie_app/workflow.xml | 19 ++++++++++++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/import.txt b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/import.txt index b20259414..8922b6ac6 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/import.txt +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/import.txt @@ -7,4 +7,5 @@ community_organization classpath eu/dnetlib/dhp/wf/subworkflows/resulttocommunit result_project classpath eu/dnetlib/dhp/wf/subworkflows/projecttoresult/oozie_app community_project classpath eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromproject/oozie_app community_sem_rel classpath eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/oozie_app -country_propagation classpath eu/dnetlib/dhp/wf/subworkflows/countrypropagation/oozie_app \ No newline at end of file +country_propagation classpath eu/dnetlib/dhp/wf/subworkflows/countrypropagation/oozie_app +person_propagation classpath eu/dnetlib/dhp/wf/subworkflows/person/oozie_app \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/workflow.xml index 8e91707b6..4351cd595 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/wf/main/oozie_app/workflow.xml @@ -122,6 +122,7 @@ ${wf:conf('resumeFrom') eq 'CommunityProject'} ${wf:conf('resumeFrom') eq 'CommunitySemanticRelation'} ${wf:conf('resumeFrom') eq 'CountryPropagation'} + ${wf:conf('resumeFrom') eq 'PersonPropagation'} @@ -291,10 +292,24 @@ + + + + + + ${wf:appPath()}/person_propagation + + + + + sourcePath + ${outputPath} + + + - ${wf:appPath()}/country_propagation @@ -319,6 +334,8 @@ + + From 01679c935a96c796a45eac96a7e15e5ea44f312d Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 15:27:06 +0200 Subject: [PATCH 167/239] [person] added test class to be implemented --- .../dhp/person/PersonPropagationJobTest.java | 95 +++++++++++++++++++ .../dhp/person/graph/dataset/part-00000 | 0 .../graph/otherresearchproduct/part-00000 | 0 .../dhp/person/graph/publication/part-00000 | 0 .../dhp/person/graph/relation/part-00000 | 1 + .../dhp/person/graph/software/part-00000 | 0 6 files changed, 96 insertions(+) create mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/person/PersonPropagationJobTest.java create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/dataset/part-00000 create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/software/part-00000 diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/person/PersonPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/person/PersonPropagationJobTest.java new file mode 100644 index 000000000..43f913d3d --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/person/PersonPropagationJobTest.java @@ -0,0 +1,95 @@ + +package eu.dnetlib.dhp.person; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; + +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +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.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob; +import scala.Tuple2; + +public class PersonPropagationJobTest { + + private static final Logger log = LoggerFactory.getLogger(PersonPropagationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(PersonPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(PersonPropagationJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(PersonPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testPersonPropagation() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/personpropagation/graph") + .getPath(); + + SparkExtractPersonRelations + .main( + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", sourcePath, + "--outputPath", workingDir.toString() + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + //TODO write assertions and find relevant information for hte resource files + } + + + +} diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/dataset/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/dataset/part-00000 new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 new file mode 100644 index 000000000..a17560e55 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 @@ -0,0 +1 @@ +{"clazz":"eu.dnetlib.dhp.schema.oaf.Relation","payload":{"collectedfrom":[{"key":"10|openaire____::806360c771262b4d6770e7cdf04b5c5a","value":"ORCID","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.91","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:crosswalk:orcid","classname":"Imported from ORCID","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"relType":"personPerson","subRelType":"coAuthorship","relClass":"hasCoAuthor","source":"30|orcid_______::028da52095190c6573d6bf9dba4c8ede","target":"30|orcid_______::8791a84ea413592878d6fe191f0ed35f","validated":true,"validationDate":null,"properties":[]}} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/software/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/software/part-00000 new file mode 100644 index 000000000..e69de29bb From a7699558ed38382618911b1d7bca62f9e738d36d Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 16:15:12 +0200 Subject: [PATCH 168/239] [person] - --- .../java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java | 2 ++ .../eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 | 1 + 2 files changed, 3 insertions(+) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java index 34bd976ea..6caeef478 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java @@ -140,6 +140,8 @@ public class SparkExtractPersonRelations { .flatMap( (FlatMapFunction) r -> getAuthorshipRelations(r), Encoders.bean(Relation.class)) +// .groupByKey((MapFunction) r-> r.getSource()+r.getTarget(), Encoders.STRING() ) +// .mapGroups((MapGroupsFunction) (k,it) -> it.next(), Encoders.bean(Relation.class) ) .map( (MapFunction>) r -> new Tuple2<>( r.getSource() + r.getRelClass() + r.getTarget(), r), diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 index e69de29bb..47a3fdccb 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/otherresearchproduct/part-00000 @@ -0,0 +1 @@ +{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "resourcetype": {"classid": "Taxonomic treatment", "classname": "Taxonomic treatment", "schemename": "dnet:dataCite_resource", "schemeid": "dnet:dataCite_resource"}, "pid": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.5281/zenodo.10249277"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.5281/zenodo.10249277"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.5281/zenodo.10249277"}], "bestaccessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "relevantdate": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "issued", "classname": "issued", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2023-11-28"}], "contributor": [], "id": "50|doi_________::fa6db8629c4a8d13ec21e445b309d1c8", "description": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "11.1 Saltia papposa (Forrsk.) Moq., Prodr. [A. P. de Candolle] 13(2): 325. 1849 \\u2261 Achyranthes papposa Forssk., Fl. Aegypt.-Arab.: 48. 1775. Lectotype (designate here): \\u2014 YEMEN. Zabid, s.d., Forssk\\u00e5l 205 (C10001569!, image of the lectotype available at https://plants.jstor. org/stable/viewer/10.5555/al.ap.specimen.c10001569?page=1); isolectotypes: C10001570! (image of the isolectotype available at https://plants.jstor.org/stable/viewer/10.5555/al.ap.specimen.c10001570?page=1) and BM000950560! (image of the isolectotype available at https://data.nhm.ac.uk/object/c634a45c-983a-42f3-9c4d-1d1b06f5f88b/1691539200000). Typification of the name Achyranthes papposa:\\u2014 Forssk\\u00e5l (1775: 48) published Achyranthes papposa by giving a short diagnosis (\\u201c foliis alternis; crassiusculis; lineari-cuneatis, obtusis \\u201d) and a detailed description; the provenance [\\u201c Zeb\\u00edd \\u201d (currently Zabid), a city of W-Yemen] is reported [see also Forssk\\u00e5l (1775: CVII) who indicated \\u201cMi.\\u201d as provenance of A. papposa, \\u201cMi.\\u201d meaning \\u201cMontium Regionis Inferior\\u201d (Forssk\\u00e5l 1775: CI)]. We traced two specimens at C, where Forsskal\\u2019s herbarium and types are mostly preserved (HUH Index of Botanists 2013c), i.e. viz. C10001569 and C10001570, both collected at Zabid; a further specimen is kept at BM (BM000950560) and it was annotated by Frank Nigel Hepper as an isotype. These three specimens are part of the original material for A. papposa (see also Hepper & Friis 1994). C10001569 bears a branch of a plant with more flowers than in C10001570. Since the morphology of the flowers is important to identify Saltia papposa (Townsend 1993), we here designate C10001569 as the lectotype of A. papposa. C10001570 and BM000950560 are isolectotypes. Chorology:\\u2015 Endemic to the Arabian Peninsula (Saudi Arabia and Yemen; POWO 2023). Occurrence in Saudi Arabia:\\u2015 Doubtfully in Makkah (Miller & Cope 1996). We did not trace any specimen collected in Saudi Arabia, but it is not impossible that Saltia papposa occurs in the country, probably in the south-eastern coastal area (Jazan?) (see also Ghazanfar & Fisher 2013: 178\\u2013179)."}], "eoscifguidelines": [], "author": [{"surname": "Hassan", "name": "Walaa A.", "pid": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "orcid_pending", "classname": "Open Researcher and Contributor ID", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "0000-0001-7605-9058"}], "rank": 1, "affiliation": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "Botany and Microbiology Department, Faculty of Science, Beni-Suef University, Beni-Suef, Egypt & azmeyw @ gmail. com; https: // orcid. org / 0000 - 0001 - 7605 - 9058"}], "fullname": "Hassan, Walaa A."}, {"surname": "Al-Shaye", "name": "Najla A.", "pid": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "orcid_pending", "classname": "Open Researcher and Contributor ID", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "0000-0002-0447-8613"}], "rank": 2, "affiliation": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "Department of Biology, College of Science, Princess Nourah bint Abdulrahman University, Riyadh, Saudi Arabia & naaalshaye @ pnu. edu. sa; https: // orcid. org / 0000 - 0002 - 0447 - 8613"}], "fullname": "Al-Shaye, Najla A."}, {"surname": "Iamonico", "name": "Duilio", "pid": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "orcid_pending", "classname": "Open Researcher and Contributor ID", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "0000-0001-5491-7568"}], "rank": 3, "affiliation": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "Department of Environmental Biology, Univeristy of Rome Sapienza, 00185 Rome, Italy & duilio. iamonico @ uniroma 1. it; https: // orcid. org / 0000 - 0001 - 5491 - 7568"}], "fullname": "Iamonico, Duilio"}], "contactgroup": [], "collectedfrom": [{"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}], "instance": [{"refereed": {"classid": "0002", "classname": "nonPeerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, "license": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "CC 0"}, "url": ["http://dx.doi.org/10.5281/zenodo.10249277", "http://treatment.plazi.org/id/97224201FFE29001FF4C6AB685F912EB"], "pid": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.5281/zenodo.10249277"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.5281/zenodo.10249277"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.5281/zenodo.10249277"}], "instanceTypeMapping": [{"originalType": "Taxonomic treatment", "vocabularyName": "openaire::coar_resource_types_3_1"}], "alternateIdentifier": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "oai", "classname": "Open Archives Initiative", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "oai:zenodo.org:10249277"}], "dateofacceptance": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "2023-11-28"}, "collectedfrom": {"value": "ZENODO", "key": "10|opendoar____::358aee4cc897452c00244351e4d91f69"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0020", "classname": "Other ORP type", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}], "dateofcollection": "2023-12-21T22:24:48+0000", "fulltext": [], "dateoftransformation": "2024-01-18T06:50:15.691Z", "dateofacceptance": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "2023-11-28"}, "format": [], "tool": [], "subject": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Tracheophyta"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Magnoliopsida"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Amaranthaceae"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Saltia"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Saltia papposa"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Biodiversity"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Plantae"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Caryophyllales"}, {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "keyword", "classname": "keyword", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "Taxonomy"}], "coverage": [], "externalReference": [], "publisher": {"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "value": "Zenodo"}, "lastupdatetimestamp": 1721832280654, "language": {"classid": "und", "classname": "Undetermined", "schemename": "dnet:languages", "schemeid": "dnet:languages"}, "resulttype": {"classid": "other", "classname": "other", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "country": [], "extraInfo": [], "originalId": ["oai:zenodo.org:10249277", "50|od______2659::42fc9730cd6f5de3b0e3bfacdc347177"], "contactperson": [], "source": [], "context": [], "title": [{"dataInfo": {"invisible": false, "trust": "0.9", "deletedbyinference": false, "inferred": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "Saltia papposa Moq., Prodr."}]} \ No newline at end of file From c93bf824875fb3ffdb2220ac62fc52495e17fa01 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 17:34:34 +0200 Subject: [PATCH 169/239] [affroNewModel] extended wf definition --- .../actionmanager/bipaffiliations/oozie_app/workflow.xml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml index 2e89c07fd..88ff42dc2 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml @@ -21,6 +21,10 @@ webCrawlInputPath the path where to find the inferred affiliation relations from webCrawl + + publisherInputPath + the path where to find the inferred affiliation relations from publisher websites + outputPath the path where to store the actionset @@ -99,7 +103,7 @@ yarn cluster - Produces the atomic action with the inferred by BIP! affiliation relations (from Crossref and Pubmed) + Produces the atomic action with the inferred by OpenAIRE affiliation relations eu.dnetlib.dhp.actionmanager.bipaffiliations.PrepareAffiliationRelations dhp-aggregation-${projectVersion}.jar @@ -117,6 +121,7 @@ --openapcInputPath${openapcInputPath} --dataciteInputPath${dataciteInputPath} --webCrawlInputPath${webCrawlInputPath} + --publisherInputPath${publisherInputPath} --outputPath${outputPath} From cab8f1135fd82d2721cb8051d44550ac24b0b3eb Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 17:44:33 +0200 Subject: [PATCH 170/239] [affroNewModel] - --- .../bipaffiliations/PrepareAffiliationRelations.java | 2 +- .../dhp/actionmanager/bipaffiliations/job.properties | 6 ++++-- .../bipaffiliations/PrepareAffiliationRelationsTest.java | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 61a018a41..15c1cc376 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -34,7 +34,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import scala.Tuple2; /** - * Creates action sets for Crossref affiliation relations inferred by BIP! + * Creates action sets for Crossref affiliation relations inferred by OpenAIRE */ public class PrepareAffiliationRelations implements Serializable { diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties index ded4fe409..58124c9d1 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties @@ -32,8 +32,10 @@ spark2SqlQueryExecutionListeners=com.cloudera.spark.lineage.NavigatorQueryListen oozie.wf.application.path=${oozieTopWfApplicationPath} crossrefInputPath=/data/bip-affiliations/crossref-data.json -pubmedInputPath=/data/bip-affiliations/pubmed-data.json +pubmedInputPath=/data/bip-affiliations/pubmed-data-v4.json openapcInputPath=/data/bip-affiliations/openapc-data.json dataciteInputPath=/data/bip-affiliations/datacite-data.json +webCrawlInputPath=/data/bip-affiliations/webCrawl +publisherInputPath=/data/bip-affiliations/publishers -outputPath=/tmp/crossref-affiliations-output-v5 +outputPath=/tmp/affRoAS diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index c704bb99b..16d60f7da 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -112,7 +112,7 @@ public class PrepareAffiliationRelationsTest { .map(aa -> ((Relation) aa.getPayload())); // count the number of relations - assertEquals(162, tmp.count());// 18 + 24 + 30 * 4 = + assertEquals(162, tmp.count());// 18 + 24 + 30 * 4 = Dataset dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); dataset.createOrReplaceTempView("result"); From 32f444984e3a6277c3ffc32f45408e4033b7ba3b Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 24 Oct 2024 17:51:42 +0200 Subject: [PATCH 171/239] [person] - --- .../resources/eu/dnetlib/dhp/person/graph/publication/part-00000 | 1 + .../resources/eu/dnetlib/dhp/person/graph/relation/part-00000 | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 index e69de29bb..af1b5d55c 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/publication/part-00000 @@ -0,0 +1 @@ +{"dataInfo": {"invisible": false, "trust": "0.91", "deletedbyinference": true, "inferred": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "Harvested", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}}, "resourcetype": {"classid": "publication", "classname": "publication", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.11646/phytotaxa.379.3.5"}], "bestaccessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "relevantdate": [{"qualifier": {"classid": "created", "classname": "created", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2018-11-29"}, {"qualifier": {"classid": "published-online", "classname": "published-online", "schemename": "dnet:dataCite_date", "schemeid": "dnet:dataCite_date"}, "value": "2018-11-29"}], "collectedfrom": [{"value": "Crossref", "key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"}], "id": "50|doi_________::b2eae15cfe9b0d7f416b6dcfc84c09f9", "description": [{"value": "As part of the ongoing studies on the genus Polycarpon Linnaeus (1759: 859, 881) (see e.g., Iamonico 2015a, 2015b, 2015c, Iamonico & Domina 2015), and on the Italian loci classici (see e.g., Peruzzi et al. 2015, Brundu et al. 2015, Domina et al. 2016, Di Gristina et al. 2017, Domina et al. 2017, 2018a, 2018b), we present here a note regarding Hagaea alsinifolia Bivona-Bernardi (1815: 7\\u20138) [currently accepted (see Bartolucci et al. 2018) as Polycarpon tetraphyllum Linnaeus (1759: 881) subsp. alsinifolium (Biv.) Ball (1877: 370)]."}], "lastupdatetimestamp": 1648743612067, "author": [{"surname": "IAMONICO", "fullname": "DUILIO IAMONICO", "pid": [], "name": "DUILIO", "rank": 1}, {"surname": "DOMINA", "fullname": "GIANNIANTONIO DOMINA", "pid": [], "name": "GIANNIANTONIO", "rank": 2}], "instance": [{"refereed": {"classid": "0001", "classname": "peerReviewed", "schemename": "dnet:review_levels", "schemeid": "dnet:review_levels"}, "hostedby": {"dataInfo": {"invisible": false, "deletedbyinference": false}, "value": "Phytotaxa", "key": "10|issn___print::9336d3bbf63c241b54726a55fa38c0ef"}, "url": ["https://doi.org/10.11646/phytotaxa.379.3.5"], "pid": [{"qualifier": {"classid": "doi", "classname": "Digital Object Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.11646/phytotaxa.379.3.5"}], "instanceTypeMapping": [{"originalType": "journal-article", "typeLabel": "research article", "vocabularyName": "openaire::coar_resource_types_3_1", "typeCode": "http://purl.org/coar/resource_type/c_2df8fbb1"}, {"originalType": "http://purl.org/coar/resource_type/c_2df8fbb1", "typeLabel": "Article", "vocabularyName": "openaire::user_resource_types", "typeCode": "Article"}], "dateofacceptance": {"value": "2018-11-29"}, "collectedfrom": {"value": "Crossref", "key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"}, "accessright": {"classid": "UNKNOWN", "classname": "not available", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}], "dateofcollection": "2024-07-26T02:32:47.105", "metaResourceType": {"classid": "Research Literature", "classname": "Research Literature", "schemename": "openaire::meta_resource_types", "schemeid": "openaire::meta_resource_types"}, "context": [], "journal": {"issnPrinted": "1179-3155", "vol": "379", "sp": "267", "issnOnline": "1179-3163", "name": "Phytotaxa"}, "subject": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "0106 biological sciences"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "0301 basic medicine"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "03 medical and health sciences"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "03010801 Mycology/Symbiosis"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "030108 mycology & parasitology"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "010603 evolutionary biology"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": ""}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "01 natural sciences"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "subject:fos", "classname": "Inferred by OpenAIRE", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "update", "invisible": false, "trust": "0.5"}, "qualifier": {"classid": "FOS", "classname": "Fields of Science and Technology classification", "schemename": "dnet:subject_classification_typologies", "schemeid": "dnet:subject_classification_typologies"}, "value": "01060304 Pollination/Angiosperms"}], "externalReference": [], "publisher": {"value": "Magnolia Press"}, "eoscifguidelines": [], "language": {"classid": "und", "classname": "Undetermined", "schemename": "dnet:languages", "schemeid": "dnet:languages"}, "resulttype": {"classid": "publication", "classname": "publication", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "country": [], "title": [{"qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "Epitypification of Hagaea alsinifolia (Polyycarpon tetraphyllum subsp. Alsinifolium, Caryophyllaceae)"}], "originalId": ["10.11646/phytotaxa.379.3.5", "50|doiboost____|b2eae15cfe9b0d7f416b6dcfc84c09f9"], "source": [{"value": "Crossref"}], "dateofacceptance": {"value": "2018-11-29"}} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 index a17560e55..e69de29bb 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/person/graph/relation/part-00000 @@ -1 +0,0 @@ -{"clazz":"eu.dnetlib.dhp.schema.oaf.Relation","payload":{"collectedfrom":[{"key":"10|openaire____::806360c771262b4d6770e7cdf04b5c5a","value":"ORCID","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.91","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:crosswalk:orcid","classname":"Imported from ORCID","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"relType":"personPerson","subRelType":"coAuthorship","relClass":"hasCoAuthor","source":"30|orcid_______::028da52095190c6573d6bf9dba4c8ede","target":"30|orcid_______::8791a84ea413592878d6fe191f0ed35f","validated":true,"validationDate":null,"properties":[]}} \ No newline at end of file From e75326d6ec712d534c34efd50a48c5a18a7a358a Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 25 Oct 2024 09:13:54 +0200 Subject: [PATCH 172/239] [FundersMatchFromCrossref] added match from CrossRef to DFG unidentified project --- .../scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index d8292a631..824c7ff52 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -560,7 +560,11 @@ case object Crossref2Oaf { "10.13039/501100000266" | "10.13039/501100006041" | "10.13039/501100000265" | "10.13039/501100000270" | "10.13039/501100013589" | "10.13039/501100000271" => generateSimpleRelationFromAward(funder, "ukri________", a => a) - + //DFG + case "10.13039/501100001659" => + val targetId = getProjectId("dfgf________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) case _ => logger.debug("no match for " + funder.DOI.get) } From 842cc75dae0b11bcc0f4974cf6fe199813f7696d Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 25 Oct 2024 09:42:52 +0200 Subject: [PATCH 173/239] [AffRo] fix name --- .../dhp/actionmanager/bipaffiliations/job.properties | 12 ++++++------ .../bipaffiliations/oozie_app/workflow.xml | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties index 58124c9d1..c61830cba 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/job.properties @@ -31,11 +31,11 @@ spark2SqlQueryExecutionListeners=com.cloudera.spark.lineage.NavigatorQueryListen # The following is needed as a property of a workflow oozie.wf.application.path=${oozieTopWfApplicationPath} -crossrefInputPath=/data/bip-affiliations/crossref-data.json -pubmedInputPath=/data/bip-affiliations/pubmed-data-v4.json -openapcInputPath=/data/bip-affiliations/openapc-data.json -dataciteInputPath=/data/bip-affiliations/datacite-data.json -webCrawlInputPath=/data/bip-affiliations/webCrawl -publisherInputPath=/data/bip-affiliations/publishers +crossrefInputPath=/data/openaire-affiliations/crossref-data.json +pubmedInputPath=/data/openaire-affiliations/pubmed-data-v4.json +openapcInputPath=/data/openaire-affiliations/openapc-data.json +dataciteInputPath=/data/openaire-affiliations/datacite-data.json +webCrawlInputPath=/data/openaire-affiliations/webCrawl +publisherInputPath=/data/openaire-affiliations/publishers outputPath=/tmp/affRoAS diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml index 88ff42dc2..2e65aaa5e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/bipaffiliations/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + From 1fce7d5a0f467b478993d408c1103dbd7d895acf Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 25 Oct 2024 10:05:17 +0200 Subject: [PATCH 174/239] [Person] remove the isolated nodes from the person set --- .../person/SparkExtractPersonRelations.java | 28 +++++++++++++++++-- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java index 6caeef478..3892498df 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/person/SparkExtractPersonRelations.java @@ -16,10 +16,8 @@ import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.sql.*; 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; @@ -84,9 +82,33 @@ public class SparkExtractPersonRelations { spark, sourcePath, workingPath); + removeIsolatedPerson(spark,sourcePath, workingPath); }); } + private static void removeIsolatedPerson(SparkSession spark, String sourcePath, String workingPath) { + Dataset personDataset = spark.read().schema(Encoders.bean(Person.class).schema()) + .json(sourcePath + "person") + .as(Encoders.bean(Person.class)); + + Dataset relationDataset = spark.read().schema(Encoders.bean(Relation.class).schema()) + .json(sourcePath + "relation") + .as(Encoders.bean(Relation.class)); + + personDataset.join(relationDataset, personDataset.col("id").equalTo(relationDataset.col("source")), "left_semi") + .write() + .option("compression","gzip") + .mode(SaveMode.Overwrite) + .json(workingPath + "person"); + + spark.read().schema(Encoders.bean(Person.class).schema()) + .json(workingPath + "person") + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .json(sourcePath + "person"); + } + private static void extractRelations(SparkSession spark, String sourcePath, String workingPath) { Dataset> relationDataset = spark From 0fb6af5586ac0532745bd9fde1347b1c972fcf8a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 29 Jan 2024 18:12:33 +0100 Subject: [PATCH 175/239] Updated main pom dependency against dhp-schema, from 8.0.1 to 9.0.0. The new fields included in the updated schema module are populated by the Solr JSON payload mapping, which also limits the number of authors serialised to 200. --- .../dhp/schema/oaf/utils/ModelHardLimits.java | 1 + .../model/ProvisionModelSupport.java | 19 ++++++++++++++++++- pom.xml | 2 +- 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java index 36d138ba1..74cd1b42a 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/ModelHardLimits.java @@ -12,6 +12,7 @@ public class ModelHardLimits { public static final int MAX_EXTERNAL_ENTITIES = 50; public static final int MAX_AUTHORS = 200; + public static final int MAX_RELATED_AUTHORS = 20; public static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000; public static final int MAX_TITLE_LENGTH = 5000; public static final int MAX_TITLES = 10; diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 4a2326453..bc02b595f 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,6 +5,7 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -150,6 +151,12 @@ public class ProvisionModelSupport { rr.setPublisher(re.getPublisher()); rr.setResulttype(mapQualifier(re.getResulttype())); rr.setTitle(Optional.ofNullable(re.getTitle()).map(StructuredProperty::getValue).orElse(null)); + rr.setDescription(StringUtils.left(re.getDescription(), ModelHardLimits.MAX_RELATED_ABSTRACT_LENGTH)); + rr.setAuthor(Optional.ofNullable(re.getAuthor()) + .map(aa -> aa.stream() + .limit(ModelHardLimits.MAX_RELATED_AUTHORS) + .collect(Collectors.toList())) + .orElse(null)); if (relation.getValidated() == null) { relation.setValidated(false); @@ -378,6 +385,7 @@ public class ProvisionModelSupport { rs.setPubliclyFunded(r.getPubliclyFunded()); rs.setTransformativeAgreement(r.getTransformativeAgreement()); rs.setExternalReference(mapExternalReference(r.getExternalReference())); + rs.setBestinstancetype(mapQualifier(r.getBestInstancetype())); rs.setInstance(mapInstances(r.getInstance())); if (r instanceof Publication) { @@ -667,14 +675,23 @@ public class ProvisionModelSupport { } private static List asAuthor(List authorList) { + return asAuthor(authorList, ModelHardLimits.MAX_AUTHORS); + } + + private static List asAuthor(List authorList, int maxAuthors) { return Optional .ofNullable(authorList) .map( authors -> authors .stream() + .limit(maxAuthors) .map( a -> Author - .newInstance(a.getFullname(), a.getName(), a.getSurname(), a.getRank(), asPid(a.getPid()))) + .newInstance( + StringUtils.left(a.getFullname(), ModelHardLimits.MAX_AUTHOR_FULLNAME_LENGTH), + a.getName(), + a.getSurname(), + a.getRank(), asPid(a.getPid()))) .collect(Collectors.toList())) .orElse(null); } diff --git a/pom.xml b/pom.xml index e1d99f25b..9480ddfc0 100644 --- a/pom.xml +++ b/pom.xml @@ -937,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [8.0.1] + [9.0.0] cdh5.9.2 3.5 11.0.2 From 32fa579b809138f9ae5d52d661423b1cebf4fbe4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 28 Oct 2024 10:03:02 +0100 Subject: [PATCH 176/239] [graph provision] select the longest abstract --- .../dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java index 63f3c2ead..add1c80fa 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import java.util.Comparator; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -167,8 +168,9 @@ public class CreateRelatedEntitiesJob_phase1 { result .getDescription() .stream() - .findFirst() + .filter(d -> Objects.nonNull(d.getValue())) .map(Field::getValue) + .max(Comparator.comparingInt(String::length)) .ifPresent( d -> re.setDescription(StringUtils.left(d, ModelHardLimits.MAX_RELATED_ABSTRACT_LENGTH))); } From 6fd50266f11c39d7b71600d8da9fc010da4037e5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 28 Oct 2024 10:42:46 +0100 Subject: [PATCH 177/239] translate 'otherresearchproduct' into 'other' when setting the related record type --- .../dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index bc02b595f..69aa940c9 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -109,7 +109,7 @@ public class ProvisionModelSupport { RelatedRecord rr = new RelatedRecord(); final RelatedEntity re = rew.getTarget(); - final RecordType relatedRecordType = RecordType.valueOf(re.getType()); + final RecordType relatedRecordType = RecordType.fromString(re.getType()); final Relation relation = rew.getRelation(); final String relationProvenance = Optional .ofNullable(relation.getDataInfo()) From 5916346ba17cfcbc8a9ba378e4f6f1972c1b729e Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 28 Oct 2024 12:18:50 +0100 Subject: [PATCH 178/239] [TransformativeAgreement] fix to remove the file downloaded from a previous run of the workflow --- .../transformativeagreement/oozie_app/workflow.xml | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/transformativeagreement/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/transformativeagreement/oozie_app/workflow.xml index 0c5b1c119..a11e3350e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/transformativeagreement/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/transformativeagreement/oozie_app/workflow.xml @@ -24,7 +24,7 @@ - ${wf:conf('resumeFrom') eq 'DownloadDump'} + ${wf:conf('resumeFrom') eq 'DownloadDump'} @@ -33,6 +33,14 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + ${jobTracker} From 56224e034ab1b19bfc21d5dab38ea49d6da62529 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Mon, 28 Oct 2024 12:05:56 +0100 Subject: [PATCH 179/239] Fill the new mergedIds field when generating dedup records Filter out dedup records composed of invisible records only Filter out mergerels that have not been used when creating the dedup record (ungrouping of cliques) --- .../dhp/oa/dedup/DedupRecordFactory.java | 15 ++++++-- .../dhp/oa/dedup/SparkCreateDedupRecord.java | 36 +++++++++++++++++-- pom.xml | 2 +- 3 files changed, 47 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index 36ed4d7c1..44482cfdb 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -2,14 +2,13 @@ package eu.dnetlib.dhp.oa.dedup; import java.util.*; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.beanutils.BeanUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.FlatMapGroupsFunction; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.sql.*; import eu.dnetlib.dhp.oa.dedup.model.Identifier; @@ -107,6 +106,8 @@ public class DedupRecordFactory { final HashSet acceptanceDate = new HashSet<>(); + boolean isVisible = false; + while (it.hasNext()) { Tuple3 t = it.next(); OafEntity entity = t._3(); @@ -114,6 +115,7 @@ public class DedupRecordFactory { if (entity == null) { aliases.add(t._2()); } else { + isVisible = isVisible || !entity.getDataInfo().getInvisible(); cliques.add(entity); if (acceptanceDate.size() < MAX_ACCEPTANCE_DATE) { @@ -129,13 +131,20 @@ public class DedupRecordFactory { } - if (acceptanceDate.size() >= MAX_ACCEPTANCE_DATE || cliques.isEmpty()) { + if (!isVisible || acceptanceDate.size() >= MAX_ACCEPTANCE_DATE || cliques.isEmpty()) { return Collections.emptyIterator(); } OafEntity mergedEntity = MergeUtils.mergeGroup(dedupId, cliques.iterator()); // dedup records do not have date of transformation attribute mergedEntity.setDateoftransformation(null); + mergedEntity + .setMergedIds( + Stream + .concat(cliques.stream().map(OafEntity::getId), aliases.stream()) + .distinct() + .sorted() + .collect(Collectors.toList())); return Stream .concat( diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java index 6989ec54b..6f5f40e43 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java @@ -5,11 +5,11 @@ import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PROVENANCE_ACTION import static eu.dnetlib.dhp.schema.common.ModelConstants.PROVENANCE_DEDUP; import java.io.IOException; +import java.util.Arrays; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.*; import org.dom4j.DocumentException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -17,6 +17,7 @@ import org.xml.sax.SAXException; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.OafEntity; @@ -25,6 +26,7 @@ import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.pace.config.DedupConfig; +import scala.collection.JavaConverters; public class SparkCreateDedupRecord extends AbstractSparkAction { @@ -85,6 +87,36 @@ public class SparkCreateDedupRecord extends AbstractSparkAction { .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputPath); + + log.info("Updating mergerels for: '{}'", subEntity); + final Dataset dedupIds = spark + .read() + .schema("`id` STRING, `mergedIds` ARRAY") + .json(outputPath) + .selectExpr("id as source", "explode(mergedIds) as target"); + spark + .read() + .load(mergeRelPath) + .where("relClass == 'merges'") + .join(dedupIds, JavaConverters.asScalaBuffer(Arrays.asList("source", "target")).toSeq(), "left_semi") + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .save(workingPath + "/mergerel_filtered"); + + final Dataset validRels = spark.read().load(workingPath + "/mergerel_filtered"); + + final Dataset filteredMergeRels = validRels + .union( + validRels + .withColumnRenamed("source", "source_tmp") + .withColumnRenamed("target", "target_tmp") + .withColumn("relClass", functions.lit(ModelConstants.IS_MERGED_IN)) + .withColumnRenamed("target_tmp", "source") + .withColumnRenamed("source_tmp", "target")); + + saveParquet(filteredMergeRels, mergeRelPath, SaveMode.Overwrite); + removeOutputDir(spark, workingPath + "/mergerel_filtered"); } } diff --git a/pom.xml b/pom.xml index e1d99f25b..9480ddfc0 100644 --- a/pom.xml +++ b/pom.xml @@ -937,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [8.0.1] + [9.0.0] cdh5.9.2 3.5 11.0.2 From e6ca382debc8c9ee96e008bfa71a6ebd4083a76a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 28 Oct 2024 13:52:06 +0100 Subject: [PATCH 180/239] using scala 2.11 converters --- .../java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java index 6f5f40e43..29394cb12 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java @@ -98,7 +98,9 @@ public class SparkCreateDedupRecord extends AbstractSparkAction { .read() .load(mergeRelPath) .where("relClass == 'merges'") - .join(dedupIds, JavaConverters.asScalaBuffer(Arrays.asList("source", "target")).toSeq(), "left_semi") + .join( + dedupIds, JavaConverters.asScalaBufferConverter(Arrays.asList("source", "target")).asScala(), + "left_semi") .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") From 9b4415cb674c5fedfa8644a035c2bfc66414dfe2 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 28 Oct 2024 13:56:25 +0100 Subject: [PATCH 181/239] using _the right_ scala 2.11 converters --- .../java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java index 29394cb12..bd17d05eb 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java @@ -26,6 +26,7 @@ import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.pace.config.DedupConfig; +import scala.collection.JavaConversions; import scala.collection.JavaConverters; public class SparkCreateDedupRecord extends AbstractSparkAction { @@ -98,9 +99,7 @@ public class SparkCreateDedupRecord extends AbstractSparkAction { .read() .load(mergeRelPath) .where("relClass == 'merges'") - .join( - dedupIds, JavaConverters.asScalaBufferConverter(Arrays.asList("source", "target")).asScala(), - "left_semi") + .join(dedupIds, JavaConversions.asScalaBuffer(Arrays.asList("source", "target")), "left_semi") .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") From e4504fd98de52f20b201ff3985a18792241751f1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 28 Oct 2024 15:32:09 +0100 Subject: [PATCH 182/239] [Person] fixed project identifier creation --- .../personentity/ExtractPerson.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 1131f85e9..bf2c19c3d 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -25,6 +25,7 @@ import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.*; import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,10 +44,7 @@ import eu.dnetlib.dhp.common.person.Coauthors; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Person; -import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner; @@ -70,7 +68,11 @@ public class ExtractPerson implements Serializable { private static final String PMCID_PREFIX = "50|pmcid_______::"; private static final String ROR_PREFIX = "20|ror_________::"; - private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______"; + private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + + IdentifierFactory.ID_PREFIX_SEPARATOR + ModelConstants.ORCID + "_______"; + private static final String PROJECT_ID_PREFIX = ModelSupport.getIdPrefix(Project.class) + + IdentifierFactory.ID_PREFIX_SEPARATOR; + public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid"; public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID"; public static final String FUNDER_AUTHORS_CLASSID = "sysimport:crosswalk:funderdatabase"; @@ -173,7 +175,7 @@ public class ExtractPerson implements Serializable { } } catch (IOException e) { - e.printStackTrace(); + throw new RuntimeException(e); } } @@ -191,7 +193,7 @@ public class ExtractPerson implements Serializable { private static Relation getProjectRelation(String project, String orcid, String role) { String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); - String target = project.substring(0, 14) + String target = PROJECT_ID_PREFIX + project.substring(0, 14) + IdentifierFactory.md5(project.substring(15)); List properties = new ArrayList<>(); From 499892b67c4549bd0cebe6bc07873a4e553eda3a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 29 Oct 2024 09:51:30 +0100 Subject: [PATCH 183/239] [graph raw] rule out empty PIDs --- .../graph/raw/AbstractMdRecordToOafMapper.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index a85f47d99..2436a272c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -657,13 +657,21 @@ public abstract class AbstractMdRecordToOafMapper { final Node n = (Node) o; final String classId = n.valueOf(xpathClassId).trim(); if (this.vocs.termExists(schemeId, classId)) { - res - .add( - HashableStructuredProperty - .newInstance(n.getText(), this.vocs.getTermAsQualifier(schemeId, classId), info)); + final String value = n.getText(); + if (StringUtils.isNotBlank(value)) { + res + .add( + HashableStructuredProperty + .newInstance(value, this.vocs.getTermAsQualifier(schemeId, classId), info)); + } } } - return Lists.newArrayList(res); + return res + .stream() + .filter(Objects::nonNull) + .filter(p -> StringUtils.isNotBlank(p.getValue())) + .filter(p -> StringUtils.isNotBlank(p.getValue().trim())) + .collect(Collectors.toList()); } protected List prepareListStructProps( From 5ca031c8d6a780b46adc2b1b0b4a7a5307cf9c73 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 29 Oct 2024 13:48:41 +0100 Subject: [PATCH 184/239] [graph raw] rule out empty PIDs --- .../java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java | 1 + .../main/java/eu/dnetlib/dhp/schema/oaf/utils/PidCleaner.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java index 2c77c3b37..da245d67c 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/IdentifierFactory.java @@ -204,6 +204,7 @@ public class IdentifierFactory implements Serializable { .map( pp -> pp .stream() + .filter(p -> StringUtils.isNotBlank(p.getValue())) // filter away PIDs provided by a DS that is not considered an authority for the // given PID Type .filter(p -> shouldFilterPidByCriteria(collectedFrom, p, mapHandles)) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidCleaner.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidCleaner.java index 114c2b3af..678ed71dd 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidCleaner.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidCleaner.java @@ -26,7 +26,7 @@ public class PidCleaner { String value = Optional .ofNullable(pidValue) .map(String::trim) - .orElseThrow(() -> new IllegalArgumentException("PID value cannot be empty")); + .orElseThrow(() -> new IllegalArgumentException("PID (" + pidType + ") value cannot be empty")); switch (pidType) { From 69aee609ef32c6e8121e08042b25d779f8a40dcb Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 29 Oct 2024 15:53:04 +0100 Subject: [PATCH 185/239] [bulktag] align type to community api --- .../eu/dnetlib/dhp/api/model/CommunityContentprovider.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java index 9fab5a80c..8e0ea598c 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java @@ -13,13 +13,13 @@ public class CommunityContentprovider { private String openaireId; private SelectionConstraints selectioncriteria; - private String enabled; + private Boolean enabled; - public String getEnabled() { + public Boolean getEnabled() { return enabled; } - public void setEnabled(String enabled) { + public void setEnabled(Boolean enabled) { this.enabled = enabled; } From 323c76eafca0e992c7b7dd749fd31fdc2eb840e2 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 30 Oct 2024 07:35:30 +0100 Subject: [PATCH 186/239] patch relations job: removed non necessary logging --- .../eu/dnetlib/dhp/oa/graph/raw/PatchRelationsApplication.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/PatchRelationsApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/PatchRelationsApplication.java index 615b4a824..dce6cd39d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/PatchRelationsApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/PatchRelationsApplication.java @@ -80,9 +80,6 @@ public class PatchRelationsApplication { final Dataset rels = readPath(spark, relationPath, Relation.class); final Dataset idMapping = readPath(spark, idMappingPath, RelationIdMapping.class); - log.info("relations: {}", rels.count()); - log.info("idMapping: {}", idMapping.count()); - final Dataset bySource = rels .joinWith(idMapping, rels.col("source").equalTo(idMapping.col("oldId")), "left") .map((MapFunction, Relation>) t -> { From 26cdc7e439e8035a31a21f4895a4d521aa85d1e0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 30 Oct 2024 07:35:47 +0100 Subject: [PATCH 187/239] Avoid NPEs in MergeUtils --- .../main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index e01813110..79a12d630 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -715,7 +715,9 @@ public class MergeUtils { private static String spKeyExtractor(StructuredProperty sp) { return Optional .ofNullable(sp) - .map(s -> Joiner.on("||").join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) + .map(s -> Joiner.on("||") + .useForNull("") + .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) .orElse(null); } From a877c76d70a10afdd888f84a33cbe683c2f78755 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 30 Oct 2024 11:24:25 +0100 Subject: [PATCH 188/239] make MergeUtils.selectOldestDate less prone to errors when receiving invalid date formats --- .../schema/oaf/utils/GraphCleaningFunctions.java | 2 +- .../dnetlib/dhp/schema/oaf/utils/MergeUtils.java | 15 +++++---------- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index fdfd63a15..b6574da16 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -810,7 +810,7 @@ public class GraphCleaningFunctions extends CleaningFunctions { return author; } - private static Optional cleanDateField(Field dateofacceptance) { + public static Optional cleanDateField(Field dateofacceptance) { return Optional .ofNullable(dateofacceptance) .map(Field::getValue) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index 79a12d630..4c411a155 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -654,16 +654,9 @@ public class MergeUtils { } private static Field selectOldestDate(Field d1, Field d2) { - if (d1 == null || StringUtils.isBlank(d1.getValue())) { + if (!GraphCleaningFunctions.cleanDateField(d1).isPresent()) { return d2; - } else if (d2 == null || StringUtils.isBlank(d2.getValue())) { - return d1; - } - - if (StringUtils.contains(d1.getValue(), "null")) { - return d2; - } - if (StringUtils.contains(d2.getValue(), "null")) { + } else if (!GraphCleaningFunctions.cleanDateField(d2).isPresent()) { return d1; } @@ -715,7 +708,9 @@ public class MergeUtils { private static String spKeyExtractor(StructuredProperty sp) { return Optional .ofNullable(sp) - .map(s -> Joiner.on("||") + .map( + s -> Joiner + .on("||") .useForNull("") .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) .orElse(null); From a42c8b7c8541d3225d5d087469af9b7ff5fb9aa7 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 30 Oct 2024 11:25:17 +0100 Subject: [PATCH 189/239] person table directory produced by the workflows raw_all and merge graphs --- .../dhp/oa/graph/merge/oozie_app/workflow.xml | 28 +++++++++++++++++++ .../oa/graph/raw_all/oozie_app/workflow.xml | 27 ++++++++++++++++++ 2 files changed, 55 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml index a8d0d5068..3444e3afe 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml @@ -68,6 +68,7 @@ + @@ -260,6 +261,33 @@ + + + yarn + cluster + Merge person + eu.dnetlib.dhp.oa.graph.merge.MergeGraphTableSparkJob + dhp-graph-mapper-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --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 + + --betaInputPath${betaInputGraphPath}/person + --prodInputPath${prodInputGraphPath}/person + --outputPath${graphOutputPath}/person + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Person + --priority${priority} + + + + + yarn 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 ff927fe52..1b3cb1111 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 @@ -649,6 +649,7 @@ + @@ -860,6 +861,32 @@ + + + yarn + cluster + MergeClaims_person + 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${graphOutputPath} + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Person + + + + + From a8ed5a3b048b560a6c2b794834e27e4adcdcac97 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Mon, 4 Nov 2024 17:45:28 +0100 Subject: [PATCH 190/239] Organized getters and setters in the PMArticle class for better readability and maintainability. --- dhp-shade-package/pom.xml | 122 +++---- .../personentity/ExtractPerson.java | 8 +- .../dnetlib/dhp/sx/bio/pubmed/PMArticle.java | 312 +++++------------- .../sx/bio/ebi/baseline_to_oaf_params.json | 7 +- .../sx/bio/ebi/SparkCreatePubmedDump.scala | 90 +++++ .../dnetlib/dhp/sx/bio/pubmed/PMParser2.scala | 264 +++++++++++++++ .../dhp/sx/graph/bio/single_pubmed.xml | 222 +++++++++++++ .../dnetlib/dhp/sx/bio/BioScholixTest.scala | 28 +- 8 files changed, 754 insertions(+), 299 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala create mode 100644 dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml diff --git a/dhp-shade-package/pom.xml b/dhp-shade-package/pom.xml index d8e17ed46..c4f9b262e 100644 --- a/dhp-shade-package/pom.xml +++ b/dhp-shade-package/pom.xml @@ -26,16 +26,16 @@ - - eu.dnetlib.dhp - dhp-actionmanager - ${project.version} - - + + + eu.dnetlib.dhp + dhp-aggregation + ${project.version} + @@ -56,61 +56,61 @@ - - eu.dnetlib.dhp - dhp-graph-mapper - ${project.version} - - - eu.dnetlib.dhp - dhp-graph-provision - ${project.version} - - - eu.dnetlib.dhp - dhp-impact-indicators - ${project.version} - - - eu.dnetlib.dhp - dhp-stats-actionsets - ${project.version} - - - eu.dnetlib.dhp - dhp-stats-hist-snaps - ${project.version} - - - eu.dnetlib.dhp - dhp-stats-monitor-irish - ${project.version} - - - eu.dnetlib.dhp - dhp-stats-promote - ${project.version} - - - eu.dnetlib.dhp - dhp-stats-update - ${project.version} - - - eu.dnetlib.dhp - dhp-swh - ${project.version} - - - eu.dnetlib.dhp - dhp-usage-raw-data-update - ${project.version} - - - eu.dnetlib.dhp - dhp-usage-stats-build - ${project.version} - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index bf2c19c3d..db31bb43f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -15,6 +15,7 @@ import java.util.stream.Collectors; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -29,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.spark_project.jetty.util.StringUtil; import com.fasterxml.jackson.databind.ObjectMapper; @@ -206,7 +206,7 @@ public class ExtractPerson implements Serializable { null); relation.setValidated(true); - if (StringUtil.isNotBlank(role)) { + if (StringUtils.isNotBlank(role)) { KeyValue kv = new KeyValue(); kv.setKey("role"); kv.setValue(role); @@ -439,13 +439,13 @@ public class ExtractPerson implements Serializable { null); relation.setValidated(true); - if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtil.isNotBlank(row.getStartDate())) { + if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtils.isNotBlank(row.getStartDate())) { KeyValue kv = new KeyValue(); kv.setKey("startDate"); kv.setValue(row.getStartDate()); properties.add(kv); } - if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtil.isNotBlank(row.getEndDate())) { + if (Optional.ofNullable(row.getEndDate()).isPresent() && StringUtils.isNotBlank(row.getEndDate())) { KeyValue kv = new KeyValue(); kv.setKey("endDate"); kv.setValue(row.getEndDate()); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMArticle.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMArticle.java index 3fb814606..6191f6446 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMArticle.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMArticle.java @@ -8,259 +8,115 @@ import java.util.List; /** * This class represent an instance of Pubmed Article extracted from the native XML * - * @author Sandro La Bruzzo */ - public class PMArticle implements Serializable { - /** - * the Pubmed Identifier - */ private String pmid; - private String pmcId; - - /** - * the DOI - */ private String doi; - /** - * the Pubmed Date extracted from Specifies a date significant to either the article's history or the citation's processing. - * All dates will have a , , and elements. Some may have an , , and element(s). - */ private String date; - /** - * This is an 'envelop' element that contains various elements describing the journal cited; i.e., ISSN, Volume, Issue, and PubDate and author name(s), however, it does not contain data itself. - */ private PMJournal journal; - /** - * The full journal title (taken from NLM cataloging data following NLM rules for how to compile a serial name) is exported in this element. Some characters that are not part of the NLM MEDLINE/PubMed Character Set reside in a relatively small number of full journal titles. The NLM journal title abbreviation is exported in the element. - */ private String title; - /** - * English-language abstracts are taken directly from the published article. - * If the article does not have a published abstract, the National Library of Medicine does not create one, - * thus the record lacks the and elements. However, in the absence of a formally - * labeled abstract in the published article, text from a substantive "summary", "summary and conclusions" or "conclusions and summary" may be used. - */ private String description; - /** - * the language in which an article was published is recorded in . - * All entries are three letter abbreviations stored in lower case, such as eng, fre, ger, jpn, etc. When a single - * record contains more than one language value the XML export program extracts the languages in alphabetic order by the 3-letter language value. - * Some records provided by collaborating data producers may contain the value und to identify articles whose language is undetermined. - */ private String language; - - /** - * NLM controlled vocabulary, Medical Subject Headings (MeSH®), is used to characterize the content of the articles represented by MEDLINE citations. * - */ - private final List subjects = new ArrayList<>(); - /** - * This element is used to identify the type of article indexed for MEDLINE; - * it characterizes the nature of the information or the manner in which it is conveyed as well as the type of - * research support received (e.g., Review, Letter, Retracted Publication, Clinical Conference, Research Support, N.I.H., Extramural). - */ - private final List publicationTypes = new ArrayList<>(); - /** - * Personal and collective (corporate) author names published with the article are found in . - */ + private List subjects; + private List publicationTypes = new ArrayList<>(); private List authors = new ArrayList<>(); + private List grants = new ArrayList<>(); - /** - * contains the research grant or contract number (or both) that designates financial support by any agency of the United States Public Health Service - * or any institute of the National Institutes of Health. Additionally, beginning in late 2005, grant numbers are included for many other US and non-US funding agencies and organizations. - */ - private final List grants = new ArrayList<>(); - - /** - * get the DOI - * @return a DOI - */ - public String getDoi() { - return doi; - } - - /** - * Set the DOI - * @param doi a DOI - */ - public void setDoi(String doi) { - this.doi = doi; - } - - /** - * get the Pubmed Identifier - * @return the PMID - */ public String getPmid() { return pmid; } - /** - * set the Pubmed Identifier - * @param pmid the Pubmed Identifier - */ public void setPmid(String pmid) { this.pmid = pmid; } - /** - * the Pubmed Date extracted from Specifies a date significant to either the article's history or the citation's processing. - * All dates will have a , , and elements. Some may have an , , and element(s). - * - * @return the Pubmed Date - */ - public String getDate() { - return date; - } - - /** - * Set the pubmed Date - * @param date - */ - public void setDate(String date) { - this.date = date; - } - - /** - * The full journal title (taken from NLM cataloging data following NLM rules for how to compile a serial name) is exported in this element. - * Some characters that are not part of the NLM MEDLINE/PubMed Character Set reside in a relatively small number of full journal titles. - * The NLM journal title abbreviation is exported in the element. - * - * @return the pubmed Journal Extracted - */ - public PMJournal getJournal() { - return journal; - } - - /** - * Set the mapped pubmed Journal - * @param journal - */ - public void setJournal(PMJournal journal) { - this.journal = journal; - } - - /** - * contains the entire title of the journal article. is always in English; - * those titles originally published in a non-English language and translated for are enclosed in square brackets. - * All titles end with a period unless another punctuation mark such as a question mark or bracket is present. - * Explanatory information about the title itself is enclosed in parentheses, e.g.: (author's transl). - * Corporate/collective authors may appear at the end of for citations up to about the year 2000. - * - * @return the extracted pubmed Title - */ - public String getTitle() { - return title; - } - - /** - * set the pubmed title - * @param title - */ - public void setTitle(String title) { - this.title = title; - } - - /** - * English-language abstracts are taken directly from the published article. - * If the article does not have a published abstract, the National Library of Medicine does not create one, - * thus the record lacks the and elements. However, in the absence of a formally - * labeled abstract in the published article, text from a substantive "summary", "summary and conclusions" or "conclusions and summary" may be used. - * - * @return the Mapped Pubmed Article Abstracts - */ - public String getDescription() { - return description; - } - - /** - * Set the Mapped Pubmed Article Abstracts - * @param description - */ - public void setDescription(String description) { - this.description = description; - } - - /** - * Personal and collective (corporate) author names published with the article are found in . - * - * @return get the Mapped Authors lists - */ - public List getAuthors() { - return authors; - } - - /** - * Set the Mapped Authors lists - * @param authors - */ - public void setAuthors(List authors) { - this.authors = authors; - } - - /** - * This element is used to identify the type of article indexed for MEDLINE; - * it characterizes the nature of the information or the manner in which it is conveyed as well as the type of - * research support received (e.g., Review, Letter, Retracted Publication, Clinical Conference, Research Support, N.I.H., Extramural). - * - * @return the mapped Subjects - */ - public List getSubjects() { - return subjects; - } - - /** - * - * the language in which an article was published is recorded in . - * All entries are three letter abbreviations stored in lower case, such as eng, fre, ger, jpn, etc. When a single - * record contains more than one language value the XML export program extracts the languages in alphabetic order by the 3-letter language value. - * Some records provided by collaborating data producers may contain the value und to identify articles whose language is undetermined. - * - * @return The mapped Language - */ - public String getLanguage() { - return language; - } - - /** - * - * Set The mapped Language - * - * @param language the mapped Language - */ - public void setLanguage(String language) { - this.language = language; - } - - /** - * This element is used to identify the type of article indexed for MEDLINE; - * it characterizes the nature of the information or the manner in which it is conveyed as well as the type of - * research support received (e.g., Review, Letter, Retracted Publication, Clinical Conference, Research Support, N.I.H., Extramural). - * - * @return the mapped Publication Type - */ - public List getPublicationTypes() { - return publicationTypes; - } - - /** - * contains the research grant or contract number (or both) that designates financial support by any agency of the United States Public Health Service - * or any institute of the National Institutes of Health. Additionally, beginning in late 2005, grant numbers are included for many other US and non-US funding agencies and organizations. - * @return the mapped grants - */ - - public List getGrants() { - return grants; - } - public String getPmcId() { return pmcId; } - public PMArticle setPmcId(String pmcId) { + public void setPmcId(String pmcId) { this.pmcId = pmcId; - return this; + } + + public String getDoi() { + return doi; + } + + public void setDoi(String doi) { + this.doi = doi; + } + + public String getDate() { + return date; + } + + public void setDate(String date) { + this.date = date; + } + + public PMJournal getJournal() { + return journal; + } + + public void setJournal(PMJournal journal) { + this.journal = journal; + } + + public String getTitle() { + return title; + } + + public void setTitle(String title) { + this.title = title; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public String getLanguage() { + return language; + } + + public void setLanguage(String language) { + this.language = language; + } + + public List getSubjects() { + return subjects; + } + + public void setSubjects(List subjects) { + this.subjects = subjects; + } + + public List getPublicationTypes() { + return publicationTypes; + } + + public void setPublicationTypes(List publicationTypes) { + this.publicationTypes = publicationTypes; + } + + public List getAuthors() { + return authors; + } + + public void setAuthors(List authors) { + this.authors = authors; + } + + public List getGrants() { + return grants; + } + + public void setGrants(List grants) { + this.grants = grants; } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json index 3ba83764d..8326fab0f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json @@ -1,8 +1,7 @@ [ {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, {"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true}, - {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true}, - {"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the oaf path ", "paramRequired": true}, - {"paramName":"s", "paramLongName":"skipUpdate", "paramDescription": "skip update ", "paramRequired": false}, - {"paramName":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true} + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the baseline path", "paramRequired": true}, + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the mdstore path to save", "paramRequired": true} + ] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala new file mode 100644 index 000000000..c21bfd7c3 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala @@ -0,0 +1,90 @@ +package eu.dnetlib.dhp.sx.bio.ebi + +import com.fasterxml.jackson.databind.ObjectMapper +import eu.dnetlib.dhp.application.{AbstractScalaApplication, ArgumentApplicationParser} +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup +import eu.dnetlib.dhp.schema.oaf.Oaf +import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PMParser2, PubMedToOaf} +import eu.dnetlib.dhp.utils.ISLookupClientFactory +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +import java.io.ByteArrayInputStream +import javax.xml.stream.XMLInputFactory + +class SparkCreatePubmedDump(propertyPath: String, args: Array[String], log: Logger) + extends AbstractScalaApplication(propertyPath, args, log: Logger) { + + /** Here all the spark applications runs this method + * where the whole logic of the spark node is defined + */ + override def run(): Unit = { + val isLookupUrl: String = parser.get("isLookupUrl") + log.info("isLookupUrl: {}", isLookupUrl) + val sourcePath = parser.get("sourcePath") + log.info(s"SourcePath is '$sourcePath'") + val targetPath = parser.get("targetPath") + log.info(s"TargetPath is '$targetPath'") + + val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl) + val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService) + + createPubmedDump(spark, sourcePath, targetPath, vocabularies) + + } + + def createPubmedDump( + spark: SparkSession, + sourcePath: String, + targetPath: String, + vocabularies: VocabularyGroup + ): Unit = { + require(spark != null) + + implicit val PMEncoder: Encoder[PMArticle] = Encoders.bean(classOf[PMArticle]) + + import spark.implicits._ + val df = spark.read.option("lineSep", "").text(sourcePath) + val mapper = new ObjectMapper() + df.as[String] + .map(s => { + val id = s.indexOf("") + if (id >= 0) s"${s.substring(id)}" else null + }) + .filter(s => s != null) + .map { i => + try { + new PMParser2().parse(i) + } catch { + case _: Exception => { + throw new RuntimeException(s"Error parsing article: $i") + } + } + } + .dropDuplicates("pmid") + .map { a => + val oaf = PubMedToOaf.convert(a, vocabularies) + if (oaf != null) + mapper.writeValueAsString(oaf) + else + null + } + .as[String] + .filter(s => s != null) + .write + .option("compression", "gzip") + .mode("overwrite") + .text(targetPath) + } +} + +object SparkCreatePubmedDump { + + def main(args: Array[String]): Unit = { + val log: Logger = LoggerFactory.getLogger(getClass) + + new SparkCreatePubmedDump("/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json", args, log).initialize().run() + + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala new file mode 100644 index 000000000..c9e868185 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala @@ -0,0 +1,264 @@ +package eu.dnetlib.dhp.sx.bio.pubmed + +import org.apache.commons.lang3.StringUtils + +import javax.xml.stream.XMLEventReader +import scala.collection.JavaConverters._ +import scala.xml.{MetaData, NodeSeq} +import scala.xml.pull.{EvElemEnd, EvElemStart, EvText} + +class PMParser2 { + + /** Extracts the value of an attribute from a MetaData object. + * @param attrs the MetaData object + * @param key the key of the attribute + * @return the value of the attribute or null if the attribute is not found + */ + private def extractAttributes(attrs: MetaData, key: String): String = { + + val res = attrs.get(key) + if (res.isDefined) { + val s = res.get + if (s != null && s.nonEmpty) + s.head.text + else + null + } else null + } + + /** Validates and formats a date given the year, month, and day as strings. + * + * @param year the year as a string + * @param month the month as a string + * @param day the day as a string + * @return the formatted date as "YYYY-MM-DD" or null if the date is invalid + */ + private def validate_Date(year: String, month: String, day: String): String = { + try { + f"${year.toInt}-${month.toInt}%02d-${day.toInt}%02d" + + } catch { + case _: Throwable => null + } + } + + /** Extracts the grant information from a NodeSeq object. + * + * @param gNode the NodeSeq object + * @return the grant information or an empty list if the grant information is not found + */ + private def extractGrant(gNode: NodeSeq): List[PMGrant] = { + gNode + .map(node => { + val grantId = (node \ "GrantID").text + val agency = (node \ "Agency").text + val country = (node \ "Country").text + new PMGrant(grantId, agency, country) + }) + .toList + } + + /** Extracts the journal information from a NodeSeq object. + * + * @param jNode the NodeSeq object + * @return the journal information or null if the journal information is not found + */ + private def extractJournal(jNode: NodeSeq): PMJournal = { + val journal = new PMJournal + journal.setTitle((jNode \ "Title").text) + journal.setIssn((jNode \ "ISSN").text) + journal.setVolume((jNode \ "JournalIssue" \ "Volume").text) + journal.setIssue((jNode \ "JournalIssue" \ "Issue").text) + if (journal.getTitle != null && StringUtils.isNotEmpty(journal.getTitle)) + journal + else + null + } + + private def extractAuthors(aNode: NodeSeq): List[PMAuthor] = { + aNode + .map(author => { + val a = new PMAuthor + a.setLastName((author \ "LastName").text) + a.setForeName((author \ "ForeName").text) + a + }) + .toList + } + + def parse(input: String): PMArticle = { + val xml = scala.xml.XML.loadString(input) + val article = new PMArticle + + val grantNodes = xml \ "MedlineCitation" \\ "Grant" + article.setGrants(extractGrant(grantNodes).asJava) + + val journal = xml \ "MedlineCitation" \ "Article" \ "Journal" + article.setJournal(extractJournal(journal)) + + val authors = xml \ "MedlineCitation" \ "Article" \ "AuthorList" \ "Author" + + article.setAuthors( + authors + .map(author => { + val a = new PMAuthor + a.setLastName((author \ "LastName").text) + a.setForeName((author \ "ForeName").text) + a + }) + .toList + .asJava + ) + + val pmId = xml \ "MedlineCitation" \ "PMID" + + val articleIds = xml \ "PubmedData" \ "ArticleIdList" \ "ArticleId" + articleIds.foreach(articleId => { + val idType = (articleId \ "@IdType").text + val id = articleId.text + if ("doi".equalsIgnoreCase(idType)) article.setDoi(id) + if ("pmc".equalsIgnoreCase(idType)) article.setPmcId(id) + }) + article.setPmid(pmId.text) + + val pubMedPubDate = xml \ "MedlineCitation" \ "DateCompleted" + val currentDate = + validate_Date((pubMedPubDate \ "Year").text, (pubMedPubDate \ "Month").text, (pubMedPubDate \ "Day").text) + if (currentDate != null) article.setDate(currentDate) + + val articleTitle = xml \ "MedlineCitation" \ "Article" \ "ArticleTitle" + article.setTitle(articleTitle.text) + + val abstractText = xml \ "MedlineCitation" \ "Article" \ "Abstract" \ "AbstractText" + if (abstractText != null && abstractText.text != null && abstractText.text.nonEmpty) + article.setDescription(abstractText.text.split("\n").map(s => s.trim).mkString(" ").trim) + + val language = xml \ "MedlineCitation" \ "Article" \ "Language" + article.setLanguage(language.text) + + val subjects = xml \ "MedlineCitation" \ "MeshHeadingList" \ "MeshHeading" + article.setSubjects( + subjects + .take(20) + .map(subject => { + val descriptorName = (subject \ "DescriptorName").text + val ui = (subject \ "DescriptorName" \ "@UI").text + val s = new PMSubject + s.setValue(descriptorName) + s.setMeshId(ui) + s + }) + .toList + .asJava + ) + val publicationTypes = xml \ "MedlineCitation" \ "Article" \ "PublicationTypeList" \ "PublicationType" + article.setPublicationTypes( + publicationTypes + .map(pt => { + val s = new PMSubject + s.setValue(pt.text) + s + }) + .toList + .asJava + ) + + article + } + + def parse2(xml: XMLEventReader): PMArticle = { + var currentArticle: PMArticle = null + var currentSubject: PMSubject = null + var currentAuthor: PMAuthor = null + var currentJournal: PMJournal = null + var currentGrant: PMGrant = null + var currNode: String = null + var currentYear = "0" + var currentMonth = "01" + var currentDay = "01" + var currentArticleType: String = null + + while (xml.hasNext) { + val ne = xml.next + ne match { + case EvElemStart(_, label, attrs, _) => + currNode = label + + label match { + case "PubmedArticle" => currentArticle = new PMArticle + case "Author" => currentAuthor = new PMAuthor + case "Journal" => currentJournal = new PMJournal + case "Grant" => currentGrant = new PMGrant + case "PublicationType" | "DescriptorName" => + currentSubject = new PMSubject + currentSubject.setMeshId(extractAttributes(attrs, "UI")) + case "ArticleId" => currentArticleType = extractAttributes(attrs, "IdType") + case _ => + } + case EvElemEnd(_, label) => + label match { + case "PubmedArticle" => return currentArticle + case "Author" => currentArticle.getAuthors.add(currentAuthor) + case "Journal" => currentArticle.setJournal(currentJournal) + case "Grant" => currentArticle.getGrants.add(currentGrant) + case "PubMedPubDate" => + if (currentArticle.getDate == null) + currentArticle.setDate(validate_Date(currentYear, currentMonth, currentDay)) + case "PubDate" => currentJournal.setDate(s"$currentYear-$currentMonth-$currentDay") + case "DescriptorName" => currentArticle.getSubjects.add(currentSubject) + case "PublicationType" => currentArticle.getPublicationTypes.add(currentSubject) + case _ => + } + case EvText(text) => + if (currNode != null && text.trim.nonEmpty) + currNode match { + case "ArticleTitle" => { + if (currentArticle.getTitle == null) + currentArticle.setTitle(text.trim) + else + currentArticle.setTitle(currentArticle.getTitle + text.trim) + } + case "AbstractText" => { + if (currentArticle.getDescription == null) + currentArticle.setDescription(text.trim) + else + currentArticle.setDescription(currentArticle.getDescription + text.trim) + } + case "PMID" => currentArticle.setPmid(text.trim) + case "ArticleId" => + if ("doi".equalsIgnoreCase(currentArticleType)) currentArticle.setDoi(text.trim) + if ("pmc".equalsIgnoreCase(currentArticleType)) currentArticle.setPmcId(text.trim) + case "Language" => currentArticle.setLanguage(text.trim) + case "ISSN" => currentJournal.setIssn(text.trim) + case "GrantID" => currentGrant.setGrantID(text.trim) + case "Agency" => currentGrant.setAgency(text.trim) + case "Country" => if (currentGrant != null) currentGrant.setCountry(text.trim) + case "Year" => currentYear = text.trim + case "Month" => currentMonth = text.trim + case "Day" => currentDay = text.trim + case "Volume" => currentJournal.setVolume(text.trim) + case "Issue" => currentJournal.setIssue(text.trim) + case "PublicationType" | "DescriptorName" => currentSubject.setValue(text.trim) + case "LastName" => { + if (currentAuthor != null) + currentAuthor.setLastName(text.trim) + } + case "ForeName" => + if (currentAuthor != null) + currentAuthor.setForeName(text.trim) + case "Title" => + if (currentJournal.getTitle == null) + currentJournal.setTitle(text.trim) + else + currentJournal.setTitle(currentJournal.getTitle + text.trim) + case _ => + + } + case _ => + } + + } + null + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml new file mode 100644 index 000000000..4b4d860d7 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml @@ -0,0 +1,222 @@ + + + 37885214 + + 2024 + 02 + 14 + + + 2024 + 02 + 14 + +
+ + 2752-7549 + + 40 + 5 + + 2023 Sep-Oct + + + Journal of pediatric hematology/oncology nursing + J Pediatr Hematol Oncol Nurs + + Care Needs of Parents of Children With Cancer in a Low-Middle-Income Country. + + 295-304 + + 10.1177/27527530231193972 + + Background: Mapping out actual supportive care needs assists nurses in providing holistic individualized care. This study aimed to explore the care needs of parents of children with cancer in the Philippines. Method: Guided by the Supportive Care Needs Framework (SCNF), this study used an embedded mixed-method design with the quantitative revised Cancer Patient Needs Questionnaire and qualitative semistructured interviews to describe parents' care needs and priorities. Results: Filipino parents (N = 156) of children with cancer have various care needs which could be classified along the SCNF categories-practical, informational, spiritual, physical, emotional, and physical needs as ranked from highest to lowest. A number of variables were significantly associated with care needs. Solid tumor diagnosis was associated with greater practical, emotional, and psychosocial care needs; having a child who had undergone surgery was associated with more practical and spiritual care needs; and being within one year of the child's diagnosis was associated with practical, psychosocial, and spiritual care needs. Parent priority needs included (a) addressing financial needs; (b) access to temporary housing to minimize treatment-related costs; (c) support groups among parents of children with cancer as a source of information; (d) financial and social support between members of family and partners of parents of children with cancer; and (e) using prayer to facilitate acceptance. Conclusions: Supportive care needs of parents of children with cancer are important components of care that should be given recognition to enhance holistic individualized care throughout the childhood cancer experience. + + + + Banayat + Aprille Campos + AC + 0000-0001-9339-9871 + + College of Nursing, University of the Philippines Manila, Manila, Philippines. + + + + Abad + Peter James B + PJB + + College of Nursing, University of the Philippines Manila, Manila, Philippines. + + + + Bonito + Sheila R + SR + + College of Nursing, University of the Philippines Manila, Manila, Philippines. + + + + Manahan + Lydia T + LT + + College of Nursing, University of the Philippines Manila, Manila, Philippines. + + + + Peralta + Arnold B + AB + + College of Nursing, University of the Philippines Manila, Manila, Philippines. + + + + eng + + Journal Article + + + 2023 + 10 + 26 + +
+ + United States + J Pediatr Hematol Oncol Nurs + 9918282681506676 + 2752-7530 + + IM + + + Child + + + Humans + + + Parents + psychology + + + Social Support + + + Spirituality + + + Religion + + + Neoplasms + therapy + + + + cancer + mixed methods + parent + pediatric + research + supportive care + + Declaration of Conflicting InterestsThe author(s) declared no potential conflicts of interest with respect to the research, authorship, and/or publication of this article. +
+ + + + 2024 + 2 + 12 + 18 + 42 + + + 2023 + 10 + 27 + 6 + 42 + + + 2023 + 10 + 27 + 3 + 43 + + + ppublish + + 37885214 + 10.1177/27527530231193972 + + +
+ +30522158 +32769323 +34061701 +34661197 +34837091 +35035475 +35211699 +35557982 +35782783 +35795240 +35832688 +35847411 +36081602 +36081858 +36468085 +36468934 +36580086 +36589526 +36619609 +36649460 +36654909 +36655054 +36700856 +36705625 +36713939 +36714172 +36741203 +36741905 +36743825 +36788221 +36844926 +36846546 +36935776 +36946757 +36972191 +37034422 +37124311 +37152108 +37171968 +37273889 +37333905 +37387733 +37431449 +37576947 +37601162 +37711214 +37901290 +37981909 +37981945 +37982005 +38037601 +38037602 +38150730 +38274640 +38332671 +38334184 +38335456 +38349506 +38349576 +38353676 + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala index c4af14c40..1374b741d 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala @@ -5,7 +5,10 @@ import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest import eu.dnetlib.dhp.schema.oaf.utils.PidType import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Result} import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved -import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMParser, PMSubject, PubMedToOaf} +import eu.dnetlib.dhp.sx.bio.ebi.SparkCreatePubmedDump +import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PMParser2, PMSubject, PubMedToOaf} +import org.apache.commons.io.IOUtils +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession} import org.json4s.DefaultFormats import org.json4s.JsonAST.{JField, JObject, JString} import org.json4s.jackson.JsonMethods.parse @@ -13,8 +16,9 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.junit.jupiter.MockitoExtension +import org.slf4j.LoggerFactory -import java.io.{BufferedReader, InputStream, InputStreamReader} +import java.io.{BufferedReader, ByteArrayInputStream, InputStream, InputStreamReader} import java.util.zip.GZIPInputStream import javax.xml.stream.XMLInputFactory import scala.collection.JavaConverters._ @@ -48,6 +52,17 @@ class BioScholixTest extends AbstractVocabularyTest { } } + @Test + def testParsingPubmed2(): Unit = { + val mapper = new ObjectMapper() + val xml = IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml")) + val parser = new PMParser2() + val article = parser.parse(xml) + + println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(article)) + + } + @Test def testEBIData() = { val inputFactory = XMLInputFactory.newInstance @@ -124,6 +139,15 @@ class BioScholixTest extends AbstractVocabularyTest { } } + @Test + def testPubmedSplitting(): Unit = { + + val spark: SparkSession = SparkSession.builder().appName("test").master("local").getOrCreate() + new SparkCreatePubmedDump("", Array.empty, LoggerFactory.getLogger(getClass)) + .createPubmedDump(spark, "/home/sandro/Downloads/pubmed", "/home/sandro/Downloads/pubmed_mapped", vocabularies) + + } + @Test def testPubmedOriginalID(): Unit = { val article: PMArticle = new PMArticle From c1cef5d685373ad28dac341b56fc459cd43ff606 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Tue, 5 Nov 2024 10:38:40 +0100 Subject: [PATCH 191/239] removed old library joda time replaced with standard java.time introduced in java 8 --- .../plugin/gtr2/Gtr2PublicationsIterator.java | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java index 5b8f48680..779c43712 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java @@ -1,6 +1,8 @@ package eu.dnetlib.dhp.collection.plugin.gtr2; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -16,9 +18,7 @@ import org.dom4j.Document; import org.dom4j.DocumentException; import org.dom4j.DocumentHelper; import org.dom4j.Element; -import org.joda.time.DateTime; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,7 +33,7 @@ public class Gtr2PublicationsIterator implements Iterator { private static final Logger log = LoggerFactory.getLogger(Gtr2PublicationsIterator.class); private final HttpConnector2 connector; - private static final DateTimeFormatter simpleDateTimeFormatter = DateTimeFormat.forPattern("yyyy-MM-dd"); + private static final DateTimeFormatter simpleDateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); private static final int MAX_ATTEMPTS = 10; @@ -41,7 +41,7 @@ public class Gtr2PublicationsIterator implements Iterator { private int currPage; private int endPage; private boolean incremental = false; - private DateTime fromDate; + private LocalDate fromDate; private final Map cache = new HashMap<>(); @@ -188,28 +188,28 @@ public class Gtr2PublicationsIterator implements Iterator { private Document loadURL(final String cleanUrl, final int attempt) { try { - log.debug(" * Downloading Url: " + cleanUrl); - final byte[] bytes = this.connector.getInputSource(cleanUrl).getBytes("UTF-8"); + log.debug(" * Downloading Url: {}", cleanUrl); + final byte[] bytes = this.connector.getInputSource(cleanUrl).getBytes(StandardCharsets.UTF_8); return DocumentHelper.parseText(new String(bytes)); } catch (final Throwable e) { - log.error("Error dowloading url: " + cleanUrl + ", attempt = " + attempt, e); + log.error("Error dowloading url: {}, attempt = {}", cleanUrl, attempt, e); if (attempt >= MAX_ATTEMPTS) { - throw new RuntimeException("Error dowloading url: " + cleanUrl, e); + throw new RuntimeException("Error downloading url: " + cleanUrl, e); } try { Thread.sleep(60000); // I wait for a minute } catch (final InterruptedException e1) { - throw new RuntimeException("Error dowloading url: " + cleanUrl, e); + throw new RuntimeException("Error downloading url: " + cleanUrl, e); } return loadURL(cleanUrl, attempt + 1); } } - private DateTime parseDate(final String s) { - return DateTime.parse(s.contains("T") ? s.substring(0, s.indexOf("T")) : s, simpleDateTimeFormatter); + private LocalDate parseDate(final String s) { + return LocalDate.parse(s.contains("T") ? s.substring(0, s.indexOf("T")) : s, simpleDateTimeFormatter); } - private boolean isAfter(final String d, final DateTime fromDate) { + private boolean isAfter(final String d, final LocalDate fromDate) { return StringUtils.isNotBlank(d) && parseDate(d).isAfter(fromDate); } } From 973aa7dca6508e2c1e82fa7c775b3d40c18fe45b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 6 Nov 2024 12:29:06 +0100 Subject: [PATCH 192/239] [dedup] force the Relation schema when reading the merge rels --- .../java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java index c7efce4d7..b0bc314e2 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java @@ -69,6 +69,7 @@ public class SparkPropagateRelation extends AbstractSparkAction { Dataset mergeRels = spark .read() + .schema(REL_BEAN_ENC.schema()) .load(DedupUtility.createMergeRelPath(workingPath, "*", "*")) .as(REL_BEAN_ENC); From f7bb53fe7895105f4a7a73b7de14cd6b4121589e Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 Nov 2024 01:04:43 +0100 Subject: [PATCH 193/239] [orcid enrichment] added missing workflow parameter: workingDir --- .../resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml index 4031da15a..1ece2c0be 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml @@ -51,6 +51,7 @@ --orcidPath${orcidPath} --targetPath${targetPath} --graphPath${graphPath} + --workingDir${workingDir} --masteryarn From 8f5171557e20ed58f69f7abe3af1ad0a85b10ba3 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Thu, 7 Nov 2024 12:22:34 +0100 Subject: [PATCH 194/239] Remove ORCID information when the same ORCID ID is used multiple times in the same result for different authors --- .../oaf/utils/GraphCleaningFunctions.java | 36 ++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index b6574da16..9153a6476 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -2,7 +2,6 @@ package eu.dnetlib.dhp.schema.oaf.utils; import static eu.dnetlib.dhp.schema.common.ModelConstants.*; -import static eu.dnetlib.dhp.schema.common.ModelConstants.OPENAIRE_META_RESOURCE_TYPE; import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.getProvenance; import java.net.MalformedURLException; @@ -696,6 +695,7 @@ public class GraphCleaningFunctions extends CleaningFunctions { } } + // set ORCID_PENDING to all orcid values that are not coming from ORCID provenance for (Author a : r.getAuthor()) { if (Objects.isNull(a.getPid())) { a.setPid(Lists.newArrayList()); @@ -752,6 +752,40 @@ public class GraphCleaningFunctions extends CleaningFunctions { .collect(Collectors.toList())); } } + + // Identify clashing ORCIDS:that is same ORCID associated to multiple authors in this result + Map clashing_orcid = new HashMap<>(); + + for (Author a : r.getAuthor()) { + a + .getPid() + .stream() + .filter( + p -> StringUtils + .contains(StringUtils.lowerCase(p.getQualifier().getClassid()), ORCID_PENDING)) + .map(StructuredProperty::getValue) + .distinct() + .forEach(orcid -> clashing_orcid.compute(orcid, (k, v) -> (v == null) ? 1 : v + 1)); + } + + Set clashing = clashing_orcid + .entrySet() + .stream() + .filter(ee -> ee.getValue() > 1) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + + // filter out clashing orcids + for (Author a : r.getAuthor()) { + a + .setPid( + a + .getPid() + .stream() + .filter(p -> !clashing.contains(p.getValue())) + .collect(Collectors.toList())); + } + } if (value instanceof Publication) { From 6fd9ec856608c3ca9baeedfa3677a64287483d92 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 7 Nov 2024 13:55:31 +0100 Subject: [PATCH 195/239] [danishfunders] added link for danish funders versus the unidentified project for IRFD (501100004836) CF (501100002808) and NNF(501100009708) --- .../doiboost/crossref/Crossref2Oaf.scala | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index f284a063e..bf11ed0a8 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -566,7 +566,23 @@ case object Crossref2Oaf { queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) case _ => logger.debug("no match for " + funder.DOI.get) - + //Add for Danish funders + //Independent Research Fund Denmark (IRFD) + case "10.13039/501100004836" => + val targetId = getProjectId("irfd________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + //Carlsberg Foundation (CF) + case "10.13039/501100002808" => + val targetId = getProjectId("cf__________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + //Novo Nordisk Foundation (NNF) + case "10.13039/501100009708" => + val targetId = getProjectId("nnf_________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + case _ => logger.debug("no match for " + funder.DOI.get) } } else { From 0d0904f4ec2b1d4424d5d8d5d6d782049c0544cd Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Mon, 11 Nov 2024 10:27:23 +0100 Subject: [PATCH 196/239] updated workflow baseline to direct transform on OAF --- .../sx/bio/ebi/baseline_to_oaf_params.json | 8 ++--- .../dhp/sx/bio/pubmed/oozie_app/workflow.xml | 17 ++++------- .../sx/bio/ebi/SparkCreatePubmedDump.scala | 29 ++++++++++++------- 3 files changed, 27 insertions(+), 27 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json index 8326fab0f..0fcc03266 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json @@ -1,7 +1,7 @@ [ - {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true}, - {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the baseline path", "paramRequired": true}, - {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the mdstore path to save", "paramRequired": true} + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the baseline path", "paramRequired": true}, + {"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the mdstore path to save", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml index 30eb41469..0f4c5c249 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml @@ -16,11 +16,6 @@ mdStoreManagerURI the path of the cleaned mdstore
- - skipUpdate - false - The request block size -
@@ -44,16 +39,16 @@ --mdStoreManagerURI${mdStoreManagerURI} - + - + yarn cluster - Convert Baseline to OAF Dataset - eu.dnetlib.dhp.sx.bio.ebi.SparkCreateBaselineDataFrame + Convert Baseline Pubmed to OAF Dataset + eu.dnetlib.dhp.sx.bio.ebi.SparkCreatePubmedDump dhp-aggregation-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} @@ -65,12 +60,10 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --workingPath${baselineWorkingPath} + --sourcePath${baselineWorkingPath} --mdstoreOutputVersion${wf:actionData('StartTransaction')['mdStoreVersion']} --masteryarn --isLookupUrl${isLookupUrl} - --hdfsServerUri${nameNode} - --skipUpdate${skipUpdate} diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala index c21bfd7c3..1bdd2a4bc 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala @@ -1,18 +1,14 @@ package eu.dnetlib.dhp.sx.bio.ebi import com.fasterxml.jackson.databind.ObjectMapper -import eu.dnetlib.dhp.application.{AbstractScalaApplication, ArgumentApplicationParser} +import eu.dnetlib.dhp.application.AbstractScalaApplication import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup -import eu.dnetlib.dhp.schema.oaf.Oaf -import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PMParser2, PubMedToOaf} +import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion +import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMParser2, PubMedToOaf} import eu.dnetlib.dhp.utils.ISLookupClientFactory -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession} +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} import org.slf4j.{Logger, LoggerFactory} -import java.io.ByteArrayInputStream -import javax.xml.stream.XMLInputFactory - class SparkCreatePubmedDump(propertyPath: String, args: Array[String], log: Logger) extends AbstractScalaApplication(propertyPath, args, log: Logger) { @@ -24,16 +20,26 @@ class SparkCreatePubmedDump(propertyPath: String, args: Array[String], log: Logg log.info("isLookupUrl: {}", isLookupUrl) val sourcePath = parser.get("sourcePath") log.info(s"SourcePath is '$sourcePath'") - val targetPath = parser.get("targetPath") - log.info(s"TargetPath is '$targetPath'") + val mdstoreOutputVersion = parser.get("mdstoreOutputVersion") + log.info(s"mdstoreOutputVersion is '$mdstoreOutputVersion'") + val mapper = new ObjectMapper() + val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion]) + val outputBasePath = cleanedMdStoreVersion.getHdfsPath + log.info(s"outputBasePath is '$outputBasePath'") val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl) val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService) - createPubmedDump(spark, sourcePath, targetPath, vocabularies) + createPubmedDump(spark, sourcePath, outputBasePath, vocabularies) } + /** This method creates a dump of the pubmed articles + * @param spark the spark session + * @param sourcePath the path of the source file + * @param targetPath the path of the target file + * @param vocabularies the vocabularies + */ def createPubmedDump( spark: SparkSession, sourcePath: String, @@ -54,6 +60,7 @@ class SparkCreatePubmedDump(propertyPath: String, args: Array[String], log: Logg }) .filter(s => s != null) .map { i => + //remove try catch try { new PMParser2().parse(i) } catch { From 19ce783e58f4f9176f4dc9a98d9bb250dc615e0d Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Mon, 11 Nov 2024 12:28:02 +0100 Subject: [PATCH 197/239] renamed workflow --- .../eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml index 0f4c5c249..2a51b4050 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/bio/pubmed/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + baselineWorkingPath From 474f365286b7f83b8d58e4969277b3e67ebdd0cc Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Mon, 11 Nov 2024 12:37:27 +0100 Subject: [PATCH 198/239] removed wrong test --- .../test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala index 1374b741d..c942ca132 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala @@ -6,9 +6,9 @@ import eu.dnetlib.dhp.schema.oaf.utils.PidType import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Result} import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved import eu.dnetlib.dhp.sx.bio.ebi.SparkCreatePubmedDump -import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PMParser2, PMSubject, PubMedToOaf} +import eu.dnetlib.dhp.sx.bio.pubmed._ import org.apache.commons.io.IOUtils -import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession} +import org.apache.spark.sql.SparkSession import org.json4s.DefaultFormats import org.json4s.JsonAST.{JField, JObject, JString} import org.json4s.jackson.JsonMethods.parse @@ -18,13 +18,12 @@ import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.junit.jupiter.MockitoExtension import org.slf4j.LoggerFactory -import java.io.{BufferedReader, ByteArrayInputStream, InputStream, InputStreamReader} +import java.io.{BufferedReader, InputStream, InputStreamReader} import java.util.zip.GZIPInputStream import javax.xml.stream.XMLInputFactory import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import scala.io.Source -import scala.xml.pull.XMLEventReader @ExtendWith(Array(classOf[MockitoExtension])) class BioScholixTest extends AbstractVocabularyTest { @@ -139,7 +138,6 @@ class BioScholixTest extends AbstractVocabularyTest { } } - @Test def testPubmedSplitting(): Unit = { val spark: SparkSession = SparkSession.builder().appName("test").master("local").getOrCreate() From b0283fe94c168b87176f283f414ef2c4dfd3cdab Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 11 Nov 2024 14:57:57 +0100 Subject: [PATCH 199/239] [person] fix provenance of pid in person when it is orcid (classid entityregistry to avoid the cleaning put orcid_pending) --- .../dhp/actionmanager/personentity/ExtractPerson.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index bf2c19c3d..6830f2291 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -345,7 +345,16 @@ public class ExtractPerson implements Serializable { OafMapperUtils .structuredProperty( op.getOrcid(), ModelConstants.ORCID, ModelConstants.ORCID_CLASSNAME, - ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, null)); + ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, + OafMapperUtils.dataInfo(false, + null, + false, + false, + OafMapperUtils.qualifier(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY, + ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY, + ModelConstants.DNET_PID_TYPES, + ModelConstants.DNET_PID_TYPES), + "0.91"))); person.setDateofcollection(op.getLastModifiedDate()); person.setOriginalId(Arrays.asList(op.getOrcid())); person.setDataInfo(ORCIDDATAINFO); From f1ea9da5bcda277451416253982c24233e40d87b Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 11 Nov 2024 15:37:56 +0100 Subject: [PATCH 200/239] [person] checked type in inferenceprovenance --- .../bipaffiliations/PrepareAffiliationRelations.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java index 15c1cc376..75e58e665 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelations.java @@ -104,22 +104,22 @@ public class PrepareAffiliationRelations implements Serializable { .listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); JavaPairRDD crossrefRelations = prepareAffiliationRelationsNewModel( - spark, crossrefInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::crossref"); + spark, crossrefInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":crossref"); JavaPairRDD pubmedRelations = prepareAffiliationRelations( - spark, pubmedInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::pubmed"); + spark, pubmedInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":pubmed"); JavaPairRDD openAPCRelations = prepareAffiliationRelationsNewModel( - spark, openapcInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::openapc"); + spark, openapcInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":openapc"); JavaPairRDD dataciteRelations = prepareAffiliationRelationsNewModel( - spark, dataciteInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::datacite"); + spark, dataciteInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":datacite"); JavaPairRDD webCrawlRelations = prepareAffiliationRelationsNewModel( - spark, webcrawlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::rawaff"); + spark, webcrawlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":rawaff"); JavaPairRDD publisherRelations = prepareAffiliationRelationFromPublisherNewModel( - spark, publisherlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + "::webcrawl"); + spark, publisherlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":webcrawl"); crossrefRelations .union(pubmedRelations) From 250f101779a16ffbec1c16d3e0dc1050d6533c87 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 11 Nov 2024 16:04:06 +0100 Subject: [PATCH 201/239] [person] fixed issue in creating project identifier for the graph for person->project relations --- .../dhp/actionmanager/personentity/ExtractPerson.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 6830f2291..6976def4c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -15,6 +15,7 @@ import java.util.stream.Collectors; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -193,8 +194,8 @@ public class ExtractPerson implements Serializable { private static Relation getProjectRelation(String project, String orcid, String role) { String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid); - String target = PROJECT_ID_PREFIX + project.substring(0, 14) - + IdentifierFactory.md5(project.substring(15)); + String target = PROJECT_ID_PREFIX + StringUtils.substringBefore(project, "::") + "::" + + IdentifierFactory.md5(StringUtils.substringAfter(project, "::")); List properties = new ArrayList<>(); Relation relation = OafMapperUtils From 6c5df761e21d5aea6c203cb2ca2374b33a9219e5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 12 Nov 2024 14:18:04 +0100 Subject: [PATCH 202/239] enforce resulttype based on the dnet:result_typologies vocabulary and upon merge --- .../dhp/oa/merge/GroupEntitiesSparkJob.java | 7 ++- .../dhp/schema/oaf/utils/MergeUtils.java | 62 ++++++++++++++++--- .../dhp/oa/dedup/DedupRecordFactory.java | 2 +- .../dhp/oa/dedup/DatasetMergerTest.java | 4 +- .../raw/AbstractMdRecordToOafMapper.java | 11 ++-- .../raw/GenerateEntitiesApplication.java | 2 +- .../dhp/sx/graph/SparkCreateInputGraph.scala | 2 +- 7 files changed, 66 insertions(+), 24 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java index 24de1a787..98ec09277 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java @@ -2,8 +2,7 @@ package eu.dnetlib.dhp.oa.merge; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import static org.apache.spark.sql.functions.col; -import static org.apache.spark.sql.functions.when; +import static org.apache.spark.sql.functions.*; import java.util.Map; import java.util.Optional; @@ -135,7 +134,9 @@ public class GroupEntitiesSparkJob { .applyCoarVocabularies(entity, vocs), OAFENTITY_KRYO_ENC) .groupByKey((MapFunction) OafEntity::getId, Encoders.STRING()) - .mapGroups((MapGroupsFunction) MergeUtils::mergeById, OAFENTITY_KRYO_ENC) + .mapGroups( + (MapGroupsFunction) (key, group) -> MergeUtils.mergeById(group, vocs), + OAFENTITY_KRYO_ENC) .map( (MapFunction>) t -> new Tuple2<>( t.getClass().getName(), t), diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index 4c411a155..d7e08fca7 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -23,24 +23,30 @@ import org.apache.commons.lang3.tuple.Pair; import com.github.sisyphsu.dateparser.DateParserUtils; import com.google.common.base.Joiner; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.merge.AuthorMerger; import eu.dnetlib.dhp.schema.common.AccessRightComparator; +import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; public class MergeUtils { - public static T mergeById(String s, Iterator oafEntityIterator) { - return mergeGroup(s, oafEntityIterator, true); + public static T mergeById(Iterator oafEntityIterator, VocabularyGroup vocs) { + return mergeGroup(oafEntityIterator, true, vocs); } - public static T mergeGroup(String s, Iterator oafEntityIterator) { - return mergeGroup(s, oafEntityIterator, false); + public static T mergeGroup(Iterator oafEntityIterator) { + return mergeGroup(oafEntityIterator, false); } - public static T mergeGroup(String s, Iterator oafEntityIterator, - boolean checkDelegateAuthority) { + public static T mergeGroup(Iterator oafEntityIterator, boolean checkDelegateAuthority) { + return mergeGroup(oafEntityIterator, checkDelegateAuthority, null); + } + + public static T mergeGroup(Iterator oafEntityIterator, + boolean checkDelegateAuthority, VocabularyGroup vocs) { ArrayList sortedEntities = new ArrayList<>(); oafEntityIterator.forEachRemaining(sortedEntities::add); @@ -49,13 +55,49 @@ public class MergeUtils { Iterator it = sortedEntities.iterator(); T merged = it.next(); - while (it.hasNext()) { - merged = checkedMerge(merged, it.next(), checkDelegateAuthority); + if (!it.hasNext() && merged instanceof Result && vocs != null) { + return enforceResultType(vocs, (Result) merged); + } else { + while (it.hasNext()) { + merged = checkedMerge(merged, it.next(), checkDelegateAuthority); + } } - return merged; } + private static T enforceResultType(VocabularyGroup vocs, Result mergedResult) { + if (Optional.ofNullable(mergedResult.getInstance()).map(List::isEmpty).orElse(true)) { + return (T) mergedResult; + } else { + final Instance i = mergedResult.getInstance().get(0); + + if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { + return (T) mergedResult; + } else { + final Qualifier expectedResultType = vocs + .getSynonymAsQualifier( + ModelConstants.DNET_RESULT_TYPOLOGIES, + i.getInstancetype().getClassid()); + + // there is a clash among the result types + if (!expectedResultType.getClassid().equals(mergedResult.getResulttype().getClassid())) { + try { + String resulttype = expectedResultType.getClassid(); + if (EntityType.otherresearchproduct.toString().equals(resulttype)) { + resulttype = "other"; + } + Result result = (Result) ModelSupport.oafTypes.get(resulttype).newInstance(); + return (T) mergeResultFields(result, mergedResult); + } catch (InstantiationException | IllegalAccessException e) { + throw new IllegalStateException(e); + } + } else { + return (T) mergedResult; + } + } + } + } + public static T checkedMerge(final T left, final T right, boolean checkDelegateAuthority) { return (T) merge(left, right, checkDelegateAuthority); } @@ -106,7 +148,7 @@ public class MergeUtils { return mergeSoftware((Software) left, (Software) right); } - return mergeResultFields((Result) left, (Result) right); + return left; } else if (sameClass(left, right, Datasource.class)) { // TODO final int trust = compareTrust(left, right); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index 44482cfdb..f6a436543 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -135,7 +135,7 @@ public class DedupRecordFactory { return Collections.emptyIterator(); } - OafEntity mergedEntity = MergeUtils.mergeGroup(dedupId, cliques.iterator()); + OafEntity mergedEntity = MergeUtils.mergeGroup(cliques.iterator()); // dedup records do not have date of transformation attribute mergedEntity.setDateoftransformation(null); mergedEntity diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java index 726814c43..a79047590 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/DatasetMergerTest.java @@ -46,8 +46,8 @@ class DatasetMergerTest implements Serializable { } @Test - void datasetMergerTest() throws InstantiationException, IllegalAccessException, InvocationTargetException { - Dataset pub_merged = MergeUtils.mergeGroup(dedupId, datasets.stream().map(Tuple2::_2).iterator()); + void datasetMergerTest() { + Dataset pub_merged = MergeUtils.mergeGroup(datasets.stream().map(Tuple2::_2).iterator()); // verify id assertEquals(dedupId, pub_merged.getId()); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 2436a272c..ba6887a2e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -155,7 +155,7 @@ public abstract class AbstractMdRecordToOafMapper { final List instances = prepareInstances(doc, entityInfo, collectedFrom, hostedBy); - final String type = getResultType(doc, instances); + final String type = getResultType(instances); return createOafs(doc, type, instances, collectedFrom, entityInfo, lastUpdateTimestamp); } catch (final DocumentException e) { @@ -164,10 +164,9 @@ public abstract class AbstractMdRecordToOafMapper { } } - protected String getResultType(final Document doc, final List instances) { - final String type = doc.valueOf("//dr:CobjCategory/@type"); + protected String getResultType(final List instances) { - if (StringUtils.isBlank(type) && this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { + if (this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { final String instanceType = instances .stream() .map(i -> i.getInstancetype().getClassid()) @@ -178,9 +177,9 @@ public abstract class AbstractMdRecordToOafMapper { .ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType)) .map(Qualifier::getClassid) .orElse("0000"); + } else { + throw new IllegalStateException("Missing vocabulary: " + ModelConstants.DNET_RESULT_TYPOLOGIES); } - - return type; } private KeyValue getProvenanceDatasource(final Document doc, final String xpathId, final String xpathName) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java index c3806c211..357fae470 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java @@ -133,7 +133,7 @@ public class GenerateEntitiesApplication extends AbstractMigrationApplication { inputRdd .keyBy(oaf -> ModelSupport.idFn().apply(oaf)) .groupByKey() - .map(t -> MergeUtils.mergeGroup(t._1, t._2.iterator())), + .map(t -> MergeUtils.mergeGroup(t._2.iterator())), // .mapToPair(oaf -> new Tuple2<>(ModelSupport.idFn().apply(oaf), oaf)) // .reduceByKey(MergeUtils::merge) // .map(Tuple2::_2), diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala index d94a23947..42299cd34 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala @@ -133,7 +133,7 @@ object SparkCreateInputGraph { val ds: Dataset[T] = spark.read.load(sourcePath).as[T] ds.groupByKey(_.getId) - .mapGroups { (id, it) => MergeUtils.mergeGroup(id, it.asJava).asInstanceOf[T] } + .mapGroups { (id, it) => MergeUtils.mergeGroup(it.asJava).asInstanceOf[T] } // .reduceGroups { (x: T, y: T) => MergeUtils.merge(x, y).asInstanceOf[T] } // .map(_) .write From 07f267bb10911d62e30a2b299db3c50fcd1746a2 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 13 Nov 2024 08:14:26 +0100 Subject: [PATCH 203/239] fix vocabulary lookup in mergeutils --- .../dhp/schema/oaf/utils/MergeUtils.java | 425 +++++++++--------- 1 file changed, 215 insertions(+), 210 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index d7e08fca7..dc76860f8 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -16,6 +16,8 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.schema.common.EntityType; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -23,10 +25,8 @@ import org.apache.commons.lang3.tuple.Pair; import com.github.sisyphsu.dateparser.DateParserUtils; import com.google.common.base.Joiner; -import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.merge.AuthorMerger; import eu.dnetlib.dhp.schema.common.AccessRightComparator; -import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; @@ -46,7 +46,7 @@ public class MergeUtils { } public static T mergeGroup(Iterator oafEntityIterator, - boolean checkDelegateAuthority, VocabularyGroup vocs) { + boolean checkDelegateAuthority, VocabularyGroup vocs) { ArrayList sortedEntities = new ArrayList<>(); oafEntityIterator.forEachRemaining(sortedEntities::add); @@ -74,11 +74,16 @@ public class MergeUtils { if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { return (T) mergedResult; } else { - final Qualifier expectedResultType = vocs - .getSynonymAsQualifier( + final Qualifier expectedResultType = vocs.lookupTermBySynonym( ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid()); + if (Objects.isNull(expectedResultType)) { + throw new IllegalArgumentException( + "instance type not bound to any result type in dnet:result_typologies: " + + i.getInstancetype().getClassid()); + } + // there is a clash among the result types if (!expectedResultType.getClassid().equals(mergedResult.getResulttype().getClassid())) { try { @@ -117,10 +122,10 @@ public class MergeUtils { return mergeRelation((Relation) left, (Relation) right); } else { throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); } } @@ -159,10 +164,10 @@ public class MergeUtils { return mergeProject((Project) left, (Project) right); } else { throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); } } @@ -253,7 +258,7 @@ public class MergeUtils { } private static List mergeLists(final List left, final List right, int trust, - Function keyExtractor, BinaryOperator merger) { + Function keyExtractor, BinaryOperator merger) { if (left == null || left.isEmpty()) { return right != null ? right : new ArrayList<>(); } else if (right == null || right.isEmpty()) { @@ -264,11 +269,11 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return new ArrayList<>(Stream - .concat(h.stream(), l.stream()) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.toMap(keyExtractor, v -> v, merger, LinkedHashMap::new)) - .values()); + .concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toMap(keyExtractor, v -> v, merger, LinkedHashMap::new)) + .values()); } private static List unionDistinctLists(final List left, final List right, int trust) { @@ -282,10 +287,10 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return Stream - .concat(h.stream(), l.stream()) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.toList()); + .concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toList()); } private static List unionDistinctListOfString(final List l, final List r) { @@ -296,10 +301,10 @@ public class MergeUtils { } return Stream - .concat(l.stream(), r.stream()) - .filter(StringUtils::isNotBlank) - .distinct() - .collect(Collectors.toList()); + .concat(l.stream(), r.stream()) + .filter(StringUtils::isNotBlank) + .distinct() + .collect(Collectors.toList()); } // TODO review @@ -325,7 +330,7 @@ public class MergeUtils { } private static List unionTitle(List left, List right, - int trust) { + int trust) { if (left == null) { return right; } else if (right == null) { @@ -336,10 +341,10 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return Stream - .concat(h.stream(), l.stream()) - .filter(Objects::isNull) - .distinct() - .collect(Collectors.toList()); + .concat(h.stream(), l.stream()) + .filter(Objects::isNull) + .distinct() + .collect(Collectors.toList()); } /** @@ -374,8 +379,8 @@ public class MergeUtils { merged.setPid(mergeLists(merged.getPid(), enrich.getPid(), trust, MergeUtils::spKeyExtractor, (p1, p2) -> p1)); merged.setDateofcollection(LocalDateTime.now().toString()); merged - .setDateoftransformation( - chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); + .setDateoftransformation( + chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); merged.setExtraInfo(unionDistinctLists(merged.getExtraInfo(), enrich.getExtraInfo(), trust)); // When merging records OAI provenance becomes null merged.setOaiprovenance(null); @@ -392,7 +397,7 @@ public class MergeUtils { checkArgument(Objects.equals(merge.getTarget(), enrich.getTarget()), "target ids must be equal"); checkArgument(Objects.equals(merge.getRelType(), enrich.getRelType()), "relType(s) must be equal"); checkArgument( - Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); + Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); checkArgument(Objects.equals(merge.getRelClass(), enrich.getRelClass()), "relClass(es) must be equal"); // merge.setProvenance(mergeLists(merge.getProvenance(), enrich.getProvenance())); @@ -403,10 +408,10 @@ public class MergeUtils { merge.setValidationDate(ModelSupport.oldest(merge.getValidationDate(), enrich.getValidationDate())); } catch (ParseException e) { throw new IllegalArgumentException(String - .format( - "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), - merge.getTarget(), - merge.getValidationDate())); + .format( + "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), + merge.getTarget(), + merge.getValidationDate())); } // TODO keyvalue merge @@ -420,7 +425,7 @@ public class MergeUtils { T merge = mergeOafEntityFields(original, enrich, trust); if (merge.getProcessingchargeamount() == null - || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { + || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { merge.setProcessingchargeamount(enrich.getProcessingchargeamount()); merge.setProcessingchargecurrency(enrich.getProcessingchargecurrency()); } @@ -452,8 +457,8 @@ public class MergeUtils { } merge - .setDateofacceptance( - mergeDateOfAcceptance(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); + .setDateofacceptance( + mergeDateOfAcceptance(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); merge.setPublisher(coalesce(merge.getPublisher(), enrich.getPublisher())); merge.setEmbargoenddate(coalesce(merge.getEmbargoenddate(), enrich.getEmbargoenddate())); @@ -468,7 +473,7 @@ public class MergeUtils { merge.setCoverage(unionDistinctLists(merge.getCoverage(), enrich.getCoverage(), trust)); if (enrich.getBestaccessright() != null - && new AccessRightComparator<>() + && new AccessRightComparator<>() .compare(enrich.getBestaccessright(), merge.getBestaccessright()) < 0) { merge.setBestaccessright(enrich.getBestaccessright()); } @@ -481,8 +486,8 @@ public class MergeUtils { // ok merge - .setExternalReference( - mergeExternalReference(merge.getExternalReference(), enrich.getExternalReference(), trust)); + .setExternalReference( + mergeExternalReference(merge.getExternalReference(), enrich.getExternalReference(), trust)); // instance enrichment or union // review instance equals => add pid to comparision @@ -490,17 +495,17 @@ public class MergeUtils { merge.setInstance(mergeInstances(merge.getInstance(), enrich.getInstance(), trust)); } else { final List enrichmentInstances = isAnEnrichment(merge) ? merge.getInstance() - : enrich.getInstance(); + : enrich.getInstance(); final List enrichedInstances = isAnEnrichment(merge) ? enrich.getInstance() - : merge.getInstance(); + : merge.getInstance(); if (isAnEnrichment(merge)) merge.setDataInfo(enrich.getDataInfo()); merge.setInstance(enrichInstances(enrichedInstances, enrichmentInstances)); } merge - .setEoscifguidelines( - mergeEosciifguidelines(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); + .setEoscifguidelines( + mergeEosciifguidelines(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); merge.setIsGreen(booleanOR(merge.getIsGreen(), enrich.getIsGreen())); // OK but should be list of values merge.setOpenAccessColor(coalesce(merge.getOpenAccessColor(), enrich.getOpenAccessColor())); @@ -526,7 +531,7 @@ public class MergeUtils { LocalDate enrich_date = LocalDate.parse(enrich.getValue(), DateTimeFormatter.ISO_DATE); if (enrich_date.getYear() > 1300 - && (merge_date.getYear() < 1300 || merge_date.isAfter(enrich_date))) { + && (merge_date.getYear() < 1300 || merge_date.isAfter(enrich_date))) { return enrich; } } catch (NullPointerException | DateTimeParseException e) { @@ -544,56 +549,56 @@ public class MergeUtils { private static List mergeInstances(List v1, List v2, int trust) { return mergeLists( - v1, v2, trust, - MergeUtils::instanceKeyExtractor, - MergeUtils::instanceMerger); + v1, v2, trust, + MergeUtils::instanceKeyExtractor, + MergeUtils::instanceMerger); } private static List mergeEosciifguidelines(List v1, List v2, - int trust) { + int trust) { return mergeLists( - v1, v2, trust, er -> Joiner - .on("||") - .useForNull("") - .join(er.getCode(), er.getLabel(), er.getUrl(), er.getSemanticRelation()), - (r, l) -> r); + v1, v2, trust, er -> Joiner + .on("||") + .useForNull("") + .join(er.getCode(), er.getLabel(), er.getUrl(), er.getSemanticRelation()), + (r, l) -> r); } private static List mergeExternalReference(List v1, - List v2, int trust) { + List v2, int trust) { return mergeLists( - v1, v2, trust, er -> Joiner - .on(',') - .useForNull("") - .join( - er.getSitename(), er.getLabel(), - er.getUrl(), toString(er.getQualifier()), er.getRefidentifier(), - er.getQuery(), toString(er.getDataInfo())), - (r, l) -> r); + v1, v2, trust, er -> Joiner + .on(',') + .useForNull("") + .join( + er.getSitename(), er.getLabel(), + er.getUrl(), toString(er.getQualifier()), er.getRefidentifier(), + er.getQuery(), toString(er.getDataInfo())), + (r, l) -> r); } private static String toString(DataInfo di) { return Joiner - .on(',') - .useForNull("") - .join( - di.getInvisible(), di.getInferred(), di.getDeletedbyinference(), di.getTrust(), - di.getInferenceprovenance(), toString(di.getProvenanceaction())); + .on(',') + .useForNull("") + .join( + di.getInvisible(), di.getInferred(), di.getDeletedbyinference(), di.getTrust(), + di.getInferenceprovenance(), toString(di.getProvenanceaction())); } private static String toString(Qualifier q) { return Joiner - .on(',') - .useForNull("") - .join(q.getClassid(), q.getClassname(), q.getSchemeid(), q.getSchemename()); + .on(',') + .useForNull("") + .join(q.getClassid(), q.getClassname(), q.getSchemeid(), q.getSchemename()); } private static String toString(StructuredProperty sp) { return Joiner - .on(',') - .useForNull("") - .join(toString(sp.getQualifier()), sp.getValue()); + .on(',') + .useForNull("") + .join(toString(sp.getQualifier()), sp.getValue()); } private static List mergeStructuredProperties(List v1, List v2, int trust) { @@ -632,17 +637,17 @@ public class MergeUtils { // 2. @@ // 3. || return String - .join( - "::", - kvKeyExtractor(i.getHostedby()), - kvKeyExtractor(i.getCollectedfrom()), - qualifierKeyExtractor(i.getAccessright()), - qualifierKeyExtractor(i.getInstancetype()), - Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), - Optional - .ofNullable(i.getPid()) - .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) - .orElse(null)); + .join( + "::", + kvKeyExtractor(i.getHostedby()), + kvKeyExtractor(i.getCollectedfrom()), + qualifierKeyExtractor(i.getAccessright()), + qualifierKeyExtractor(i.getInstancetype()), + Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), + Optional + .ofNullable(i.getPid()) + .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) + .orElse(null)); } private static Instance instanceMerger(Instance i1, Instance i2) { @@ -653,30 +658,30 @@ public class MergeUtils { i.setInstancetype(i1.getInstancetype()); i.setPid(mergeLists(i1.getPid(), i2.getPid(), 0, MergeUtils::spKeyExtractor, (sp1, sp2) -> sp1)); i - .setAlternateIdentifier( - mergeLists( - i1.getAlternateIdentifier(), i2.getAlternateIdentifier(), 0, MergeUtils::spKeyExtractor, - (sp1, sp2) -> sp1)); + .setAlternateIdentifier( + mergeLists( + i1.getAlternateIdentifier(), i2.getAlternateIdentifier(), 0, MergeUtils::spKeyExtractor, + (sp1, sp2) -> sp1)); i - .setRefereed( - Collections - .min( - Stream.of(i1.getRefereed(), i2.getRefereed()).collect(Collectors.toList()), - new RefereedComparator())); + .setRefereed( + Collections + .min( + Stream.of(i1.getRefereed(), i2.getRefereed()).collect(Collectors.toList()), + new RefereedComparator())); i - .setInstanceTypeMapping( - mergeLists( - i1.getInstanceTypeMapping(), i2.getInstanceTypeMapping(), 0, - MergeUtils::instanceTypeMappingKeyExtractor, (itm1, itm2) -> itm1)); + .setInstanceTypeMapping( + mergeLists( + i1.getInstanceTypeMapping(), i2.getInstanceTypeMapping(), 0, + MergeUtils::instanceTypeMappingKeyExtractor, (itm1, itm2) -> itm1)); i.setFulltext(selectFulltext(i1.getFulltext(), i2.getFulltext())); i.setDateofacceptance(selectOldestDate(i1.getDateofacceptance(), i2.getDateofacceptance())); i.setLicense(coalesce(i1.getLicense(), i2.getLicense())); i.setProcessingchargeamount(coalesce(i1.getProcessingchargeamount(), i2.getProcessingchargeamount())); i.setProcessingchargecurrency(coalesce(i1.getProcessingchargecurrency(), i2.getProcessingchargecurrency())); i - .setMeasures( - mergeLists(i1.getMeasures(), i2.getMeasures(), 0, MergeUtils::measureKeyExtractor, (m1, m2) -> m1)); + .setMeasures( + mergeLists(i1.getMeasures(), i2.getMeasures(), 0, MergeUtils::measureKeyExtractor, (m1, m2) -> m1)); i.setUrl(unionDistinctListOfString(i1.getUrl(), i2.getUrl())); @@ -685,14 +690,14 @@ public class MergeUtils { private static String measureKeyExtractor(Measure m) { return String - .join( - "::", - m.getId(), - m - .getUnit() - .stream() - .map(KeyValue::getKey) - .collect(Collectors.joining("::"))); + .join( + "::", + m.getId(), + m + .getUnit() + .stream() + .map(KeyValue::getKey) + .collect(Collectors.joining("::"))); } private static Field selectOldestDate(Field d1, Field d2) { @@ -703,16 +708,16 @@ public class MergeUtils { } return Stream - .of(d1, d2) - .min( - Comparator - .comparing( - f -> DateParserUtils - .parseDate(f.getValue()) - .toInstant() - .atZone(ZoneId.systemDefault()) - .toLocalDate())) - .orElse(d1); + .of(d1, d2) + .min( + Comparator + .comparing( + f -> DateParserUtils + .parseDate(f.getValue()) + .toInstant() + .atZone(ZoneId.systemDefault()) + .toLocalDate())) + .orElse(d1); } private static String selectFulltext(String ft1, String ft2) { @@ -727,12 +732,12 @@ public class MergeUtils { private static String instanceTypeMappingKeyExtractor(InstanceTypeMapping itm) { return String - .join( - "::", - itm.getOriginalType(), - itm.getTypeCode(), - itm.getTypeLabel(), - itm.getVocabularyName()); + .join( + "::", + itm.getOriginalType(), + itm.getTypeCode(), + itm.getTypeLabel(), + itm.getVocabularyName()); } private static String kvKeyExtractor(KeyValue kv) { @@ -749,13 +754,13 @@ public class MergeUtils { private static String spKeyExtractor(StructuredProperty sp) { return Optional - .ofNullable(sp) - .map( - s -> Joiner - .on("||") - .useForNull("") - .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) - .orElse(null); + .ofNullable(sp) + .map( + s -> Joiner + .on("||") + .useForNull("") + .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) + .orElse(null); } private static T mergeORP(T original, T enrich) { @@ -777,8 +782,8 @@ public class MergeUtils { merge.setLicense(unionDistinctLists(merge.getLicense(), enrich.getLicense(), trust)); merge.setCodeRepositoryUrl(chooseReference(merge.getCodeRepositoryUrl(), enrich.getCodeRepositoryUrl(), trust)); merge - .setProgrammingLanguage( - chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); + .setProgrammingLanguage( + chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); return merge; } @@ -792,11 +797,11 @@ public class MergeUtils { merge.setSize(chooseReference(merge.getSize(), enrich.getSize(), trust)); merge.setVersion(chooseReference(merge.getVersion(), enrich.getVersion(), trust)); merge - .setLastmetadataupdate( - chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); + .setLastmetadataupdate( + chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); merge - .setMetadataversionnumber( - chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); + .setMetadataversionnumber( + chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); merge.setGeolocation(unionDistinctLists(merge.getGeolocation(), enrich.getGeolocation(), trust)); return merge; @@ -818,26 +823,26 @@ public class MergeUtils { merged.setLegalshortname(chooseReference(merged.getLegalshortname(), enrich.getLegalshortname(), trust)); merged.setLegalname(chooseReference(merged.getLegalname(), enrich.getLegalname(), trust)); merged - .setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); + .setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust)); merged.setLogourl(chooseReference(merged.getLogourl(), enrich.getLogourl(), trust)); merged.setEclegalbody(chooseReference(merged.getEclegalbody(), enrich.getEclegalbody(), trust)); merged.setEclegalperson(chooseReference(merged.getEclegalperson(), enrich.getEclegalperson(), trust)); merged.setEcnonprofit(chooseReference(merged.getEcnonprofit(), enrich.getEcnonprofit(), trust)); merged - .setEcresearchorganization( - chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); + .setEcresearchorganization( + chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); merged - .setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); + .setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); merged - .setEcinternationalorganizationeurinterests( - chooseReference( - merged.getEcinternationalorganizationeurinterests(), - enrich.getEcinternationalorganizationeurinterests(), trust)); + .setEcinternationalorganizationeurinterests( + chooseReference( + merged.getEcinternationalorganizationeurinterests(), + enrich.getEcinternationalorganizationeurinterests(), trust)); merged - .setEcinternationalorganization( - chooseReference( - merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); + .setEcinternationalorganization( + chooseReference( + merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); merged.setEcenterprise(chooseReference(merged.getEcenterprise(), enrich.getEcenterprise(), trust)); merged.setEcsmevalidated(chooseReference(merged.getEcsmevalidated(), enrich.getEcsmevalidated(), trust)); merged.setEcnutscode(chooseReference(merged.getEcnutscode(), enrich.getEcnutscode(), trust)); @@ -861,8 +866,8 @@ public class MergeUtils { merged.setDuration(chooseReference(merged.getDuration(), enrich.getDuration(), trust)); merged.setEcsc39(chooseReference(merged.getEcsc39(), enrich.getEcsc39(), trust)); merged - .setOamandatepublications( - chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); + .setOamandatepublications( + chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); merged.setEcarticle29_3(chooseReference(merged.getEcarticle29_3(), enrich.getEcarticle29_3(), trust)); merged.setSubjects(unionDistinctLists(merged.getSubjects(), enrich.getSubjects(), trust)); merged.setFundingtree(unionDistinctLists(merged.getFundingtree(), enrich.getFundingtree(), trust)); @@ -888,8 +893,8 @@ public class MergeUtils { } merged - .setH2020classification( - unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); + .setH2020classification( + unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); return merged; } @@ -916,7 +921,7 @@ public class MergeUtils { * @return list of instances possibly enriched */ private static List enrichInstances(final List toEnrichInstances, - final List enrichmentInstances) { + final List enrichmentInstances) { final List enrichmentResult = new ArrayList<>(); if (toEnrichInstances == null) { @@ -954,42 +959,42 @@ public class MergeUtils { */ private static Map toInstanceMap(final List ri) { return ri - .stream() - .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) - .flatMap(i -> { - final List> result = new ArrayList<>(); - if (i.getPid() != null) - i - .getPid() - .stream() - .filter(MergeUtils::validPid) - .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); - if (i.getAlternateIdentifier() != null) - i - .getAlternateIdentifier() - .stream() - .filter(MergeUtils::validPid) - .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); - return result.stream(); - }) - .collect( - Collectors - .toMap( - Pair::getLeft, - Pair::getRight, - (a, b) -> a)); + .stream() + .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) + .flatMap(i -> { + final List> result = new ArrayList<>(); + if (i.getPid() != null) + i + .getPid() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + if (i.getAlternateIdentifier() != null) + i + .getAlternateIdentifier() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + return result.stream(); + }) + .collect( + Collectors + .toMap( + Pair::getLeft, + Pair::getRight, + (a, b) -> a)); } private static boolean isFromDelegatedAuthority(Result r) { return Optional - .ofNullable(r.getInstance()) - .map( - instance -> instance - .stream() - .filter(i -> Objects.nonNull(i.getCollectedfrom())) - .map(i -> i.getCollectedfrom().getKey()) - .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) - .orElse(false); + .ofNullable(r.getInstance()) + .map( + instance -> instance + .stream() + .filter(i -> Objects.nonNull(i.getCollectedfrom())) + .map(i -> i.getCollectedfrom().getKey()) + .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) + .orElse(false); } /** @@ -1025,15 +1030,15 @@ public class MergeUtils { * @return the list */ private static List findEnrichmentsByPID(final List pids, - final Map enrichments) { + final Map enrichments) { if (pids == null || enrichments == null) return null; return pids - .stream() - .map(MergeUtils::extractKeyFromPid) - .map(enrichments::get) - .filter(Objects::nonNull) - .collect(Collectors.toList()); + .stream() + .map(MergeUtils::extractKeyFromPid) + .map(enrichments::get) + .filter(Objects::nonNull) + .collect(Collectors.toList()); } /** @@ -1044,8 +1049,8 @@ public class MergeUtils { */ private static boolean isAnEnrichment(OafEntity e) { return e.getDataInfo() != null && - e.getDataInfo().getProvenanceaction() != null - && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); + e.getDataInfo().getProvenanceaction() != null + && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); } /** @@ -1068,17 +1073,17 @@ public class MergeUtils { merge.setHostedby(firstNonNull(merge.getHostedby(), enrichment.getHostedby())); merge.setUrl(unionDistinctLists(merge.getUrl(), enrichment.getUrl(), 0)); merge - .setDistributionlocation( - firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); + .setDistributionlocation( + firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); merge.setCollectedfrom(firstNonNull(merge.getCollectedfrom(), enrichment.getCollectedfrom())); // pid and alternateId are used for matching merge.setDateofacceptance(firstNonNull(merge.getDateofacceptance(), enrichment.getDateofacceptance())); merge - .setProcessingchargeamount( - firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); + .setProcessingchargeamount( + firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); merge - .setProcessingchargecurrency( - firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); + .setProcessingchargecurrency( + firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); merge.setRefereed(firstNonNull(merge.getRefereed(), enrichment.getRefereed())); merge.setMeasures(unionDistinctLists(merge.getMeasures(), enrichment.getMeasures(), 0)); merge.setFulltext(firstNonNull(merge.getFulltext(), enrichment.getFulltext())); @@ -1086,14 +1091,14 @@ public class MergeUtils { private static int compareTrust(Oaf a, Oaf b) { String left = Optional - .ofNullable(a.getDataInfo()) - .map(DataInfo::getTrust) - .orElse("0.0"); + .ofNullable(a.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); String right = Optional - .ofNullable(b.getDataInfo()) - .map(DataInfo::getTrust) - .orElse("0.0"); + .ofNullable(b.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); return left.compareTo(right); } From a1d5ad5c2609c91b60b97600163072d551dcc440 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 13 Nov 2024 09:51:13 +0100 Subject: [PATCH 204/239] code formatted --- .../dhp/schema/oaf/utils/MergeUtils.java | 423 +++++++++--------- .../personentity/ExtractPerson.java | 22 +- .../plugin/gtr2/Gtr2PublicationsIterator.java | 6 +- 3 files changed, 228 insertions(+), 223 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index dc76860f8..cd8506583 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -16,8 +16,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; -import eu.dnetlib.dhp.schema.common.EntityType; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -25,8 +23,10 @@ import org.apache.commons.lang3.tuple.Pair; import com.github.sisyphsu.dateparser.DateParserUtils; import com.google.common.base.Joiner; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.merge.AuthorMerger; import eu.dnetlib.dhp.schema.common.AccessRightComparator; +import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; @@ -46,7 +46,7 @@ public class MergeUtils { } public static T mergeGroup(Iterator oafEntityIterator, - boolean checkDelegateAuthority, VocabularyGroup vocs) { + boolean checkDelegateAuthority, VocabularyGroup vocs) { ArrayList sortedEntities = new ArrayList<>(); oafEntityIterator.forEachRemaining(sortedEntities::add); @@ -74,14 +74,15 @@ public class MergeUtils { if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { return (T) mergedResult; } else { - final Qualifier expectedResultType = vocs.lookupTermBySynonym( + final Qualifier expectedResultType = vocs + .lookupTermBySynonym( ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid()); if (Objects.isNull(expectedResultType)) { throw new IllegalArgumentException( - "instance type not bound to any result type in dnet:result_typologies: " + - i.getInstancetype().getClassid()); + "instance type not bound to any result type in dnet:result_typologies: " + + i.getInstancetype().getClassid()); } // there is a clash among the result types @@ -122,10 +123,10 @@ public class MergeUtils { return mergeRelation((Relation) left, (Relation) right); } else { throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); } } @@ -164,10 +165,10 @@ public class MergeUtils { return mergeProject((Project) left, (Project) right); } else { throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); } } @@ -258,7 +259,7 @@ public class MergeUtils { } private static List mergeLists(final List left, final List right, int trust, - Function keyExtractor, BinaryOperator merger) { + Function keyExtractor, BinaryOperator merger) { if (left == null || left.isEmpty()) { return right != null ? right : new ArrayList<>(); } else if (right == null || right.isEmpty()) { @@ -269,11 +270,11 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return new ArrayList<>(Stream - .concat(h.stream(), l.stream()) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.toMap(keyExtractor, v -> v, merger, LinkedHashMap::new)) - .values()); + .concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toMap(keyExtractor, v -> v, merger, LinkedHashMap::new)) + .values()); } private static List unionDistinctLists(final List left, final List right, int trust) { @@ -287,10 +288,10 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return Stream - .concat(h.stream(), l.stream()) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.toList()); + .concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toList()); } private static List unionDistinctListOfString(final List l, final List r) { @@ -301,10 +302,10 @@ public class MergeUtils { } return Stream - .concat(l.stream(), r.stream()) - .filter(StringUtils::isNotBlank) - .distinct() - .collect(Collectors.toList()); + .concat(l.stream(), r.stream()) + .filter(StringUtils::isNotBlank) + .distinct() + .collect(Collectors.toList()); } // TODO review @@ -330,7 +331,7 @@ public class MergeUtils { } private static List unionTitle(List left, List right, - int trust) { + int trust) { if (left == null) { return right; } else if (right == null) { @@ -341,10 +342,10 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return Stream - .concat(h.stream(), l.stream()) - .filter(Objects::isNull) - .distinct() - .collect(Collectors.toList()); + .concat(h.stream(), l.stream()) + .filter(Objects::isNull) + .distinct() + .collect(Collectors.toList()); } /** @@ -379,8 +380,8 @@ public class MergeUtils { merged.setPid(mergeLists(merged.getPid(), enrich.getPid(), trust, MergeUtils::spKeyExtractor, (p1, p2) -> p1)); merged.setDateofcollection(LocalDateTime.now().toString()); merged - .setDateoftransformation( - chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); + .setDateoftransformation( + chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); merged.setExtraInfo(unionDistinctLists(merged.getExtraInfo(), enrich.getExtraInfo(), trust)); // When merging records OAI provenance becomes null merged.setOaiprovenance(null); @@ -397,7 +398,7 @@ public class MergeUtils { checkArgument(Objects.equals(merge.getTarget(), enrich.getTarget()), "target ids must be equal"); checkArgument(Objects.equals(merge.getRelType(), enrich.getRelType()), "relType(s) must be equal"); checkArgument( - Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); + Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); checkArgument(Objects.equals(merge.getRelClass(), enrich.getRelClass()), "relClass(es) must be equal"); // merge.setProvenance(mergeLists(merge.getProvenance(), enrich.getProvenance())); @@ -408,10 +409,10 @@ public class MergeUtils { merge.setValidationDate(ModelSupport.oldest(merge.getValidationDate(), enrich.getValidationDate())); } catch (ParseException e) { throw new IllegalArgumentException(String - .format( - "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), - merge.getTarget(), - merge.getValidationDate())); + .format( + "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), + merge.getTarget(), + merge.getValidationDate())); } // TODO keyvalue merge @@ -425,7 +426,7 @@ public class MergeUtils { T merge = mergeOafEntityFields(original, enrich, trust); if (merge.getProcessingchargeamount() == null - || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { + || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { merge.setProcessingchargeamount(enrich.getProcessingchargeamount()); merge.setProcessingchargecurrency(enrich.getProcessingchargecurrency()); } @@ -457,8 +458,8 @@ public class MergeUtils { } merge - .setDateofacceptance( - mergeDateOfAcceptance(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); + .setDateofacceptance( + mergeDateOfAcceptance(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); merge.setPublisher(coalesce(merge.getPublisher(), enrich.getPublisher())); merge.setEmbargoenddate(coalesce(merge.getEmbargoenddate(), enrich.getEmbargoenddate())); @@ -473,7 +474,7 @@ public class MergeUtils { merge.setCoverage(unionDistinctLists(merge.getCoverage(), enrich.getCoverage(), trust)); if (enrich.getBestaccessright() != null - && new AccessRightComparator<>() + && new AccessRightComparator<>() .compare(enrich.getBestaccessright(), merge.getBestaccessright()) < 0) { merge.setBestaccessright(enrich.getBestaccessright()); } @@ -486,8 +487,8 @@ public class MergeUtils { // ok merge - .setExternalReference( - mergeExternalReference(merge.getExternalReference(), enrich.getExternalReference(), trust)); + .setExternalReference( + mergeExternalReference(merge.getExternalReference(), enrich.getExternalReference(), trust)); // instance enrichment or union // review instance equals => add pid to comparision @@ -495,17 +496,17 @@ public class MergeUtils { merge.setInstance(mergeInstances(merge.getInstance(), enrich.getInstance(), trust)); } else { final List enrichmentInstances = isAnEnrichment(merge) ? merge.getInstance() - : enrich.getInstance(); + : enrich.getInstance(); final List enrichedInstances = isAnEnrichment(merge) ? enrich.getInstance() - : merge.getInstance(); + : merge.getInstance(); if (isAnEnrichment(merge)) merge.setDataInfo(enrich.getDataInfo()); merge.setInstance(enrichInstances(enrichedInstances, enrichmentInstances)); } merge - .setEoscifguidelines( - mergeEosciifguidelines(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); + .setEoscifguidelines( + mergeEosciifguidelines(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); merge.setIsGreen(booleanOR(merge.getIsGreen(), enrich.getIsGreen())); // OK but should be list of values merge.setOpenAccessColor(coalesce(merge.getOpenAccessColor(), enrich.getOpenAccessColor())); @@ -531,7 +532,7 @@ public class MergeUtils { LocalDate enrich_date = LocalDate.parse(enrich.getValue(), DateTimeFormatter.ISO_DATE); if (enrich_date.getYear() > 1300 - && (merge_date.getYear() < 1300 || merge_date.isAfter(enrich_date))) { + && (merge_date.getYear() < 1300 || merge_date.isAfter(enrich_date))) { return enrich; } } catch (NullPointerException | DateTimeParseException e) { @@ -549,56 +550,56 @@ public class MergeUtils { private static List mergeInstances(List v1, List v2, int trust) { return mergeLists( - v1, v2, trust, - MergeUtils::instanceKeyExtractor, - MergeUtils::instanceMerger); + v1, v2, trust, + MergeUtils::instanceKeyExtractor, + MergeUtils::instanceMerger); } private static List mergeEosciifguidelines(List v1, List v2, - int trust) { + int trust) { return mergeLists( - v1, v2, trust, er -> Joiner - .on("||") - .useForNull("") - .join(er.getCode(), er.getLabel(), er.getUrl(), er.getSemanticRelation()), - (r, l) -> r); + v1, v2, trust, er -> Joiner + .on("||") + .useForNull("") + .join(er.getCode(), er.getLabel(), er.getUrl(), er.getSemanticRelation()), + (r, l) -> r); } private static List mergeExternalReference(List v1, - List v2, int trust) { + List v2, int trust) { return mergeLists( - v1, v2, trust, er -> Joiner - .on(',') - .useForNull("") - .join( - er.getSitename(), er.getLabel(), - er.getUrl(), toString(er.getQualifier()), er.getRefidentifier(), - er.getQuery(), toString(er.getDataInfo())), - (r, l) -> r); + v1, v2, trust, er -> Joiner + .on(',') + .useForNull("") + .join( + er.getSitename(), er.getLabel(), + er.getUrl(), toString(er.getQualifier()), er.getRefidentifier(), + er.getQuery(), toString(er.getDataInfo())), + (r, l) -> r); } private static String toString(DataInfo di) { return Joiner - .on(',') - .useForNull("") - .join( - di.getInvisible(), di.getInferred(), di.getDeletedbyinference(), di.getTrust(), - di.getInferenceprovenance(), toString(di.getProvenanceaction())); + .on(',') + .useForNull("") + .join( + di.getInvisible(), di.getInferred(), di.getDeletedbyinference(), di.getTrust(), + di.getInferenceprovenance(), toString(di.getProvenanceaction())); } private static String toString(Qualifier q) { return Joiner - .on(',') - .useForNull("") - .join(q.getClassid(), q.getClassname(), q.getSchemeid(), q.getSchemename()); + .on(',') + .useForNull("") + .join(q.getClassid(), q.getClassname(), q.getSchemeid(), q.getSchemename()); } private static String toString(StructuredProperty sp) { return Joiner - .on(',') - .useForNull("") - .join(toString(sp.getQualifier()), sp.getValue()); + .on(',') + .useForNull("") + .join(toString(sp.getQualifier()), sp.getValue()); } private static List mergeStructuredProperties(List v1, List v2, int trust) { @@ -637,17 +638,17 @@ public class MergeUtils { // 2. @@ // 3. || return String - .join( - "::", - kvKeyExtractor(i.getHostedby()), - kvKeyExtractor(i.getCollectedfrom()), - qualifierKeyExtractor(i.getAccessright()), - qualifierKeyExtractor(i.getInstancetype()), - Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), - Optional - .ofNullable(i.getPid()) - .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) - .orElse(null)); + .join( + "::", + kvKeyExtractor(i.getHostedby()), + kvKeyExtractor(i.getCollectedfrom()), + qualifierKeyExtractor(i.getAccessright()), + qualifierKeyExtractor(i.getInstancetype()), + Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), + Optional + .ofNullable(i.getPid()) + .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) + .orElse(null)); } private static Instance instanceMerger(Instance i1, Instance i2) { @@ -658,30 +659,30 @@ public class MergeUtils { i.setInstancetype(i1.getInstancetype()); i.setPid(mergeLists(i1.getPid(), i2.getPid(), 0, MergeUtils::spKeyExtractor, (sp1, sp2) -> sp1)); i - .setAlternateIdentifier( - mergeLists( - i1.getAlternateIdentifier(), i2.getAlternateIdentifier(), 0, MergeUtils::spKeyExtractor, - (sp1, sp2) -> sp1)); + .setAlternateIdentifier( + mergeLists( + i1.getAlternateIdentifier(), i2.getAlternateIdentifier(), 0, MergeUtils::spKeyExtractor, + (sp1, sp2) -> sp1)); i - .setRefereed( - Collections - .min( - Stream.of(i1.getRefereed(), i2.getRefereed()).collect(Collectors.toList()), - new RefereedComparator())); + .setRefereed( + Collections + .min( + Stream.of(i1.getRefereed(), i2.getRefereed()).collect(Collectors.toList()), + new RefereedComparator())); i - .setInstanceTypeMapping( - mergeLists( - i1.getInstanceTypeMapping(), i2.getInstanceTypeMapping(), 0, - MergeUtils::instanceTypeMappingKeyExtractor, (itm1, itm2) -> itm1)); + .setInstanceTypeMapping( + mergeLists( + i1.getInstanceTypeMapping(), i2.getInstanceTypeMapping(), 0, + MergeUtils::instanceTypeMappingKeyExtractor, (itm1, itm2) -> itm1)); i.setFulltext(selectFulltext(i1.getFulltext(), i2.getFulltext())); i.setDateofacceptance(selectOldestDate(i1.getDateofacceptance(), i2.getDateofacceptance())); i.setLicense(coalesce(i1.getLicense(), i2.getLicense())); i.setProcessingchargeamount(coalesce(i1.getProcessingchargeamount(), i2.getProcessingchargeamount())); i.setProcessingchargecurrency(coalesce(i1.getProcessingchargecurrency(), i2.getProcessingchargecurrency())); i - .setMeasures( - mergeLists(i1.getMeasures(), i2.getMeasures(), 0, MergeUtils::measureKeyExtractor, (m1, m2) -> m1)); + .setMeasures( + mergeLists(i1.getMeasures(), i2.getMeasures(), 0, MergeUtils::measureKeyExtractor, (m1, m2) -> m1)); i.setUrl(unionDistinctListOfString(i1.getUrl(), i2.getUrl())); @@ -690,14 +691,14 @@ public class MergeUtils { private static String measureKeyExtractor(Measure m) { return String - .join( - "::", - m.getId(), - m - .getUnit() - .stream() - .map(KeyValue::getKey) - .collect(Collectors.joining("::"))); + .join( + "::", + m.getId(), + m + .getUnit() + .stream() + .map(KeyValue::getKey) + .collect(Collectors.joining("::"))); } private static Field selectOldestDate(Field d1, Field d2) { @@ -708,16 +709,16 @@ public class MergeUtils { } return Stream - .of(d1, d2) - .min( - Comparator - .comparing( - f -> DateParserUtils - .parseDate(f.getValue()) - .toInstant() - .atZone(ZoneId.systemDefault()) - .toLocalDate())) - .orElse(d1); + .of(d1, d2) + .min( + Comparator + .comparing( + f -> DateParserUtils + .parseDate(f.getValue()) + .toInstant() + .atZone(ZoneId.systemDefault()) + .toLocalDate())) + .orElse(d1); } private static String selectFulltext(String ft1, String ft2) { @@ -732,12 +733,12 @@ public class MergeUtils { private static String instanceTypeMappingKeyExtractor(InstanceTypeMapping itm) { return String - .join( - "::", - itm.getOriginalType(), - itm.getTypeCode(), - itm.getTypeLabel(), - itm.getVocabularyName()); + .join( + "::", + itm.getOriginalType(), + itm.getTypeCode(), + itm.getTypeLabel(), + itm.getVocabularyName()); } private static String kvKeyExtractor(KeyValue kv) { @@ -754,13 +755,13 @@ public class MergeUtils { private static String spKeyExtractor(StructuredProperty sp) { return Optional - .ofNullable(sp) - .map( - s -> Joiner - .on("||") - .useForNull("") - .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) - .orElse(null); + .ofNullable(sp) + .map( + s -> Joiner + .on("||") + .useForNull("") + .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) + .orElse(null); } private static T mergeORP(T original, T enrich) { @@ -782,8 +783,8 @@ public class MergeUtils { merge.setLicense(unionDistinctLists(merge.getLicense(), enrich.getLicense(), trust)); merge.setCodeRepositoryUrl(chooseReference(merge.getCodeRepositoryUrl(), enrich.getCodeRepositoryUrl(), trust)); merge - .setProgrammingLanguage( - chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); + .setProgrammingLanguage( + chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); return merge; } @@ -797,11 +798,11 @@ public class MergeUtils { merge.setSize(chooseReference(merge.getSize(), enrich.getSize(), trust)); merge.setVersion(chooseReference(merge.getVersion(), enrich.getVersion(), trust)); merge - .setLastmetadataupdate( - chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); + .setLastmetadataupdate( + chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); merge - .setMetadataversionnumber( - chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); + .setMetadataversionnumber( + chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); merge.setGeolocation(unionDistinctLists(merge.getGeolocation(), enrich.getGeolocation(), trust)); return merge; @@ -823,26 +824,26 @@ public class MergeUtils { merged.setLegalshortname(chooseReference(merged.getLegalshortname(), enrich.getLegalshortname(), trust)); merged.setLegalname(chooseReference(merged.getLegalname(), enrich.getLegalname(), trust)); merged - .setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); + .setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust)); merged.setLogourl(chooseReference(merged.getLogourl(), enrich.getLogourl(), trust)); merged.setEclegalbody(chooseReference(merged.getEclegalbody(), enrich.getEclegalbody(), trust)); merged.setEclegalperson(chooseReference(merged.getEclegalperson(), enrich.getEclegalperson(), trust)); merged.setEcnonprofit(chooseReference(merged.getEcnonprofit(), enrich.getEcnonprofit(), trust)); merged - .setEcresearchorganization( - chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); + .setEcresearchorganization( + chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); merged - .setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); + .setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); merged - .setEcinternationalorganizationeurinterests( - chooseReference( - merged.getEcinternationalorganizationeurinterests(), - enrich.getEcinternationalorganizationeurinterests(), trust)); + .setEcinternationalorganizationeurinterests( + chooseReference( + merged.getEcinternationalorganizationeurinterests(), + enrich.getEcinternationalorganizationeurinterests(), trust)); merged - .setEcinternationalorganization( - chooseReference( - merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); + .setEcinternationalorganization( + chooseReference( + merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); merged.setEcenterprise(chooseReference(merged.getEcenterprise(), enrich.getEcenterprise(), trust)); merged.setEcsmevalidated(chooseReference(merged.getEcsmevalidated(), enrich.getEcsmevalidated(), trust)); merged.setEcnutscode(chooseReference(merged.getEcnutscode(), enrich.getEcnutscode(), trust)); @@ -866,8 +867,8 @@ public class MergeUtils { merged.setDuration(chooseReference(merged.getDuration(), enrich.getDuration(), trust)); merged.setEcsc39(chooseReference(merged.getEcsc39(), enrich.getEcsc39(), trust)); merged - .setOamandatepublications( - chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); + .setOamandatepublications( + chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); merged.setEcarticle29_3(chooseReference(merged.getEcarticle29_3(), enrich.getEcarticle29_3(), trust)); merged.setSubjects(unionDistinctLists(merged.getSubjects(), enrich.getSubjects(), trust)); merged.setFundingtree(unionDistinctLists(merged.getFundingtree(), enrich.getFundingtree(), trust)); @@ -893,8 +894,8 @@ public class MergeUtils { } merged - .setH2020classification( - unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); + .setH2020classification( + unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); return merged; } @@ -921,7 +922,7 @@ public class MergeUtils { * @return list of instances possibly enriched */ private static List enrichInstances(final List toEnrichInstances, - final List enrichmentInstances) { + final List enrichmentInstances) { final List enrichmentResult = new ArrayList<>(); if (toEnrichInstances == null) { @@ -959,42 +960,42 @@ public class MergeUtils { */ private static Map toInstanceMap(final List ri) { return ri - .stream() - .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) - .flatMap(i -> { - final List> result = new ArrayList<>(); - if (i.getPid() != null) - i - .getPid() - .stream() - .filter(MergeUtils::validPid) - .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); - if (i.getAlternateIdentifier() != null) - i - .getAlternateIdentifier() - .stream() - .filter(MergeUtils::validPid) - .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); - return result.stream(); - }) - .collect( - Collectors - .toMap( - Pair::getLeft, - Pair::getRight, - (a, b) -> a)); + .stream() + .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) + .flatMap(i -> { + final List> result = new ArrayList<>(); + if (i.getPid() != null) + i + .getPid() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + if (i.getAlternateIdentifier() != null) + i + .getAlternateIdentifier() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + return result.stream(); + }) + .collect( + Collectors + .toMap( + Pair::getLeft, + Pair::getRight, + (a, b) -> a)); } private static boolean isFromDelegatedAuthority(Result r) { return Optional - .ofNullable(r.getInstance()) - .map( - instance -> instance - .stream() - .filter(i -> Objects.nonNull(i.getCollectedfrom())) - .map(i -> i.getCollectedfrom().getKey()) - .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) - .orElse(false); + .ofNullable(r.getInstance()) + .map( + instance -> instance + .stream() + .filter(i -> Objects.nonNull(i.getCollectedfrom())) + .map(i -> i.getCollectedfrom().getKey()) + .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) + .orElse(false); } /** @@ -1030,15 +1031,15 @@ public class MergeUtils { * @return the list */ private static List findEnrichmentsByPID(final List pids, - final Map enrichments) { + final Map enrichments) { if (pids == null || enrichments == null) return null; return pids - .stream() - .map(MergeUtils::extractKeyFromPid) - .map(enrichments::get) - .filter(Objects::nonNull) - .collect(Collectors.toList()); + .stream() + .map(MergeUtils::extractKeyFromPid) + .map(enrichments::get) + .filter(Objects::nonNull) + .collect(Collectors.toList()); } /** @@ -1049,8 +1050,8 @@ public class MergeUtils { */ private static boolean isAnEnrichment(OafEntity e) { return e.getDataInfo() != null && - e.getDataInfo().getProvenanceaction() != null - && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); + e.getDataInfo().getProvenanceaction() != null + && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); } /** @@ -1073,17 +1074,17 @@ public class MergeUtils { merge.setHostedby(firstNonNull(merge.getHostedby(), enrichment.getHostedby())); merge.setUrl(unionDistinctLists(merge.getUrl(), enrichment.getUrl(), 0)); merge - .setDistributionlocation( - firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); + .setDistributionlocation( + firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); merge.setCollectedfrom(firstNonNull(merge.getCollectedfrom(), enrichment.getCollectedfrom())); // pid and alternateId are used for matching merge.setDateofacceptance(firstNonNull(merge.getDateofacceptance(), enrichment.getDateofacceptance())); merge - .setProcessingchargeamount( - firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); + .setProcessingchargeamount( + firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); merge - .setProcessingchargecurrency( - firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); + .setProcessingchargecurrency( + firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); merge.setRefereed(firstNonNull(merge.getRefereed(), enrichment.getRefereed())); merge.setMeasures(unionDistinctLists(merge.getMeasures(), enrichment.getMeasures(), 0)); merge.setFulltext(firstNonNull(merge.getFulltext(), enrichment.getFulltext())); @@ -1091,14 +1092,14 @@ public class MergeUtils { private static int compareTrust(Oaf a, Oaf b) { String left = Optional - .ofNullable(a.getDataInfo()) - .map(DataInfo::getTrust) - .orElse("0.0"); + .ofNullable(a.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); String right = Optional - .ofNullable(b.getDataInfo()) - .map(DataInfo::getTrust) - .orElse("0.0"); + .ofNullable(b.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); return left.compareTo(right); } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java index 06924f05a..05f083740 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java @@ -346,15 +346,19 @@ public class ExtractPerson implements Serializable { .structuredProperty( op.getOrcid(), ModelConstants.ORCID, ModelConstants.ORCID_CLASSNAME, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, - OafMapperUtils.dataInfo(false, - null, - false, - false, - OafMapperUtils.qualifier(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY, - ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY, - ModelConstants.DNET_PID_TYPES, - ModelConstants.DNET_PID_TYPES), - "0.91"))); + OafMapperUtils + .dataInfo( + false, + null, + false, + false, + OafMapperUtils + .qualifier( + ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY, + ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY, + ModelConstants.DNET_PID_TYPES, + ModelConstants.DNET_PID_TYPES), + "0.91"))); person.setDateofcollection(op.getLastModifiedDate()); person.setOriginalId(Arrays.asList(op.getOrcid())); person.setDataInfo(ORCIDDATAINFO); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java index 779c43712..1b1ff8db4 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/gtr2/Gtr2PublicationsIterator.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.collection.plugin.gtr2; import java.nio.charset.StandardCharsets; import java.time.LocalDate; +import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -18,7 +19,6 @@ import org.dom4j.Document; import org.dom4j.DocumentException; import org.dom4j.DocumentHelper; import org.dom4j.Element; -import java.time.format.DateTimeFormatter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -188,11 +188,11 @@ public class Gtr2PublicationsIterator implements Iterator { private Document loadURL(final String cleanUrl, final int attempt) { try { - log.debug(" * Downloading Url: {}", cleanUrl); + log.debug(" * Downloading Url: {}", cleanUrl); final byte[] bytes = this.connector.getInputSource(cleanUrl).getBytes(StandardCharsets.UTF_8); return DocumentHelper.parseText(new String(bytes)); } catch (final Throwable e) { - log.error("Error dowloading url: {}, attempt = {}", cleanUrl, attempt, e); + log.error("Error dowloading url: {}, attempt = {}", cleanUrl, attempt, e); if (attempt >= MAX_ATTEMPTS) { throw new RuntimeException("Error downloading url: " + cleanUrl, e); } From 03c262ccb981bed4d6351705026e699963c9e4fc Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Wed, 13 Nov 2024 10:56:17 +0100 Subject: [PATCH 205/239] Crossref: generate canonical openaire id for results in affiliation relationship --- .gitignore | 1 + .../eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 6fafc7055..ef9144ae3 100644 --- a/.gitignore +++ b/.gitignore @@ -28,3 +28,4 @@ spark-warehouse /**/.scalafmt.conf /.java-version /dhp-shade-package/dependency-reduced-pom.xml +/**/job.properties diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index e7d68920b..d3a68c92e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -673,11 +673,12 @@ case object Crossref2Oaf { val doi = input.getString(0) val rorId = input.getString(1) - val pubId = s"50|${PidType.doi.toString.padTo(12, "_")}::${DoiCleaningRule.clean(doi)}" + + val pubId = IdentifierFactory.idFromPid("50", "doi", DoiCleaningRule.clean(doi), true) val affId = GenerateRorActionSetJob.calculateOpenaireId(rorId) val r: Relation = new Relation - DoiCleaningRule.clean(doi) + r.setSource(pubId) r.setTarget(affId) r.setRelType(ModelConstants.RESULT_ORGANIZATION) From fb1f0f8850b867f758fffdf9751ec9e4d2543db5 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 7 Nov 2024 14:05:02 +0100 Subject: [PATCH 206/239] [danishfunders] added the possibility to link also versus a specif award if present in the metadata --- .../collection/crossref/Crossref2Oaf.scala | 21 ++++++++++++++++++- .../doiboost/crossref/Crossref2Oaf.scala | 3 +++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index e7d68920b..59a12bc03 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -978,7 +978,26 @@ case object Crossref2Oaf { case "10.13039/501100010790" => generateSimpleRelationFromAward(funder, "erasmusplus_", a => a) case _ => logger.debug("no match for " + funder.DOI.get) - + //Add for Danish funders + //Independent Research Fund Denmark (IRFD) + case "10.13039/501100004836" => + generateSimpleRelationFromAward(funder, "irfd________", a => a) + val targetId = getProjectId("irfd________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + //Carlsberg Foundation (CF) + case "10.13039/501100002808" => + generateSimpleRelationFromAward(funder, "cf__________", a => a) + val targetId = getProjectId("cf__________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + //Novo Nordisk Foundation (NNF) + case "10.13039/501100009708" => + generateSimpleRelationFromAward(funder, "nnf___________", a => a) + val targetId = getProjectId("nnf_________", "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + case _ => logger.debug("no match for " + funder.DOI.get) } } else { diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index bf11ed0a8..031a04058 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -569,16 +569,19 @@ case object Crossref2Oaf { //Add for Danish funders //Independent Research Fund Denmark (IRFD) case "10.13039/501100004836" => + generateSimpleRelationFromAward(funder, "irfd________", a => a) val targetId = getProjectId("irfd________", "1e5e62235d094afd01cd56e65112fc63") queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) //Carlsberg Foundation (CF) case "10.13039/501100002808" => + generateSimpleRelationFromAward(funder, "cf__________", a => a) val targetId = getProjectId("cf__________", "1e5e62235d094afd01cd56e65112fc63") queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) //Novo Nordisk Foundation (NNF) case "10.13039/501100009708" => + generateSimpleRelationFromAward(funder, "nnf___________", a => a) val targetId = getProjectId("nnf_________", "1e5e62235d094afd01cd56e65112fc63") queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) From ac0a94d62d7c34fc3953d47a9a2263ac5cfadb82 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 13 Nov 2024 16:26:59 +0100 Subject: [PATCH 207/239] updated pubmed parser to add also ORCID id and affiliation string to authors --- .../dhp/sx/bio/pubmed/PMAffiliation.java | 39 +++ .../dnetlib/dhp/sx/bio/pubmed/PMAuthor.java | 39 +++ .../dhp/sx/bio/pubmed/PMIdentifier.java | 53 +++++ .../dnetlib/dhp/sx/bio/pubmed/PMParser2.scala | 30 ++- .../dhp/sx/bio/pubmed/PubMedToOaf.scala | 6 + .../dhp/sx/graph/bio/single_pubmed.xml | 223 +++++++----------- .../dnetlib/dhp/sx/bio/BioScholixTest.scala | 64 ++++- 7 files changed, 300 insertions(+), 154 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java new file mode 100644 index 000000000..54aba8715 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java @@ -0,0 +1,39 @@ +package eu.dnetlib.dhp.sx.bio.pubmed; + +/** + * The type Pubmed Affiliation. + * + * @author Sandro La Bruzzo + */ +public class PMAffiliation { + + private String name; + + private PMIdentifier identifier; + + public PMAffiliation() { + + } + public PMAffiliation(String name, PMIdentifier identifier) { + this.name = name; + this.identifier = identifier; + } + + public String getName() { + return name; + } + + public PMAffiliation setName(String name) { + this.name = name; + return this; + } + + public PMIdentifier getIdentifier() { + return identifier; + } + + public PMAffiliation setIdentifier(PMIdentifier identifier) { + this.identifier = identifier; + return this; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java index 68ef6459e..b0df25663 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java @@ -12,6 +12,8 @@ public class PMAuthor implements Serializable { private String lastName; private String foreName; + private PMIdentifier identifier; + private PMAffiliation affiliation; /** * Gets last name. @@ -59,4 +61,41 @@ public class PMAuthor implements Serializable { .format("%s, %s", this.foreName != null ? this.foreName : "", this.lastName != null ? this.lastName : ""); } + /** + * Gets identifier. + * + * @return the identifier + */ + public PMIdentifier getIdentifier() { + return identifier; + } + + /** + * Sets identifier. + * + * @param identifier the identifier + */ + public void setIdentifier(PMIdentifier identifier) { + this.identifier = identifier; + } + + /** + * Gets affiliation. + * + * @return the affiliation + */ + public PMAffiliation getAffiliation() { + return affiliation; + } + + /** + * Sets affiliation. + * + * @param affiliation the affiliation + */ + public void setAffiliation(PMAffiliation affiliation) { + this.affiliation = affiliation; + } + + } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java new file mode 100644 index 000000000..0c8c55e40 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java @@ -0,0 +1,53 @@ +package eu.dnetlib.dhp.sx.bio.pubmed; + +public class PMIdentifier { + + private String pid; + private String type; + + + public PMIdentifier(String pid, String type) { + this.pid = cleanPid(pid); + this.type = type; + } + + public PMIdentifier() { + + } + + private String cleanPid(String pid) { + + if (pid == null) { + return null; + } + + // clean ORCID ID in the form 0000000163025705 to 0000-0001-6302-5705 + if (pid.matches("[0-9]{15}[0-9X]")) { + return pid.replaceAll("(.{4})(.{4})(.{4})(.{4})", "$1-$2-$3-$4"); + } + + // clean ORCID in the form http://orcid.org/0000-0001-8567-3543 to 0000-0001-8567-3543 + if (pid.matches("http://orcid.org/[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{4}")) { + return pid.replaceAll("http://orcid.org/", ""); + } + return pid; + } + + public String getPid() { + return pid; + } + + public PMIdentifier setPid(String pid) { + this.pid = cleanPid(pid); + return this; + } + + public String getType() { + return type; + } + + public PMIdentifier setType(String type) { + this.type = type; + return this; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala index c9e868185..2eb4bea65 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala @@ -81,6 +81,26 @@ class PMParser2 { val a = new PMAuthor a.setLastName((author \ "LastName").text) a.setForeName((author \ "ForeName").text) + val id = (author \ "Identifier").text + val idType =(author \ "Identifier" \ "@Source").text + + if(id != null && id.nonEmpty && idType != null && idType.nonEmpty) { + a.setIdentifier(new PMIdentifier(id, idType)) + } + + + val affiliation = (author \ "AffiliationInfo" \ "Affiliation").text + val affiliationId = (author \ "AffiliationInfo" \ "Identifier").text + val affiliationIdType = (author \ "AffiliationInfo" \ "Identifier" \ "@Source").text + + if(affiliation != null && affiliation.nonEmpty) { + val aff = new PMAffiliation() + aff.setName(affiliation) + if(affiliationId != null && affiliationId.nonEmpty && affiliationIdType != null && affiliationIdType.nonEmpty) { + aff.setIdentifier(new PMIdentifier(affiliationId, affiliationIdType)) + } + a.setAffiliation(aff) + } a }) .toList @@ -99,15 +119,7 @@ class PMParser2 { val authors = xml \ "MedlineCitation" \ "Article" \ "AuthorList" \ "Author" article.setAuthors( - authors - .map(author => { - val a = new PMAuthor - a.setLastName((author \ "LastName").text) - a.setForeName((author \ "ForeName").text) - a - }) - .toList - .asJava + extractAuthors(authors).asJava ) val pmId = xml \ "MedlineCitation" \ "PMID" diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala index d59d73bd0..5e14c731a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala @@ -294,6 +294,12 @@ object PubMedToOaf { author.setName(a.getForeName) author.setSurname(a.getLastName) author.setFullname(a.getFullName) + if(a.getIdentifier != null) { + author.setPid(List(OafMapperUtils.structuredProperty(a.getIdentifier.getPid, + OafMapperUtils.qualifier(a.getIdentifier.getType,a.getIdentifier.getType,ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES), dataInfo)).asJava) + } + if (a.getAffiliation!= null) + author.setRawAffiliationString(List(a.getAffiliation.getName).asJava) author.setRank(index + 1) author }(collection.breakOut) diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml index 4b4d860d7..c2e503f57 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml @@ -1,77 +1,56 @@ - - 37885214 + + 37318999 2024 02 - 14 + 09 2024 02 - 14 + 09
- 2752-7549 + 1522-1229 - 40 - 5 + 47 + 3 - 2023 Sep-Oct + 2023 + Sep + 01 - Journal of pediatric hematology/oncology nursing - J Pediatr Hematol Oncol Nurs + Advances in physiology education + Adv Physiol Educ - Care Needs of Parents of Children With Cancer in a Low-Middle-Income Country. + Providing the choice of in-person or videoconference attendance in a clinical physiology course may harm learning outcomes for the entire cohort. - 295-304 + 548-556 - 10.1177/27527530231193972 + 10.1152/advan.00160.2022 - Background: Mapping out actual supportive care needs assists nurses in providing holistic individualized care. This study aimed to explore the care needs of parents of children with cancer in the Philippines. Method: Guided by the Supportive Care Needs Framework (SCNF), this study used an embedded mixed-method design with the quantitative revised Cancer Patient Needs Questionnaire and qualitative semistructured interviews to describe parents' care needs and priorities. Results: Filipino parents (N = 156) of children with cancer have various care needs which could be classified along the SCNF categories-practical, informational, spiritual, physical, emotional, and physical needs as ranked from highest to lowest. A number of variables were significantly associated with care needs. Solid tumor diagnosis was associated with greater practical, emotional, and psychosocial care needs; having a child who had undergone surgery was associated with more practical and spiritual care needs; and being within one year of the child's diagnosis was associated with practical, psychosocial, and spiritual care needs. Parent priority needs included (a) addressing financial needs; (b) access to temporary housing to minimize treatment-related costs; (c) support groups among parents of children with cancer as a source of information; (d) financial and social support between members of family and partners of parents of children with cancer; and (e) using prayer to facilitate acceptance. Conclusions: Supportive care needs of parents of children with cancer are important components of care that should be given recognition to enhance holistic individualized care throughout the childhood cancer experience. + Clinical Physiology 1 and 2 are flipped classes in which students watch prerecorded videos before class. During the 3-h class, students take practice assessments, work in groups on critical thinking exercises, work through case studies, and engage in drawing exercises. Due to the COVID pandemic, these courses were transitioned from in-person classes to online classes. Despite the university's return-to-class policy, some students were reluctant to return to in-person classes; therefore during the 2021-2022 academic year, Clinical Physiology 1 and 2 were offered as flipped, hybrid courses. In a hybrid format, students either attended the synchronous class in person or online. Here we evaluate the learning outcomes and the perceptions of the learning experience for students who attended Clinical Physiology 1 and 2 either online (2020-2021) or in a hybrid format (2021-2022). In addition to exam scores, in-class surveys and end of course evaluations were compiled to describe the student experience in the flipped hybrid setting. Retrospective linear mixed-model regression analysis of exam scores revealed that a hybrid modality (2021-2022) was associated with lower exam scores when controlling for sex, graduate/undergraduate status, delivery method, and the order in which the courses were taken (F test: F = 8.65, df1 = 2, df2 = 179.28, P = 0.0003). In addition, being a Black Indigenous Person of Color (BIPOC) student is associated with a lower exam score, controlling for the same previous factors (F test: F = 4.23, df1 = 1, df2 = 130.28, P = 0.04), albeit with lower confidence; the BIPOC representation in this sample is small (BIPOC: n = 144; total: n = 504). There is no significant interaction between the hybrid modality and race, meaning that BIPOC and White students are both negatively affected in a hybrid flipped course. Instructors should consider carefully about offering hybrid courses and build in extra student support.NEW & NOTEWORTHY The transition from online to in-person teaching has been as challenging as the original transition to remote teaching with the onset of the pandemic. Since not all students were ready to return to the classroom, students could choose to take this course in person or online. This arrangement provided flexibility and opportunities for innovative class activities for students but introduced tradeoffs in lower test scores from the hybrid modality than fully online or fully in-person modalities. - Banayat - Aprille Campos - AC - 0000-0001-9339-9871 + Anderson + Lisa Carney + LC + 0000-0003-2261-1921 - College of Nursing, University of the Philippines Manila, Manila, Philippines. + Department of Integrative Biology and Physiology, University of Minnesota, Minneapolis, Minnesota, United States. + https://ror.org/017zqws13 - Abad - Peter James B - PJB + Jacobson + Tate + T - College of Nursing, University of the Philippines Manila, Manila, Philippines. - - - - Bonito - Sheila R - SR - - College of Nursing, University of the Philippines Manila, Manila, Philippines. - - - - Manahan - Lydia T - LT - - College of Nursing, University of the Philippines Manila, Manila, Philippines. - - - - Peralta - Arnold B - AB - - College of Nursing, University of the Philippines Manila, Manila, Philippines. + Department of Statistics, University of Minnesota, Minneapolis, Minnesota, United States. @@ -81,142 +60,98 @@ 2023 - 10 - 26 + 06 + 15
United States - J Pediatr Hematol Oncol Nurs - 9918282681506676 - 2752-7530 + Adv Physiol Educ + 100913944 + 1043-4046 IM - Child + Physiology + education + + + Retrospective Studies + + + Learning + + + Pandemics + + + COVID-19 + + + Regression Analysis + + + Students Humans - Parents - psychology + Male - Social Support + Female - Spirituality + White People - Religion + Black People - Neoplasms - therapy + Education, Distance + + + Curriculum - cancer - mixed methods - parent - pediatric - research - supportive care + flipped teaching + hybrid teaching + inequity + learning outcomes + responsive teaching - Declaration of Conflicting InterestsThe author(s) declared no potential conflicts of interest with respect to the research, authorship, and/or publication of this article.
- 2024 - 2 - 12 - 18 - 42 + 2023 + 7 + 21 + 6 + 44 2023 - 10 - 27 - 6 - 42 + 6 + 15 + 19 + 14 2023 - 10 - 27 - 3 - 43 + 6 + 15 + 12 + 53 ppublish - 37885214 - 10.1177/27527530231193972 + 37318999 + 10.1152/advan.00160.2022
- -30522158 -32769323 -34061701 -34661197 -34837091 -35035475 -35211699 -35557982 -35782783 -35795240 -35832688 -35847411 -36081602 -36081858 -36468085 -36468934 -36580086 -36589526 -36619609 -36649460 -36654909 -36655054 -36700856 -36705625 -36713939 -36714172 -36741203 -36741905 -36743825 -36788221 -36844926 -36846546 -36935776 -36946757 -36972191 -37034422 -37124311 -37152108 -37171968 -37273889 -37333905 -37387733 -37431449 -37576947 -37601162 -37711214 -37901290 -37981909 -37981945 -37982005 -38037601 -38037602 -38150730 -38274640 -38332671 -38334184 -38335456 -38349506 -38349576 -38353676 - \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala index c942ca132..4a926df01 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala @@ -19,9 +19,11 @@ import org.mockito.junit.jupiter.MockitoExtension import org.slf4j.LoggerFactory import java.io.{BufferedReader, InputStream, InputStreamReader} +import java.util.regex.Pattern import java.util.zip.GZIPInputStream import javax.xml.stream.XMLInputFactory import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.collection.mutable.ListBuffer import scala.io.Source @@ -51,6 +53,64 @@ class BioScholixTest extends AbstractVocabularyTest { } } + @Test + def testPid(): Unit = { + val pids = List( + "0000000163025705", + "000000018494732X", + "0000000308873343", + "0000000335964515", + "0000000333457333", + "0000000335964515", + "0000000302921949", + + "http://orcid.org/0000-0001-8567-3543", + "http://orcid.org/0000-0001-7868-8528", + "0000-0001-9189-1440", + "0000-0003-3727-9247", + "0000-0001-7246-1058", + "000000033962389X", + "0000000330371470", + "0000000171236123", + "0000000272569752", + "0000000293231371", + "http://orcid.org/0000-0003-3345-7333", + "0000000340145688", + "http://orcid.org/0000-0003-4894-1689" + ) + + pids.foreach(pid => { + val pidCleaned = new PMIdentifier(pid, "ORCID").getPid + // assert pid is in the format of ORCID + println(pidCleaned) + assertTrue(pidCleaned.matches("[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{3}[0-9X]")) + }) + } + + def extractAffiliation(s: String): List[String] = { + val regex: String = "(.*)<\\/Affiliation>" + val pattern = Pattern.compile(regex, Pattern.MULTILINE) + val matcher = pattern.matcher(s) + val l: mutable.ListBuffer[String] = mutable.ListBuffer() + while (matcher.find()) { + l += matcher.group(1) + } + l.toList + } + + case class AuthorPID(pidType: String, pid: String) {} + + def extractAuthorIdentifier(s: String): List[AuthorPID] = { + val regex: String = "(.*)<\\/Identifier>" + val pattern = Pattern.compile(regex, Pattern.MULTILINE) + val matcher = pattern.matcher(s) + val l: mutable.ListBuffer[AuthorPID] = mutable.ListBuffer() + while (matcher.find()) { + l += AuthorPID(pidType = matcher.group(1), pid = matcher.group(2)) + } + l.toList + } + @Test def testParsingPubmed2(): Unit = { val mapper = new ObjectMapper() @@ -58,7 +118,9 @@ class BioScholixTest extends AbstractVocabularyTest { val parser = new PMParser2() val article = parser.parse(xml) - println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(article)) +// println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(article)) + + println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(PubMedToOaf.convert(article, vocabularies))) } From 4a3b173ca2d917c52de1671c352d1296ac211736 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 13 Nov 2024 16:27:00 +0100 Subject: [PATCH 208/239] defaults to 0000 - Unknown in case the instance type lookup in the dnet:result_typologies doesn't find a corresponding result type binding --- .../dhp/schema/oaf/utils/MergeUtils.java | 12 +--- .../raw/AbstractMdRecordToOafMapper.java | 55 ++++++++++--------- .../dhp/oa/graph/raw/OafToOafMapper.java | 4 +- .../dhp/oa/graph/raw/OdfToOafMapper.java | 4 +- 4 files changed, 36 insertions(+), 39 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index dc76860f8..c9b235fd6 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -74,15 +74,9 @@ public class MergeUtils { if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { return (T) mergedResult; } else { - final Qualifier expectedResultType = vocs.lookupTermBySynonym( - ModelConstants.DNET_RESULT_TYPOLOGIES, - i.getInstancetype().getClassid()); - - if (Objects.isNull(expectedResultType)) { - throw new IllegalArgumentException( - "instance type not bound to any result type in dnet:result_typologies: " + - i.getInstancetype().getClassid()); - } + final Qualifier expectedResultType = Optional + .ofNullable(vocs.lookupTermBySynonym(ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid())) + .orElse(OafMapperUtils.unknown(ModelConstants.DNET_RESULT_TYPOLOGIES, ModelConstants.DNET_RESULT_TYPOLOGIES)); // there is a clash among the result types if (!expectedResultType.getClassid().equals(mergedResult.getResulttype().getClassid())) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index ba6887a2e..be84778f5 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -153,30 +153,33 @@ public abstract class AbstractMdRecordToOafMapper { final DataInfo entityInfo = prepareDataInfo(doc, this.invisible); final long lastUpdateTimestamp = new Date().getTime(); - final List instances = prepareInstances(doc, entityInfo, collectedFrom, hostedBy); + final Instance instance = prepareInstances(doc, entityInfo, collectedFrom, hostedBy); - final String type = getResultType(instances); + if (!Optional.ofNullable(instance.getInstancetype()) + .map(Qualifier::getClassid) + .filter(StringUtils::isNotBlank) + .isPresent()) { + return Lists.newArrayList(); + } - return createOafs(doc, type, instances, collectedFrom, entityInfo, lastUpdateTimestamp); + final String type = getResultType(instance); + + return createOafs(doc, type, instance, collectedFrom, entityInfo, lastUpdateTimestamp); } catch (final DocumentException e) { log.error("Error with record:\n" + xml); return Lists.newArrayList(); } } - protected String getResultType(final List instances) { - + protected String getResultType(final Instance instance) { if (this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { - final String instanceType = instances - .stream() - .map(i -> i.getInstancetype().getClassid()) - .findFirst() - .filter(s -> !UNKNOWN.equalsIgnoreCase(s)) - .orElse("0000"); // Unknown - return Optional - .ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType)) - .map(Qualifier::getClassid) - .orElse("0000"); + return Optional.ofNullable(instance.getInstancetype()) + .map(Qualifier::getClassid) + .map(instanceType -> Optional + .ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType)) + .map(Qualifier::getClassid) + .orElse("0000")) + .orElse("0000"); } else { throw new IllegalStateException("Missing vocabulary: " + ModelConstants.DNET_RESULT_TYPOLOGIES); } @@ -196,12 +199,12 @@ public abstract class AbstractMdRecordToOafMapper { protected List createOafs( final Document doc, final String type, - final List instances, + final Instance instance, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { - final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp); + final OafEntity entity = createEntity(doc, type, instance, collectedFrom, info, lastUpdateTimestamp); final Set originalId = Sets.newHashSet(entity.getOriginalId()); originalId.add(entity.getId()); @@ -234,19 +237,19 @@ public abstract class AbstractMdRecordToOafMapper { private OafEntity createEntity(final Document doc, final String type, - final List instances, + final Instance instance, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { switch (type.toLowerCase()) { case "publication": final Publication p = new Publication(); - populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(p, doc, instance, collectedFrom, info, lastUpdateTimestamp); p.setJournal(prepareJournal(doc, info)); return p; case "dataset": final Dataset d = new Dataset(); - populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(d, doc, instance, collectedFrom, info, lastUpdateTimestamp); d.setStoragedate(prepareDatasetStorageDate(doc, info)); d.setDevice(prepareDatasetDevice(doc, info)); d.setSize(prepareDatasetSize(doc, info)); @@ -257,7 +260,7 @@ public abstract class AbstractMdRecordToOafMapper { return d; case "software": final Software s = new Software(); - populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(s, doc, instance, collectedFrom, info, lastUpdateTimestamp); s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info)); s.setLicense(prepareSoftwareLicenses(doc, info)); s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info)); @@ -267,7 +270,7 @@ public abstract class AbstractMdRecordToOafMapper { case "otherresearchproducts": default: final OtherResearchProduct o = new OtherResearchProduct(); - populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(o, doc, instance, collectedFrom, info, lastUpdateTimestamp); o.setContactperson(prepareOtherResearchProductContactPersons(doc, info)); o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info)); o.setTool(prepareOtherResearchProductTools(doc, info)); @@ -414,7 +417,7 @@ public abstract class AbstractMdRecordToOafMapper { private void populateResultFields( final Result r, final Document doc, - final List instances, + final Instance instance, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { @@ -448,8 +451,8 @@ public abstract class AbstractMdRecordToOafMapper { r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES r.setProcessingchargeamount(field(doc.valueOf("//oaf:processingchargeamount"), info)); r.setProcessingchargecurrency(field(doc.valueOf("//oaf:processingchargeamount/@currency"), info)); - r.setInstance(instances); - r.setBestaccessright(OafMapperUtils.createBestAccessRights(instances)); + r.setInstance(Arrays.asList(instance)); + r.setBestaccessright(OafMapperUtils.createBestAccessRights(Arrays.asList(instance))); r.setEoscifguidelines(prepareEOSCIfGuidelines(doc, info)); } @@ -508,7 +511,7 @@ public abstract class AbstractMdRecordToOafMapper { protected abstract Qualifier prepareResourceType(Document doc, DataInfo info); - protected abstract List prepareInstances( + protected abstract Instance prepareInstances( Document doc, DataInfo info, KeyValue collectedfrom, diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index 98da48f9e..33351e91f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -135,7 +135,7 @@ public class OafToOafMapper extends AbstractMdRecordToOafMapper { } @Override - protected List prepareInstances( + protected Instance prepareInstances( final Document doc, final DataInfo info, final KeyValue collectedfrom, @@ -197,7 +197,7 @@ public class OafToOafMapper extends AbstractMdRecordToOafMapper { instance.getUrl().addAll(validUrl); } - return Lists.newArrayList(instance); + return instance; } /** diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index ad61304a0..a811aad46 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -126,7 +126,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { } @Override - protected List prepareInstances( + protected Instance prepareInstances( final Document doc, final DataInfo info, final KeyValue collectedfrom, @@ -210,7 +210,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { instance.setUrl(new ArrayList<>()); instance.getUrl().addAll(validUrl); } - return Arrays.asList(instance); + return instance; } protected String trimAndDecodeUrl(String url) { From b95672b4204667f1b011a7b6ed281b7fcbb3525c Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 15 Nov 2024 09:16:18 +0100 Subject: [PATCH 209/239] mergeUtils set the result identifier when enforcing the result type --- .../dhp/schema/oaf/utils/MergeUtils.java | 451 +++++++++--------- 1 file changed, 229 insertions(+), 222 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index c9b235fd6..c092f6035 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -16,8 +16,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; -import eu.dnetlib.dhp.schema.common.EntityType; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -25,8 +23,10 @@ import org.apache.commons.lang3.tuple.Pair; import com.github.sisyphsu.dateparser.DateParserUtils; import com.google.common.base.Joiner; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.merge.AuthorMerger; import eu.dnetlib.dhp.schema.common.AccessRightComparator; +import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; @@ -46,7 +46,7 @@ public class MergeUtils { } public static T mergeGroup(Iterator oafEntityIterator, - boolean checkDelegateAuthority, VocabularyGroup vocs) { + boolean checkDelegateAuthority, VocabularyGroup vocs) { ArrayList sortedEntities = new ArrayList<>(); oafEntityIterator.forEachRemaining(sortedEntities::add); @@ -74,22 +74,29 @@ public class MergeUtils { if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { return (T) mergedResult; } else { - final Qualifier expectedResultType = Optional - .ofNullable(vocs.lookupTermBySynonym(ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid())) - .orElse(OafMapperUtils.unknown(ModelConstants.DNET_RESULT_TYPOLOGIES, ModelConstants.DNET_RESULT_TYPOLOGIES)); + final String expectedResultType = Optional + .ofNullable( + vocs + .lookupTermBySynonym( + ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid())) + .orElse(ModelConstants.ORP_DEFAULT_RESULTTYPE) + .getClassid(); // there is a clash among the result types - if (!expectedResultType.getClassid().equals(mergedResult.getResulttype().getClassid())) { - try { - String resulttype = expectedResultType.getClassid(); - if (EntityType.otherresearchproduct.toString().equals(resulttype)) { - resulttype = "other"; - } - Result result = (Result) ModelSupport.oafTypes.get(resulttype).newInstance(); - return (T) mergeResultFields(result, mergedResult); - } catch (InstantiationException | IllegalAccessException e) { - throw new IllegalStateException(e); - } + if (!expectedResultType.equals(mergedResult.getResulttype().getClassid())) { + + Result result = (Result) Optional + .ofNullable(ModelSupport.oafTypes.get(expectedResultType)) + .map(r -> { + try { + return r.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new IllegalStateException(e); + } + }) + .orElse(new OtherResearchProduct()); + result.setId(mergedResult.getId()); + return (T) mergeResultFields(result, mergedResult); } else { return (T) mergedResult; } @@ -116,10 +123,10 @@ public class MergeUtils { return mergeRelation((Relation) left, (Relation) right); } else { throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); } } @@ -158,10 +165,10 @@ public class MergeUtils { return mergeProject((Project) left, (Project) right); } else { throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); } } @@ -252,7 +259,7 @@ public class MergeUtils { } private static List mergeLists(final List left, final List right, int trust, - Function keyExtractor, BinaryOperator merger) { + Function keyExtractor, BinaryOperator merger) { if (left == null || left.isEmpty()) { return right != null ? right : new ArrayList<>(); } else if (right == null || right.isEmpty()) { @@ -263,11 +270,11 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return new ArrayList<>(Stream - .concat(h.stream(), l.stream()) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.toMap(keyExtractor, v -> v, merger, LinkedHashMap::new)) - .values()); + .concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toMap(keyExtractor, v -> v, merger, LinkedHashMap::new)) + .values()); } private static List unionDistinctLists(final List left, final List right, int trust) { @@ -281,10 +288,10 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return Stream - .concat(h.stream(), l.stream()) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.toList()); + .concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toList()); } private static List unionDistinctListOfString(final List l, final List r) { @@ -295,10 +302,10 @@ public class MergeUtils { } return Stream - .concat(l.stream(), r.stream()) - .filter(StringUtils::isNotBlank) - .distinct() - .collect(Collectors.toList()); + .concat(l.stream(), r.stream()) + .filter(StringUtils::isNotBlank) + .distinct() + .collect(Collectors.toList()); } // TODO review @@ -324,7 +331,7 @@ public class MergeUtils { } private static List unionTitle(List left, List right, - int trust) { + int trust) { if (left == null) { return right; } else if (right == null) { @@ -335,10 +342,10 @@ public class MergeUtils { List l = trust >= 0 ? right : left; return Stream - .concat(h.stream(), l.stream()) - .filter(Objects::isNull) - .distinct() - .collect(Collectors.toList()); + .concat(h.stream(), l.stream()) + .filter(Objects::isNull) + .distinct() + .collect(Collectors.toList()); } /** @@ -373,8 +380,8 @@ public class MergeUtils { merged.setPid(mergeLists(merged.getPid(), enrich.getPid(), trust, MergeUtils::spKeyExtractor, (p1, p2) -> p1)); merged.setDateofcollection(LocalDateTime.now().toString()); merged - .setDateoftransformation( - chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); + .setDateoftransformation( + chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); merged.setExtraInfo(unionDistinctLists(merged.getExtraInfo(), enrich.getExtraInfo(), trust)); // When merging records OAI provenance becomes null merged.setOaiprovenance(null); @@ -391,7 +398,7 @@ public class MergeUtils { checkArgument(Objects.equals(merge.getTarget(), enrich.getTarget()), "target ids must be equal"); checkArgument(Objects.equals(merge.getRelType(), enrich.getRelType()), "relType(s) must be equal"); checkArgument( - Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); + Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); checkArgument(Objects.equals(merge.getRelClass(), enrich.getRelClass()), "relClass(es) must be equal"); // merge.setProvenance(mergeLists(merge.getProvenance(), enrich.getProvenance())); @@ -402,10 +409,10 @@ public class MergeUtils { merge.setValidationDate(ModelSupport.oldest(merge.getValidationDate(), enrich.getValidationDate())); } catch (ParseException e) { throw new IllegalArgumentException(String - .format( - "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), - merge.getTarget(), - merge.getValidationDate())); + .format( + "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), + merge.getTarget(), + merge.getValidationDate())); } // TODO keyvalue merge @@ -419,7 +426,7 @@ public class MergeUtils { T merge = mergeOafEntityFields(original, enrich, trust); if (merge.getProcessingchargeamount() == null - || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { + || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { merge.setProcessingchargeamount(enrich.getProcessingchargeamount()); merge.setProcessingchargecurrency(enrich.getProcessingchargecurrency()); } @@ -451,8 +458,8 @@ public class MergeUtils { } merge - .setDateofacceptance( - mergeDateOfAcceptance(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); + .setDateofacceptance( + mergeDateOfAcceptance(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); merge.setPublisher(coalesce(merge.getPublisher(), enrich.getPublisher())); merge.setEmbargoenddate(coalesce(merge.getEmbargoenddate(), enrich.getEmbargoenddate())); @@ -467,7 +474,7 @@ public class MergeUtils { merge.setCoverage(unionDistinctLists(merge.getCoverage(), enrich.getCoverage(), trust)); if (enrich.getBestaccessright() != null - && new AccessRightComparator<>() + && new AccessRightComparator<>() .compare(enrich.getBestaccessright(), merge.getBestaccessright()) < 0) { merge.setBestaccessright(enrich.getBestaccessright()); } @@ -480,8 +487,8 @@ public class MergeUtils { // ok merge - .setExternalReference( - mergeExternalReference(merge.getExternalReference(), enrich.getExternalReference(), trust)); + .setExternalReference( + mergeExternalReference(merge.getExternalReference(), enrich.getExternalReference(), trust)); // instance enrichment or union // review instance equals => add pid to comparision @@ -489,17 +496,17 @@ public class MergeUtils { merge.setInstance(mergeInstances(merge.getInstance(), enrich.getInstance(), trust)); } else { final List enrichmentInstances = isAnEnrichment(merge) ? merge.getInstance() - : enrich.getInstance(); + : enrich.getInstance(); final List enrichedInstances = isAnEnrichment(merge) ? enrich.getInstance() - : merge.getInstance(); + : merge.getInstance(); if (isAnEnrichment(merge)) merge.setDataInfo(enrich.getDataInfo()); merge.setInstance(enrichInstances(enrichedInstances, enrichmentInstances)); } merge - .setEoscifguidelines( - mergeEosciifguidelines(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); + .setEoscifguidelines( + mergeEosciifguidelines(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); merge.setIsGreen(booleanOR(merge.getIsGreen(), enrich.getIsGreen())); // OK but should be list of values merge.setOpenAccessColor(coalesce(merge.getOpenAccessColor(), enrich.getOpenAccessColor())); @@ -525,7 +532,7 @@ public class MergeUtils { LocalDate enrich_date = LocalDate.parse(enrich.getValue(), DateTimeFormatter.ISO_DATE); if (enrich_date.getYear() > 1300 - && (merge_date.getYear() < 1300 || merge_date.isAfter(enrich_date))) { + && (merge_date.getYear() < 1300 || merge_date.isAfter(enrich_date))) { return enrich; } } catch (NullPointerException | DateTimeParseException e) { @@ -543,56 +550,56 @@ public class MergeUtils { private static List mergeInstances(List v1, List v2, int trust) { return mergeLists( - v1, v2, trust, - MergeUtils::instanceKeyExtractor, - MergeUtils::instanceMerger); + v1, v2, trust, + MergeUtils::instanceKeyExtractor, + MergeUtils::instanceMerger); } private static List mergeEosciifguidelines(List v1, List v2, - int trust) { + int trust) { return mergeLists( - v1, v2, trust, er -> Joiner - .on("||") - .useForNull("") - .join(er.getCode(), er.getLabel(), er.getUrl(), er.getSemanticRelation()), - (r, l) -> r); + v1, v2, trust, er -> Joiner + .on("||") + .useForNull("") + .join(er.getCode(), er.getLabel(), er.getUrl(), er.getSemanticRelation()), + (r, l) -> r); } private static List mergeExternalReference(List v1, - List v2, int trust) { + List v2, int trust) { return mergeLists( - v1, v2, trust, er -> Joiner - .on(',') - .useForNull("") - .join( - er.getSitename(), er.getLabel(), - er.getUrl(), toString(er.getQualifier()), er.getRefidentifier(), - er.getQuery(), toString(er.getDataInfo())), - (r, l) -> r); + v1, v2, trust, er -> Joiner + .on(',') + .useForNull("") + .join( + er.getSitename(), er.getLabel(), + er.getUrl(), toString(er.getQualifier()), er.getRefidentifier(), + er.getQuery(), toString(er.getDataInfo())), + (r, l) -> r); } private static String toString(DataInfo di) { return Joiner - .on(',') - .useForNull("") - .join( - di.getInvisible(), di.getInferred(), di.getDeletedbyinference(), di.getTrust(), - di.getInferenceprovenance(), toString(di.getProvenanceaction())); + .on(',') + .useForNull("") + .join( + di.getInvisible(), di.getInferred(), di.getDeletedbyinference(), di.getTrust(), + di.getInferenceprovenance(), toString(di.getProvenanceaction())); } private static String toString(Qualifier q) { return Joiner - .on(',') - .useForNull("") - .join(q.getClassid(), q.getClassname(), q.getSchemeid(), q.getSchemename()); + .on(',') + .useForNull("") + .join(q.getClassid(), q.getClassname(), q.getSchemeid(), q.getSchemename()); } private static String toString(StructuredProperty sp) { return Joiner - .on(',') - .useForNull("") - .join(toString(sp.getQualifier()), sp.getValue()); + .on(',') + .useForNull("") + .join(toString(sp.getQualifier()), sp.getValue()); } private static List mergeStructuredProperties(List v1, List v2, int trust) { @@ -631,17 +638,17 @@ public class MergeUtils { // 2. @@ // 3. || return String - .join( - "::", - kvKeyExtractor(i.getHostedby()), - kvKeyExtractor(i.getCollectedfrom()), - qualifierKeyExtractor(i.getAccessright()), - qualifierKeyExtractor(i.getInstancetype()), - Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), - Optional - .ofNullable(i.getPid()) - .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) - .orElse(null)); + .join( + "::", + kvKeyExtractor(i.getHostedby()), + kvKeyExtractor(i.getCollectedfrom()), + qualifierKeyExtractor(i.getAccessright()), + qualifierKeyExtractor(i.getInstancetype()), + Optional.ofNullable(i.getUrl()).map(u -> String.join("@@", u)).orElse(null), + Optional + .ofNullable(i.getPid()) + .map(pp -> pp.stream().map(MergeUtils::spKeyExtractor).collect(Collectors.joining("@@"))) + .orElse(null)); } private static Instance instanceMerger(Instance i1, Instance i2) { @@ -652,30 +659,30 @@ public class MergeUtils { i.setInstancetype(i1.getInstancetype()); i.setPid(mergeLists(i1.getPid(), i2.getPid(), 0, MergeUtils::spKeyExtractor, (sp1, sp2) -> sp1)); i - .setAlternateIdentifier( - mergeLists( - i1.getAlternateIdentifier(), i2.getAlternateIdentifier(), 0, MergeUtils::spKeyExtractor, - (sp1, sp2) -> sp1)); + .setAlternateIdentifier( + mergeLists( + i1.getAlternateIdentifier(), i2.getAlternateIdentifier(), 0, MergeUtils::spKeyExtractor, + (sp1, sp2) -> sp1)); i - .setRefereed( - Collections - .min( - Stream.of(i1.getRefereed(), i2.getRefereed()).collect(Collectors.toList()), - new RefereedComparator())); + .setRefereed( + Collections + .min( + Stream.of(i1.getRefereed(), i2.getRefereed()).collect(Collectors.toList()), + new RefereedComparator())); i - .setInstanceTypeMapping( - mergeLists( - i1.getInstanceTypeMapping(), i2.getInstanceTypeMapping(), 0, - MergeUtils::instanceTypeMappingKeyExtractor, (itm1, itm2) -> itm1)); + .setInstanceTypeMapping( + mergeLists( + i1.getInstanceTypeMapping(), i2.getInstanceTypeMapping(), 0, + MergeUtils::instanceTypeMappingKeyExtractor, (itm1, itm2) -> itm1)); i.setFulltext(selectFulltext(i1.getFulltext(), i2.getFulltext())); i.setDateofacceptance(selectOldestDate(i1.getDateofacceptance(), i2.getDateofacceptance())); i.setLicense(coalesce(i1.getLicense(), i2.getLicense())); i.setProcessingchargeamount(coalesce(i1.getProcessingchargeamount(), i2.getProcessingchargeamount())); i.setProcessingchargecurrency(coalesce(i1.getProcessingchargecurrency(), i2.getProcessingchargecurrency())); i - .setMeasures( - mergeLists(i1.getMeasures(), i2.getMeasures(), 0, MergeUtils::measureKeyExtractor, (m1, m2) -> m1)); + .setMeasures( + mergeLists(i1.getMeasures(), i2.getMeasures(), 0, MergeUtils::measureKeyExtractor, (m1, m2) -> m1)); i.setUrl(unionDistinctListOfString(i1.getUrl(), i2.getUrl())); @@ -684,14 +691,14 @@ public class MergeUtils { private static String measureKeyExtractor(Measure m) { return String - .join( - "::", - m.getId(), - m - .getUnit() - .stream() - .map(KeyValue::getKey) - .collect(Collectors.joining("::"))); + .join( + "::", + m.getId(), + m + .getUnit() + .stream() + .map(KeyValue::getKey) + .collect(Collectors.joining("::"))); } private static Field selectOldestDate(Field d1, Field d2) { @@ -702,16 +709,16 @@ public class MergeUtils { } return Stream - .of(d1, d2) - .min( - Comparator - .comparing( - f -> DateParserUtils - .parseDate(f.getValue()) - .toInstant() - .atZone(ZoneId.systemDefault()) - .toLocalDate())) - .orElse(d1); + .of(d1, d2) + .min( + Comparator + .comparing( + f -> DateParserUtils + .parseDate(f.getValue()) + .toInstant() + .atZone(ZoneId.systemDefault()) + .toLocalDate())) + .orElse(d1); } private static String selectFulltext(String ft1, String ft2) { @@ -726,12 +733,12 @@ public class MergeUtils { private static String instanceTypeMappingKeyExtractor(InstanceTypeMapping itm) { return String - .join( - "::", - itm.getOriginalType(), - itm.getTypeCode(), - itm.getTypeLabel(), - itm.getVocabularyName()); + .join( + "::", + itm.getOriginalType(), + itm.getTypeCode(), + itm.getTypeLabel(), + itm.getVocabularyName()); } private static String kvKeyExtractor(KeyValue kv) { @@ -748,13 +755,13 @@ public class MergeUtils { private static String spKeyExtractor(StructuredProperty sp) { return Optional - .ofNullable(sp) - .map( - s -> Joiner - .on("||") - .useForNull("") - .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) - .orElse(null); + .ofNullable(sp) + .map( + s -> Joiner + .on("||") + .useForNull("") + .join(qualifierKeyExtractor(s.getQualifier()), s.getValue())) + .orElse(null); } private static T mergeORP(T original, T enrich) { @@ -776,8 +783,8 @@ public class MergeUtils { merge.setLicense(unionDistinctLists(merge.getLicense(), enrich.getLicense(), trust)); merge.setCodeRepositoryUrl(chooseReference(merge.getCodeRepositoryUrl(), enrich.getCodeRepositoryUrl(), trust)); merge - .setProgrammingLanguage( - chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); + .setProgrammingLanguage( + chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); return merge; } @@ -791,11 +798,11 @@ public class MergeUtils { merge.setSize(chooseReference(merge.getSize(), enrich.getSize(), trust)); merge.setVersion(chooseReference(merge.getVersion(), enrich.getVersion(), trust)); merge - .setLastmetadataupdate( - chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); + .setLastmetadataupdate( + chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); merge - .setMetadataversionnumber( - chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); + .setMetadataversionnumber( + chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); merge.setGeolocation(unionDistinctLists(merge.getGeolocation(), enrich.getGeolocation(), trust)); return merge; @@ -817,26 +824,26 @@ public class MergeUtils { merged.setLegalshortname(chooseReference(merged.getLegalshortname(), enrich.getLegalshortname(), trust)); merged.setLegalname(chooseReference(merged.getLegalname(), enrich.getLegalname(), trust)); merged - .setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); + .setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust)); merged.setLogourl(chooseReference(merged.getLogourl(), enrich.getLogourl(), trust)); merged.setEclegalbody(chooseReference(merged.getEclegalbody(), enrich.getEclegalbody(), trust)); merged.setEclegalperson(chooseReference(merged.getEclegalperson(), enrich.getEclegalperson(), trust)); merged.setEcnonprofit(chooseReference(merged.getEcnonprofit(), enrich.getEcnonprofit(), trust)); merged - .setEcresearchorganization( - chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); + .setEcresearchorganization( + chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); merged - .setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); + .setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); merged - .setEcinternationalorganizationeurinterests( - chooseReference( - merged.getEcinternationalorganizationeurinterests(), - enrich.getEcinternationalorganizationeurinterests(), trust)); + .setEcinternationalorganizationeurinterests( + chooseReference( + merged.getEcinternationalorganizationeurinterests(), + enrich.getEcinternationalorganizationeurinterests(), trust)); merged - .setEcinternationalorganization( - chooseReference( - merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); + .setEcinternationalorganization( + chooseReference( + merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); merged.setEcenterprise(chooseReference(merged.getEcenterprise(), enrich.getEcenterprise(), trust)); merged.setEcsmevalidated(chooseReference(merged.getEcsmevalidated(), enrich.getEcsmevalidated(), trust)); merged.setEcnutscode(chooseReference(merged.getEcnutscode(), enrich.getEcnutscode(), trust)); @@ -860,8 +867,8 @@ public class MergeUtils { merged.setDuration(chooseReference(merged.getDuration(), enrich.getDuration(), trust)); merged.setEcsc39(chooseReference(merged.getEcsc39(), enrich.getEcsc39(), trust)); merged - .setOamandatepublications( - chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); + .setOamandatepublications( + chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); merged.setEcarticle29_3(chooseReference(merged.getEcarticle29_3(), enrich.getEcarticle29_3(), trust)); merged.setSubjects(unionDistinctLists(merged.getSubjects(), enrich.getSubjects(), trust)); merged.setFundingtree(unionDistinctLists(merged.getFundingtree(), enrich.getFundingtree(), trust)); @@ -887,8 +894,8 @@ public class MergeUtils { } merged - .setH2020classification( - unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); + .setH2020classification( + unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); return merged; } @@ -915,7 +922,7 @@ public class MergeUtils { * @return list of instances possibly enriched */ private static List enrichInstances(final List toEnrichInstances, - final List enrichmentInstances) { + final List enrichmentInstances) { final List enrichmentResult = new ArrayList<>(); if (toEnrichInstances == null) { @@ -953,42 +960,42 @@ public class MergeUtils { */ private static Map toInstanceMap(final List ri) { return ri - .stream() - .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) - .flatMap(i -> { - final List> result = new ArrayList<>(); - if (i.getPid() != null) - i - .getPid() - .stream() - .filter(MergeUtils::validPid) - .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); - if (i.getAlternateIdentifier() != null) - i - .getAlternateIdentifier() - .stream() - .filter(MergeUtils::validPid) - .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); - return result.stream(); - }) - .collect( - Collectors - .toMap( - Pair::getLeft, - Pair::getRight, - (a, b) -> a)); + .stream() + .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) + .flatMap(i -> { + final List> result = new ArrayList<>(); + if (i.getPid() != null) + i + .getPid() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + if (i.getAlternateIdentifier() != null) + i + .getAlternateIdentifier() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + return result.stream(); + }) + .collect( + Collectors + .toMap( + Pair::getLeft, + Pair::getRight, + (a, b) -> a)); } private static boolean isFromDelegatedAuthority(Result r) { return Optional - .ofNullable(r.getInstance()) - .map( - instance -> instance - .stream() - .filter(i -> Objects.nonNull(i.getCollectedfrom())) - .map(i -> i.getCollectedfrom().getKey()) - .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) - .orElse(false); + .ofNullable(r.getInstance()) + .map( + instance -> instance + .stream() + .filter(i -> Objects.nonNull(i.getCollectedfrom())) + .map(i -> i.getCollectedfrom().getKey()) + .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) + .orElse(false); } /** @@ -1024,15 +1031,15 @@ public class MergeUtils { * @return the list */ private static List findEnrichmentsByPID(final List pids, - final Map enrichments) { + final Map enrichments) { if (pids == null || enrichments == null) return null; return pids - .stream() - .map(MergeUtils::extractKeyFromPid) - .map(enrichments::get) - .filter(Objects::nonNull) - .collect(Collectors.toList()); + .stream() + .map(MergeUtils::extractKeyFromPid) + .map(enrichments::get) + .filter(Objects::nonNull) + .collect(Collectors.toList()); } /** @@ -1043,8 +1050,8 @@ public class MergeUtils { */ private static boolean isAnEnrichment(OafEntity e) { return e.getDataInfo() != null && - e.getDataInfo().getProvenanceaction() != null - && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); + e.getDataInfo().getProvenanceaction() != null + && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); } /** @@ -1067,17 +1074,17 @@ public class MergeUtils { merge.setHostedby(firstNonNull(merge.getHostedby(), enrichment.getHostedby())); merge.setUrl(unionDistinctLists(merge.getUrl(), enrichment.getUrl(), 0)); merge - .setDistributionlocation( - firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); + .setDistributionlocation( + firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); merge.setCollectedfrom(firstNonNull(merge.getCollectedfrom(), enrichment.getCollectedfrom())); // pid and alternateId are used for matching merge.setDateofacceptance(firstNonNull(merge.getDateofacceptance(), enrichment.getDateofacceptance())); merge - .setProcessingchargeamount( - firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); + .setProcessingchargeamount( + firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); merge - .setProcessingchargecurrency( - firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); + .setProcessingchargecurrency( + firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); merge.setRefereed(firstNonNull(merge.getRefereed(), enrichment.getRefereed())); merge.setMeasures(unionDistinctLists(merge.getMeasures(), enrichment.getMeasures(), 0)); merge.setFulltext(firstNonNull(merge.getFulltext(), enrichment.getFulltext())); @@ -1085,14 +1092,14 @@ public class MergeUtils { private static int compareTrust(Oaf a, Oaf b) { String left = Optional - .ofNullable(a.getDataInfo()) - .map(DataInfo::getTrust) - .orElse("0.0"); + .ofNullable(a.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); String right = Optional - .ofNullable(b.getDataInfo()) - .map(DataInfo::getTrust) - .orElse("0.0"); + .ofNullable(b.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); return left.compareTo(right); } From 5f512f510e92a717f1e536b9be9db15399d42805 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 15 Nov 2024 09:16:51 +0100 Subject: [PATCH 210/239] code formatting --- .../raw/AbstractMdRecordToOafMapper.java | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index be84778f5..881d3202c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -155,10 +155,11 @@ public abstract class AbstractMdRecordToOafMapper { final Instance instance = prepareInstances(doc, entityInfo, collectedFrom, hostedBy); - if (!Optional.ofNullable(instance.getInstancetype()) - .map(Qualifier::getClassid) - .filter(StringUtils::isNotBlank) - .isPresent()) { + if (!Optional + .ofNullable(instance.getInstancetype()) + .map(Qualifier::getClassid) + .filter(StringUtils::isNotBlank) + .isPresent()) { return Lists.newArrayList(); } @@ -173,13 +174,16 @@ public abstract class AbstractMdRecordToOafMapper { protected String getResultType(final Instance instance) { if (this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { - return Optional.ofNullable(instance.getInstancetype()) - .map(Qualifier::getClassid) - .map(instanceType -> Optional - .ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType)) - .map(Qualifier::getClassid) - .orElse("0000")) - .orElse("0000"); + return Optional + .ofNullable(instance.getInstancetype()) + .map(Qualifier::getClassid) + .map( + instanceType -> Optional + .ofNullable( + this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType)) + .map(Qualifier::getClassid) + .orElse("0000")) + .orElse("0000"); } else { throw new IllegalStateException("Missing vocabulary: " + ModelConstants.DNET_RESULT_TYPOLOGIES); } From cf7d9a32ab847d50bcafe03bc66795d2719e7d03 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 15 Nov 2024 09:17:28 +0100 Subject: [PATCH 211/239] disable autoBroadcastJoin in the cleaning workflow --- .../eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml index 2512fc5bc..01aaadae5 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/clean/oozie_app/workflow.xml @@ -162,6 +162,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=15000 --inputPath${graphInputPath}/publication @@ -197,6 +198,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=8000 --inputPath${graphInputPath}/dataset @@ -232,6 +234,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=5000 --inputPath${graphInputPath}/otherresearchproduct @@ -267,6 +270,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=2000 --inputPath${graphInputPath}/software @@ -302,6 +306,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=1000 --inputPath${graphInputPath}/datasource @@ -337,6 +342,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=1000 --inputPath${graphInputPath}/organization @@ -372,6 +378,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=2000 --inputPath${graphInputPath}/project @@ -407,6 +414,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=2000 --inputPath${graphInputPath}/person @@ -442,6 +450,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.autoBroadcastJoinThreshold=-1 --conf spark.sql.shuffle.partitions=20000 --inputPath${graphInputPath}/relation From 9e439f5ecaea8c92ca7db4e62158289766fee84e Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 15 Nov 2024 12:19:26 +0100 Subject: [PATCH 212/239] map the abstracts considering both the datacite and the dc nsPrefix --- .../dhp/oa/graph/raw/OdfToOafMapper.java | 2 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 23 +++++ .../dhp/oa/graph/raw/odf_guidelines4.xml | 95 +++++++++++++++++++ 3 files changed, 119 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_guidelines4.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index a811aad46..efe05eb68 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -319,7 +319,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { @Override protected List> prepareDescriptions(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//*[local-name()='description' and ./@descriptionType='Abstract']", info); + return prepareListFields(doc, "//datacite:description[./@descriptionType='Abstract'] | //dc:description", info); } @Override diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 2cf3ea0c0..ea9503d17 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -906,6 +906,29 @@ class MappersTest { assertEquals("IT", p.getCountry().get(0).getClassid()); assertEquals("FR", p.getCountry().get(1).getClassid()); assertEquals("DE", p.getCountry().get(2).getClassid()); + + assertNotNull(p.getDescription()); + assertEquals(1, p.getDescription().size()); + assertNotNull(p.getDescription().get(0)); + assertTrue(StringUtils.isNotBlank(p.getDescription().get(0).getValue())); + } + + @Test + void testODFRecord_guidelines4() throws IOException { + final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_guidelines4.xml"))); + final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); + + final Publication p = (Publication) list.get(0); + assertValidId(p.getId()); + assertValidId(p.getCollectedfrom().get(0).getKey()); + assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue())); + + assertNotNull(p.getDescription()); + assertEquals(2, p.getDescription().size()); + assertNotNull(p.getDescription().get(0)); + assertTrue(StringUtils.isNotBlank(p.getDescription().get(0).getValue())); + assertNotNull(p.getDescription().get(1)); + assertTrue(StringUtils.isNotBlank(p.getDescription().get(1).getValue())); } @Test diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_guidelines4.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_guidelines4.xml new file mode 100644 index 000000000..4f390afb7 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_guidelines4.xml @@ -0,0 +1,95 @@ + + +
+ od______1261::66de86a37fa980b9b8f4cefdf1c5a0c5 + oai:air.unimi.it:2434/636308 + 2020-09-04T14:36:48.411Z + od______1261 + oai:air.unimi.it:2434/636308 + 2022-10-14T14:17:28Z + com_2434_73555 + col_2434_73557 + openaire + 2024-11-07T06:17:52.454Z +
+ + + + I processi incorpati di costruzione della conoscenza nelle pratiche di cura + + + + S. Visioli + rp17381 + University of Milan + 53567 + + + L. Zannini + 0000-0001-7287-6616 + rp31993 + University of Milan + 5556 + + + 487 + 2 + 506 + 8 + METIS + + 10.30557/MT00041 + http://hdl.handle.net/2434/636308 + + + 2018 + 2018 + 2019-05-23 + + ita + journal article + In ambito educativo, il corpo è stato tradizionalmente marginalizzato o rifiutato come fonte di conoscenza. Negli ultimi decenni, la letteratura pedagogica ha manifestato crescente interesse per il ruolo che ha il corpo dell’insegnante, nel pianificare e realizzare l’insegnamento, sfidando le ideologie e le epistemologie dominanti, secondo le quali la nostra mente è la fonte primaria di apprendimento. Al contrario, una pedagogia della conoscenza incorpata (embodiment) considera il corpo centrale nel nostro conoscere (le pratiche d’insegnamento) e nel nostro essere (insegnanti). Analogamente, la letteratura delle professioni del caring si è rivelata molto attenta ai temi della conoscenza incorpata, ossia di quel sapere che è presente in un corpo, spesso in modo tacito. Le infermiere hanno da tempo esplorato il ruolo del loro corpo nei processi di costruzione della conoscenza, come anche il ruolo del corpo dello studente, e addirittura del paziente, nel conoscere la malattia. In questo paper riportiamo alcune riflessioni ed esperienze sulla conoscenza incorpata nelle pratiche assistenziali, mettendo in luce come questa, benché spesso tacita, abbia un ruolo centrale nell’aver cura delle persone malate. + In educational contexts, the body has been traditionally marginalized or rejected as a source of knowledge. In the last decades, some pedagogical contributions have shown interest for the role that the teacher’s body has in planning and implementing teaching, challenging dominant ideologies and epistemologies that tell us our minds are the primary sources of learning. Conversely, a pedagogy of embodiment makes the body central in our knowing (educational practices) and being (a teacher). Similarly, literature from the caring professions has revealed very receptive to-ward issues related to embodied knowledge, that is to say the knowledge that the body owns, which is frequently tacit. Since many years, nurses have explored the role of body in the processes of knowledge building, as well as the role of the student’s body, and even the patient’s body, in knowing the illness. In this paper we report some reflections and experiences concerning embodied knowledge in the nursing practice, highlighting that embodied knowledge, even if tacit, has a pivotal role when caring for sick people. + application/pdf + 2434/636308 + open access + + body; educational practices; embodied knowledge; nursing; teaching + + + + 413926 bytes + + + https://air.unimi.it/bitstream/2434/636308/2/Finale%20pubblicato.pdf + + https://air.unimi.it/bitstream/2434/636308/2/Finale%20pubblicato.pdf + 0001 + 2018-01-01 + OPEN + ita + + + +
\ No newline at end of file From a1297082e2df74e672512334238143542ad323ce Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Tue, 19 Nov 2024 14:57:18 +0100 Subject: [PATCH 213/239] Crossref Enhancements: -Accurate Review Type Assignment: Resolved an issue identified in ticket https://support.openaire.eu/issues/9525#note-13. When a relationship of "is-review-of" is detected, the publication type is now correctly set to "Review." -Enhanced Author Affiliation Data: Implemented Miriam's suggestion by including a new field, "RawAffiliationString," in each author entry. This additional data provides a more granular level of detail regarding author affiliations, potentially improving discoverability and research analysis. --- .../collection/crossref/Crossref2Oaf.scala | 67 ++--- .../collection/crossref/affiliationTest.json | 232 ++++++++++++++++++ .../crossref/CrossrefMappingTest.scala | 12 +- 3 files changed, 265 insertions(+), 46 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/crossref/affiliationTest.json diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index e7d68920b..e15312e43 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -37,7 +37,7 @@ case class mappingAuthor( family: Option[String], sequence: Option[String], ORCID: Option[String], - affiliation: Option[mappingAffiliation] + affiliation: Option[List[mappingAffiliation]] ) {} case class funderInfo(id: String, uri: String, name: String, synonym: List[String]) {} @@ -457,15 +457,14 @@ case object Crossref2Oaf { } //Mapping Author - val authorList: List[mappingAuthor] = - (json \ "author").extract[List[mappingAuthor]].filter(a => a.family.isDefined) + val authorList: List[mappingAuthor] = (json \ "author").extract[List[mappingAuthor]].filter(a => a.family.isDefined) val sorted_list = authorList.sortWith((a: mappingAuthor, b: mappingAuthor) => a.sequence.isDefined && a.sequence.get.equalsIgnoreCase("first") ) result.setAuthor(sorted_list.zipWithIndex.map { case (a, index) => - generateAuhtor(a.given.orNull, a.family.get, a.ORCID.orNull, index) + generateAuthor(a.given.orNull, a.family.get, a.ORCID.orNull, index, a.affiliation) }.asJava) // Mapping instance @@ -504,18 +503,6 @@ case object Crossref2Oaf { ) } - val is_review = json \ "relation" \ "is-review-of" \ "id" - - if (is_review != JNothing) { - instance.setInstancetype( - OafMapperUtils.qualifier( - "0015", - "peerReviewed", - ModelConstants.DNET_REVIEW_LEVELS, - ModelConstants.DNET_REVIEW_LEVELS - ) - ) - } if (doi.startsWith("10.3410") || doi.startsWith("10.12703")) instance.setHostedby( @@ -569,17 +556,24 @@ case object Crossref2Oaf { result } + + def generateIdentifier(oaf: Result, doi: String): String = { val id = DHPUtils.md5(doi.toLowerCase) s"50|doiboost____|$id" } - def generateAuhtor(given: String, family: String, orcid: String, index: Int): Author = { + private def generateAuthor(given: String, family: String, orcid: String, index: Int, affiliation: Option[List[mappingAffiliation]]): Author = { val a = new Author a.setName(given) a.setSurname(family) a.setFullname(s"$given $family") a.setRank(index + 1) + + // Adding Raw affiliation if it's defined + if (affiliation.isDefined) { + a.setRawAffiliationString(affiliation.get.map(a => a.name).asJava) + } if (StringUtils.isNotBlank(orcid)) a.setPid( List( @@ -705,11 +699,21 @@ case object Crossref2Oaf { val objectType = (json \ "type").extractOrElse[String](null) if (objectType == null) return resultList - val typology = getTypeQualifier(objectType, vocabularies) + + + // If the item has a relations is-review-of, then we force it to a peer-review + val is_review = json \ "relation" \ "is-review-of" \ "id" + var force_to_review = false + if (is_review != JNothing) { + force_to_review = true + } + + val typology = getTypeQualifier(if (force_to_review) "peer-review" else objectType, vocabularies) if (typology == null) return List() + val result = generateItemFromType(typology._2) if (result == null) return List() @@ -757,33 +761,6 @@ case object Crossref2Oaf { else resultList } - - // if (uw != null) { -// result.getCollectedfrom.add(createUnpayWallCollectedFrom()) -// val i: Instance = new Instance() -// i.setCollectedfrom(createUnpayWallCollectedFrom()) -// if (uw.best_oa_location != null) { -// -// i.setUrl(List(uw.best_oa_location.url).asJava) -// if (uw.best_oa_location.license.isDefined) { -// i.setLicense(field[String](uw.best_oa_location.license.get, null)) -// } -// -// val colour = get_unpaywall_color(uw.oa_status) -// if (colour.isDefined) { -// val a = new AccessRight -// a.setClassid(ModelConstants.ACCESS_RIGHT_OPEN) -// a.setClassname(ModelConstants.ACCESS_RIGHT_OPEN) -// a.setSchemeid(ModelConstants.DNET_ACCESS_MODES) -// a.setSchemename(ModelConstants.DNET_ACCESS_MODES) -// a.setOpenAccessRoute(colour.get) -// i.setAccessright(a) -// } -// i.setPid(result.getPid) -// result.getInstance().add(i) -// } -// } - } private def createCiteRelation(source: Result, targetPid: String, targetPidType: String): List[Relation] = { diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/crossref/affiliationTest.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/crossref/affiliationTest.json new file mode 100644 index 000000000..201138e45 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/crossref/affiliationTest.json @@ -0,0 +1,232 @@ +{ + "indexed": { + "date-parts": [ + [ + 2022, + 4, + 3 + ] + ], + "date-time": "2022-04-03T01:45:59Z", + "timestamp": 1648950359167 + }, + "reference-count": 0, + "publisher": "American Society of Clinical Oncology (ASCO)", + "issue": "18_suppl", + "content-domain": { + "domain": [], + "crossmark-restriction": false + }, + "short-container-title": [ + "JCO" + ], + "published-print": { + "date-parts": [ + [ + 2007, + 6, + 20 + ] + ] + }, + "abstract": " 3507 Purpose: To detect IGF-1R on circulating tumor cells (CTCs) as a biomarker in the clinical development of a monoclonal human antibody, CP-751,871, targeting IGF-1R. Experimental Design: An automated sample preparation and analysis system for enumerating CTCs (Celltracks) was adapted for detecting IGF-1R positive CTCs with a diagnostic antibody targeting a different IGF-1R epitope to CP-751,871. This assay was utilized in three phase I trials of CP-751,871 as a single agent or with chemotherapy and was validated using cell lines and blood samples from healthy volunteers and patients with metastatic carcinoma. Results: There was no interference between the analytical and therapeutic antibodies. CP-751,871 was well tolerated as a single agent, and in combination with docetaxel or carboplatin and paclitaxel, at doses ranging from 0.05 mg/kg to 20 mg/kg. Eighty patients were enrolled on phase 1 studies of CP-751,871, with 47 (59%) patients having CTCs detected during the study. Prior to treatment 26 patients (33%) had CTCs, with 23 having detectable IGF-1R positive CTCs. CP-751,871 alone, and CP-751,871 with cytotoxic chemotherapy, decreased CTCs and IGF-1R positive CTCs; these increased towards the end of the 21-day cycle in some patients, falling again with retreatment. CTCs were commonest in advanced hormone refractory prostate cancer (11/20). Detectable IGF-1R expression on CTCs before treatment with CP-751,871 and docetaxel was associated with a higher frequency of PSA decline by more than 50% (6/10 vs 2/8 patients). A relationship was observed between sustained falls in CTCs counts and PSA declines by more than 50%. Conclusions: IGF-1R expression is detectable by immunofluorescence on CTCs. These data support the further evaluation of CTCs in pharmacodynamic studies and patient selection, particularly in advanced prostate cancer. No significant financial relationships to disclose. ", + "DOI": "10.1200/jco.2007.25.18_suppl.3507", + "type": "journal-article", + "created": { + "date-parts": [ + [ + 2020, + 3, + 6 + ] + ], + "date-time": "2020-03-06T20:50:42Z", + "timestamp": 1583527842000 + }, + "page": "3507-3507", + "source": "Crossref", + "is-referenced-by-count": 0, + "title": [ + "Circulating tumor cells expressing the insulin growth factor-1 receptor (IGF-1R): Method of detection, incidence and potential applications" + ], + "prefix": "10.1200", + "volume": "25", + "author": [ + { + "given": "J. S.", + "family": "de Bono", + "sequence": "first", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "A.", + "family": "Adjei", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "G.", + "family": "Attard", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "M.", + "family": "Pollak", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "P.", + "family": "Fong", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "P.", + "family": "Haluska", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "L.", + "family": "Roberts", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "D.", + "family": "Chainese", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "L.", + "family": "Terstappen", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + }, + { + "given": "A.", + "family": "Gualberto", + "sequence": "additional", + "affiliation": [ + { + "name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA" + } + ] + } + ], + "member": "233", + "container-title": [ + "Journal of Clinical Oncology" + ], + "original-title": [], + "language": "en", + "deposited": { + "date-parts": [ + [ + 2020, + 3, + 6 + ] + ], + "date-time": "2020-03-06T20:51:03Z", + "timestamp": 1583527863000 + }, + "score": 1, + "resource": { + "primary": { + "URL": "http://ascopubs.org/doi/10.1200/jco.2007.25.18_suppl.3507" + } + }, + "subtitle": [], + "short-title": [], + "issued": { + "date-parts": [ + [ + 2007, + 6, + 20 + ] + ] + }, + "references-count": 0, + "journal-issue": { + "issue": "18_suppl", + "published-print": { + "date-parts": [ + [ + 2007, + 6, + 20 + ] + ] + } + }, + "alternative-id": [ + "10.1200/jco.2007.25.18_suppl.3507" + ], + "URL": "http://dx.doi.org/10.1200/jco.2007.25.18_suppl.3507", + "relation": {}, + "ISSN": [ + "0732-183X", + "1527-7755" + ], + "issn-type": [ + { + "value": "0732-183X", + "type": "print" + }, + { + "value": "1527-7755", + "type": "electronic" + } + ], + "subject": [], + "published": { + "date-parts": [ + [ + 2007, + 6, + 20 + ] + ] + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala index c3ea884eb..f6f71ca66 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala @@ -25,8 +25,18 @@ class CrossrefMappingTest extends AbstractVocabularyTest { val input = IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/issn_pub.json"), "utf-8") - println(Crossref2Oaf.convert(input, vocabularies, TransformationType.All)) + Crossref2Oaf.convert(input, vocabularies, TransformationType.All).foreach(record => { + println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(record)) + }) } + + @Test + def mappingAffiliation(): Unit = { + val input = + IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/affiliationTest.json"), "utf-8") + val data = Crossref2Oaf.convert(input, vocabularies, TransformationType.OnlyResult) + println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(data.head)) + } } From 5d344323983e5aa6cff4cd02557bf40d0dc6c5dd Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 19 Nov 2024 15:12:04 +0100 Subject: [PATCH 214/239] align MergeUtils with beta branch --- .../dhp/schema/oaf/utils/MergeUtils.java | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java index cd8506583..c092f6035 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -74,29 +74,29 @@ public class MergeUtils { if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { return (T) mergedResult; } else { - final Qualifier expectedResultType = vocs - .lookupTermBySynonym( - ModelConstants.DNET_RESULT_TYPOLOGIES, - i.getInstancetype().getClassid()); - - if (Objects.isNull(expectedResultType)) { - throw new IllegalArgumentException( - "instance type not bound to any result type in dnet:result_typologies: " + - i.getInstancetype().getClassid()); - } + final String expectedResultType = Optional + .ofNullable( + vocs + .lookupTermBySynonym( + ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid())) + .orElse(ModelConstants.ORP_DEFAULT_RESULTTYPE) + .getClassid(); // there is a clash among the result types - if (!expectedResultType.getClassid().equals(mergedResult.getResulttype().getClassid())) { - try { - String resulttype = expectedResultType.getClassid(); - if (EntityType.otherresearchproduct.toString().equals(resulttype)) { - resulttype = "other"; - } - Result result = (Result) ModelSupport.oafTypes.get(resulttype).newInstance(); - return (T) mergeResultFields(result, mergedResult); - } catch (InstantiationException | IllegalAccessException e) { - throw new IllegalStateException(e); - } + if (!expectedResultType.equals(mergedResult.getResulttype().getClassid())) { + + Result result = (Result) Optional + .ofNullable(ModelSupport.oafTypes.get(expectedResultType)) + .map(r -> { + try { + return r.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new IllegalStateException(e); + } + }) + .orElse(new OtherResearchProduct()); + result.setId(mergedResult.getId()); + return (T) mergeResultFields(result, mergedResult); } else { return (T) mergedResult; } From 4e55ddc547451c18b9e14cce8244015e6825b3cf Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 19 Nov 2024 16:50:42 +0100 Subject: [PATCH 215/239] [PubMed aggregation] storing contents into mdStoreVersion/store --- .../dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala index 1bdd2a4bc..adac9ffb6 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/ebi/SparkCreatePubmedDump.scala @@ -2,9 +2,13 @@ package eu.dnetlib.dhp.sx.bio.ebi import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.application.AbstractScalaApplication +import eu.dnetlib.dhp.common.Constants +import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH, MDSTORE_SIZE_PATH} import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMParser2, PubMedToOaf} +import eu.dnetlib.dhp.transformation.TransformSparkJobNode +import eu.dnetlib.dhp.utils.DHPUtils.writeHdfsFile import eu.dnetlib.dhp.utils.ISLookupClientFactory import org.apache.spark.sql.{Encoder, Encoders, SparkSession} import org.slf4j.{Logger, LoggerFactory} @@ -82,7 +86,10 @@ class SparkCreatePubmedDump(propertyPath: String, args: Array[String], log: Logg .write .option("compression", "gzip") .mode("overwrite") - .text(targetPath) + .text(targetPath + MDSTORE_DATA_PATH) + + val mdStoreSize = spark.read.text(targetPath + MDSTORE_DATA_PATH).count + writeHdfsFile(spark.sparkContext.hadoopConfiguration, "" + mdStoreSize, targetPath + MDSTORE_SIZE_PATH) } } From 496007188a720b18ce8301b6050c6e0b924522d1 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 20 Nov 2024 09:50:09 +0100 Subject: [PATCH 216/239] Added assertion on CrossrefMappingTest --- .../crossref/CrossrefMappingTest.scala | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala index f6f71ca66..12ca14ba1 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala @@ -3,12 +3,15 @@ package eu.dnetlib.dhp.collection.crossref import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest import eu.dnetlib.dhp.collection.crossref.Crossref2Oaf.TransformationType +import eu.dnetlib.dhp.schema.oaf.Publication import org.apache.commons.io.IOUtils -import org.junit.jupiter.api.{BeforeEach, Test} +import org.junit.jupiter.api.{Assertions, BeforeEach, Test} import org.junit.jupiter.api.extension.ExtendWith import org.mockito.junit.jupiter.MockitoExtension import org.slf4j.{Logger, LoggerFactory} +import scala.collection.JavaConverters.asScalaBufferConverter + @ExtendWith(Array(classOf[MockitoExtension])) class CrossrefMappingTest extends AbstractVocabularyTest { @@ -26,7 +29,7 @@ class CrossrefMappingTest extends AbstractVocabularyTest { IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/issn_pub.json"), "utf-8") Crossref2Oaf.convert(input, vocabularies, TransformationType.All).foreach(record => { - println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(record)) + Assertions.assertNotNull(record) }) } @@ -37,6 +40,16 @@ class CrossrefMappingTest extends AbstractVocabularyTest { val input = IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/affiliationTest.json"), "utf-8") val data = Crossref2Oaf.convert(input, vocabularies, TransformationType.OnlyResult) + data.foreach(record => { + Assertions.assertNotNull(record) + Assertions.assertTrue(record.isInstanceOf[Publication]) + val publication = record.asInstanceOf[Publication] + publication.getAuthor.asScala.foreach(author => { + Assertions.assertNotNull(author.getRawAffiliationString) + Assertions.assertTrue(author.getRawAffiliationString.size()>0) + + }) + }) println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(data.head)) } } From 15227f82b8acecaf7b51cddba83d68bde5f10169 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 20 Nov 2024 15:52:40 +0100 Subject: [PATCH 217/239] added related author's given name and family name in the solr json payload serialisation --- .../model/ProvisionModelSupport.java | 23 +++++++++++++++++-- pom.xml | 2 +- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 738d75189..0da0f6955 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,6 +5,7 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.solr.PersonTopic; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -39,7 +40,6 @@ import eu.dnetlib.dhp.schema.solr.OpenAccessColor; import eu.dnetlib.dhp.schema.solr.OpenAccessRoute; import eu.dnetlib.dhp.schema.solr.Organization; import eu.dnetlib.dhp.schema.solr.Person; -import eu.dnetlib.dhp.schema.solr.PersonTopic; import eu.dnetlib.dhp.schema.solr.Pid; import eu.dnetlib.dhp.schema.solr.Project; import eu.dnetlib.dhp.schema.solr.Result; @@ -174,6 +174,8 @@ public class ProvisionModelSupport { && StringUtils.isNotBlank(relation.getValidationDate())) { rr.setValidationDate(relation.getValidationDate()); } + rr.setGivenName(re.getGivenName()); + rr.setFamilyName(re.getFamilyName()); return rr; } @@ -208,11 +210,28 @@ public class ProvisionModelSupport { ps.setAlternativeNames(p.getAlternativeNames()); ps.setBiography(p.getBiography()); ps.setConsent(p.getConsent()); - // ps.setSubject(...)); + ps.setSubject(mapPersonTopics(p.getSubject())); return ps; } + private static List mapPersonTopics(List subjects) { + return Optional.ofNullable(subjects) + .map(ss -> ss.stream() + .map(ProvisionModelSupport::mapPersonTopic) + .collect(Collectors.toList())) + .orElse(null); + } + + private static PersonTopic mapPersonTopic(eu.dnetlib.dhp.schema.oaf.PersonTopic pt) { + PersonTopic topic = new PersonTopic(); + topic.setValue(pt.getValue()); + topic.setSchema(pt.getSchema()); + topic.setFromYear(pt.getFromYear()); + topic.setToYear(pt.getToYear()); + return topic; + } + private static Funding mapFunding(List fundingtree, VocabularyGroup vocs) { SAXReader reader = new SAXReader(); return Optional diff --git a/pom.xml b/pom.xml index 9480ddfc0..033d88b0b 100644 --- a/pom.xml +++ b/pom.xml @@ -937,7 +937,7 @@ 1.1.3 1.7 1.0.7 - [9.0.0] + [10.0.0] cdh5.9.2 3.5 11.0.2 From e5b04e61ff8f7f7b1d80f607e0fcb4da7c7c9f37 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Thu, 21 Nov 2024 10:20:12 +0100 Subject: [PATCH 218/239] [CommunityPatents] extends the community propagation considering also the results of type patents linked with a isrelatedto semantcis --- .../main/java/eu/dnetlib/dhp/api/Utils.java | 2 +- .../PrepareResultCommunitySetStep1.java | 51 ++++++++++++++++--- .../PrepareResultCommunitySetStep2.java | 18 ++----- 3 files changed, 49 insertions(+), 22 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java index 6079da365..1e39d99c3 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -171,7 +171,7 @@ public class Utils implements Serializable { public static List getCommunityIdList(String baseURL) throws IOException { return getValidCommunities(baseURL) .stream() - .map(community -> community.getId()) + .map(CommunityModel::getId) .collect(Collectors.toList()); } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index aede9ef05..ff496bb87 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -3,11 +3,14 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import static java.lang.String.join; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import eu.dnetlib.dhp.schema.common.ModelConstants; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; @@ -45,7 +48,7 @@ public class PrepareResultCommunitySetStep1 { /** * a dataset for example could be linked to more than one publication. For each publication linked to that dataset - * the previous query will produce a row: targetId set of community context the target could possibly inherit with + * the previous query will produce a row: targetId, set of community context the target could possibly inherit. With * the following query there will be a single row for each result linked to more than one result of the result type * currently being used */ @@ -56,6 +59,16 @@ public class PrepareResultCommunitySetStep1 { + "where length(co) > 0 " + "group by resultId"; + private static final String RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO = "select target resultId, community_context " + + "from (select id, collect_set(co.id) community_context " + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false %s " + + " and array_contains(instance.instancetype.classname, 'Patent') group by id) p " + + " JOIN " + + " (select source, target from relation " + + " where datainfo.deletedbyinference = false %s ) r ON p.id = r.source"; + public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( @@ -82,15 +95,20 @@ public class PrepareResultCommunitySetStep1 { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + final String allowedsemrel = join(",", Arrays.stream(parser.get("allowedsemrels").split(";")) + .map(value -> "'" + value.toLowerCase() + "'") + .toArray(String[]::new)); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); final String baseURL = parser.get("baseURL"); log.info("baseURL: {}", baseURL); - final List communityIdList = getCommunityList(baseURL); - log.info("communityIdList: {}", new Gson().toJson(communityIdList)); + final String communityIdList = join(",", getCommunityList(baseURL).stream() + .map(value -> "'" + value.toLowerCase() + "'") + .toArray(String[]::new)); + log.info("communityIdList: {}", new Gson().toJson(communityIdList)); final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); @@ -118,10 +136,10 @@ public class PrepareResultCommunitySetStep1 { SparkSession spark, String inputPath, String outputPath, - List allowedsemrel, + String allowedsemrel, Class resultClazz, String resultType, - List communityIdList) { + String communityIdList) { final String inputResultPath = inputPath + "/" + resultType; log.info("Reading Graph table from: {}", inputResultPath); @@ -141,8 +159,15 @@ public class PrepareResultCommunitySetStep1 { String resultContextQuery = String .format( RESULT_CONTEXT_QUERY_TEMPLATE, - getConstraintList(" lower(co.id) = '", communityIdList), - getConstraintList(" lower(relClass) = '", allowedsemrel)); + " lower(co.id) IN " + communityIdList, + " AND lower(relClass) IN " + allowedsemrel); + + String resultContextQueryIsRelatedTo = String + .format( + RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO, + " AND lower(co.id) IN " + communityIdList, + "AND lower(relClass) = '"+ + ModelConstants.IS_RELATED_TO.toLowerCase() + "'"); Dataset result_context = spark.sql(resultContextQuery); result_context.createOrReplaceTempView("result_context"); @@ -154,6 +179,16 @@ public class PrepareResultCommunitySetStep1 { .option("compression", "gzip") .mode(SaveMode.Overwrite) .json(outputResultPath); + + result_context = spark.sql(resultContextQueryIsRelatedTo); + result_context.createOrReplaceTempView("result_context"); + spark + .sql(RESULT_COMMUNITY_LIST_QUERY) + .as(Encoders.bean(ResultCommunityList.class)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Append) + .json(outputResultPath); } public static List getCommunityList(final String baseURL) throws IOException { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java index a53d3dfe3..9bebc36e5 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import java.util.ArrayList; import java.util.HashSet; import java.util.Set; @@ -76,22 +77,13 @@ public class PrepareResultCommunitySetStep2 { if (b == null) { return a; } - Set community_set = new HashSet<>(); - a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); - b - .getCommunityList() - .stream() - .forEach( - aa -> { - if (!community_set.contains(aa)) { - a.getCommunityList().add(aa); - community_set.add(aa); - } - }); + Set community_set = new HashSet<>(a.getCommunityList()); + community_set.addAll(b.getCommunityList()); + a.setCommunityList(new ArrayList<>(community_set)); return a; }) .map(Tuple2::_2) - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .map(OBJECT_MAPPER::writeValueAsString) .saveAsTextFile(outputPath, GzipCodec.class); } From 821700299a31e5201f616fd4235dcd599018d19b Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 22 Nov 2024 17:21:58 +0100 Subject: [PATCH 219/239] [patents] added test and resources --- .../PrepareResultCommunitySetStep1.java | 93 +++++++++++-------- .../ResultToCommunityJobTest.java | 55 +++++++++++ .../graph/publication/part-00000.json | 13 +++ .../graph/relation/part-00000.json | 24 +++++ 4 files changed, 147 insertions(+), 38 deletions(-) create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/relation/part-00000.json diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index ff496bb87..8f23a4cfd 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -37,7 +37,8 @@ public class PrepareResultCommunitySetStep1 { * relation */ // TODO - private static final String RESULT_CONTEXT_QUERY_TEMPLATE = "select target resultId, community_context " + private static final String RESULT_CONTEXT_QUERY_TEMPLATE = + "select target resultId, community_context " + "from (select id, collect_set(co.id) community_context " + " from result " + " lateral view explode (context) c as co " @@ -59,15 +60,26 @@ public class PrepareResultCommunitySetStep1 { + "where length(co) > 0 " + "group by resultId"; - private static final String RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO = "select target resultId, community_context " - + "from (select id, collect_set(co.id) community_context " - + " from result " - + " lateral view explode (context) c as co " - + " where datainfo.deletedbyinference = false %s " - + " and array_contains(instance.instancetype.classname, 'Patent') group by id) p " - + " JOIN " - + " (select source, target from relation " - + " where datainfo.deletedbyinference = false %s ) r ON p.id = r.source"; + private static final String RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO = + "select target as resultId, community_context " + + "from resultWithContext rwc " + + "join relatedToRelations r " + + "join patents p " + + "on rwc.id = r.source and r.target = p.id"; + + private static final String RESULT_WITH_CONTEXT = "select id, collect_set(co.id) community_context \n" + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false AND lower(co.id) IN %s" + + " group by id"; + + private static final String RESULT_PATENT = "select id " + + " from result " + + " where array_contains(instance.instancetype.classname, 'Patent')"; + + private static final String IS_RELATED_TO_RELATIONS = "select source, target " + + " from relation " + + " where lower(relClass) = 'isrelatedto' and datainfo.deletedbyinference = false"; public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -95,20 +107,18 @@ public class PrepareResultCommunitySetStep1 { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final String allowedsemrel = join(",", Arrays.stream(parser.get("allowedsemrels").split(";")) - .map(value -> "'" + value.toLowerCase() + "'") - .toArray(String[]::new)); - - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + final String allowedsemrel ="(" + join(",", + Arrays.asList(parser.get("allowedsemrels").split(";")).stream().map(value -> "'" + value.toLowerCase() + "'") + .toArray(String[]::new)) + ")"; + log.info("allowedSemRel: {}", allowedsemrel); final String baseURL = parser.get("baseURL"); log.info("baseURL: {}", baseURL); - final String communityIdList = join(",", getCommunityList(baseURL).stream() + final String communityIdList = "(" + join(",", getCommunityList(baseURL).stream() .map(value -> "'" + value.toLowerCase() + "'") - .toArray(String[]::new)); + .toArray(String[]::new)) + ")"; - log.info("communityIdList: {}", new Gson().toJson(communityIdList)); final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); @@ -156,32 +166,38 @@ public class PrepareResultCommunitySetStep1 { final String outputResultPath = outputPath + "/" + resultType; log.info("writing output results to: {}", outputResultPath); + String resultContextQuery = String .format( RESULT_CONTEXT_QUERY_TEMPLATE, - " lower(co.id) IN " + communityIdList, - " AND lower(relClass) IN " + allowedsemrel); - - String resultContextQueryIsRelatedTo = String - .format( - RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO, - " AND lower(co.id) IN " + communityIdList, - "AND lower(relClass) = '"+ - ModelConstants.IS_RELATED_TO.toLowerCase() + "'"); - + "AND lower(co.id) IN " + communityIdList, + "AND lower(relClass) IN " + allowedsemrel); Dataset result_context = spark.sql(resultContextQuery); + //result_context.createOrReplaceTempView("result_context"); + +// spark +// .sql(RESULT_COMMUNITY_LIST_QUERY) +// .as(Encoders.bean(ResultCommunityList.class)) +// .write() +// .option("compression", "gzip") +// .mode(SaveMode.Overwrite) +// .json(outputResultPath); + + Dataset rwc = spark.sql(String.format(RESULT_WITH_CONTEXT, communityIdList)); + Dataset patents = spark.sql(RESULT_PATENT); + Dataset relatedToRelations = spark.sql(IS_RELATED_TO_RELATIONS); + + rwc.createOrReplaceTempView("resultWithContext"); + + patents.createOrReplaceTempView("patents"); + + relatedToRelations.createOrReplaceTempView("relatedTorelations"); + + + result_context = result_context.unionAll( spark.sql(RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO)); + result_context.createOrReplaceTempView("result_context"); - spark - .sql(RESULT_COMMUNITY_LIST_QUERY) - .as(Encoders.bean(ResultCommunityList.class)) - .write() - .option("compression", "gzip") - .mode(SaveMode.Overwrite) - .json(outputResultPath); - - result_context = spark.sql(resultContextQueryIsRelatedTo); - result_context.createOrReplaceTempView("result_context"); spark .sql(RESULT_COMMUNITY_LIST_QUERY) .as(Encoders.bean(ResultCommunityList.class)) @@ -189,6 +205,7 @@ public class PrepareResultCommunitySetStep1 { .option("compression", "gzip") .mode(SaveMode.Append) .json(outputResultPath); + } public static List getCommunityList(final String baseURL) throws IOException { diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index 0d5b12c80..4361b6f39 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -6,8 +6,11 @@ import static org.apache.spark.sql.functions.desc; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -25,6 +28,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.Dataset; +import scala.collection.Seq; public class ResultToCommunityJobTest { @@ -271,4 +275,55 @@ public class ResultToCommunityJobTest { .get(0) .getString(0)); } + + @Test + public void prepareStep1Test() throws Exception { + /* + + + final String allowedsemrel = join(",", Arrays.stream(parser.get("allowedsemrels").split(";")) + .map(value -> "'" + value.toLowerCase() + "'") + .toArray(String[]::new)); + + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + + final String baseURL = parser.get("baseURL"); + log.info("baseURL: {}", baseURL); + */ + PrepareResultCommunitySetStep1 + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph") + .getPath(), + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", + "-outputPath", workingDir.toString() + "/preparedInfo", + "-allowedsemrels","issupplementto;issupplementedby", + "-baseURL","https://dev-openaire.d4science.org/openaire/community/" + }); + + + org.apache.spark.sql.Dataset resultCommunityList = spark.read().schema(Encoders.bean(ResultCommunityList.class).schema()) + .json(workingDir.toString() + "/preparedInfo/publication") + .as(Encoders.bean(ResultCommunityList.class)); + + Assertions.assertEquals(2, resultCommunityList.count()); + Assertions.assertEquals(1,resultCommunityList.filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'").count()); + Assertions.assertEquals(1,resultCommunityList.filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'").count()); + + ArrayList communities = resultCommunityList + .filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'") + .first().getCommunityList(); + Assertions.assertEquals(2, communities.size()); + Assertions.assertTrue(communities.stream().anyMatch(cid -> "beopen".equals(cid))); + Assertions.assertTrue(communities.stream().anyMatch(cid -> "dh-ch".equals(cid))); + + communities = resultCommunityList + .filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'") + .first().getCommunityList(); + Assertions.assertEquals(1, communities.size()); + Assertions.assertEquals("dh-ch", communities.get(0)); + } } diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json new file mode 100644 index 000000000..7957bcfd5 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json @@ -0,0 +1,13 @@ +{"author":[{"fullname":"Tanouayi, Gnon","name":"Gnon","pid":[],"rank":1,"surname":"Tanouayi"},{"fullname":"GNANDI, Kissao","name":"Kissao","pid":[],"rank":2,"surname":"Gnandi"},{"fullname":"Ouro-Sama, Kamilou","name":"Kamilou","pid":[],"rank":3,"surname":"Ouro-Sama"},{"fullname":"Ahoudi, Housséni","name":"Housséni","pid":[],"rank":4,"surname":"Ahoudi"},{"fullname":"Solitoke, Hodabalo Dhéoulaba","name":"Hodabalo Dhéoulaba","pid":[],"rank":5,"surname":"Solitoke"},{"fullname":"Badassan, Tchaa Esso-Essinam","name":"Tchaa Esso-Essinam","pid":[],"rank":6,"surname":"Badassan"},{"fullname":"Nyametso, A. Yawovi","name":"A. Yawovi","pid":[],"rank":7,"surname":"Nyametso"},{"fullname":"Agbéko, Aduayi-Akué Adoté","name":"Aduayi-Akué Adoté","pid":[],"rank":8,"surname":"Agbéko"}],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","value":"Episciences"}],"context":[],"contributor":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Importer of dst articles previously hosted by inist Eid system account"}],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2017-05-01"},"dateofcollection":"2022-03-01T00:17:08.896Z","dateoftransformation":"2022-03-01T02:57:50.566Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"This study is a contribution to the development of adsorption techniques for the removal of fluoride in natural waters. The work is carried out on a laboratory scale using local geo-materials sorbents, on the one hand the residues from the treatment of natural phosphorite of Hahotoé-Kpogamé and on the other hand the attapulgite clay mineral from the costal basin of Togo. The work carried out concerns the adsorption of fluoride on those sorbents. The following parameters are batch tested on synthetic fluoride solutions: time, solution pH, geo-material dose and fluoride concentration. The fluoride is analyzed by absorption spectrometry. The adsorption yields on the phosphorite treatment residues for aqueous fluoride solutions at an initial concentration of 10 mg/L and an adsorbent concentration of 10 g/L are 49 % at pH 6.5 and 66 % at pH 4.0. In the same experimental conditions, the yields on clay minerals are 28.2 % and 36.3 %. These yields are logically improved by increasing the adsorbent dosage (from 2 to 30 g/L). Additional tests are carried out on natural water at an initial fluoride concentration of 3.76 mg/L."},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Cette étude est une contribution au développement de techniques d’adsorption pour l’élimination du fluor dans les eaux naturelles. Les travaux ont été réalisés à l’échelle du laboratoire en utilisant comme sorbants des géo-matériaux locaux, d’une part les résidus du traitement des phosphates naturels de Hahotoé-Kpogamé et d’autre part l’argilite feuilletée du bassin sédimentaire côtier du Togo. Les travaux réalisés concernent l’adsorption du fluor sur les sorbants considérés. Les paramètres suivants ont été testés en batch sur des solutions synthétiques de fluor : le temps, le pH de la solution, la dose du géo-matériau et la concentration du fluor. Le fluor a été dosé par spectromètrie d’absorption. Les rendements d’adsorption sur les résidus de traitement des phosphates pour des solutions aqueuses de fluor à concentration initiale de 10 mg/L et une concentration en adsorbant de 10 g/L ont été de 49 % à pH 6,5 et 66 % à pH de 4,0. Dans les mêmes conditions expérimentales, les rendements sur les argilites ont été de 28,2 % et 36,3 %. Ces rendements ont logiquement été améliorés en augmentant le dosage en adsorbant (de 2 à 30 g/L). Des essais complémentaires ont été réalisés sur une eau naturelle à une concentration initiale en fluor de 3,76 mg/L."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|06cdd3ff4700::49ec404cee4e1452808aabeaffbd3072","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.4267/dechets-sciences-techniques.3534"}],"collectedfrom":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2017-05-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://eid.episciences.org/7781"]}],"language":{"classid":"fra/fre","classname":"French","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646506202085,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Foai.episciences.org%2F","datestamp":"2017-05-01","harvestDate":"2022-03-01T00:17:08.896Z","identifier":"oai:episciences.org:eid:7781","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:episciences.org:eid:7781","50|06cdd3ff4700::49ec404cee4e1452808aabeaffbd3072"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"ISSN: 2778-844X"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Environnement, Ingénierie & Développement"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Episciences.org"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"eid:7781 - Environnement, Ingénierie & Développement, 2017-05-01, N°73 - mai 2017"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"clay"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"geo-materials"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"phosphorite of Hahotoé-Kpogamé"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"removal of fluoride"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"argilite"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"élimination du fluor"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"géo-matériaux"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"phosphate de Hahotoé-Kpogamé"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[SDE.IE]Environmental Sciences/Environmental Engineering"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Défluoruration des eaux à l’aide des résidus du traitement des phosphates naturels et des argilites feuilletées"}]} +{"author":[],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::160a261e5d06fd542c2efcac6e17e08c","value":"RS Global Journals"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-02-28"},"dateofcollection":"2020-06-01T07:11:47.22Z","dateoftransformation":"2020-07-25T07:25:11.051Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"The results of treatment of 21 patients with multiple injuries, including 12 (57.2%) patients with lesions of limb bones, 9 (42.8%) patients with injuries of the pelvis treated at the Department of Traumatology number 2, 5 for the period of 2013 to 2014 were analyzed by the authors. Developed gentle immobilization of the lower limbs in patients with multiple injuries provides adequate fixation and extension of the lower limb in the intensive care period and during emergency external fixation by the device of external fixation device shin bone or hip.The introduction of surgical treatment of patients in the acute period of polytrauma, using minimally invasive fracture fixation technology allowed providing the early stabilization of the victim’s condition, to avoid diagnostic errors and obtain positive results of treatment in 98% of cases."}],"externalReference":[],"extraInfo":[],"format":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"application/pdf"}],"fulltext":[],"id":"50|07b5c0ccd4fe::e7f5459cc97865f2af6e3da964c1250b","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::160a261e5d06fd542c2efcac6e17e08c","value":"RS Global Journals"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2016-02-28"},"distributionlocation":"","hostedby":{"key":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","value":"World Science"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"license":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"https://creativecommons.org/licenses/by/4.0"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://rsglobal.pl/index.php/ws/article/view/895"]}],"journal":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"edition":"","ep":"","iss":"","issnLinking":"","issnOnline":"2413-1032","issnPrinted":"2414-6404","name":"World Science","sp":"","vol":""},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646506135978,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Frsglobal.pl%2Findex.php%2Findex%2Foai","datestamp":"2020-05-23T18:09:46Z","harvestDate":"2020-06-01T07:11:47.22Z","identifier":"oai:ojs2.rsglobal.pl:article/895","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:ojs2.rsglobal.pl:article/895","50|07b5c0ccd4fe::e7f5459cc97865f2af6e3da964c1250b"],"pid":[],"publisher":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"RS Global Sp. z O.O."},"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"World Science; Vol 3 No 2(6) (2016): World Science; 43-50"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"World Science; Том 3 № 2(6) (2016): World Science; 43-50"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2414-6404"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2413-1032"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Polytrauma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"multiple trauma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"combined injury injury"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"the severity of the damage"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"the device of external fixator"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"submersible osteosynthesis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"«damage control»"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Polytrauma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"multiple trauma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"combined injury injury"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"the severity of the damage"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"the device of external fixator"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"submersible osteosynthesis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"«damage control»"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"ЛЕЧЕНИЯ ПАЦИЕНТОВ С СОЧЕТАННЫМИ И МНОЖЕСТВЕННЫМИ ТРАВМАМИ КОНЕЧНОСТЕЙ И КОСТЕЙ ТАЗА"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"ЛЕЧЕНИЯ ПАЦИЕНТОВ С СОЧЕТАННЫМИ И МНОЖЕСТВЕННЫМИ ТРАВМАМИ КОНЕЧНОСТЕЙ И КОСТЕЙ ТАЗА"}]} +{"author":[{"fullname":"Kemppainen, Mika","name":"Mika","pid":[],"rank":1,"surname":"Kemppainen"},{"fullname":"Virkkunen, Iikka","name":"Iikka","pid":[],"rank":2,"surname":"Virkkunen"},{"fullname":"Pitkänen, Jorma","name":"Jorma","pid":[],"rank":3,"surname":"Pitkänen"},{"fullname":"Paussu, Raimo","name":"Raimo","pid":[],"rank":4,"surname":"Paussu"},{"fullname":"Hänninen, Hannu","name":"Hannu","pid":[],"rank":5,"surname":"Hänninen"}],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:34:39.606Z","dateoftransformation":"2022-02-28T14:02:46.142Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::210c52944502777ba567442480e6a76e","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/66c69ab9-a33b-4817-97cd-1f38b3b31405"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505945226,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2019-09-23T08:45:14Z","harvestDate":"2022-02-28T12:34:39.606Z","identifier":"oai:cris.vtt.fi:publications/66c69ab9-a33b-4817-97cd-1f38b3b31405","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::210c52944502777ba567442480e6a76e","oai:cris.vtt.fi:publications/66c69ab9-a33b-4817-97cd-1f38b3b31405"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Kemppainen , M , Virkkunen , I , Pitkänen , J , Paussu , R & Hänninen , H 2003 , ' Comparison of realistic artificial cracks and in-service cracks ' , The e-Journal of Nondestructive Testing & Ultrasonics , vol. 8 , no. 3 , 6 . < http://www.ndt.net/article/ecndt02/401/401.htm >"}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Comparison of realistic artificial cracks and in-service cracks"}]} +{"author":[{"fullname":"Kelhä, Väinö","name":"Väinö","pid":[],"rank":1,"surname":"Kelhä"},{"fullname":"Manninen, M.","name":"M.","pid":[],"rank":2,"surname":"Manninen"},{"fullname":"Oittinen, P.","name":"P.","pid":[],"rank":3,"surname":"Oittinen"},{"fullname":"Tiesmäki, Jarkko","name":"Jarkko","pid":[],"rank":4,"surname":"Tiesmäki"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1974-01-01"},"dateofcollection":"2022-02-28T12:32:28.547Z","dateoftransformation":"2022-02-28T14:46:21.4Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::3754cff043a1700077031ea29f8cc240","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1974-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/d2b5302f-004a-407c-8f9c-4312e0dbf679"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505968213,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2021-08-31T11:55:56Z","harvestDate":"2022-02-28T12:32:28.547Z","identifier":"oai:cris.vtt.fi:publications/d2b5302f-004a-407c-8f9c-4312e0dbf679","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::3754cff043a1700077031ea29f8cc240","oai:cris.vtt.fi:publications/d2b5302f-004a-407c-8f9c-4312e0dbf679"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Kelhä , V , Manninen , M , Oittinen , P & Tiesmäki , J 1974 , ' A parallel plate tackmeter for measuring the splitting resistance of printing inks ' , Surface Coatings International: JOCCA , vol. 57 , pp. 184-188 ."}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"A parallel plate tackmeter for measuring the splitting resistance of printing inks"}]} +{"author":[{"fullname":"Mononen, Petri","name":"Petri","pid":[],"rank":1,"surname":"Mononen"},{"fullname":"Innamaa, Satu","name":"Satu","pid":[],"rank":2,"surname":"Innamaa"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2013-01-01"},"dateofcollection":"2022-02-28T12:38:14.128Z","dateoftransformation":"2022-02-28T14:48:17.052Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::3875365f5052758953b072682e62bc80","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2013-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8602dae4-00e8-4f45-828b-65a367eb4730"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505969378,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2021-09-17T12:01:51Z","harvestDate":"2022-02-28T12:38:14.128Z","identifier":"oai:cris.vtt.fi:publications/8602dae4-00e8-4f45-828b-65a367eb4730","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:cris.vtt.fi:publications/8602dae4-00e8-4f45-828b-65a367eb4730","50|355e65625b88::3875365f5052758953b072682e62bc80"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Mononen , P & Innamaa , S 2013 , ' Enhancing journey quality : Field Operational Test of Aftermarket and Nomadic Devices in Vehicles ' , Baltic Transport Journal , pp. 46-47 ."}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Enhancing journey quality:Field Operational Test of Aftermarket and Nomadic Devices in Vehicles"}]} +{"author":[{"fullname":"Tsupari, Eemeli","name":"Eemeli","pid":[],"rank":1,"surname":"Tsupari"}],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","value":"VTT Research Information System"},{"key":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2018-10-01"},"dateofcollection":"2022-02-28T12:38:39.075Z","dateoftransformation":"2022-02-28T15:37:43.154Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Despite international agreements, global greenhouse gas (GHG) emissions have not decreased according to the targets. Consequently, our generation is creating an enormous problem for future generations. As climate change is a global problem, GHG emissions must decrease globally. Consequently, international policies are needed, actions should be effective and the impacts should be assessed with broad boundaries. In Europe, the cornerstone of climate policy is the EU Emissions Trading Scheme (EU ETS) but the rebound impacts within the EU ETS are often excluded in the assessments. This dissertation examines the impacts of major CO2 emission reduction solutions with different system boundaries, highlighting the importance of boundary selection on the results. In addition, the economic feasibilities of the selected solutions are evaluated.The case examples represent the most important sectors in terms of global CO2 emissions, such as electricity and heat production, the steel industry and transport. The studied technologies include efficient Waste-to-Energy (WtE) concepts with high power-to-heat ratio, utilisation of CO2 Capture and Storage (CCS) in different applications, replacing steel mill blast furnaces with Oxygen Blast Furnaces (OBF), Combined Heat and Power (CHP) and Carbon Capture and Utilisation (CCU) for storable fuels, which can be used for example in transportation. The results highlight the importance of the consequences in the electricity production system as well as the rebound impacts in the EU ETS. For example, the studied concepts to decrease direct GHG emissions of steel mills lead to increased power purchase from markets and consequently increase in emissions of the power system. The impacts of CCU concepts based on electrolysis increase the emissions in electricity production but enable a decrease in the usage of fossil fuels in transportation. In addition, converting electricity to storable fuels enable higher shares of variable solar and wind energy in the power systems. The consequences in the power systems are complex, including for example the impacts on electricity imports and exports, future investments and the EU ETS. Even if these impacts can be recognised by qualitative means, unambiguous quantitative consequences cannot be given. Understanding the decisive impacts of the framework and boundaries is crucial to interpreting different assessments and making effective actions and policy decisions. Solutions which decrease emissions within a narrow system boundary can actually increase the emissions of the broader system."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::54a1c76f520bb2c8da27d12e42891088","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2018-10-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0006","classname":"Doctoral thesis","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/d62ac5ef-7347-400f-95b2-59d970ceb505"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505683219,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2021-05-18T10:33:00Z","harvestDate":"2022-02-28T12:38:39.075Z","identifier":"oai:cris.vtt.fi:publications/d62ac5ef-7347-400f-95b2-59d970ceb505","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:cris.vtt.fi:publications/d62ac5ef-7347-400f-95b2-59d970ceb505","50|355e65625b88::54a1c76f520bb2c8da27d12e42891088"],"pid":[],"publisher":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Aalto University"},"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Tsupari , E 2018 , ' Impact of system boundaries on the effectiveness of climate change mitigation actions : Dissertation ' , Doctor Degree , Aalto University . < http://urn.fi/URN:ISBN:978-952-60-8358-2 >"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"energy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"environmental science"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"climate change mitigation"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"greenhouse gases"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"carbon dioxide"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"emissions trading"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"economic feasibility"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"/dk/atira/pure/sustainabledevelopmentgoals/climate_action"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"SDG 13 - Climate Action"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Impact of system boundaries on the effectiveness of climate change mitigation actions:Dissertation"}]} +{"author":[{"fullname":"Turkia, Heidi","name":"Heidi","pid":[],"rank":1,"surname":"Turkia"},{"fullname":"Sirén, Heli","name":"Heli","pid":[],"rank":2,"surname":"Sirén"},{"fullname":"Penttilä, Merja","name":"Merja","pid":[],"rank":3,"surname":"Penttilä"},{"fullname":"Pitkänen, Juha Pekka","name":"Juha Pekka","pid":[],"rank":4,"surname":"Pitkänen"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2013-02-22"},"dateofcollection":"2022-02-28T12:29:51.291Z","dateoftransformation":"2022-02-28T16:19:35.201Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"

Hydrolysates of lignocellulosic biomass, used as substrates for the sustainable production of fuels and chemicals often contain high amounts of phenolic compounds inhibiting the production microbiota. Quantification of these inhibitor compounds may help to understand possible difficulties in bioprocessing and further the development of more efficient, robust and tolerable processes. A separation method based on capillary electrophoresis with UV detection was developed for the simultaneous quantification of 10 phenolic compounds that may have inhibitor properties. Intraday relative standard deviations were less than 0.7% for migration times and between 2.6% and 6.4% for peak areas. Interday relative standard deviations were less than 3.0% for migration times and between 5.0% and 7.2% for peak areas. The method was applied to demonstrate that Saccharomyces cerevisiae was able to decrease the concentrations of vanillin, coniferyl aldehyde, syringaldehyde, acetoguaiacone and cinnamic acid during the cultivation, whereas the concentrations of phenols increased.

"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::6c232359e3b3165574cb88f0554d9264","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1016/j.chroma.2013.01.004"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2013-02-22"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/91f411d0-d8f8-4bf1-9072-345303cc776c"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505708387,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-29T08:16:51Z","harvestDate":"2022-02-28T12:29:51.291Z","identifier":"oai:cris.vtt.fi:publications/91f411d0-d8f8-4bf1-9072-345303cc776c","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:cris.vtt.fi:publications/91f411d0-d8f8-4bf1-9072-345303cc776c","50|355e65625b88::6c232359e3b3165574cb88f0554d9264"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Turkia , H , Sirén , H , Penttilä , M & Pitkänen , J P 2013 , ' Capillary electrophoresis for the monitoring of phenolic compounds in bioprocesses ' , Journal of Chromatography A , vol. 1278 , pp. 175-180 . https://doi.org/10.1016/j.chroma.2013.01.004"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"Bioprocess monitoring"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"Capillary electrophoresis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"Phenolic compounds"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Capillary electrophoresis for the monitoring of phenolic compounds in bioprocesses"}]} +{"author":[{"fullname":"Veijalainen, Jari","name":"Jari","pid":[],"rank":1,"surname":"Veijalainen"},{"fullname":"Wolski, Antoni","name":"Antoni","pid":[],"rank":2,"surname":"Wolski"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[{"dataInfo": null,"id": "dariah"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"dateofcollection":"2022-02-28T12:33:57.005Z","dateoftransformation":"2022-02-28T16:33:35.101Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::74009c567c81b4aa55c813db658734df","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0002","classname":"Book","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"]},{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"distributionlocation":"","hostedby":{"key":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","value":"VTT Research Information System"},"instancetype":{"classid":"0002","classname":"Book","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"]}, {"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"distributionlocation":"","hostedby":{"key":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","value":"VTT Research Information System"},"instancetype":{"classid":"0002","classname":"Book","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505716994,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2020-12-21T07:05:54Z","harvestDate":"2022-02-28T12:33:57.005Z","identifier":"oai:cris.vtt.fi:publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::74009c567c81b4aa55c813db658734df","oai:cris.vtt.fi:publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"],"pid":[],"publisher":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"VTT Technical Research Centre of Finland"},"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Veijalainen , J & Wolski , A 1991 , Prepare and commit certification for decentralized transaction management in rigorous multidatabases : Research Report No. J-1 . VTT Technical Research Centre of Finland ."}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Prepare and commit certification for decentralized transaction management in rigorous multidatabases:Research Report No. J-1"}]} +{"author":[{"fullname":"Hanhijärvi, Antti","name":"Antti","pid":[],"rank":1,"surname":"Hanhijärvi"},{"fullname":"Hukka, A.","name":"A.","pid":[],"rank":2,"surname":"Hukka"},{"fullname":"Paajanen, T.","name":"T.","pid":[],"rank":3,"surname":"Paajanen"},{"fullname":"Pulkkinen, P.","name":"P.","pid":[],"rank":4,"surname":"Pulkkinen"},{"fullname":"Sundman, S.","name":"S.","pid":[],"rank":5,"surname":"Sundman"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:32:33.974Z","dateoftransformation":"2022-02-28T17:38:24.191Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"The paper presents experimental results of jet drying tests of birch (Betula pendula) and spruce (Picea abies) veneers at temperatures of 140°C and 190°C. Results include drying rates for 1.5 mm thick birch veneers as well as 1.5 mm and 2.6 mm thick spruce veneers of both heartwood and sapwood. Based on the test results, material parameter values for a simplified drying model are assessed. The model is based on the use of an effective diffusion coefficient and an effective surface emission coefficient, which values are calibrated to fit to the experimental results. It is observed, that separate model parameter sets are needed for the two different species but also for occurrence of heartwood or sapwood (spruce) and different thickness values of veneers."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|dedup_wf_001::08d6f2001319c86d0e69b0f83ad75df2","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s00107-003-0379-4"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8cc91100-904f-43c5-bb3d-1cc3e0f4a4b5"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505760180,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-31T21:18:51Z","harvestDate":"2022-02-28T12:32:33.974Z","identifier":"oai:cris.vtt.fi:publications/8cc91100-904f-43c5-bb3d-1cc3e0f4a4b5","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::9cb10895b4a92b0215b85acb2c3268b9","oai:cris.vtt.fi:publications/8cc91100-904f-43c5-bb3d-1cc3e0f4a4b5"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Hanhijärvi , A , Hukka , A , Paajanen , T , Pulkkinen , P & Sundman , S 2003 , ' Experimental investigation of jet drying of birch and spruce veneers and modelling with a simplified approach ' , Holz als Roh- und Werkstoff , vol. 61 , no. 2 , pp. 83-88 . https://doi.org/10.1007/s00107-003-0379-4"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"jet drying"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"drying"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"veneers"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"birch"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"spruce"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"heartwood"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"sapwood"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Experimental investigation of jet drying of birch and spruce veneers and modelling with a simplified approach"}]} +{"author":[{"fullname":"Vainonen-Ahlgren, Elizaveta","name":"Elizaveta","pid":[],"rank":1,"surname":"Vainonen-Ahlgren"},{"fullname":"Likonen, Jari","name":"Jari","pid":[],"rank":2,"surname":"Likonen"},{"fullname":"Renvall,","pid":[],"rank":3},{"fullname":"Rohde, V.","name":"V.","pid":[],"rank":4,"surname":"Rohde"},{"fullname":"Mayer, M.","name":"M.","pid":[],"rank":5,"surname":"Mayer"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"dateofcollection":"2022-02-28T12:32:50.667Z","dateoftransformation":"2022-02-28T17:49:49.964Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"To investigate material transport in scrape-off layer plasma and long term deposition in divertor, 13CH4 was puffed at the end of 2004 and 2005 experimental campaigns into ASDEX Upgrade from the outer mid-plane.
Ex situ analyses of the tiles were performed by secondary ion mass spectrometry.
The peaks of 13C were detected below the bottom inner strike point and at the horizontal tile at the outer lower divertor. It was detected ∼21% of the total puffed 13C amount.
The deposition rate for carbon by plasma was also calculated in long term experiment. It was obtained to be 22 × 10−3 and 8.7 × 10−3 g/s for the upper (campaign 2004) and lower (campaign 2003) divertors, respectively."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1016/j.jnucmat.2007.01.026"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Patent","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505766149,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-02-01T02:35:05Z","harvestDate":"2022-02-28T12:32:50.667Z","identifier":"oai:cris.vtt.fi:publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::a29614444f5030f11e75c6c27264d272","oai:cris.vtt.fi:publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Vainonen-Ahlgren , E , Likonen , J , Renvall , Rohde , V & Mayer , M 2007 , ' Migration of 13C and deposition at ASDEX Upgrade ' , Journal of Nuclear Materials , vol. 363-365 , pp. 270-275 . https://doi.org/10.1016/j.jnucmat.2007.01.026"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"ASDEX upgrade"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"divertor"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"carbon based materials"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"erosion"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"deposition"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"ITER"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"JET"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"plasma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fusion energy"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Migration of 13C and deposition at ASDEX Upgrade"}]} +{"author":[{"fullname":"Aalto, Timo","name":"Timo","pid":[],"rank":1,"surname":"Aalto"},{"fullname":"Harjanne, Mikko","name":"Mikko","pid":[],"rank":2,"surname":"Harjanne"},{"fullname":"Kapulainen, Markku","name":"Markku","pid":[],"rank":3,"surname":"Kapulainen"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[{"dataInfo": null,"id": "beopen"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:32:37.581Z","dateoftransformation":"2022-02-28T19:39:18.717Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"A novel method has been developed for measuring the rotational angle of a fiber's or a waveguide's polarization axis with respect to a reference angle. The reference angle is the polarization axis of the measuring device. The method also gives the true polarization extinction ratio of the measured fiber or waveguide. The method is suitable for the characterization and rotational alignment of polarization-maintaining waveguides and fibers. In particular, the method can be used to rotationally align the fiber-waveguide interconnections during waveguide characterization. The measuring device is either a linear polarizer or a polarization splitter that is accurately rotated with respect to the device under test. According to the experiments with a polarization-maintaining fiber, the method is very easy and inexpensive to implement, and the angular accuracy can be better than 0.2 deg."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|openorgs____::64badd35233ba2cd4946368ef2f4cf57","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1117/1.1600730"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8cd538fb-6484-4655-81dd-47348d358fd4"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505829230,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-31T21:47:37Z","harvestDate":"2022-02-28T12:32:37.581Z","identifier":"oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::df0143af011fd82af8ac2d07b03ee8cd","oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Aalto , T , Harjanne , M & Kapulainen , M 2003 , ' Method for the rotational alignment of polarization-maintaining optical fibers and waveguides ' , Optical Engineering , vol. 42 , no. 10 , pp. 2861-2867 . https://doi.org/10.1117/1.1600730"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"optical waveguide"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization-maintaining fiber"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"characterization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fiber-waveguide coupling"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization axis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarizer"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization splitter"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Method for the rotational alignment of polarization-maintaining optical fibers and waveguides"}]} +{"author":[{"fullname":"Aalto, Timo","name":"Timo","pid":[],"rank":1,"surname":"Aalto"},{"fullname":"Harjanne, Mikko","name":"Mikko","pid":[],"rank":2,"surname":"Harjanne"},{"fullname":"Kapulainen, Markku","name":"Markku","pid":[],"rank":3,"surname":"Kapulainen"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[{"dataInfo": null,"id": "beopen"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:32:37.581Z","dateoftransformation":"2022-02-28T19:39:18.717Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"A novel method has been developed for measuring the rotational angle of a fiber's or a waveguide's polarization axis with respect to a reference angle. The reference angle is the polarization axis of the measuring device. The method also gives the true polarization extinction ratio of the measured fiber or waveguide. The method is suitable for the characterization and rotational alignment of polarization-maintaining waveguides and fibers. In particular, the method can be used to rotationally align the fiber-waveguide interconnections during waveguide characterization. The measuring device is either a linear polarizer or a polarization splitter that is accurately rotated with respect to the device under test. According to the experiments with a polarization-maintaining fiber, the method is very easy and inexpensive to implement, and the angular accuracy can be better than 0.2 deg."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|openorgs____::64badd35233ba2cd4946368ef2f4cf57","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1117/1.1600730"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8cd538fb-6484-4655-81dd-47348d358fd4"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505829230,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-31T21:47:37Z","harvestDate":"2022-02-28T12:32:37.581Z","identifier":"oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::df0143af011fd82af8ac2d07b03ee8cd","oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Aalto , T , Harjanne , M & Kapulainen , M 2003 , ' Method for the rotational alignment of polarization-maintaining optical fibers and waveguides ' , Optical Engineering , vol. 42 , no. 10 , pp. 2861-2867 . https://doi.org/10.1117/1.1600730"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"optical waveguide"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization-maintaining fiber"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"characterization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fiber-waveguide coupling"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization axis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarizer"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization splitter"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Method for the rotational alignment of polarization-maintaining optical fibers and waveguides"}]} +{"author":[{"fullname":"Penttilä, Raimo","name":"Raimo","pid":[],"rank":1,"surname":"Penttilä"},{"fullname":"Vanttaja, Ilkka","name":"Ilkka","pid":[],"rank":2,"surname":"Vanttaja"},{"fullname":"Haapamäki, Petteri","name":"Petteri","pid":[],"rank":3,"surname":"Haapamäki"},{"fullname":"Kujanpää, Veli","name":"Veli","pid":[],"rank":4,"surname":"Kujanpää"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","value":"VTT Research Information System"}],"context": [{"dataInfo":null, "id":"dh-ch"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1994-01-01"},"dateofcollection":"2022-02-28T12:35:26.769Z","dateoftransformation":"2022-02-28T19:54:10.494Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1994-01-01"},"distributionlocation":"","hostedby":{"key":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/ddbd28ea-5fbf-43e1-896f-e69856870c26"]}],"language":{"classid":"fin","classname":"Finnish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505838552,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2019-08-08T07:09:42Z","harvestDate":"2022-02-28T12:35:26.769Z","identifier":"oai:cris.vtt.fi:publications/ddbd28ea-5fbf-43e1-896f-e69856870c26","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:cris.vtt.fi:publications/ddbd28ea-5fbf-43e1-896f-e69856870c26","50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Penttilä , R , Vanttaja , I , Haapamäki , P & Kujanpää , V 1994 , ' Liimauksen ja puristusliittämisen yhdistämisellä lisää lujuutta, jäykkyyttä ja tiiveyttä ' , Ohutlevyuutiset , no. 2 , pp. 17-19 ."}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Liimauksen ja puristusliittämisen yhdistämisellä lisää lujuutta, jäykkyyttä ja tiiveyttä"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/relation/part-00000.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/relation/part-00000.json new file mode 100644 index 000000000..1733e2549 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/relation/part-00000.json @@ -0,0 +1,24 @@ +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"issupplementedby","relType":"resultOrganization","source":"50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6","subRelType":"affiliation","target":"50|pending_org_::82f63b2d21ae88596b9d8991780e9888","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"issupplementedby","relType":"resultOrganization","source":"50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6","subRelType":"affiliation","target":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","subRelType":"affiliation","target":"20|openorgs____::322ff2a6524820640bc5d1311871585e","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","subRelType":"affiliation","target":"20|openorgs____::58e60f1715d219aa6757ba0b0f2ccbce","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","subRelType":"affiliation","source":"10|issn___print::a7a2010e75d849442790955162ef4e42","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|issn___print::a7a2010e75d849442790955162ef4e43","subRelType":"affiliation","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|issn___print::a7a2010e75d849442790955162ef4e44","subRelType":"affiliation","target":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|issn___print::a7a2010e75d849442790955162ef4e45","subRelType":"affiliation","target":"20|pending_org_::c522a7c935f9fd9578122e60eeec282c","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isrelatedto","relType":"resultOrganization","source":"50|openorgs____::64badd35233ba2cd4946368ef2f4cf57","subRelType":"affiliation","target":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","subRelType":"affiliation","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isrelatedto","relType":"resultOrganization","source":"50|355e65625b88::74009c567c81b4aa55c813db658734df","subRelType":"affiliation","target":"50|dedup_wf_001::08d6f2001319c86d0e69b0f83ad75df2","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::08d6f2001319c86d0e69b0f83ad75df2","subRelType":"affiliation","target":"20|openorgs____::91a81877815afb4ebf25c1a3f3b03c5d","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","subRelType":"affiliation","target":"50|dedup_wf_001::0a1cdf269375d32ce341fdeb0e92dfa8","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0a1cdf269375d32ce341fdeb0e92dfa8","subRelType":"affiliation","target":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","subRelType":"affiliation","target":"50|dedup_wf_001::0ab92bed024ee6883c7a1244722e5eec","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0ab92bed024ee6883c7a1244722e5eec","subRelType":"affiliation","target":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","subRelType":"affiliation","target":"50|dedup_wf_001::0ca26c736ad4d15b3d5ee90a4d7853e1","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0ca26c736ad4d15b3d5ee90a4d7853e1","subRelType":"affiliation","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","subRelType":"affiliation","target":"50|dedup_wf_001::0ef8dfab3927cb4d69df0d3113f05a42","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0ef8dfab3927cb4d69df0d3113f05a42","subRelType":"affiliation","target":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","subRelType":"affiliation","target":"50|dedup_wf_001::0f488ad00253126c14a21abe6b2d406c","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0f488ad00253126c14a21abe6b2d406c","subRelType":"affiliation","target":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|pending_org_::c522a7c935f9fd9578122e60eeec282c","subRelType":"affiliation","target":"50|dedup_wf_001::12206bf78aabd7d52132477182d19147","validated":false} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::12206bf78aabd7d52132477182d19147","subRelType":"affiliation","target":"20|pending_org_::c522a7c935f9fd9578122e60eeec282c","validated":false} \ No newline at end of file From 189a7c255a52127f79db59d7cde171f216d76152 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Mon, 25 Nov 2024 16:52:13 +0100 Subject: [PATCH 220/239] [patents] added test and resources --- .../PrepareResultCommunitySetStep1.java | 11 ----------- .../ResultToCommunityJobTest.java | 2 ++ 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index 8f23a4cfd..5af2bf481 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -173,24 +173,13 @@ public class PrepareResultCommunitySetStep1 { "AND lower(co.id) IN " + communityIdList, "AND lower(relClass) IN " + allowedsemrel); Dataset result_context = spark.sql(resultContextQuery); - //result_context.createOrReplaceTempView("result_context"); - -// spark -// .sql(RESULT_COMMUNITY_LIST_QUERY) -// .as(Encoders.bean(ResultCommunityList.class)) -// .write() -// .option("compression", "gzip") -// .mode(SaveMode.Overwrite) -// .json(outputResultPath); Dataset rwc = spark.sql(String.format(RESULT_WITH_CONTEXT, communityIdList)); Dataset patents = spark.sql(RESULT_PATENT); Dataset relatedToRelations = spark.sql(IS_RELATED_TO_RELATIONS); rwc.createOrReplaceTempView("resultWithContext"); - patents.createOrReplaceTempView("patents"); - relatedToRelations.createOrReplaceTempView("relatedTorelations"); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index 4361b6f39..c1fcff4d9 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -326,4 +326,6 @@ public class ResultToCommunityJobTest { Assertions.assertEquals(1, communities.size()); Assertions.assertEquals("dh-ch", communities.get(0)); } + + } From 2e54715d715426cd66cca94f1b63fe29abbcc8ce Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 22 Nov 2024 10:05:56 +0100 Subject: [PATCH 221/239] Applying PR#512 - Sequential ActionSet promotion --- .../wf/main/oozie_app/workflow.xml | 47 ++++--------------- 1 file changed, 9 insertions(+), 38 deletions(-) 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 7ccfb342e..cdb4de908 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 @@ -135,22 +135,10 @@ --outputPath${workingDir}/action_payload_by_type --isLookupUrl${isLookupUrl} - +
- - - - - - - - - - - - ${wf:appPath()}/promote_action_payload_for_dataset_table @@ -162,7 +150,7 @@
- + @@ -177,7 +165,7 @@ - + @@ -192,7 +180,7 @@ - + @@ -207,7 +195,7 @@ - + @@ -222,7 +210,7 @@ - + @@ -237,7 +225,7 @@ - + @@ -252,7 +240,7 @@ - + @@ -267,26 +255,9 @@ - + - - - ${wf:appPath()}/promote_action_payload_for_person_table - - - - inputActionPayloadRootPath - ${workingDir}/action_payload_by_type - - - - - - - - - \ No newline at end of file From ca2d480df372b174d92f141f38cdf14b19130979 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Tue, 26 Nov 2024 13:36:27 +0100 Subject: [PATCH 222/239] [BulkTagging] added fix to consider when the set of constraints for the datasource is empty. Added check for remove constraints and advanced constraints to verify if the constraints list is empty and in that case do nothing --- .../dhp/bulktag/community/ResultTagger.java | 15 +++++++++------ .../bulktag/community/SelectionConstraints.java | 2 ++ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java index 2ea229e3e..64cbd70ba 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java @@ -130,6 +130,7 @@ public class ResultTagger implements Serializable { // log.info("Remove constraints for " + communityId); if (conf.getRemoveConstraintsMap().keySet().contains(communityId) && conf.getRemoveConstraintsMap().get(communityId).getCriteria() != null && + !conf.getRemoveConstraintsMap().get(communityId).getCriteria().isEmpty() && conf .getRemoveConstraintsMap() .get(communityId) @@ -161,29 +162,30 @@ public class ResultTagger implements Serializable { // Tagging for datasource final Set datasources = new HashSet<>(); - final Set collfrom = new HashSet<>(); + final Set cfhb = new HashSet<>(); final Set hostdby = new HashSet<>(); if (Objects.nonNull(result.getInstance())) { for (Instance i : result.getInstance()) { if (Objects.nonNull(i.getCollectedfrom()) && Objects.nonNull(i.getCollectedfrom().getKey())) { - collfrom.add(i.getCollectedfrom().getKey()); + cfhb.add(i.getCollectedfrom().getKey()); } if (Objects.nonNull(i.getHostedby()) && Objects.nonNull(i.getHostedby().getKey())) { + cfhb.add(i.getHostedby().getKey()); hostdby.add(i.getHostedby().getKey()); } } - collfrom + cfhb .forEach( dsId -> datasources .addAll( conf.getCommunityForDatasource(dsId, param))); hostdby.forEach(dsId -> { - datasources - .addAll( - conf.getCommunityForDatasource(dsId, param)); +// datasources +// .addAll( +// conf.getCommunityForDatasource(dsId, param)); if (conf.isEoscDatasource(dsId)) { datasources.add("eosc"); } @@ -226,6 +228,7 @@ public class ResultTagger implements Serializable { .forEach(communityId -> { if (!removeCommunities.contains(communityId) && conf.getSelectionConstraintsMap().get(communityId).getCriteria() != null && + !conf.getSelectionConstraintsMap().get(communityId).getCriteria().isEmpty() && conf .getSelectionConstraintsMap() .get(communityId) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java index 57cc658fc..8a23a7017 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java @@ -33,6 +33,8 @@ public class SelectionConstraints implements Serializable { // Constraints in or public boolean verifyCriteria(final Map> param) { + if (criteria.isEmpty()) + return true; for (Constraints selc : criteria) { if (selc.verifyCriteria(param)) { return true; From 0517e452e34cf30d471552f5930983a9435a91c5 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Mon, 2 Dec 2024 14:00:59 +0100 Subject: [PATCH 223/239] Fixed error on empty affiliation --- .../main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java index 54aba8715..a8dacd132 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java @@ -23,9 +23,8 @@ public class PMAffiliation { return name; } - public PMAffiliation setName(String name) { + public void setName(String name) { this.name = name; - return this; } public PMIdentifier getIdentifier() { From cc6bbbb80474553a4d3e71bc0fa80c4b84035194 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Tue, 3 Dec 2024 14:31:11 +0100 Subject: [PATCH 224/239] make setter void --- .../main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java index a8dacd132..e3dcc95dd 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java @@ -31,8 +31,7 @@ public class PMAffiliation { return identifier; } - public PMAffiliation setIdentifier(PMIdentifier identifier) { + public void setIdentifier(PMIdentifier identifier) { this.identifier = identifier; - return this; } } From 65902a87e3ae7b161cd8e682f4a3535df996e3cd Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 4 Dec 2024 13:18:17 +0100 Subject: [PATCH 225/239] support of the new apis --- .../plugin/researchfi/ResearchFiIterator.java | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java index 269a89f71..cad499962 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/researchfi/ResearchFiIterator.java @@ -6,7 +6,7 @@ import java.util.Queue; import java.util.concurrent.PriorityBlockingQueue; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.math.NumberUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.http.Header; @@ -27,25 +27,25 @@ public class ResearchFiIterator implements Iterator { private final String baseUrl; private final String authToken; - private int currPage; - private int nPages; + private String nextUrl; + private int nCalls = 0; private final Queue queue = new PriorityBlockingQueue<>(); public ResearchFiIterator(final String baseUrl, final String authToken) { this.baseUrl = baseUrl; this.authToken = authToken; - this.currPage = 0; - this.nPages = 0; + this.nextUrl = null; } private void verifyStarted() { - if (this.currPage == 0) { - try { - nextCall(); - } catch (final CollectorException e) { - throw new IllegalStateException(e); + + try { + if (this.nCalls == 0) { + this.nextUrl = invokeUrl(this.baseUrl); } + } catch (final CollectorException e) { + throw new IllegalStateException(e); } } @@ -62,9 +62,9 @@ public class ResearchFiIterator implements Iterator { synchronized (this.queue) { verifyStarted(); final String res = this.queue.poll(); - while (this.queue.isEmpty() && (this.currPage < this.nPages)) { + while (this.queue.isEmpty() && StringUtils.isNotBlank(this.nextUrl)) { try { - nextCall(); + this.nextUrl = invokeUrl(this.nextUrl); } catch (final CollectorException e) { throw new IllegalStateException(e); } @@ -73,18 +73,11 @@ public class ResearchFiIterator implements Iterator { } } - private void nextCall() throws CollectorException { + private String invokeUrl(final String url) throws CollectorException { - this.currPage += 1; + this.nCalls += 1; + String next = null; - final String url; - if (!this.baseUrl.contains("?")) { - url = String.format("%s?PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); - } else if (!this.baseUrl.contains("PageSize=")) { - url = String.format("%s&PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE); - } else { - url = String.format("%s&PageNumber=%d", this.baseUrl, this.currPage); - } log.info("Calling url: " + url); try (final CloseableHttpClient client = HttpClients.createDefault()) { @@ -94,11 +87,15 @@ public class ResearchFiIterator implements Iterator { try (final CloseableHttpResponse response = client.execute(req)) { for (final Header header : response.getAllHeaders()) { log.debug("HEADER: " + header.getName() + " = " + header.getValue()); - if ("x-page-count".equals(header.getName())) { - final int totalPages = NumberUtils.toInt(header.getValue()); - if (this.nPages != totalPages) { - this.nPages = NumberUtils.toInt(header.getValue()); - log.info("Total pages: " + totalPages); + if ("link".equals(header.getName())) { + final String s = StringUtils.substringBetween(header.getValue(), "<", ">"); + final String token = StringUtils + .substringBefore(StringUtils.substringAfter(s, "NextPageToken="), "&"); + + if (this.baseUrl.contains("?")) { + next = this.baseUrl + "&NextPageToken=" + token; + } else { + next = this.baseUrl + "?NextPageToken=" + token; } } } @@ -108,6 +105,9 @@ public class ResearchFiIterator implements Iterator { jsonArray.forEach(obj -> this.queue.add(JsonUtils.convertToXML(obj.toString()))); } + + return next; + } catch (final Throwable e) { log.warn("Error calling url: " + url, e); throw new CollectorException("Error calling url: " + url, e); From 32e2a8b34053dc8162bf20bb91c977d544d05b94 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 4 Dec 2024 13:36:21 +0100 Subject: [PATCH 226/239] implemented zenodo dump collector plugin --- .../CollectZenodoDumpCollectorPlugin.java | 96 ++++++++++++++++++ .../plugin/zenodo/ZenodoTarIterator.java | 59 +++++++++++ .../zenodo/ZenodoPluginCollectionTest.java | 35 +++++++ .../dhp/collection/zenodo/zenodo.tar.gz | Bin 0 -> 7412 bytes 4 files changed, 190 insertions(+) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/CollectZenodoDumpCollectorPlugin.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoTarIterator.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoPluginCollectionTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/zenodo/zenodo.tar.gz diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/CollectZenodoDumpCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/CollectZenodoDumpCollectorPlugin.java new file mode 100644 index 000000000..3ea29a9b0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/CollectZenodoDumpCollectorPlugin.java @@ -0,0 +1,96 @@ + +package eu.dnetlib.dhp.collection.plugin.zenodo; + +import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration; + +import java.io.IOException; +import java.io.InputStream; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; +import eu.dnetlib.dhp.common.collection.CollectorException; + +public class CollectZenodoDumpCollectorPlugin implements CollectorPlugin { + + final private Logger log = LoggerFactory.getLogger(getClass()); + + private void downloadItem(final String name, final String itemURL, final String basePath, + final FileSystem fileSystem) { + try { + final Path hdfsWritePath = new Path(String.format("%s/%s", basePath, name)); + final FSDataOutputStream fsDataOutputStream = fileSystem.create(hdfsWritePath, true); + final HttpGet request = new HttpGet(itemURL); + final int timeout = 60; // seconds + final RequestConfig config = RequestConfig + .custom() + .setConnectTimeout(timeout * 1000) + .setConnectionRequestTimeout(timeout * 1000) + .setSocketTimeout(timeout * 1000) + .build(); + log.info("Downloading url {} into {}", itemURL, hdfsWritePath.getName()); + try (CloseableHttpClient client = HttpClientBuilder.create().setDefaultRequestConfig(config).build(); + CloseableHttpResponse response = client.execute(request)) { + int responseCode = response.getStatusLine().getStatusCode(); + log.info("Response code is {}", responseCode); + if (responseCode >= 200 && responseCode < 400) { + IOUtils.copy(response.getEntity().getContent(), fsDataOutputStream); + } + } catch (Throwable eu) { + throw new RuntimeException(eu); + } + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Override + public Stream collect(ApiDescriptor api, AggregatorReport report) throws CollectorException { + try { + final String zenodoURL = api.getBaseUrl(); + final String hdfsURI = api.getParams().get("hdfsURI"); + final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(hdfsURI)); + downloadItem("zenodoDump.tar.gz", zenodoURL, "/tmp", fileSystem); + CompressionCodecFactory factory = new CompressionCodecFactory(fileSystem.getConf()); + + Path sourcePath = new Path("/tmp/zenodoDump.tar.gz"); + CompressionCodec codec = factory.getCodec(sourcePath); + InputStream gzipInputStream = null; + try { + gzipInputStream = codec.createInputStream(fileSystem.open(sourcePath)); + return iterateTar(gzipInputStream); + + } catch (IOException e) { + throw new CollectorException(e); + } finally { + log.info("Closing gzip stream"); + org.apache.hadoop.io.IOUtils.closeStream(gzipInputStream); + } + } catch (Exception e) { + throw new CollectorException(e); + } + } + + private Stream iterateTar(InputStream gzipInputStream) throws Exception { + + Iterable iterable = () -> new ZenodoTarIterator(gzipInputStream); + return StreamSupport.stream(iterable.spliterator(), false); + + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoTarIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoTarIterator.java new file mode 100644 index 000000000..8e627683e --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoTarIterator.java @@ -0,0 +1,59 @@ + +package eu.dnetlib.dhp.collection.plugin.zenodo; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.Iterator; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.io.IOUtils; + +public class ZenodoTarIterator implements Iterator, Closeable { + + private final InputStream gzipInputStream; + private final StringBuilder currentItem = new StringBuilder(); + private TarArchiveInputStream tais; + private boolean hasNext; + + public ZenodoTarIterator(InputStream gzipInputStream) { + this.gzipInputStream = gzipInputStream; + tais = new TarArchiveInputStream(gzipInputStream); + hasNext = getNextItem(); + } + + private boolean getNextItem() { + try { + TarArchiveEntry entry; + while ((entry = tais.getNextTarEntry()) != null) { + if (entry.isFile()) { + currentItem.setLength(0); + currentItem.append(IOUtils.toString(new InputStreamReader(tais))); + return true; + } + } + return false; + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean hasNext() { + return hasNext; + } + + @Override + public String next() { + final String data = currentItem.toString(); + hasNext = getNextItem(); + return data; + } + + @Override + public void close() throws IOException { + gzipInputStream.close(); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoPluginCollectionTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoPluginCollectionTest.java new file mode 100644 index 000000000..9b5cf1850 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/zenodo/ZenodoPluginCollectionTest.java @@ -0,0 +1,35 @@ + +package eu.dnetlib.dhp.collection.plugin.zenodo; + +import static org.junit.jupiter.api.Assertions.assertNotNull; + +import java.util.zip.GZIPInputStream; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.common.collection.CollectorException; + +public class ZenodoPluginCollectionTest { + + @Test + public void testZenodoIterator() throws Exception { + + final GZIPInputStream gis = new GZIPInputStream( + getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/zenodo/zenodo.tar.gz")); + try (ZenodoTarIterator it = new ZenodoTarIterator(gis)) { + Assertions.assertTrue(it.hasNext()); + int i = 0; + while (it.hasNext()) { + Assertions.assertNotNull(it.next()); + i++; + } + Assertions.assertEquals(10, i); + + } + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/zenodo/zenodo.tar.gz b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/zenodo/zenodo.tar.gz new file mode 100644 index 0000000000000000000000000000000000000000..6c06bf4e5a5d568aa6c534cc4636796d9938f97e GIT binary patch literal 7412 zcmV44`c0g6Vvm2)J$*9; zQILd$BG?2eTe;eI*ca%jx!CHh+Ka8;CuqOPKFOW~iu$oB*|FrFwjNqPe| zq*``Y6H&he#qs~5p&6x}wEXU+>u-+TGuam=sCmU1!?@QE6MHPb>q~ zJDZ-or$a^p_$O`h7>bL4`aAnIP$;y^3gnay6XugHBMlmjs6P;UrMZXq_xgOq0(F{K zbar>IM~B=WtMf_id!@N+d$iZ%K3BtOALg47J`p`q>qp#6{5XlIPe9F)4-C4<0~&D} zkW-I?LSPc;y!wYA$29Q;IgNn<#N z`YbvD83K1+C)^*f2nsc|rR-TOl+Pbk-@gWHrW3pggB)h<+vS{@EAix~^HRo?W=b&T zvCp#K{WkAw#3Vc*uYrV^G@!yRC9NTg`bbOxr+bVJ)>skT7>Fp5qyd@&=pk(p4OL@= zN^-(RjQXrgI%6O^>2fB?xdd;8P;x2~c={}&;a(|idGZUCgh|Ke5=P(t&q~+tm1bAc zo2i-oE2GiA2@dL{z}(e6nRGrfFP77@{W=FvX~Hz{7$#Szx6kOM2*hyyR%uoV7+*H| zA^1-^eh#Ee7#L)vp{P4W(^B3AyHW5w#B){Bw2Qi1G>oidX;O9Odf)?O{*TuzU@$zO zwIS`Z9h6WK!RVSVgOB^z!h)f@agE%#GIJWzd%(Y}YsaAdj(R)!w7i8&FmY<2AU&Tc zj{}UcTat!e1x6r0s%6tCI!NcWHot%{bTHNax8rw=aO*q8jd60AH?#I%Q!nH>CddxtDQi^GGre91>^jXgNH2{ zc`ssenq7w4)nohFNm^x>NiX6d_C+rIMdr@aUe-D?j;I&!WP0jol&VTlI-o<{Uu=%l~&G= z(AF5W_F}zIjp0Bo8!2am=`L3)_xjP@Oyb#}h5uMdgeRo-#qpUE2a#;#3t#jw>;7 z9dBCF046K7XD-CR@uIR|5(lz z|M?on7XR7eKljCdl-hU<{AbeegW*4`>fapx^CRULUqPuTkn|JSO13mUyk0CnNQ4>} z14L|gjt`4}stJdN{G@Tv+}TIfTZI43V!2u_7cHw?-776Vu^4fJ*@NIJ0et{Au;mF~ z^iylw8VNtaBIuGpgoHQ1)T^h!VxCcc0p8Ut+AL(3CO8ovabOYFnm!E1-Ou9I0%=nW zZk4Yyy><-Yt*7WX6+!=#PZw%`0}jq#k=g3z0Pa>n#3nR5pxj4_m(xbxu&4v>Nirla z&T0oQZ|z6OIN13z`X&p#`Z$%KCch75qESza2Y({jNEeaEyK1243SwO>;!Of$jYY+G zpx2j=vT1?*@%1KIe|T9%)3vI_s_wdvGrT?y2g&zAoM^0@Xz|4=2`te1#h}N*>+_Qq zF%0w;&n}oh0?Duoq5>N5G1I}yA{c?>#pH}p7|ib=T9UCCDvLtdOVW$Pa5`4kMm7*n zco1YGJ2?h2V^mT%AA&Gzz(hD0$28~mJK7I~&qWCFN$&k|6oaQO^KLEjET=@q>-#}5 zrd2fbqVY{bF-F-gmr!;PSzS`CZ0ee+8BYO=fyp~JqP5VcpShaIyj!X@-CDEhRNL*Q z+iq8!reiqmc1y3@7F?^@Pl5iWSQ0oQ#+tBBuIwA&6L#z4B_y7)%Q)ZPGtGyP_g$!& zh&e`Oi*Jw?h3pu%RdY-Ogj@$9*FfIQcCAvb8|7BJp;x{(9ET3%1CFe31&+%ss78(l zA{L7_b11;d1eCnh2>87d&U{+vBI$2uN`W!Go{&uOGyVx&>sBZEv^<+|ayP&KX%!n8 z`L+XGu=HCdf2|jBuNM8pN7wqlP{_mdO=1%4D+CrX&@MNCUeghnvjCGmm*>X5U78D< zJ-(5bBZh;aPSw7JTVKS8r66*xE7DjHqBBK`3Nr=9fvRQ*uAzN0XNoWc`+)^xm|G2j zAm#Iek>C--T#G(ac@|g&^&$cEDWOsVu`&^1D57`*A*eDwX5t1Cej%o0GvrzQpf|G% z0bu{tPA{T6L|shfb{5eFYe5G(S4qFg2ix36Y!Sg)Y$+phb2IN$RdyxTmvGK ztSc-ajjR7%=nmYR^hi|)O_EsP>jM8d_Ue~}k>XsZu6~nP&SzKuiG8AB7;%Xrc?dZD zj&QR5kZ^K}n;pYF;H9E%qnPp=7$G3r8xYk|M+%?a4bVdmkE#(EUGkBZbdmsM#bOQq zFAY=!C8P8p2Rcq1?{yqn0S72kcKhXveJ1_F*XZRQjy@N zn4&Co#wA93arU6sFvxo{i{Of}e~Ffx>7`2@niUngY3Cgz_PhN+ot_Vn>4cjuZ$u=aI^!oHW5FZ*-A^Xr_=S zbk4wad7sDBhiZkNTA*BxNyRcsB<}~xZs6SjN0U_Vim%fG^-52SK>2jQ_Tu~CzAqqb z@YC1H(242O49v62DHtPt5JU@~Mo@0XjYG7BWGFyBv;*cQwaU9WFOk3dk;dLYbutZI zQ6xZ~;$%3dmI;mj_%%lzg3#bURTuo0ei;wM=ki zMr6q`ds&*YDB4vrV(Xj?6{Xgk$=~ z$elbgd9O+8V{nAPE_yhWkQ*Z|`AwNcQ<=d$WdZGgw{~p~D3N%=_hArEQk)n4zj<;b z4xSk1;dH=wBES@0n0T0I%z$z5o6Rcik6-6!36z;}3RragjmH+hf6ODz|JAE{`SJ39 zO{;t}|F^uw|G&nu#s9bX|Go2nQEfa*{%_jw!{z_3sDE?#|A`o~NEAq2fN3Ug=?F>v znGV5e4wj=_OG+P02fO4g%;slkh}9NiV|1IVAP!5WG>d30Ix!uAw_RIp$aYgB7)Wa)fMaOc%&q#i`BZ$uObNnqw#xMU3CBGQPG z&*Ud@DLfJTSAU0?(=|*>FHE0#clFyPQkjZhDxUSl_hUK`0;R4EFIOj5&oSqoslAp; zckx*gLGTQoFUP2sCS4vYkd-7sfNQ70*`Lp|x%LbgTFS_aBMbw&^9eR!G~&SpuokpU zHrW4#lYbBSip<}d#9`H6lKeZzyIhLq1hiK6uer_sT}an5wDMEPo?VswTeobp?bwD< zxAc~6H(GAF)^0bdm4<6#qUckge|h#VxFPz}$^O-=_L|f@Q1-9gcI%aS;LZx5>~g%|G%l_|7pqcHvs@{m;ZlT%l{3t`Xm9sx#j;&5OP&5|8F%~b*EytYt_ah zEdRIH+;dm$MA4aCQhj|%WD@nL+FQAKV~>ID#sUwxv4Bz8ZY*%W zBg_A*SdWJP*`|B*`!DwP`!8SR*y4X%{O`{AU;dk*$AJJ*Wj$;P;AZMSJVo>aoc>)O zjP>r+Z{3BEW!GxyEvssk!EvqFhTVdQqz%zYrEWQJtySLy0aIqK3X|?h$mYP%51fHd z1O5Z2eL)VFe<@>DARnVjE7}B3rJI#txOVS)E@ep;J&RWWwZAIb0gcS~TLBXjL-2}DFWc&`VNlEcK~lr)z^ z(u;+b>`tI6+{ESt71yp5t1hBfE4OEu2mwDSL|R};x(QX=&%YM@>hDr9KLlt0H0cU_ zHC_Pk8}9xk;Smq6eh-C@{jWz9+MB)0ASu+bC zKH`!N6(fwh$#kK&>&mpwB5=+dKoyuV@__~*uH`!}+$fhXSFCgsQgvYP7WH!FmMM1q z%TWz}^o@I>X71~4ycBaNtK>H!!OOkWLX`cbJy>$s3p zzD@@?ofpJ9@^JB^_&#Ahsdb{&5}oVmmjcNrEL*&#UV0oyLPEF(ET!pkHJ@XkNf&ZJ z3gnP7pJQ&w%WSD-N<4npq5(lB?E)(y_JW&XBt~GtB^gmaQLDw`6yD`tx>{7lx<0Od zBybhnL!VB*F4X-9{Jfmn?v|O}J(3q93Q7QI}Zl`Qex7zD${xdvpr~=`>7&+C~ z-GelTjd!7CHE!B4EjRV{nl<>Xx7((RJE+zT@C2L2np+@zG5aQOKyni*D{CfHZkCwM zgYs(-vnD2=LkuGF^0qNDy}8X z2N2Vp$U!i%6nyMxIz&&K*Fkrp!Ks$bQfJ_6=s9V)pWT|Ty4EH{xdoc#mYp?M4cuES zPE{|NhU*mFN?8YkX}BhMWKV<`(8T(>yK8S?z*gPOYyG>?kbW4;G|^86Uq-LGj#d2p zBnGWrDO-;v8r`atq-{7(Redi^*Q<}DWUuqYdc+-)?(v56Uaugpqf!Nu2a!Nrm=|P| z7dWk7?vj6w08YEP9AKAJZ3rs#+tvb9A@)8w+|9t}?xofhsgt=1H2(Hpwmrck%m1%D z;`jf`ZqEMS_V)eX*EqKC|F-Y{*2w>_Jl6OBDi8eqzoqqW^Z&Q`|J(fkZT|mV^Z!j# zo29?SET9s%5(JN1a(Od#al2UHn_4XJE$tSvT`cg8Efy$0#2y%%TrAM2x7*e9TY*)l zS$ErwcEjB~On&=ffy(wtmluh{lP$sHMsNfe4B>Z`NNf z`JiAO=~8L`*{22D8aalL%U@?*!E5NAN`wKgRwIYt&w`7DTY%u!#a_0NHYGjP=Cq-~ zI0R}3;KddQ?$tJ=_&fH~#RUSoHP&z&=hS@wz8nh?QXr1S5Ikwbu*&ZLw|6zmZQDTj zRdBRtL<-DZQJI59BektIP(a0%=m7T=PsqyFn{o$O}e#RSh99O-0`Ou@|!^Xz#YH!z(a=D0zX z|L=KBR{W1Trr|Ef{}lemqcj!&Q}Mrt{1_^DTytHXC6|TFNAvl1^hGk&J&Rp`)l*47CSZ6=o1829@kz5_6r7b z;I;W^K>2O`Km>)A$I*cq66e|hmXW}+CI^>avM6bW`n~$TA&+{2T+`@o3o(Jo0v09+ zYnbM}o(LbEftg)^86lvQhr%F2(`S-S&$5q`B#(+5jM_<&g)c~_c$*>?ZwzMaMOo)1 zCa)*)2Q0z?hS^~h6^=(X5DOMOMPn1t4-pr3E&)NJ~3^4HdQ+ z-%t?h9x)xpHEKL_1X#zVZRTl=KJhpR_1yKR$b1v>)AKd~j{5AJ_^;$yo$L}qMi+iL zc&g)cGK(?$1zr3WI^_5%SnFdUR6&<*`Q=B+cshzd!24bCJ_^StpfIZ6p&z`}Eh%Dc z#8Xfg#VE~-{9D6$TS>jGpo#2WJ=uN)za!B31+gh5|NcE*n-jfkTHwj2W!2syHx+B; zXE*3Lbh#PF3Y=mw;$?n{vuQjlcV=w!CV*%T?=^?raFeg4$G3n(#qxauuJN;ISM`59 ziS>Uhwv7L&@_!zsDgR&j|Etvh@nqHi@itulr-{C*|D)>vsQN#u{?BojSgQWd6Fy|E z>i?+vKdSzZs{bRW{?Ce;SSlEJrDXMzJUn)Bcups?>|>sXrIN&ha4FgORE*|^VR;y? zgxs9h{t@_?rwEy;N&wxx1dy#t0R8w)#Q&)4a!XeH&oC@!IseBoRQ&HTnu`Ca_}|^} zKPhwC@AM>{=?I^JwvOdVi2w?TwZRDB+UVa3{}}`U1uq2tILGq_oLTU{&w`+91s%)Z zcnGo!w#q2PJ-y36QgRd)BLp@ar+M=cvE`oiQLK{mVic=B=oG|50f>%xdsoCuh(2Fq zU?kbJMVRG!0!5E>!YqqtDTWk}zCz?%0068ZH^{=1aC9cZhWMQph1m$L*cP!6<$#&C z<7hUg3EVW^%SoCe)Lea{KB8-sKm8g~~uyAbE${8f_rII)nF$WLTQm;DO*X zoTeuUD3n{nu}fVWa>$VOU04j9=p<=0hpBAAQUR|fCpV!hyB`q94pA%Pd`kZN{F`9d zHEt!&TkxJRiT!%X^=BA*x<1>uT|PTMgPxavor_tUj#AJ-h&}7TBV{(VEsS`Ub9z6` zFaIRNlpH5O6Bp8&=XWwXhbP)Q2j}Hg0k=jlpw|Q;yV$E?UL(b3rhf-(KFH!kJnH=v zZWvedFeKG&HHz7#p7q?RXg`viU7Z()Be)HLsq?akU=_e%cCzsDpLfmcbGc(ZE9RtS z^P+LvV26z3{0uz#*+rVYCvaCW$!EpzqbO)ROitlRjtF}HDan&l+$fo5X@n3#lT%UI z7;~CP5N6}mHv$a#NtFg}!e}P@Zm~_*hvP_HZ`Dxmy%9pcMp>O9a@1mEbvIyv0}anf zf`RIAisvS+E}n@EI>Gv}^HyNrV}rfkI+*oC96o$7>}gLdaDqIOQ^yQ;cio_C8v%Ep z48AI|?!b|#~|o&3^0&<)C5-Q|ww8Is0g!qjcQO*loC$l^yCmWk3a z<~?ARuFeddu9~O6(&Op%eP3ssySl>=-&EpI_Irl|T~3~Gc$P+$n1PioYcyg#<+yu5B?2eIq}lQPZoiLi7}GSF-L@>=W^Kw0YRcjd zPMSgQpk$3nspD)Q&F3L2n!4}_Xai}CvFtX8Tx#2nE?ym(%HY61tSn5hV5qZ!av1kf zOx;1RJ2;wHq}nQ(g2(>(~xOtR_0x~l(-?w5vZ)*qnG|%O@lQfb Date: Wed, 4 Dec 2024 13:37:14 +0100 Subject: [PATCH 227/239] code formatted --- .../dhp/sx/bio/pubmed/PMAffiliation.java | 42 ++++++----- .../dnetlib/dhp/sx/bio/pubmed/PMAuthor.java | 1 - .../dhp/sx/bio/pubmed/PMIdentifier.java | 74 +++++++++---------- .../collection/crossref/Crossref2Oaf.scala | 1 - .../dnetlib/dhp/sx/bio/pubmed/PMParser2.scala | 13 ++-- .../dhp/sx/bio/pubmed/PubMedToOaf.scala | 20 ++++- .../dnetlib/dhp/sx/bio/BioScholixTest.scala | 1 - 7 files changed, 82 insertions(+), 70 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java index e3dcc95dd..427eb2725 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.sx.bio.pubmed; /** @@ -7,31 +8,32 @@ package eu.dnetlib.dhp.sx.bio.pubmed; */ public class PMAffiliation { - private String name; + private String name; - private PMIdentifier identifier; + private PMIdentifier identifier; - public PMAffiliation() { + public PMAffiliation() { - } - public PMAffiliation(String name, PMIdentifier identifier) { - this.name = name; - this.identifier = identifier; - } + } - public String getName() { - return name; - } + public PMAffiliation(String name, PMIdentifier identifier) { + this.name = name; + this.identifier = identifier; + } - public void setName(String name) { - this.name = name; - } + public String getName() { + return name; + } - public PMIdentifier getIdentifier() { - return identifier; - } + public void setName(String name) { + this.name = name; + } - public void setIdentifier(PMIdentifier identifier) { - this.identifier = identifier; - } + public PMIdentifier getIdentifier() { + return identifier; + } + + public void setIdentifier(PMIdentifier identifier) { + this.identifier = identifier; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java index b0df25663..e023f2e62 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java @@ -97,5 +97,4 @@ public class PMAuthor implements Serializable { this.affiliation = affiliation; } - } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java index 0c8c55e40..6cd17a90c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java @@ -1,53 +1,53 @@ + package eu.dnetlib.dhp.sx.bio.pubmed; public class PMIdentifier { - private String pid; - private String type; + private String pid; + private String type; + public PMIdentifier(String pid, String type) { + this.pid = cleanPid(pid); + this.type = type; + } - public PMIdentifier(String pid, String type) { - this.pid = cleanPid(pid); - this.type = type; - } + public PMIdentifier() { - public PMIdentifier() { + } - } + private String cleanPid(String pid) { - private String cleanPid(String pid) { + if (pid == null) { + return null; + } - if (pid == null) { - return null; - } + // clean ORCID ID in the form 0000000163025705 to 0000-0001-6302-5705 + if (pid.matches("[0-9]{15}[0-9X]")) { + return pid.replaceAll("(.{4})(.{4})(.{4})(.{4})", "$1-$2-$3-$4"); + } - // clean ORCID ID in the form 0000000163025705 to 0000-0001-6302-5705 - if (pid.matches("[0-9]{15}[0-9X]")) { - return pid.replaceAll("(.{4})(.{4})(.{4})(.{4})", "$1-$2-$3-$4"); - } + // clean ORCID in the form http://orcid.org/0000-0001-8567-3543 to 0000-0001-8567-3543 + if (pid.matches("http://orcid.org/[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{4}")) { + return pid.replaceAll("http://orcid.org/", ""); + } + return pid; + } - // clean ORCID in the form http://orcid.org/0000-0001-8567-3543 to 0000-0001-8567-3543 - if (pid.matches("http://orcid.org/[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{4}")) { - return pid.replaceAll("http://orcid.org/", ""); - } - return pid; - } + public String getPid() { + return pid; + } - public String getPid() { - return pid; - } + public PMIdentifier setPid(String pid) { + this.pid = cleanPid(pid); + return this; + } - public PMIdentifier setPid(String pid) { - this.pid = cleanPid(pid); - return this; - } + public String getType() { + return type; + } - public String getType() { - return type; - } - - public PMIdentifier setType(String type) { - this.type = type; - return this; - } + public PMIdentifier setType(String type) { + this.type = type; + return this; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index e4a238c8f..de68ebb58 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -673,7 +673,6 @@ case object Crossref2Oaf { val doi = input.getString(0) val rorId = input.getString(1) - val pubId = IdentifierFactory.idFromPid("50", "doi", DoiCleaningRule.clean(doi), true) val affId = GenerateRorActionSetJob.calculateOpenaireId(rorId) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala index 2eb4bea65..bc9a2cf02 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala @@ -82,21 +82,22 @@ class PMParser2 { a.setLastName((author \ "LastName").text) a.setForeName((author \ "ForeName").text) val id = (author \ "Identifier").text - val idType =(author \ "Identifier" \ "@Source").text + val idType = (author \ "Identifier" \ "@Source").text - if(id != null && id.nonEmpty && idType != null && idType.nonEmpty) { + if (id != null && id.nonEmpty && idType != null && idType.nonEmpty) { a.setIdentifier(new PMIdentifier(id, idType)) } - val affiliation = (author \ "AffiliationInfo" \ "Affiliation").text - val affiliationId = (author \ "AffiliationInfo" \ "Identifier").text + val affiliationId = (author \ "AffiliationInfo" \ "Identifier").text val affiliationIdType = (author \ "AffiliationInfo" \ "Identifier" \ "@Source").text - if(affiliation != null && affiliation.nonEmpty) { + if (affiliation != null && affiliation.nonEmpty) { val aff = new PMAffiliation() aff.setName(affiliation) - if(affiliationId != null && affiliationId.nonEmpty && affiliationIdType != null && affiliationIdType.nonEmpty) { + if ( + affiliationId != null && affiliationId.nonEmpty && affiliationIdType != null && affiliationIdType.nonEmpty + ) { aff.setIdentifier(new PMIdentifier(affiliationId, affiliationIdType)) } a.setAffiliation(aff) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala index 5e14c731a..281ca0e07 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala @@ -294,11 +294,23 @@ object PubMedToOaf { author.setName(a.getForeName) author.setSurname(a.getLastName) author.setFullname(a.getFullName) - if(a.getIdentifier != null) { - author.setPid(List(OafMapperUtils.structuredProperty(a.getIdentifier.getPid, - OafMapperUtils.qualifier(a.getIdentifier.getType,a.getIdentifier.getType,ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES), dataInfo)).asJava) + if (a.getIdentifier != null) { + author.setPid( + List( + OafMapperUtils.structuredProperty( + a.getIdentifier.getPid, + OafMapperUtils.qualifier( + a.getIdentifier.getType, + a.getIdentifier.getType, + ModelConstants.DNET_PID_TYPES, + ModelConstants.DNET_PID_TYPES + ), + dataInfo + ) + ).asJava + ) } - if (a.getAffiliation!= null) + if (a.getAffiliation != null) author.setRawAffiliationString(List(a.getAffiliation.getName).asJava) author.setRank(index + 1) author diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala index 4a926df01..cb7826dbf 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala @@ -63,7 +63,6 @@ class BioScholixTest extends AbstractVocabularyTest { "0000000333457333", "0000000335964515", "0000000302921949", - "http://orcid.org/0000-0001-8567-3543", "http://orcid.org/0000-0001-7868-8528", "0000-0001-9189-1440", From 730a7751b68b0dc2c5a8fa1f97ee4e82bdb2b3f2 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 4 Dec 2024 15:03:59 +0100 Subject: [PATCH 228/239] added zenodoDump to enum of CollectorPlugin --- .../main/java/eu/dnetlib/dhp/collection/CollectorWorker.java | 3 +++ .../java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java index f63bfcb48..4c6d0653e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java @@ -7,6 +7,7 @@ import java.io.IOException; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; +import eu.dnetlib.dhp.collection.plugin.zenodo.CollectZenodoDumpCollectorPlugin; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; @@ -129,6 +130,8 @@ public class CollectorWorker extends ReportingJob { return new Gtr2PublicationsCollectorPlugin(this.clientParams); case osfPreprints: return new OsfPreprintsCollectorPlugin(this.clientParams); + case zenodoDump: + return new CollectZenodoDumpCollectorPlugin(); case other: final CollectorPlugin.NAME.OTHER_NAME plugin = Optional .ofNullable(this.api.getParams().get("other_plugin_type")) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java index 72e691579..93e65b6a7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java @@ -11,7 +11,7 @@ public interface CollectorPlugin { enum NAME { - oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications, osfPreprints; + oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications, osfPreprints, zenodoDump; public enum OTHER_NAME { mdstore_mongodb_dump, mdstore_mongodb From bde59a7c8f49cd964317a9e240dacb3be1eeec01 Mon Sep 17 00:00:00 2001 From: miconis Date: Thu, 5 Dec 2024 11:09:30 +0100 Subject: [PATCH 229/239] implementation of the utilities for the inclusion of raids in the graph --- .../java/eu/dnetlib/dhp/common/Constants.java | 4 + .../dnetlib/dhp/actionmanager/Constants.java | 8 + .../raid/GenerateRAiDActionSetJob.java | 190 ++++++++++++++++++ .../raid/model/GenerateRAiDActionSetJob.java | 2 + .../actionmanager/raid/model/RAiDEntity.java | 102 ++++++++++ .../ror/GenerateRorActionSetJob.java | 8 +- .../raid/action_set_parameters.json | 14 ++ .../raid/oozie_app/action_set_parameters.json | 0 .../raid/oozie_app/config-default.xml | 58 ++++++ .../actionmanager/raid/oozie_app/workflow.xml | 55 +++++ .../raid/GenerateRAiDActionSetJobTest.java | 112 +++++++++++ .../dhp/actionmanager/raid/raid_example.json | 6 + 12 files changed, 552 insertions(+), 7 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/action_set_parameters.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/raid/raid_example.json diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java index 0477d6399..b00199ea5 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java @@ -10,6 +10,10 @@ public class Constants { public static final Map accessRightsCoarMap = Maps.newHashMap(); public static final Map coarCodeLabelMap = Maps.newHashMap(); + public static final String RAID_NS_PREFIX = "raid________"; + public static final String RAID_DATASOURCE_NAME = "Research Activity Identifier Service (RAiD)"; + public static final String RAID_OPENAIRE_ID = ""; + public static final String ROR_NS_PREFIX = "ror_________"; public static final String ROR_OPENAIRE_ID = "10|openaire____::993a7ae7a863813cf95028b50708e222"; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java index 73b4b77cb..722415c2e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -3,6 +3,8 @@ package eu.dnetlib.dhp.actionmanager; import java.util.Optional; +import eu.dnetlib.dhp.schema.oaf.Instance; +import eu.dnetlib.dhp.schema.oaf.Qualifier; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -110,6 +112,12 @@ public class Constants { } + public static Instance getInstance(Qualifier qualifier) { + Instance instance = new Instance(); + instance.setInstancetype(qualifier); + return instance; + } + public 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/actionmanager/raid/GenerateRAiDActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java new file mode 100644 index 000000000..8e5e1bdcb --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java @@ -0,0 +1,190 @@ +package eu.dnetlib.dhp.actionmanager.raid; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.Constants; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.utils.DHPUtils; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_ID; +import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_NAME; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; +import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; + +public class GenerateRAiDActionSetJob { + + private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final List RAID_COLLECTED_FROM = listKeyValues( + OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); + + private static final Qualifier RAID_QUALIFIER = qualifier("raid:openaireinference", "raid:openaireinference", DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS); + + private static final DataInfo RAID_DATA_INFO = dataInfo( + false, OPENAIRE_DATASOURCE_NAME, true, false, RAID_QUALIFIER, "0.92"); + + public static void main(final String[] args) throws Exception { + + final String jsonConfiguration = IOUtils + .toString( + eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class + .getResourceAsStream("/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("inputPath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + + final SparkConf conf = new SparkConf(); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + removeOutputDir(spark, outputPath); + processRAiDEntities(spark, inputPath, outputPath); + }); + } + + private static void removeOutputDir(final SparkSession spark, final String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } + + static void processRAiDEntities(final SparkSession spark, + final String inputPath, + final String outputPath) { + readInputPath(spark, inputPath) + .map(GenerateRAiDActionSetJob::prepareRAiD) + .flatMap(List::iterator) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class); + + } + + protected static List> prepareRAiD(final RAiDEntity r) { + + final Date now = new Date(); + final OtherResearchProduct orp = new OtherResearchProduct(); + final List> res = new ArrayList<>(); + String raidId = calculateOpenaireId(r.getRaid()); + + orp.setId(raidId); + orp.setCollectedfrom(RAID_COLLECTED_FROM); + orp.setDataInfo(RAID_DATA_INFO); + orp.setResourcetype(RAID_QUALIFIER); + orp.setTitle( + Collections.singletonList( + structuredProperty( + r.getTitle(), + qualifier("main title", "main title", DNET_DATACITE_TITLE, DNET_DATACITE_TITLE), + RAID_DATA_INFO)) + ); + orp.setDescription(listFields(RAID_DATA_INFO, r.getSummary())); + orp.setAuthor(createAuthors(r.getAuthors())); + orp.setInstance(Collections.singletonList(eu.dnetlib.dhp.actionmanager.Constants.getInstance(RAID_QUALIFIER))); + orp.setSubject( + r.getSubjects() + .stream() + .map(s -> subject(s, qualifier(DNET_SUBJECT_KEYWORD, DNET_SUBJECT_KEYWORD, DNET_SUBJECT_TYPOLOGIES, DNET_SUBJECT_TYPOLOGIES), RAID_DATA_INFO)) + .collect(Collectors.toList()) + ); + orp.setRelevantdate( + Arrays.asList( + structuredProperty(r.getEndDate(), qualifier("endDate","endDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), RAID_DATA_INFO), + structuredProperty(r.getStartDate(), qualifier("startDate", "startDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), RAID_DATA_INFO) + ) + ); + orp.setLastupdatetimestamp(now.getTime()); + orp.setDateofcollection(r.getStartDate()); + + res.add(new AtomicAction<>(OtherResearchProduct.class, orp)); + + for(String resultId: r.getIds()) { + Relation rel1 = OafMapperUtils.getRelation( + raidId, + resultId, + ModelConstants.RESULT_RESULT, + ModelConstants.OUTCOME, + PART, + RAID_COLLECTED_FROM, + RAID_DATA_INFO, + now.getTime(), + null, + null + ); + Relation rel2 = OafMapperUtils.getRelation( + resultId, + raidId, + ModelConstants.RESULT_RESULT, + ModelConstants.OUTCOME, + IS_PART_OF, + RAID_COLLECTED_FROM, + RAID_DATA_INFO, + now.getTime(), + null, + null + ); + res.add(new AtomicAction<>(Relation.class, rel1)); + res.add(new AtomicAction<>(Relation.class, rel2)); + } + + return res; + } + + public static String calculateOpenaireId(final String raid) { + return String.format("50|%s::%s", Constants.RAID_NS_PREFIX, DHPUtils.md5(raid)); + } + + public static List createAuthors(final List author) { + return author.stream().map(s-> { + Author a = new Author(); + a.setFullname(s); + return a; + }).collect(Collectors.toList()); + } + + private static JavaRDD readInputPath( + final SparkSession spark, + final String path) { + + return spark + .read() + .json(path) + .as(Encoders.bean(RAiDEntity.class)) + .toJavaRDD(); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java new file mode 100644 index 000000000..b0aec71d3 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java @@ -0,0 +1,2 @@ +package eu.dnetlib.dhp.actionmanager.raid.model;public class GenerateRAiDActionSetJob { +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java new file mode 100644 index 000000000..bd7e28926 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java @@ -0,0 +1,102 @@ +package eu.dnetlib.dhp.actionmanager.raid.model; + +import java.io.Serializable; +import java.util.List; + +public class RAiDEntity implements Serializable { + + String raid; + List authors; + String startDate; + String endDate; + List subjects; + List titles; + List ids; + String title; + String summary; + + public RAiDEntity(){} + public RAiDEntity(String raid, List authors, String startDate, String endDate, List subjects, List titles, List ids, String title, String summary) { + this.raid = raid; + this.authors = authors; + this.startDate = startDate; + this.endDate = endDate; + this.subjects = subjects; + this.titles = titles; + this.ids = ids; + this.title = title; + this.summary = summary; + } + + public String getRaid() { + return raid; + } + + public void setRaid(String raid) { + this.raid = raid; + } + + public List getAuthors() { + return authors; + } + + public void setAuthors(List authors) { + this.authors = authors; + } + + public String getStartDate() { + return startDate; + } + + public void setStartDate(String startDate) { + this.startDate = startDate; + } + + public String getEndDate() { + return endDate; + } + + public void setEndDate(String endDate) { + this.endDate = endDate; + } + + public List getSubjects() { + return subjects; + } + + public void setSubjects(List subjects) { + this.subjects = subjects; + } + + public List getTitles() { + return titles; + } + + public void setTitles(List titles) { + this.titles = titles; + } + + public List getIds() { + return ids; + } + + public void setIds(List ids) { + this.ids = ids; + } + + public String getTitle() { + return title; + } + + public void setTitle(String title) { + this.title = title; + } + + public String getSummary() { + return summary; + } + + public void setSummary(String summary) { + this.summary = summary; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java index 5f3493d56..ce1973a7f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java @@ -21,6 +21,7 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -44,13 +45,6 @@ import eu.dnetlib.dhp.common.Constants; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Field; -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.Qualifier; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json new file mode 100644 index 000000000..2049630d2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json @@ -0,0 +1,14 @@ +[ + { + "paramName": "i", + "paramLongName": "inputPath", + "paramDescription": "the path of the input json", + "paramRequired": true + }, + { + "paramName": "o", + "paramLongName": "outputPath", + "paramDescription": "the path of the new ActionSet", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/action_set_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/action_set_parameters.json new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/config-default.xml new file mode 100644 index 000000000..a1755f329 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/config-default.xml @@ -0,0 +1,58 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + oozie.launcher.mapreduce.user.classpath.first + true + + + sparkExecutorNumber + 4 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml new file mode 100644 index 000000000..9b5aa5905 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml @@ -0,0 +1,55 @@ + + + + raidJsonInputPath + the path of the json + + + raidActionSetPath + path where to store the action set + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + yarn + cluster + ProcessRAiDFile + eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob + dhp-aggregation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --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 + + --inputPath${raidJsonInputPath} + --outputPath${raidActionSetPath} + + + + + + + diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java new file mode 100644 index 000000000..1f33f45b2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java @@ -0,0 +1,112 @@ +package eu.dnetlib.dhp.actionmanager.raid; + +import eu.dnetlib.dhp.actionmanager.opencitations.CreateOpenCitationsASTest; +import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import scala.Tuple2; + +import java.io.File; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; + +import static eu.dnetlib.dhp.actionmanager.Constants.OBJECT_MAPPER; +import static java.nio.file.Files.createTempDirectory; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class GenerateRAiDActionSetJobTest { + private static String input_path; + private static String output_path; + static SparkSession spark; + + @BeforeEach + void setUp() throws Exception { + + input_path = Paths + .get(GenerateRAiDActionSetJobTest.class.getResource("/eu/dnetlib/dhp/actionmanager/raid/raid_example.json").toURI()) + .toFile() + .getAbsolutePath(); + + output_path = createTempDirectory(GenerateRAiDActionSetJobTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + + SparkConf conf = new SparkConf(); + conf.setAppName(GenerateRAiDActionSetJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", output_path); + conf.set("hive.metastore.warehouse.dir", output_path); + + spark = SparkSession + .builder() + .appName(GenerateRAiDActionSetJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + static void cleanUp() throws Exception { + FileUtils.deleteDirectory(new File(output_path)); + } + + @Test + @Disabled + void testProcessRAiDEntities() { + GenerateRAiDActionSetJob.processRAiDEntities(spark, input_path, output_path + "/test_raid_action_set"); + + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD result = sc + .sequenceFile(output_path + "/test_raid_action_set", Text.class, Text.class) + .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) + .map(AtomicAction::getPayload); + + assertEquals(80, result.count()); + } + + @Test + void testPrepareRAiD() { + + List> atomicActions = GenerateRAiDActionSetJob.prepareRAiD(new RAiDEntity( + "-92190526", + Arrays.asList("Berli, Justin", "Le Mao, Bérénice", "Guillaume Touya", "Wenclik, Laura", "Courtial, Azelle", "Muehlenhaus, Ian", "Justin Berli", "Touya, Guillaume", "Gruget, Maïeul", "Azelle Courtial", "Ian Muhlenhaus", "Maïeul Gruget", "Marion Dumont", "Maïeul GRUGET", "Cécile Duchêne"), + "2021-09-10", + "2024-02-16", + Arrays.asList("cartography, zoom, pan, desert fog", "Road network", "zooming", "Pan-scalar maps", "pan-scalar map", "Python library", "QGIS", "map design", "landmarks", "Cartes transscalaires", "anchor", "disorientation", "[INFO]Computer Science [cs]", "[SHS.GEO]Humanities and Social Sciences/Geography", "cognitive cartography", "eye-tracking", "Computers in Earth Sciences", "Topographic map", "National Mapping Agency", "General Medicine", "Geography, Planning and Development", "multi-scales", "pan-scalar maps", "Selection", "cartography", "General Earth and Planetary Sciences", "progressiveness", "map generalisation", "Eye-tracker", "zoom", "algorithms", "Map Design", "cartography, map generalisation, zoom, multi-scale map", "Interactive maps", "Map generalisation", "Earth and Planetary Sciences (miscellaneous)", "Cartographic generalization", "rivers", "Benchmark", "General Environmental Science", "open source", "drawing", "Constraint", "Multi-scale maps"), + Arrays.asList("Where do people look at during multi-scale map tasks?", "FogDetector survey raw data", "Collection of cartographic disorientation stories", "Anchorwhat dataset", "BasqueRoads: A Benchmark for Road Network Selection", "Progressive river network selection for pan-scalar maps", "BasqueRoads, a dataset to benchmark road selection algorithms", "Missing the city for buildings? A critical review of pan-scalar map generalization and design in contemporary zoomable maps", "Empirical approach to advance the generalisation of multi-scale maps", "L'Alpe d'Huez: a dataset to benchmark topographic map generalisation", "eye-tracking data from a survey on zooming in a pan-scalar map", "Material of the experiment 'More is Less' from the MapMuxing project", "Cartagen4py, an open source Python library for map generalisation", "L’Alpe d’Huez: A Benchmark for Topographic Map Generalisation"), + Arrays.asList("50|doi_dedup___::6915135e0aa39f913394513f809ae58a", "50|doi_dedup___::754e3c283639bc6e104c925ff3e34007", "50|doi_dedup___::13517477f3c1261d57a3364363ce6ce0", "50|doi_dedup___::675b16c73accc4e7242bbb4ed9b3724a", "50|doi_dedup___::94ce09906b2d7d37eb2206cea8a50153", "50|dedup_wf_002::cc575d5ca5651ff8c3029a3a76e7e70a", "50|doi_dedup___::c5e52baddda17c755d1bae012a97dc13", "50|doi_dedup___::4f5f38c9e08fe995f7278963183f8ad4", "50|doi_dedup___::a9bc4453273b2d02648a5cb453195042", "50|doi_dedup___::5e893dc0cb7624a33f41c9b428bd59f7", "50|doi_dedup___::c1ecdef48fd9be811a291deed950e1c5", "50|doi_dedup___::9e93c8f2d97c35de8a6a57a5b53ef283", "50|dedup_wf_002::d08be0ed27b13d8a880e891e08d093ea", "50|doi_dedup___::f8d8b3b9eddeca2fc0e3bc9e63996555"), + "Exploring Multi-Scale Map Generalization and Design", + "This project aims to advance the generalization of multi-scale maps by investigating the impact of different design elements on user experience. The research involves collecting and analyzing data from various sources, including surveys, eye-tracking studies, and user experiments. The goal is to identify best practices for map generalization and design, with a focus on reducing disorientation and improving information retrieval during exploration. The project has led to the development of several datasets, including BasqueRoads, AnchorWhat, and L'Alpe d'Huez, which can be used to benchmark road selection algorithms and topographic map generalization techniques. The research has also resulted in the creation of a Python library, Cartagen4py, for map generalization. The findings of this project have the potential to improve the design and usability of multi-scale maps, making them more effective tools for navigation and information retrieval." + )); + + OtherResearchProduct orp = (OtherResearchProduct) atomicActions.get(0).getPayload(); + Relation rel = (Relation) atomicActions.get(1).getPayload(); + + assertEquals("Exploring Multi-Scale Map Generalization and Design", orp.getTitle().get(0).getValue()); + assertEquals("50|raid________::759a564ce5cc7360cab030c517c7366b", rel.getSource()); + assertEquals("50|doi_dedup___::6915135e0aa39f913394513f809ae58a", rel.getTarget()); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/raid/raid_example.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/raid/raid_example.json new file mode 100644 index 000000000..7694b605c --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/raid/raid_example.json @@ -0,0 +1,6 @@ +{"raid": "-9222092103004099540", "authors": ["Department of Archaeology & Museums", "Department of Archaeology and Museums", "Department Of Archaeology & Museums"], "subjects": ["Begamganj", "Raisen", "Bhopal", "Budhni", "Malwa site survey", "सीहोर", "Gauharganj", "बुधनी", "Budni", "Berasia"], "titles": ["Malwa site survey : Raisen District, Begamganj Tahsīl, photographic documentation", "Malwa site survey : Bhopal District, photographic documentation (version 1, TIFF files)", "Malwa site survey : Raisen District, Gauharganj Tahsīl, village finds", "Malwa site survey : Sehore सीहोर District, Budni Tahsīl, photographic documentation (part 1)", "Malwa site survey: Bhopal District, Berasia Tahsīl, photographic documentation (with villages named)", "Malwa site survey : Sehore सीहोर District, Budni Tahsīl, photographic documentation (part 2)", "Malwa site survey : Bhopal District, photographic documentation (version 2, JPEG files)"], "ids": ["50|doi_dedup___::7523d165970830dd857e6cbea4302adf", "50|doi_dedup___::02309ae8a9fae291df321e317f5c5330", "50|doi_dedup___::95347ba2c4264414fab39712ee7fe481", "50|doi_dedup___::970aa708fe667596754fd02a708780f5", "50|doi_dedup___::b7cd9128cc53b1257a4f000347f339b0", "50|doi_dedup___::c7d65da0ecedef4d2c702b9db197d90c", "50|doi_dedup___::addbb67cf5046e340f342ba091bcebfa"], "title": "Documentation of Malwa Region", "summary": "This project involves the documentation of the Malwa region through photographic surveys. The surveys were conducted by the Department of Archaeology and Museums, Madhya Pradesh, and cover various districts and tahsils. The documentation includes photographic records of sites, villages, and other relevant features. The project aims to provide a comprehensive understanding of the region's cultural and historical significance.", "startDate": "2019-03-06", "endDate": "2019-03-08"} +{"raid": "-9221424331076109424", "authors": ["Hutchings, Judy", "Ward, Catherine", "Baban, Adriana", "D��nil��, Ingrid", "Frantz, Inga", "Gardner, Frances", "Lachman, Jamie", "Lachman, Jamie M.", "Foran, Heather", "Heinrichs, Nina", "Murphy, Hugh", "B��ban, Adriana", "Raleva, Marija", "Fang, Xiangming", "Jansen, Elena", "Taut, Diana", "Foran, Heather M.", "T��ut, Diana", "Ward, Catherine L.", "Williams, Margiad", "Lesco, Galina", "Brühl, Antonia"], "subjects": ["3. Good health", "5. Gender equality", "Criminology not elsewhere classified", "1. No poverty", "2. Zero hunger"], "titles": ["sj-docx-1-vaw-10.1177_10778012231188090 - Supplemental material for Co-Occurrence of Intimate Partner Violence Against Mothers and Maltreatment of Their Children With Behavioral Problems in Eastern Europe", "Hunger in vulnerable families in Southeastern Europe: Associations with health and violence", "Prevention of child mental health problems through parenting interventions in Southeastern Europe (RISE): study protocol for a multi-site randomised controlled trial"], "ids": ["50|doi_dedup___::a70015063e5400dae2e097ee10b4a589", "50|doi_dedup___::6e1d12026fcde9087724622ccdeed430", "50|doi_dedup___::5b7bd5d46c5d95e2ef5b36663504a67e"], "title": "Exploring the Impact of Hunger and Violence on Child Health in Southeastern Europe", "summary": "This study aims to investigate the relationship between hunger, violence, and child health in vulnerable families in Southeastern Europe. The research will explore the experiences of families in FYR Macedonia, Republic of Moldova, and Romania, and examine the associations between hunger, maltreatment, and other health indicators. The study will also test the efficacy of a parenting intervention targeting child behavioral problems in alleviating these issues. The findings of this research will contribute to the development of effective interventions to address the complex needs of vulnerable families in the region.", "startDate": "2019-06-04", "endDate": "2023-01-01"} +{"raid": "-9219052635741785098", "authors": ["Berli, Justin", "Le Mao, Bérénice", "Guillaume Touya", "Wenclik, Laura", "Courtial, Azelle", "Muehlenhaus, Ian", "Justin Berli", "Touya, Guillaume", "Gruget, Maïeul", "Azelle Courtial", "Ian Muhlenhaus", "Maïeul Gruget", "Marion Dumont", "Maïeul GRUGET", "Cécile Duchêne"], "subjects": ["cartography, zoom, pan, desert fog", "Road network", "zooming", "Pan-scalar maps", "pan-scalar map", "Python library", "QGIS", "map design", "landmarks", "Cartes transscalaires", "anchor", "disorientation", "[INFO]Computer Science [cs]", "[SHS.GEO]Humanities and Social Sciences/Geography", "cognitive cartography", "eye-tracking", "Computers in Earth Sciences", "Topographic map", "National Mapping Agency", "General Medicine", "Geography, Planning and Development", "multi-scales", "pan-scalar maps", "Selection", "cartography", "General Earth and Planetary Sciences", "progressiveness", "map generalisation", "Eye-tracker", "zoom", "algorithms", "Map Design", "cartography, map generalisation, zoom, multi-scale map", "Interactive maps", "Map generalisation", "Earth and Planetary Sciences (miscellaneous)", "Cartographic generalization", "rivers", "Benchmark", "General Environmental Science", "open source", "drawing", "Constraint", "Multi-scale maps"], "titles": ["Where do people look at during multi-scale map tasks?", "FogDetector survey raw data", "Collection of cartographic disorientation stories", "Anchorwhat dataset", "BasqueRoads: A Benchmark for Road Network Selection", "Progressive river network selection for pan-scalar maps", "BasqueRoads, a dataset to benchmark road selection algorithms", "Missing the city for buildings? A critical review of pan-scalar map generalization and design in contemporary zoomable maps", "Empirical approach to advance the generalisation of multi-scale maps", "L'Alpe d'Huez: a dataset to benchmark topographic map generalisation", "eye-tracking data from a survey on zooming in a pan-scalar map", "Material of the experiment \"More is Less\" from the MapMuxing project", "Cartagen4py, an open source Python library for map generalisation", "L’Alpe d’Huez: A Benchmark for Topographic Map Generalisation"], "ids": ["50|doi_dedup___::6915135e0aa39f913394513f809ae58a", "50|doi_dedup___::754e3c283639bc6e104c925ff3e34007", "50|doi_dedup___::13517477f3c1261d57a3364363ce6ce0", "50|doi_dedup___::675b16c73accc4e7242bbb4ed9b3724a", "50|doi_dedup___::94ce09906b2d7d37eb2206cea8a50153", "50|dedup_wf_002::cc575d5ca5651ff8c3029a3a76e7e70a", "50|doi_dedup___::c5e52baddda17c755d1bae012a97dc13", "50|doi_dedup___::4f5f38c9e08fe995f7278963183f8ad4", "50|doi_dedup___::a9bc4453273b2d02648a5cb453195042", "50|doi_dedup___::5e893dc0cb7624a33f41c9b428bd59f7", "50|doi_dedup___::c1ecdef48fd9be811a291deed950e1c5", "50|doi_dedup___::9e93c8f2d97c35de8a6a57a5b53ef283", "50|dedup_wf_002::d08be0ed27b13d8a880e891e08d093ea", "50|doi_dedup___::f8d8b3b9eddeca2fc0e3bc9e63996555"], "title": "Exploring Multi-Scale Map Generalization and Design", "summary": "This project aims to advance the generalization of multi-scale maps by investigating the impact of different design elements on user experience. The research involves collecting and analyzing data from various sources, including surveys, eye-tracking studies, and user experiments. The goal is to identify best practices for map generalization and design, with a focus on reducing disorientation and improving information retrieval during exploration. The project has led to the development of several datasets, including BasqueRoads, AnchorWhat, and L'Alpe d'Huez, which can be used to benchmark road selection algorithms and topographic map generalization techniques. The research has also resulted in the creation of a Python library, Cartagen4py, for map generalization. The findings of this project have the potential to improve the design and usability of multi-scale maps, making them more effective tools for navigation and information retrieval.", "startDate": "2021-09-10", "endDate": "2024-02-16"} +{"raid": "-9216828847055450272", "authors": ["Grey, Alan", "Gorelov, Sergey", "Pall, Szilard", "Merz, Pascal", "Justin A., Lemkul", "Szilárd Páll", "Pasquadibisceglie, Andrea", "Kutzner, Carsten", "Schulz, Roland", "Nabet, Julien", "Abraham, Mark", "Jalalypour, Farzaneh", "Lundborg, Magnus", "Gray, Alan", "Villa, Alessandra", "Berk Hess", "Santuz, Hubert", "Irrgang, M. Eric", "Wingbermuehle, Sebastian", "Lemkul, Justin A.", "Jordan, Joe", "Pellegrino, Michele", "Doijade, Mahesh", "Shvetsov, Alexey", "Hess, Berk", "Behera, Sudarshan", "Andrey Alekseenko", "Shugaeva, Tatiana", "Fleischmann, Stefan", "Bergh, Cathrine", "Morozov, Dmitry", "Adam Hospital", "Briand, Eliane", "Lindahl, Erik", "Brown, Ania", "Marta Lloret Llinares", "Miletic, Vedran", "Alekseenko, Andrey", "Gouaillardet, Gilles", "Fiorin, Giacomo", "Basov, Vladimir"], "subjects": ["webinar"], "titles": ["Introduction to HPC: molecular dynamics simulations with GROMACS: log files", "BioExcel webinar #73: Competency frameworks to support training design and professional development", "Introduction to HPC: molecular dynamics simulations with GROMACS: output files - Devana", "GROMACS 2024.0 Manual", "BioExcel Webinar #71: GROMACS-PMX for accurate estimation of free energy differences", "Introduction to HPC: molecular dynamics simulations with GROMACS: input files", "BioExcel Webinar #68: What's new in GROMACS 2023", "BioExcel Webinar #69: BioBB-Wfs and BioBB-API, integrated web-based platform and programmatic interface for biomolecular simulations workflows using the BioExcel Building Blocks library", "GROMACS 2024-beta Source code"], "ids": ["50|doi_dedup___::8318fbc815ee1943c3269be7567f220b", "50|doi_dedup___::9530e03fb2aac63e82b18a40dc09e32c", "50|doi_dedup___::30174ab31075e76a428ca5b4f4d236b8", "50|doi_________::70b7c6dce09ae6f1361d22913fdf95eb", "50|doi_dedup___::337dd48600618f3c06257edd750d6201", "50|doi_dedup___::d622992ba9077617f37ebd268b3e806d", "50|doi_dedup___::0b0bcc6825d6c052c37882fd5cfc1e8c", "50|doi_dedup___::4b1541a7cee32527c65ace5d1ed57335", "50|doi_dedup___::1379861df59bd755e4fb39b9f95ffbd3"], "title": "Exploring High-Performance Computing and Biomolecular Simulations", "summary": "This project involves exploring high-performance computing (HPC) and biomolecular simulations using GROMACS. The objectives include understanding molecular dynamics simulations, log files, input files, and output files. Additionally, the project aims to explore competency frameworks for professional development, specifically in the field of computational biomolecular research. The tools and techniques used will include GROMACS, BioExcel Building Blocks, and competency frameworks. The expected outcomes include a deeper understanding of HPC and biomolecular simulations, as well as the development of skills in using GROMACS and BioExcel Building Blocks. The project will also contribute to the development of competency frameworks for professional development in the field of computational biomolecular research.", "startDate": "2023-04-25", "endDate": "2024-01-30"} +{"raid": "-9210544816395499758", "authors": ["Bateson, Melissa", "Andrews, Clare", "Verhulst, Simon", "Nettle, Daniel", "Zuidersma, Erica"], "subjects": ["2. Zero hunger"], "titles": ["Exposure to food insecurity increases energy storage and reduces somatic maintenance in European starlings", "Data and code archive for Andrews et al. 'Exposure to food insecurity increases energy storage and reduces somatic maintenance in European starlings'"], "ids": ["50|doi_dedup___::176117239be06189523c253e0ca9c5ec", "50|doi_dedup___::343e0b0ddf0d54763a89a62af1f7a379"], "title": "Investigating the Effects of Food Insecurity on Energy Storage and Somatic Maintenance in European Starlings", "summary": "This study examines the impact of food insecurity on energy storage and somatic maintenance in European starlings. The research involved exposing juvenile starlings to either uninterrupted food availability or a regime of unpredictable food unavailability. The results show that birds exposed to food insecurity stored more energy, but at the expense of somatic maintenance and repair. The study provides insights into the adaptive responses of birds to food scarcity and the trade-offs involved in energy storage and maintenance.", "startDate": "2021-06-28", "endDate": "2021-06-28"} +{"raid": "-9208499171224730388", "authors": ["Maniati, Eleni", "Bakker, Bjorn", "McClelland, Sarah E.", "Shaikh, Nadeem", "De Angelis, Simone", "Johnson, Sarah C.", "Wang, Jun", "Foijer, Floris", "Spierings, Diana C. J.", "Boemo, Michael A.", "Wardenaar, René", "Mazzagatti, Alice"], "subjects": [], "titles": ["Additional file 2 of Replication stress generates distinctive landscapes of DNA copy number alterations and chromosome scale losses", "Additional file 5 of Replication stress generates distinctive landscapes of DNA copy number alterations and chromosome scale losses"], "ids": ["50|doi_dedup___::a1bfeb173971f74a274fab8bdd78a4bc", "50|doi_dedup___::3d6e151aaeb2f7c40a320207fdd80ade"], "title": "Analysis of DNA Copy Number Alterations and Chromosome Scale Losses", "summary": "This study analyzed the effects of replication stress on DNA copy number alterations and chromosome scale losses. The results show distinctive landscapes of these alterations and losses, which were further investigated in additional files. The study provides valuable insights into the mechanisms of replication stress and its impact on genomic stability.", "startDate": "2022-01-01", "endDate": "2022-01-01"} \ No newline at end of file From 6af3fd16b6362c4a9357f44ba779b7822f5b7033 Mon Sep 17 00:00:00 2001 From: miconis Date: Thu, 5 Dec 2024 14:39:42 +0100 Subject: [PATCH 230/239] attributes fixes --- .../java/eu/dnetlib/pace/tree/DateRange.java | 90 ++--- .../eu/dnetlib/pace/tree/JsonListMatch.java | 2 +- .../pace/comparators/ComparatorTest.java | 30 +- .../dnetlib/dhp/actionmanager/Constants.java | 4 +- .../raid/GenerateRAiDActionSetJob.java | 309 ++++++++++-------- .../raid/model/GenerateRAiDActionSetJob.java | 5 +- .../actionmanager/raid/model/RAiDEntity.java | 154 ++++----- .../ror/GenerateRorActionSetJob.java | 2 +- .../dhp/sx/bio/pubmed/PMAffiliation.java | 44 +-- .../dnetlib/dhp/sx/bio/pubmed/PMAuthor.java | 1 - .../dhp/sx/bio/pubmed/PMIdentifier.java | 74 ++--- .../raid/oozie_app/action_set_parameters.json | 0 .../actionmanager/raid/oozie_app/workflow.xml | 2 - .../collection/crossref/Crossref2Oaf.scala | 1 - .../dnetlib/dhp/sx/bio/pubmed/PMParser2.scala | 13 +- .../dhp/sx/bio/pubmed/PubMedToOaf.scala | 20 +- .../raid/GenerateRAiDActionSetJobTest.java | 203 +++++++----- .../dnetlib/dhp/sx/bio/BioScholixTest.scala | 1 - .../dhp/bulktag/community/ResultTagger.java | 4 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 3 +- .../model/ProvisionModelSupport.java | 15 +- 21 files changed, 546 insertions(+), 431 deletions(-) delete mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/action_set_parameters.json diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java index c913109a4..194677e6e 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java @@ -1,10 +1,5 @@ -package eu.dnetlib.pace.tree; -import com.wcohen.ss.AbstractStringDistance; -import eu.dnetlib.pace.config.Config; -import eu.dnetlib.pace.tree.support.AbstractStringComparator; -import eu.dnetlib.pace.tree.support.ComparatorClass; -import org.joda.time.DateTime; +package eu.dnetlib.pace.tree; import java.time.DateTimeException; import java.time.LocalDate; @@ -13,55 +8,62 @@ import java.time.format.DateTimeFormatter; import java.util.Locale; import java.util.Map; +import org.joda.time.DateTime; + +import com.wcohen.ss.AbstractStringDistance; + +import eu.dnetlib.pace.config.Config; +import eu.dnetlib.pace.tree.support.AbstractStringComparator; +import eu.dnetlib.pace.tree.support.ComparatorClass; + @ComparatorClass("dateRange") public class DateRange extends AbstractStringComparator { - int YEAR_RANGE; + int YEAR_RANGE; - public DateRange(Map params) { - super(params, new com.wcohen.ss.JaroWinkler()); - YEAR_RANGE = Integer.parseInt(params.getOrDefault("year_range", "3")); - } + public DateRange(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + YEAR_RANGE = Integer.parseInt(params.getOrDefault("year_range", "3")); + } - public DateRange(final double weight) { - super(weight, new com.wcohen.ss.JaroWinkler()); - } + public DateRange(final double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } - protected DateRange(final double weight, final AbstractStringDistance ssalgo) { - super(weight, ssalgo); - } + protected DateRange(final double weight, final AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } - public static boolean isNumeric(String str) { - return str.matches("\\d+"); //match a number with optional '-' and decimal. - } + public static boolean isNumeric(String str) { + return str.matches("\\d+"); // match a number with optional '-' and decimal. + } - @Override - public double distance(final String a, final String b, final Config conf) { - if (a.isEmpty() || b.isEmpty()) { - return -1.0; // return -1 if a field is missing - } + @Override + public double distance(final String a, final String b, final Config conf) { + if (a.isEmpty() || b.isEmpty()) { + return -1.0; // return -1 if a field is missing + } - try { - DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ENGLISH); - LocalDate d1 = LocalDate.parse(a, formatter); - LocalDate d2 = LocalDate.parse(b, formatter); - Period period = Period.between(d1, d2); + try { + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ENGLISH); + LocalDate d1 = LocalDate.parse(a, formatter); + LocalDate d2 = LocalDate.parse(b, formatter); + Period period = Period.between(d1, d2); - return period.getYears() <= YEAR_RANGE? 1.0 : 0.0; - } - catch (DateTimeException e) { - return -1.0; - } + return period.getYears() <= YEAR_RANGE ? 1.0 : 0.0; + } catch (DateTimeException e) { + return -1.0; + } - } + } - @Override - public double getWeight() { - return super.weight; - } + @Override + public double getWeight() { + return super.weight; + } - @Override - protected double normalize(final double d) { - return d; - } + @Override + protected double normalize(final double d) { + return d; + } } diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java index e95d9206e..d9558df90 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java @@ -62,7 +62,7 @@ public class JsonListMatch extends AbstractListComparator { Set types = Sets.intersection(typesA, typesB); - if (types.isEmpty()) // if no common type, it is impossible to compare + if (types.isEmpty()) // if no common type, it is impossible to compare return -1; ca = ca.stream().filter(s -> types.contains(s.split("::")[0])).collect(Collectors.toSet()); diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index 83539de4a..0abde84bc 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -72,14 +72,34 @@ public class ComparatorTest extends AbstractPaceTest { CodeMatch codeMatch = new CodeMatch(params); // names have different codes - assertEquals(0.0, codeMatch.distance("physical oceanography at ctd station june 1998 ev02a", "physical oceanography at ctd station june 1998 ir02", conf)); + assertEquals( + 0.0, + codeMatch + .distance( + "physical oceanography at ctd station june 1998 ev02a", + "physical oceanography at ctd station june 1998 ir02", conf)); // names have same code - assertEquals(1.0, codeMatch.distance("physical oceanography at ctd station june 1998 ev02a", "physical oceanography at ctd station june 1998 ev02a", conf)); + assertEquals( + 1.0, + codeMatch + .distance( + "physical oceanography at ctd station june 1998 ev02a", + "physical oceanography at ctd station june 1998 ev02a", conf)); // code is not in both names - assertEquals(-1, codeMatch.distance("physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998 ev02a", conf)); - assertEquals(1.0, codeMatch.distance("physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998", conf)); + assertEquals( + -1, + codeMatch + .distance( + "physical oceanography at ctd station june 1998", + "physical oceanography at ctd station june 1998 ev02a", conf)); + assertEquals( + 1.0, + codeMatch + .distance( + "physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998", + conf)); } @Test @@ -275,7 +295,7 @@ public class ComparatorTest extends AbstractPaceTest { Arrays .asList( "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"grid\",\"classname\":\"GRID Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"grid_1\"}", - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_1\"}"), + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_1\"}"), "authors"); List b = createFieldList( Arrays diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java index 722415c2e..394cc22a3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -3,8 +3,6 @@ package eu.dnetlib.dhp.actionmanager; import java.util.Optional; -import eu.dnetlib.dhp.schema.oaf.Instance; -import eu.dnetlib.dhp.schema.oaf.Qualifier; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -15,6 +13,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Instance; +import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.dhp.schema.oaf.Subject; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java index 8e5e1bdcb..3b2405956 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java @@ -1,15 +1,15 @@ + package eu.dnetlib.dhp.actionmanager.raid; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.Constants; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import eu.dnetlib.dhp.utils.DHPUtils; +import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_ID; +import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_NAME; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; +import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; + +import java.util.*; +import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.SequenceFileOutputFormat; @@ -19,172 +19,191 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.Constants; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; -import java.util.*; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_ID; -import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_NAME; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import static eu.dnetlib.dhp.schema.common.ModelConstants.*; -import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; - public class GenerateRAiDActionSetJob { - private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class); + private static final Logger log = LoggerFactory + .getLogger(eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final List RAID_COLLECTED_FROM = listKeyValues( - OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); + private static final List RAID_COLLECTED_FROM = listKeyValues( + OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); - private static final Qualifier RAID_QUALIFIER = qualifier("raid:openaireinference", "raid:openaireinference", DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS); + private static final Qualifier RAID_QUALIFIER = qualifier("0049", "Research Activity Identifier", DNET_PUBLICATION_RESOURCE, DNET_PUBLICATION_RESOURCE); - private static final DataInfo RAID_DATA_INFO = dataInfo( - false, OPENAIRE_DATASOURCE_NAME, true, false, RAID_QUALIFIER, "0.92"); + private static final Qualifier RAID_INFERENCE_QUALIFIER = qualifier( + "raid:openaireinference", "Inferred by OpenAIRE", DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS); - public static void main(final String[] args) throws Exception { + private static final DataInfo RAID_DATA_INFO = dataInfo( + false, OPENAIRE_DATASOURCE_NAME, true, false, RAID_INFERENCE_QUALIFIER, "0.92"); - final String jsonConfiguration = IOUtils - .toString( - eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class - .getResourceAsStream("/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json")); + public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final String jsonConfiguration = IOUtils + .toString( + eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class + .getResourceAsStream("/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json")); - parser.parseArgument(args); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - final Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + parser.parseArgument(args); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); - final String inputPath = parser.get("inputPath"); - log.info("inputPath: {}", inputPath); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}: ", outputPath); + final String inputPath = parser.get("inputPath"); + log.info("inputPath: {}", inputPath); - final SparkConf conf = new SparkConf(); + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); - runWithSparkSession(conf, isSparkSessionManaged, spark -> { - removeOutputDir(spark, outputPath); - processRAiDEntities(spark, inputPath, outputPath); - }); - } + final SparkConf conf = new SparkConf(); - private static void removeOutputDir(final SparkSession spark, final String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + removeOutputDir(spark, outputPath); + processRAiDEntities(spark, inputPath, outputPath); + }); + } - static void processRAiDEntities(final SparkSession spark, - final String inputPath, - final String outputPath) { - readInputPath(spark, inputPath) - .map(GenerateRAiDActionSetJob::prepareRAiD) - .flatMap(List::iterator) - .mapToPair( - aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), - new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class); + private static void removeOutputDir(final SparkSession spark, final String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } - } + static void processRAiDEntities(final SparkSession spark, + final String inputPath, + final String outputPath) { + readInputPath(spark, inputPath) + .map(GenerateRAiDActionSetJob::prepareRAiD) + .flatMap(List::iterator) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class); - protected static List> prepareRAiD(final RAiDEntity r) { + } - final Date now = new Date(); - final OtherResearchProduct orp = new OtherResearchProduct(); - final List> res = new ArrayList<>(); - String raidId = calculateOpenaireId(r.getRaid()); + protected static List> prepareRAiD(final RAiDEntity r) { - orp.setId(raidId); - orp.setCollectedfrom(RAID_COLLECTED_FROM); - orp.setDataInfo(RAID_DATA_INFO); - orp.setResourcetype(RAID_QUALIFIER); - orp.setTitle( - Collections.singletonList( - structuredProperty( - r.getTitle(), - qualifier("main title", "main title", DNET_DATACITE_TITLE, DNET_DATACITE_TITLE), - RAID_DATA_INFO)) - ); - orp.setDescription(listFields(RAID_DATA_INFO, r.getSummary())); - orp.setAuthor(createAuthors(r.getAuthors())); - orp.setInstance(Collections.singletonList(eu.dnetlib.dhp.actionmanager.Constants.getInstance(RAID_QUALIFIER))); - orp.setSubject( - r.getSubjects() - .stream() - .map(s -> subject(s, qualifier(DNET_SUBJECT_KEYWORD, DNET_SUBJECT_KEYWORD, DNET_SUBJECT_TYPOLOGIES, DNET_SUBJECT_TYPOLOGIES), RAID_DATA_INFO)) - .collect(Collectors.toList()) - ); - orp.setRelevantdate( - Arrays.asList( - structuredProperty(r.getEndDate(), qualifier("endDate","endDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), RAID_DATA_INFO), - structuredProperty(r.getStartDate(), qualifier("startDate", "startDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), RAID_DATA_INFO) - ) - ); - orp.setLastupdatetimestamp(now.getTime()); - orp.setDateofcollection(r.getStartDate()); + final Date now = new Date(); + final OtherResearchProduct orp = new OtherResearchProduct(); + final List> res = new ArrayList<>(); + String raidId = calculateOpenaireId(r.getRaid()); - res.add(new AtomicAction<>(OtherResearchProduct.class, orp)); + orp.setId(raidId); + orp.setCollectedfrom(RAID_COLLECTED_FROM); + orp.setDataInfo(RAID_DATA_INFO); + orp + .setTitle( + Collections + .singletonList( + structuredProperty( + r.getTitle(), + qualifier("main title", "main title", DNET_DATACITE_TITLE, DNET_DATACITE_TITLE), + RAID_DATA_INFO))); + orp.setDescription(listFields(RAID_DATA_INFO, r.getSummary())); +// orp.setAuthor(createAuthors(r.getAuthors())); + orp.setInstance(Collections.singletonList(eu.dnetlib.dhp.actionmanager.Constants.getInstance(RAID_QUALIFIER))); + orp + .setSubject( + r + .getSubjects() + .stream() + .map( + s -> subject( + s, + qualifier( + DNET_SUBJECT_KEYWORD, DNET_SUBJECT_KEYWORD, DNET_SUBJECT_TYPOLOGIES, + DNET_SUBJECT_TYPOLOGIES), + RAID_DATA_INFO)) + .collect(Collectors.toList())); + orp + .setRelevantdate( + Arrays + .asList( + structuredProperty( + r.getEndDate(), qualifier("endDate", "endDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), + RAID_DATA_INFO), + structuredProperty( + r.getStartDate(), + qualifier("startDate", "startDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), + RAID_DATA_INFO))); + orp.setLastupdatetimestamp(now.getTime()); + orp.setDateofacceptance(field(r.getStartDate(), RAID_DATA_INFO)); - for(String resultId: r.getIds()) { - Relation rel1 = OafMapperUtils.getRelation( - raidId, - resultId, - ModelConstants.RESULT_RESULT, - ModelConstants.OUTCOME, - PART, - RAID_COLLECTED_FROM, - RAID_DATA_INFO, - now.getTime(), - null, - null - ); - Relation rel2 = OafMapperUtils.getRelation( - resultId, - raidId, - ModelConstants.RESULT_RESULT, - ModelConstants.OUTCOME, - IS_PART_OF, - RAID_COLLECTED_FROM, - RAID_DATA_INFO, - now.getTime(), - null, - null - ); - res.add(new AtomicAction<>(Relation.class, rel1)); - res.add(new AtomicAction<>(Relation.class, rel2)); - } + res.add(new AtomicAction<>(OtherResearchProduct.class, orp)); - return res; - } + for (String resultId : r.getIds()) { + Relation rel1 = OafMapperUtils + .getRelation( + raidId, + resultId, + ModelConstants.RESULT_RESULT, + PART, + HAS_PART, + RAID_COLLECTED_FROM, + RAID_DATA_INFO, + now.getTime(), + null, + null); + Relation rel2 = OafMapperUtils + .getRelation( + resultId, + raidId, + ModelConstants.RESULT_RESULT, + PART, + IS_PART_OF, + RAID_COLLECTED_FROM, + RAID_DATA_INFO, + now.getTime(), + null, + null); + res.add(new AtomicAction<>(Relation.class, rel1)); + res.add(new AtomicAction<>(Relation.class, rel2)); + } - public static String calculateOpenaireId(final String raid) { - return String.format("50|%s::%s", Constants.RAID_NS_PREFIX, DHPUtils.md5(raid)); - } + return res; + } - public static List createAuthors(final List author) { - return author.stream().map(s-> { - Author a = new Author(); - a.setFullname(s); - return a; - }).collect(Collectors.toList()); - } + public static String calculateOpenaireId(final String raid) { + return String.format("50|%s::%s", Constants.RAID_NS_PREFIX, DHPUtils.md5(raid)); + } - private static JavaRDD readInputPath( - final SparkSession spark, - final String path) { + public static List createAuthors(final List author) { + return author.stream().map(s -> { + Author a = new Author(); + a.setFullname(s); + return a; + }).collect(Collectors.toList()); + } - return spark - .read() - .json(path) - .as(Encoders.bean(RAiDEntity.class)) - .toJavaRDD(); + private static JavaRDD readInputPath( + final SparkSession spark, + final String path) { - } + return spark + .read() + .json(path) + .as(Encoders.bean(RAiDEntity.class)) + .toJavaRDD(); + + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java index b0aec71d3..856b52e18 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/GenerateRAiDActionSetJob.java @@ -1,2 +1,5 @@ -package eu.dnetlib.dhp.actionmanager.raid.model;public class GenerateRAiDActionSetJob { + +package eu.dnetlib.dhp.actionmanager.raid.model; + +public class GenerateRAiDActionSetJob { } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java index bd7e28926..1203b28a7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/model/RAiDEntity.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.actionmanager.raid.model; import java.io.Serializable; @@ -5,98 +6,101 @@ import java.util.List; public class RAiDEntity implements Serializable { - String raid; - List authors; - String startDate; - String endDate; - List subjects; - List titles; - List ids; - String title; - String summary; + String raid; + List authors; + String startDate; + String endDate; + List subjects; + List titles; + List ids; + String title; + String summary; - public RAiDEntity(){} - public RAiDEntity(String raid, List authors, String startDate, String endDate, List subjects, List titles, List ids, String title, String summary) { - this.raid = raid; - this.authors = authors; - this.startDate = startDate; - this.endDate = endDate; - this.subjects = subjects; - this.titles = titles; - this.ids = ids; - this.title = title; - this.summary = summary; - } + public RAiDEntity() { + } - public String getRaid() { - return raid; - } + public RAiDEntity(String raid, List authors, String startDate, String endDate, List subjects, + List titles, List ids, String title, String summary) { + this.raid = raid; + this.authors = authors; + this.startDate = startDate; + this.endDate = endDate; + this.subjects = subjects; + this.titles = titles; + this.ids = ids; + this.title = title; + this.summary = summary; + } - public void setRaid(String raid) { - this.raid = raid; - } + public String getRaid() { + return raid; + } - public List getAuthors() { - return authors; - } + public void setRaid(String raid) { + this.raid = raid; + } - public void setAuthors(List authors) { - this.authors = authors; - } + public List getAuthors() { + return authors; + } - public String getStartDate() { - return startDate; - } + public void setAuthors(List authors) { + this.authors = authors; + } - public void setStartDate(String startDate) { - this.startDate = startDate; - } + public String getStartDate() { + return startDate; + } - public String getEndDate() { - return endDate; - } + public void setStartDate(String startDate) { + this.startDate = startDate; + } - public void setEndDate(String endDate) { - this.endDate = endDate; - } + public String getEndDate() { + return endDate; + } - public List getSubjects() { - return subjects; - } + public void setEndDate(String endDate) { + this.endDate = endDate; + } - public void setSubjects(List subjects) { - this.subjects = subjects; - } + public List getSubjects() { + return subjects; + } - public List getTitles() { - return titles; - } + public void setSubjects(List subjects) { + this.subjects = subjects; + } - public void setTitles(List titles) { - this.titles = titles; - } + public List getTitles() { + return titles; + } - public List getIds() { - return ids; - } + public void setTitles(List titles) { + this.titles = titles; + } - public void setIds(List ids) { - this.ids = ids; - } + public List getIds() { + return ids; + } - public String getTitle() { - return title; - } + public void setIds(List ids) { + this.ids = ids; + } - public void setTitle(String title) { - this.title = title; - } + public String getTitle() { + return title; + } - public String getSummary() { - return summary; - } + public void setTitle(String title) { + this.title = title; + } - public void setSummary(String summary) { - this.summary = summary; - } + public String getSummary() { + return summary; + } + + public void setSummary(String summary) { + this.summary = summary; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java index ce1973a7f..6e8f48bda 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/ror/GenerateRorActionSetJob.java @@ -21,7 +21,6 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -45,6 +44,7 @@ import eu.dnetlib.dhp.common.Constants; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java index a8dacd132..5ac1920ea 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAffiliation.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.sx.bio.pubmed; /** @@ -7,32 +8,33 @@ package eu.dnetlib.dhp.sx.bio.pubmed; */ public class PMAffiliation { - private String name; + private String name; - private PMIdentifier identifier; + private PMIdentifier identifier; - public PMAffiliation() { + public PMAffiliation() { - } - public PMAffiliation(String name, PMIdentifier identifier) { - this.name = name; - this.identifier = identifier; - } + } - public String getName() { - return name; - } + public PMAffiliation(String name, PMIdentifier identifier) { + this.name = name; + this.identifier = identifier; + } - public void setName(String name) { - this.name = name; - } + public String getName() { + return name; + } - public PMIdentifier getIdentifier() { - return identifier; - } + public void setName(String name) { + this.name = name; + } - public PMAffiliation setIdentifier(PMIdentifier identifier) { - this.identifier = identifier; - return this; - } + public PMIdentifier getIdentifier() { + return identifier; + } + + public PMAffiliation setIdentifier(PMIdentifier identifier) { + this.identifier = identifier; + return this; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java index b0df25663..e023f2e62 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMAuthor.java @@ -97,5 +97,4 @@ public class PMAuthor implements Serializable { this.affiliation = affiliation; } - } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java index 0c8c55e40..6cd17a90c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/sx/bio/pubmed/PMIdentifier.java @@ -1,53 +1,53 @@ + package eu.dnetlib.dhp.sx.bio.pubmed; public class PMIdentifier { - private String pid; - private String type; + private String pid; + private String type; + public PMIdentifier(String pid, String type) { + this.pid = cleanPid(pid); + this.type = type; + } - public PMIdentifier(String pid, String type) { - this.pid = cleanPid(pid); - this.type = type; - } + public PMIdentifier() { - public PMIdentifier() { + } - } + private String cleanPid(String pid) { - private String cleanPid(String pid) { + if (pid == null) { + return null; + } - if (pid == null) { - return null; - } + // clean ORCID ID in the form 0000000163025705 to 0000-0001-6302-5705 + if (pid.matches("[0-9]{15}[0-9X]")) { + return pid.replaceAll("(.{4})(.{4})(.{4})(.{4})", "$1-$2-$3-$4"); + } - // clean ORCID ID in the form 0000000163025705 to 0000-0001-6302-5705 - if (pid.matches("[0-9]{15}[0-9X]")) { - return pid.replaceAll("(.{4})(.{4})(.{4})(.{4})", "$1-$2-$3-$4"); - } + // clean ORCID in the form http://orcid.org/0000-0001-8567-3543 to 0000-0001-8567-3543 + if (pid.matches("http://orcid.org/[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{4}")) { + return pid.replaceAll("http://orcid.org/", ""); + } + return pid; + } - // clean ORCID in the form http://orcid.org/0000-0001-8567-3543 to 0000-0001-8567-3543 - if (pid.matches("http://orcid.org/[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{4}")) { - return pid.replaceAll("http://orcid.org/", ""); - } - return pid; - } + public String getPid() { + return pid; + } - public String getPid() { - return pid; - } + public PMIdentifier setPid(String pid) { + this.pid = cleanPid(pid); + return this; + } - public PMIdentifier setPid(String pid) { - this.pid = cleanPid(pid); - return this; - } + public String getType() { + return type; + } - public String getType() { - return type; - } - - public PMIdentifier setType(String type) { - this.type = type; - return this; - } + public PMIdentifier setType(String type) { + this.type = type; + return this; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/action_set_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/action_set_parameters.json deleted file mode 100644 index e69de29bb..000000000 diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml index 9b5aa5905..d3392596f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/raid/oozie_app/workflow.xml @@ -20,8 +20,6 @@ - - diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index e4a238c8f..de68ebb58 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -673,7 +673,6 @@ case object Crossref2Oaf { val doi = input.getString(0) val rorId = input.getString(1) - val pubId = IdentifierFactory.idFromPid("50", "doi", DoiCleaningRule.clean(doi), true) val affId = GenerateRorActionSetJob.calculateOpenaireId(rorId) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala index 2eb4bea65..bc9a2cf02 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PMParser2.scala @@ -82,21 +82,22 @@ class PMParser2 { a.setLastName((author \ "LastName").text) a.setForeName((author \ "ForeName").text) val id = (author \ "Identifier").text - val idType =(author \ "Identifier" \ "@Source").text + val idType = (author \ "Identifier" \ "@Source").text - if(id != null && id.nonEmpty && idType != null && idType.nonEmpty) { + if (id != null && id.nonEmpty && idType != null && idType.nonEmpty) { a.setIdentifier(new PMIdentifier(id, idType)) } - val affiliation = (author \ "AffiliationInfo" \ "Affiliation").text - val affiliationId = (author \ "AffiliationInfo" \ "Identifier").text + val affiliationId = (author \ "AffiliationInfo" \ "Identifier").text val affiliationIdType = (author \ "AffiliationInfo" \ "Identifier" \ "@Source").text - if(affiliation != null && affiliation.nonEmpty) { + if (affiliation != null && affiliation.nonEmpty) { val aff = new PMAffiliation() aff.setName(affiliation) - if(affiliationId != null && affiliationId.nonEmpty && affiliationIdType != null && affiliationIdType.nonEmpty) { + if ( + affiliationId != null && affiliationId.nonEmpty && affiliationIdType != null && affiliationIdType.nonEmpty + ) { aff.setIdentifier(new PMIdentifier(affiliationId, affiliationIdType)) } a.setAffiliation(aff) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala index 5e14c731a..281ca0e07 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala @@ -294,11 +294,23 @@ object PubMedToOaf { author.setName(a.getForeName) author.setSurname(a.getLastName) author.setFullname(a.getFullName) - if(a.getIdentifier != null) { - author.setPid(List(OafMapperUtils.structuredProperty(a.getIdentifier.getPid, - OafMapperUtils.qualifier(a.getIdentifier.getType,a.getIdentifier.getType,ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES), dataInfo)).asJava) + if (a.getIdentifier != null) { + author.setPid( + List( + OafMapperUtils.structuredProperty( + a.getIdentifier.getPid, + OafMapperUtils.qualifier( + a.getIdentifier.getType, + a.getIdentifier.getType, + ModelConstants.DNET_PID_TYPES, + ModelConstants.DNET_PID_TYPES + ), + dataInfo + ) + ).asJava + ) } - if (a.getAffiliation!= null) + if (a.getAffiliation != null) author.setRawAffiliationString(List(a.getAffiliation.getName).asJava) author.setRank(index + 1) author diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java index 1f33f45b2..9417822af 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJobTest.java @@ -1,11 +1,16 @@ + package eu.dnetlib.dhp.actionmanager.raid; -import eu.dnetlib.dhp.actionmanager.opencitations.CreateOpenCitationsASTest; -import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; -import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Relation; +import static java.nio.file.Files.createTempDirectory; + +import static eu.dnetlib.dhp.actionmanager.Constants.OBJECT_MAPPER; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.File; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; + import org.apache.commons.io.FileUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; @@ -20,93 +25,141 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; + +import eu.dnetlib.dhp.actionmanager.opencitations.CreateOpenCitationsASTest; +import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Relation; import scala.Tuple2; -import java.io.File; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.List; - -import static eu.dnetlib.dhp.actionmanager.Constants.OBJECT_MAPPER; -import static java.nio.file.Files.createTempDirectory; -import static org.junit.jupiter.api.Assertions.assertEquals; - public class GenerateRAiDActionSetJobTest { - private static String input_path; - private static String output_path; - static SparkSession spark; + private static String input_path; + private static String output_path; + static SparkSession spark; - @BeforeEach - void setUp() throws Exception { + @BeforeEach + void setUp() throws Exception { - input_path = Paths - .get(GenerateRAiDActionSetJobTest.class.getResource("/eu/dnetlib/dhp/actionmanager/raid/raid_example.json").toURI()) - .toFile() - .getAbsolutePath(); + input_path = Paths + .get( + GenerateRAiDActionSetJobTest.class + .getResource("/eu/dnetlib/dhp/actionmanager/raid/raid_example.json") + .toURI()) + .toFile() + .getAbsolutePath(); - output_path = createTempDirectory(GenerateRAiDActionSetJobTest.class.getSimpleName() + "-") - .toAbsolutePath() - .toString(); + output_path = createTempDirectory(GenerateRAiDActionSetJobTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); - SparkConf conf = new SparkConf(); - conf.setAppName(GenerateRAiDActionSetJobTest.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(GenerateRAiDActionSetJobTest.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", output_path); - conf.set("hive.metastore.warehouse.dir", output_path); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", output_path); + conf.set("hive.metastore.warehouse.dir", output_path); - spark = SparkSession - .builder() - .appName(GenerateRAiDActionSetJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(GenerateRAiDActionSetJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - static void cleanUp() throws Exception { - FileUtils.deleteDirectory(new File(output_path)); - } + @AfterAll + static void cleanUp() throws Exception { + FileUtils.deleteDirectory(new File(output_path)); + } - @Test - @Disabled - void testProcessRAiDEntities() { - GenerateRAiDActionSetJob.processRAiDEntities(spark, input_path, output_path + "/test_raid_action_set"); + @Test + @Disabled + void testProcessRAiDEntities() { + GenerateRAiDActionSetJob.processRAiDEntities(spark, input_path, output_path + "/test_raid_action_set"); - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD result = sc - .sequenceFile(output_path + "/test_raid_action_set", Text.class, Text.class) - .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) - .map(AtomicAction::getPayload); + JavaRDD result = sc + .sequenceFile(output_path + "/test_raid_action_set", Text.class, Text.class) + .map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class)) + .map(AtomicAction::getPayload); - assertEquals(80, result.count()); - } + assertEquals(80, result.count()); + } - @Test - void testPrepareRAiD() { + @Test + void testPrepareRAiD() { - List> atomicActions = GenerateRAiDActionSetJob.prepareRAiD(new RAiDEntity( - "-92190526", - Arrays.asList("Berli, Justin", "Le Mao, Bérénice", "Guillaume Touya", "Wenclik, Laura", "Courtial, Azelle", "Muehlenhaus, Ian", "Justin Berli", "Touya, Guillaume", "Gruget, Maïeul", "Azelle Courtial", "Ian Muhlenhaus", "Maïeul Gruget", "Marion Dumont", "Maïeul GRUGET", "Cécile Duchêne"), - "2021-09-10", - "2024-02-16", - Arrays.asList("cartography, zoom, pan, desert fog", "Road network", "zooming", "Pan-scalar maps", "pan-scalar map", "Python library", "QGIS", "map design", "landmarks", "Cartes transscalaires", "anchor", "disorientation", "[INFO]Computer Science [cs]", "[SHS.GEO]Humanities and Social Sciences/Geography", "cognitive cartography", "eye-tracking", "Computers in Earth Sciences", "Topographic map", "National Mapping Agency", "General Medicine", "Geography, Planning and Development", "multi-scales", "pan-scalar maps", "Selection", "cartography", "General Earth and Planetary Sciences", "progressiveness", "map generalisation", "Eye-tracker", "zoom", "algorithms", "Map Design", "cartography, map generalisation, zoom, multi-scale map", "Interactive maps", "Map generalisation", "Earth and Planetary Sciences (miscellaneous)", "Cartographic generalization", "rivers", "Benchmark", "General Environmental Science", "open source", "drawing", "Constraint", "Multi-scale maps"), - Arrays.asList("Where do people look at during multi-scale map tasks?", "FogDetector survey raw data", "Collection of cartographic disorientation stories", "Anchorwhat dataset", "BasqueRoads: A Benchmark for Road Network Selection", "Progressive river network selection for pan-scalar maps", "BasqueRoads, a dataset to benchmark road selection algorithms", "Missing the city for buildings? A critical review of pan-scalar map generalization and design in contemporary zoomable maps", "Empirical approach to advance the generalisation of multi-scale maps", "L'Alpe d'Huez: a dataset to benchmark topographic map generalisation", "eye-tracking data from a survey on zooming in a pan-scalar map", "Material of the experiment 'More is Less' from the MapMuxing project", "Cartagen4py, an open source Python library for map generalisation", "L’Alpe d’Huez: A Benchmark for Topographic Map Generalisation"), - Arrays.asList("50|doi_dedup___::6915135e0aa39f913394513f809ae58a", "50|doi_dedup___::754e3c283639bc6e104c925ff3e34007", "50|doi_dedup___::13517477f3c1261d57a3364363ce6ce0", "50|doi_dedup___::675b16c73accc4e7242bbb4ed9b3724a", "50|doi_dedup___::94ce09906b2d7d37eb2206cea8a50153", "50|dedup_wf_002::cc575d5ca5651ff8c3029a3a76e7e70a", "50|doi_dedup___::c5e52baddda17c755d1bae012a97dc13", "50|doi_dedup___::4f5f38c9e08fe995f7278963183f8ad4", "50|doi_dedup___::a9bc4453273b2d02648a5cb453195042", "50|doi_dedup___::5e893dc0cb7624a33f41c9b428bd59f7", "50|doi_dedup___::c1ecdef48fd9be811a291deed950e1c5", "50|doi_dedup___::9e93c8f2d97c35de8a6a57a5b53ef283", "50|dedup_wf_002::d08be0ed27b13d8a880e891e08d093ea", "50|doi_dedup___::f8d8b3b9eddeca2fc0e3bc9e63996555"), - "Exploring Multi-Scale Map Generalization and Design", - "This project aims to advance the generalization of multi-scale maps by investigating the impact of different design elements on user experience. The research involves collecting and analyzing data from various sources, including surveys, eye-tracking studies, and user experiments. The goal is to identify best practices for map generalization and design, with a focus on reducing disorientation and improving information retrieval during exploration. The project has led to the development of several datasets, including BasqueRoads, AnchorWhat, and L'Alpe d'Huez, which can be used to benchmark road selection algorithms and topographic map generalization techniques. The research has also resulted in the creation of a Python library, Cartagen4py, for map generalization. The findings of this project have the potential to improve the design and usability of multi-scale maps, making them more effective tools for navigation and information retrieval." - )); + List> atomicActions = GenerateRAiDActionSetJob + .prepareRAiD( + new RAiDEntity( + "-92190526", + Arrays + .asList( + "Berli, Justin", "Le Mao, Bérénice", "Guillaume Touya", "Wenclik, Laura", + "Courtial, Azelle", "Muehlenhaus, Ian", "Justin Berli", "Touya, Guillaume", + "Gruget, Maïeul", "Azelle Courtial", "Ian Muhlenhaus", "Maïeul Gruget", "Marion Dumont", + "Maïeul GRUGET", "Cécile Duchêne"), + "2021-09-10", + "2024-02-16", + Arrays + .asList( + "cartography, zoom, pan, desert fog", "Road network", "zooming", "Pan-scalar maps", + "pan-scalar map", "Python library", "QGIS", "map design", "landmarks", + "Cartes transscalaires", "anchor", "disorientation", "[INFO]Computer Science [cs]", + "[SHS.GEO]Humanities and Social Sciences/Geography", "cognitive cartography", + "eye-tracking", "Computers in Earth Sciences", "Topographic map", "National Mapping Agency", + "General Medicine", "Geography, Planning and Development", "multi-scales", + "pan-scalar maps", "Selection", "cartography", "General Earth and Planetary Sciences", + "progressiveness", "map generalisation", "Eye-tracker", "zoom", "algorithms", "Map Design", + "cartography, map generalisation, zoom, multi-scale map", "Interactive maps", + "Map generalisation", "Earth and Planetary Sciences (miscellaneous)", + "Cartographic generalization", "rivers", "Benchmark", "General Environmental Science", + "open source", "drawing", "Constraint", "Multi-scale maps"), + Arrays + .asList( + "Where do people look at during multi-scale map tasks?", "FogDetector survey raw data", + "Collection of cartographic disorientation stories", "Anchorwhat dataset", + "BasqueRoads: A Benchmark for Road Network Selection", + "Progressive river network selection for pan-scalar maps", + "BasqueRoads, a dataset to benchmark road selection algorithms", + "Missing the city for buildings? A critical review of pan-scalar map generalization and design in contemporary zoomable maps", + "Empirical approach to advance the generalisation of multi-scale maps", + "L'Alpe d'Huez: a dataset to benchmark topographic map generalisation", + "eye-tracking data from a survey on zooming in a pan-scalar map", + "Material of the experiment 'More is Less' from the MapMuxing project", + "Cartagen4py, an open source Python library for map generalisation", + "L’Alpe d’Huez: A Benchmark for Topographic Map Generalisation"), + Arrays + .asList( + "50|doi_dedup___::6915135e0aa39f913394513f809ae58a", + "50|doi_dedup___::754e3c283639bc6e104c925ff3e34007", + "50|doi_dedup___::13517477f3c1261d57a3364363ce6ce0", + "50|doi_dedup___::675b16c73accc4e7242bbb4ed9b3724a", + "50|doi_dedup___::94ce09906b2d7d37eb2206cea8a50153", + "50|dedup_wf_002::cc575d5ca5651ff8c3029a3a76e7e70a", + "50|doi_dedup___::c5e52baddda17c755d1bae012a97dc13", + "50|doi_dedup___::4f5f38c9e08fe995f7278963183f8ad4", + "50|doi_dedup___::a9bc4453273b2d02648a5cb453195042", + "50|doi_dedup___::5e893dc0cb7624a33f41c9b428bd59f7", + "50|doi_dedup___::c1ecdef48fd9be811a291deed950e1c5", + "50|doi_dedup___::9e93c8f2d97c35de8a6a57a5b53ef283", + "50|dedup_wf_002::d08be0ed27b13d8a880e891e08d093ea", + "50|doi_dedup___::f8d8b3b9eddeca2fc0e3bc9e63996555"), + "Exploring Multi-Scale Map Generalization and Design", + "This project aims to advance the generalization of multi-scale maps by investigating the impact of different design elements on user experience. The research involves collecting and analyzing data from various sources, including surveys, eye-tracking studies, and user experiments. The goal is to identify best practices for map generalization and design, with a focus on reducing disorientation and improving information retrieval during exploration. The project has led to the development of several datasets, including BasqueRoads, AnchorWhat, and L'Alpe d'Huez, which can be used to benchmark road selection algorithms and topographic map generalization techniques. The research has also resulted in the creation of a Python library, Cartagen4py, for map generalization. The findings of this project have the potential to improve the design and usability of multi-scale maps, making them more effective tools for navigation and information retrieval.")); - OtherResearchProduct orp = (OtherResearchProduct) atomicActions.get(0).getPayload(); - Relation rel = (Relation) atomicActions.get(1).getPayload(); + OtherResearchProduct orp = (OtherResearchProduct) atomicActions.get(0).getPayload(); + Relation rel = (Relation) atomicActions.get(1).getPayload(); - assertEquals("Exploring Multi-Scale Map Generalization and Design", orp.getTitle().get(0).getValue()); - assertEquals("50|raid________::759a564ce5cc7360cab030c517c7366b", rel.getSource()); - assertEquals("50|doi_dedup___::6915135e0aa39f913394513f809ae58a", rel.getTarget()); + assertEquals("Exploring Multi-Scale Map Generalization and Design", orp.getTitle().get(0).getValue()); + assertEquals("50|raid________::759a564ce5cc7360cab030c517c7366b", rel.getSource()); + assertEquals("50|doi_dedup___::6915135e0aa39f913394513f809ae58a", rel.getTarget()); - } + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala index 4a926df01..cb7826dbf 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/sx/bio/BioScholixTest.scala @@ -63,7 +63,6 @@ class BioScholixTest extends AbstractVocabularyTest { "0000000333457333", "0000000335964515", "0000000302921949", - "http://orcid.org/0000-0001-8567-3543", "http://orcid.org/0000-0001-7868-8528", "0000-0001-9189-1440", diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java index 64cbd70ba..0d6c81627 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java @@ -130,7 +130,7 @@ public class ResultTagger implements Serializable { // log.info("Remove constraints for " + communityId); if (conf.getRemoveConstraintsMap().keySet().contains(communityId) && conf.getRemoveConstraintsMap().get(communityId).getCriteria() != null && - !conf.getRemoveConstraintsMap().get(communityId).getCriteria().isEmpty() && + !conf.getRemoveConstraintsMap().get(communityId).getCriteria().isEmpty() && conf .getRemoveConstraintsMap() .get(communityId) @@ -228,7 +228,7 @@ public class ResultTagger implements Serializable { .forEach(communityId -> { if (!removeCommunities.contains(communityId) && conf.getSelectionConstraintsMap().get(communityId).getCriteria() != null && - !conf.getSelectionConstraintsMap().get(communityId).getCriteria().isEmpty() && + !conf.getSelectionConstraintsMap().get(communityId).getCriteria().isEmpty() && conf .getSelectionConstraintsMap() .get(communityId) diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index ea9503d17..e1710db54 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -915,7 +915,8 @@ class MappersTest { @Test void testODFRecord_guidelines4() throws IOException { - final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_guidelines4.xml"))); + final String xml = IOUtils + .toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_guidelines4.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); final Publication p = (Publication) list.get(0); diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 0da0f6955..2c977a390 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.solr.PersonTopic; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -40,6 +39,7 @@ import eu.dnetlib.dhp.schema.solr.OpenAccessColor; import eu.dnetlib.dhp.schema.solr.OpenAccessRoute; import eu.dnetlib.dhp.schema.solr.Organization; import eu.dnetlib.dhp.schema.solr.Person; +import eu.dnetlib.dhp.schema.solr.PersonTopic; import eu.dnetlib.dhp.schema.solr.Pid; import eu.dnetlib.dhp.schema.solr.Project; import eu.dnetlib.dhp.schema.solr.Result; @@ -216,11 +216,14 @@ public class ProvisionModelSupport { } private static List mapPersonTopics(List subjects) { - return Optional.ofNullable(subjects) - .map(ss -> ss.stream() - .map(ProvisionModelSupport::mapPersonTopic) - .collect(Collectors.toList())) - .orElse(null); + return Optional + .ofNullable(subjects) + .map( + ss -> ss + .stream() + .map(ProvisionModelSupport::mapPersonTopic) + .collect(Collectors.toList())) + .orElse(null); } private static PersonTopic mapPersonTopic(eu.dnetlib.dhp.schema.oaf.PersonTopic pt) { From fed13e083e8d9370d7d16885c17c13a8aa1a8d46 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Thu, 5 Dec 2024 15:21:32 +0100 Subject: [PATCH 231/239] Fix: do not import joda formatting --- .../java/eu/dnetlib/pace/tree/DateRange.java | 88 +++++++++---------- .../eu/dnetlib/pace/tree/JsonListMatch.java | 2 +- .../pace/comparators/ComparatorTest.java | 30 +++++-- .../dhp/bulktag/community/ResultTagger.java | 4 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 3 +- .../model/ProvisionModelSupport.java | 15 ++-- 6 files changed, 83 insertions(+), 59 deletions(-) diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java index c913109a4..6349c944b 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/DateRange.java @@ -1,10 +1,5 @@ -package eu.dnetlib.pace.tree; -import com.wcohen.ss.AbstractStringDistance; -import eu.dnetlib.pace.config.Config; -import eu.dnetlib.pace.tree.support.AbstractStringComparator; -import eu.dnetlib.pace.tree.support.ComparatorClass; -import org.joda.time.DateTime; +package eu.dnetlib.pace.tree; import java.time.DateTimeException; import java.time.LocalDate; @@ -13,55 +8,60 @@ import java.time.format.DateTimeFormatter; import java.util.Locale; import java.util.Map; +import com.wcohen.ss.AbstractStringDistance; + +import eu.dnetlib.pace.config.Config; +import eu.dnetlib.pace.tree.support.AbstractStringComparator; +import eu.dnetlib.pace.tree.support.ComparatorClass; + @ComparatorClass("dateRange") public class DateRange extends AbstractStringComparator { - int YEAR_RANGE; + int YEAR_RANGE; - public DateRange(Map params) { - super(params, new com.wcohen.ss.JaroWinkler()); - YEAR_RANGE = Integer.parseInt(params.getOrDefault("year_range", "3")); - } + public DateRange(Map params) { + super(params, new com.wcohen.ss.JaroWinkler()); + YEAR_RANGE = Integer.parseInt(params.getOrDefault("year_range", "3")); + } - public DateRange(final double weight) { - super(weight, new com.wcohen.ss.JaroWinkler()); - } + public DateRange(final double weight) { + super(weight, new com.wcohen.ss.JaroWinkler()); + } - protected DateRange(final double weight, final AbstractStringDistance ssalgo) { - super(weight, ssalgo); - } + protected DateRange(final double weight, final AbstractStringDistance ssalgo) { + super(weight, ssalgo); + } - public static boolean isNumeric(String str) { - return str.matches("\\d+"); //match a number with optional '-' and decimal. - } + public static boolean isNumeric(String str) { + return str.matches("\\d+"); // match a number with optional '-' and decimal. + } - @Override - public double distance(final String a, final String b, final Config conf) { - if (a.isEmpty() || b.isEmpty()) { - return -1.0; // return -1 if a field is missing - } + @Override + public double distance(final String a, final String b, final Config conf) { + if (a.isEmpty() || b.isEmpty()) { + return -1.0; // return -1 if a field is missing + } - try { - DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ENGLISH); - LocalDate d1 = LocalDate.parse(a, formatter); - LocalDate d2 = LocalDate.parse(b, formatter); - Period period = Period.between(d1, d2); + try { + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ENGLISH); + LocalDate d1 = LocalDate.parse(a, formatter); + LocalDate d2 = LocalDate.parse(b, formatter); + Period period = Period.between(d1, d2); - return period.getYears() <= YEAR_RANGE? 1.0 : 0.0; - } - catch (DateTimeException e) { - return -1.0; - } + return period.getYears() <= YEAR_RANGE ? 1.0 : 0.0; + } catch (DateTimeException e) { + return -1.0; + } - } + } - @Override - public double getWeight() { - return super.weight; - } + @Override + public double getWeight() { + return super.weight; + } - @Override - protected double normalize(final double d) { - return d; - } + @Override + protected double normalize(final double d) { + return d; + } } diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java index e95d9206e..d9558df90 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/JsonListMatch.java @@ -62,7 +62,7 @@ public class JsonListMatch extends AbstractListComparator { Set types = Sets.intersection(typesA, typesB); - if (types.isEmpty()) // if no common type, it is impossible to compare + if (types.isEmpty()) // if no common type, it is impossible to compare return -1; ca = ca.stream().filter(s -> types.contains(s.split("::")[0])).collect(Collectors.toSet()); diff --git a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java index 83539de4a..0abde84bc 100644 --- a/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java +++ b/dhp-pace-core/src/test/java/eu/dnetlib/pace/comparators/ComparatorTest.java @@ -72,14 +72,34 @@ public class ComparatorTest extends AbstractPaceTest { CodeMatch codeMatch = new CodeMatch(params); // names have different codes - assertEquals(0.0, codeMatch.distance("physical oceanography at ctd station june 1998 ev02a", "physical oceanography at ctd station june 1998 ir02", conf)); + assertEquals( + 0.0, + codeMatch + .distance( + "physical oceanography at ctd station june 1998 ev02a", + "physical oceanography at ctd station june 1998 ir02", conf)); // names have same code - assertEquals(1.0, codeMatch.distance("physical oceanography at ctd station june 1998 ev02a", "physical oceanography at ctd station june 1998 ev02a", conf)); + assertEquals( + 1.0, + codeMatch + .distance( + "physical oceanography at ctd station june 1998 ev02a", + "physical oceanography at ctd station june 1998 ev02a", conf)); // code is not in both names - assertEquals(-1, codeMatch.distance("physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998 ev02a", conf)); - assertEquals(1.0, codeMatch.distance("physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998", conf)); + assertEquals( + -1, + codeMatch + .distance( + "physical oceanography at ctd station june 1998", + "physical oceanography at ctd station june 1998 ev02a", conf)); + assertEquals( + 1.0, + codeMatch + .distance( + "physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998", + conf)); } @Test @@ -275,7 +295,7 @@ public class ComparatorTest extends AbstractPaceTest { Arrays .asList( "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"grid\",\"classname\":\"GRID Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"grid_1\"}", - "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_1\"}"), + "{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_1\"}"), "authors"); List b = createFieldList( Arrays diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java index 64cbd70ba..0d6c81627 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java @@ -130,7 +130,7 @@ public class ResultTagger implements Serializable { // log.info("Remove constraints for " + communityId); if (conf.getRemoveConstraintsMap().keySet().contains(communityId) && conf.getRemoveConstraintsMap().get(communityId).getCriteria() != null && - !conf.getRemoveConstraintsMap().get(communityId).getCriteria().isEmpty() && + !conf.getRemoveConstraintsMap().get(communityId).getCriteria().isEmpty() && conf .getRemoveConstraintsMap() .get(communityId) @@ -228,7 +228,7 @@ public class ResultTagger implements Serializable { .forEach(communityId -> { if (!removeCommunities.contains(communityId) && conf.getSelectionConstraintsMap().get(communityId).getCriteria() != null && - !conf.getSelectionConstraintsMap().get(communityId).getCriteria().isEmpty() && + !conf.getSelectionConstraintsMap().get(communityId).getCriteria().isEmpty() && conf .getSelectionConstraintsMap() .get(communityId) diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index ea9503d17..e1710db54 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -915,7 +915,8 @@ class MappersTest { @Test void testODFRecord_guidelines4() throws IOException { - final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_guidelines4.xml"))); + final String xml = IOUtils + .toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_guidelines4.xml"))); final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); final Publication p = (Publication) list.get(0); diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 0da0f6955..2c977a390 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; -import eu.dnetlib.dhp.schema.solr.PersonTopic; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -40,6 +39,7 @@ import eu.dnetlib.dhp.schema.solr.OpenAccessColor; import eu.dnetlib.dhp.schema.solr.OpenAccessRoute; import eu.dnetlib.dhp.schema.solr.Organization; import eu.dnetlib.dhp.schema.solr.Person; +import eu.dnetlib.dhp.schema.solr.PersonTopic; import eu.dnetlib.dhp.schema.solr.Pid; import eu.dnetlib.dhp.schema.solr.Project; import eu.dnetlib.dhp.schema.solr.Result; @@ -216,11 +216,14 @@ public class ProvisionModelSupport { } private static List mapPersonTopics(List subjects) { - return Optional.ofNullable(subjects) - .map(ss -> ss.stream() - .map(ProvisionModelSupport::mapPersonTopic) - .collect(Collectors.toList())) - .orElse(null); + return Optional + .ofNullable(subjects) + .map( + ss -> ss + .stream() + .map(ProvisionModelSupport::mapPersonTopic) + .collect(Collectors.toList())) + .orElse(null); } private static PersonTopic mapPersonTopic(eu.dnetlib.dhp.schema.oaf.PersonTopic pt) { From fd1038b44d10e17146d0aac51841818460872eca Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Fri, 6 Dec 2024 09:12:06 +0100 Subject: [PATCH 232/239] removed a sneaky break that was committed by mistake. --- .../java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java index 11f4c55d8..8172456bb 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java @@ -154,7 +154,6 @@ public class ORCIDExtractor extends Thread { extractedItem++; if (extractedItem % 100000 == 0) { log.info("Thread {}: Extracted {} items", id, extractedItem); - break; } } } From 1c144a4dcb951319ed88a7cac4825837c5385316 Mon Sep 17 00:00:00 2001 From: miconis Date: Fri, 6 Dec 2024 09:18:10 +0100 Subject: [PATCH 233/239] minor change --- .../dhp/actionmanager/raid/GenerateRAiDActionSetJob.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java index 3b2405956..c82934cdb 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java @@ -43,7 +43,8 @@ public class GenerateRAiDActionSetJob { private static final List RAID_COLLECTED_FROM = listKeyValues( OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME); - private static final Qualifier RAID_QUALIFIER = qualifier("0049", "Research Activity Identifier", DNET_PUBLICATION_RESOURCE, DNET_PUBLICATION_RESOURCE); + private static final Qualifier RAID_QUALIFIER = qualifier( + "0049", "Research Activity Identifier", DNET_PUBLICATION_RESOURCE, DNET_PUBLICATION_RESOURCE); private static final Qualifier RAID_INFERENCE_QUALIFIER = qualifier( "raid:openaireinference", "Inferred by OpenAIRE", DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS); From dade7d5bb86d4030fc2b69f3a26940e055e216eb Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 6 Dec 2024 10:02:07 +0100 Subject: [PATCH 234/239] minor changes --- .../java/eu/dnetlib/dhp/common/Constants.java | 5 ++-- .../raid/GenerateRAiDActionSetJob.java | 27 +++++++------------ 2 files changed, 13 insertions(+), 19 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java index b00199ea5..6a4bb34d3 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java @@ -11,8 +11,9 @@ public class Constants { public static final Map coarCodeLabelMap = Maps.newHashMap(); public static final String RAID_NS_PREFIX = "raid________"; - public static final String RAID_DATASOURCE_NAME = "Research Activity Identifier Service (RAiD)"; - public static final String RAID_OPENAIRE_ID = ""; + + public static final String END_DATE = "endDate"; + public static final String START_DATE = "startDate"; public static final String ROR_NS_PREFIX = "ror_________"; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java index c82934cdb..e67e7171f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/raid/GenerateRAiDActionSetJob.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.actionmanager.raid; import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_ID; import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_NAME; +import static eu.dnetlib.dhp.common.Constants.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static eu.dnetlib.dhp.schema.common.ModelConstants.*; import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; @@ -24,7 +25,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.Constants; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; @@ -120,8 +120,10 @@ public class GenerateRAiDActionSetJob { qualifier("main title", "main title", DNET_DATACITE_TITLE, DNET_DATACITE_TITLE), RAID_DATA_INFO))); orp.setDescription(listFields(RAID_DATA_INFO, r.getSummary())); -// orp.setAuthor(createAuthors(r.getAuthors())); - orp.setInstance(Collections.singletonList(eu.dnetlib.dhp.actionmanager.Constants.getInstance(RAID_QUALIFIER))); + + Instance instance = new Instance(); + instance.setInstancetype(RAID_QUALIFIER); + orp.setInstance(Collections.singletonList(instance)); orp .setSubject( r @@ -140,11 +142,11 @@ public class GenerateRAiDActionSetJob { Arrays .asList( structuredProperty( - r.getEndDate(), qualifier("endDate", "endDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), + r.getEndDate(), qualifier(END_DATE, END_DATE, DNET_DATACITE_DATE, DNET_DATACITE_DATE), RAID_DATA_INFO), structuredProperty( r.getStartDate(), - qualifier("startDate", "startDate", DNET_DATACITE_DATE, DNET_DATACITE_DATE), + qualifier(START_DATE, START_DATE, DNET_DATACITE_DATE, DNET_DATACITE_DATE), RAID_DATA_INFO))); orp.setLastupdatetimestamp(now.getTime()); orp.setDateofacceptance(field(r.getStartDate(), RAID_DATA_INFO)); @@ -159,11 +161,7 @@ public class GenerateRAiDActionSetJob { ModelConstants.RESULT_RESULT, PART, HAS_PART, - RAID_COLLECTED_FROM, - RAID_DATA_INFO, - now.getTime(), - null, - null); + orp); Relation rel2 = OafMapperUtils .getRelation( resultId, @@ -171,11 +169,7 @@ public class GenerateRAiDActionSetJob { ModelConstants.RESULT_RESULT, PART, IS_PART_OF, - RAID_COLLECTED_FROM, - RAID_DATA_INFO, - now.getTime(), - null, - null); + orp); res.add(new AtomicAction<>(Relation.class, rel1)); res.add(new AtomicAction<>(Relation.class, rel2)); } @@ -184,7 +178,7 @@ public class GenerateRAiDActionSetJob { } public static String calculateOpenaireId(final String raid) { - return String.format("50|%s::%s", Constants.RAID_NS_PREFIX, DHPUtils.md5(raid)); + return String.format("50|%s::%s", RAID_NS_PREFIX, DHPUtils.md5(raid)); } public static List createAuthors(final List author) { @@ -204,7 +198,6 @@ public class GenerateRAiDActionSetJob { .json(path) .as(Encoders.bean(RAiDEntity.class)) .toJavaRDD(); - } } From 8a5ba8df45d6fb1b570853307fb99f465d0667f5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 6 Dec 2024 10:03:11 +0100 Subject: [PATCH 235/239] minor changes --- .../main/java/eu/dnetlib/dhp/actionmanager/Constants.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java index 394cc22a3..d7ad7fcb9 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -112,12 +112,6 @@ public class Constants { } - public static Instance getInstance(Qualifier qualifier) { - Instance instance = new Instance(); - instance.setInstancetype(qualifier); - return instance; - } - public static void removeOutputDir(SparkSession spark, String path) { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); } From ee84db7a6a5ff1e271f5d6a0cfdfcc07e1a59ac9 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 6 Dec 2024 12:20:13 +0100 Subject: [PATCH 236/239] [communityfromsemrelpropagation] added filtering to remove the deletedbyinference and invisible results --- .../PrepareResultCommunitySetStep1.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index 5af2bf481..764390442 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -70,7 +70,7 @@ public class PrepareResultCommunitySetStep1 { private static final String RESULT_WITH_CONTEXT = "select id, collect_set(co.id) community_context \n" + " from result " + " lateral view explode (context) c as co " + - " where datainfo.deletedbyinference = false AND lower(co.id) IN %s" + + " where lower(co.id) IN %s" + " group by id"; private static final String RESULT_PATENT = "select id " + @@ -160,7 +160,8 @@ public class PrepareResultCommunitySetStep1 { Dataset relation = readPath(spark, inputRelationPath, Relation.class); relation.createOrReplaceTempView("relation"); - Dataset result = readPath(spark, inputResultPath, resultClazz); + Dataset result = readPath(spark, inputResultPath, resultClazz) + .where("datainfo.deletedbyinference != true AND datainfo.invisible != true"); result.createOrReplaceTempView("result"); final String outputResultPath = outputPath + "/" + resultType; From 666155bafaa04db7153c98d91b911d88e9cb2f01 Mon Sep 17 00:00:00 2001 From: Miriam Baglioni Date: Fri, 6 Dec 2024 12:26:41 +0100 Subject: [PATCH 237/239] [communityfromsemrelpropagation] changed resource to have deletedbyinference = false. --- .../graph/publication/part-00000.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json index 7957bcfd5..ae9083c65 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph/publication/part-00000.json @@ -7,7 +7,7 @@ {"author":[{"fullname":"Turkia, Heidi","name":"Heidi","pid":[],"rank":1,"surname":"Turkia"},{"fullname":"Sirén, Heli","name":"Heli","pid":[],"rank":2,"surname":"Sirén"},{"fullname":"Penttilä, Merja","name":"Merja","pid":[],"rank":3,"surname":"Penttilä"},{"fullname":"Pitkänen, Juha Pekka","name":"Juha Pekka","pid":[],"rank":4,"surname":"Pitkänen"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2013-02-22"},"dateofcollection":"2022-02-28T12:29:51.291Z","dateoftransformation":"2022-02-28T16:19:35.201Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"

Hydrolysates of lignocellulosic biomass, used as substrates for the sustainable production of fuels and chemicals often contain high amounts of phenolic compounds inhibiting the production microbiota. Quantification of these inhibitor compounds may help to understand possible difficulties in bioprocessing and further the development of more efficient, robust and tolerable processes. A separation method based on capillary electrophoresis with UV detection was developed for the simultaneous quantification of 10 phenolic compounds that may have inhibitor properties. Intraday relative standard deviations were less than 0.7% for migration times and between 2.6% and 6.4% for peak areas. Interday relative standard deviations were less than 3.0% for migration times and between 5.0% and 7.2% for peak areas. The method was applied to demonstrate that Saccharomyces cerevisiae was able to decrease the concentrations of vanillin, coniferyl aldehyde, syringaldehyde, acetoguaiacone and cinnamic acid during the cultivation, whereas the concentrations of phenols increased.

"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::6c232359e3b3165574cb88f0554d9264","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1016/j.chroma.2013.01.004"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2013-02-22"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/91f411d0-d8f8-4bf1-9072-345303cc776c"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505708387,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-29T08:16:51Z","harvestDate":"2022-02-28T12:29:51.291Z","identifier":"oai:cris.vtt.fi:publications/91f411d0-d8f8-4bf1-9072-345303cc776c","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:cris.vtt.fi:publications/91f411d0-d8f8-4bf1-9072-345303cc776c","50|355e65625b88::6c232359e3b3165574cb88f0554d9264"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Turkia , H , Sirén , H , Penttilä , M & Pitkänen , J P 2013 , ' Capillary electrophoresis for the monitoring of phenolic compounds in bioprocesses ' , Journal of Chromatography A , vol. 1278 , pp. 175-180 . https://doi.org/10.1016/j.chroma.2013.01.004"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"Bioprocess monitoring"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"Capillary electrophoresis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"Phenolic compounds"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Capillary electrophoresis for the monitoring of phenolic compounds in bioprocesses"}]} {"author":[{"fullname":"Veijalainen, Jari","name":"Jari","pid":[],"rank":1,"surname":"Veijalainen"},{"fullname":"Wolski, Antoni","name":"Antoni","pid":[],"rank":2,"surname":"Wolski"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[{"dataInfo": null,"id": "dariah"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"dateofcollection":"2022-02-28T12:33:57.005Z","dateoftransformation":"2022-02-28T16:33:35.101Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::74009c567c81b4aa55c813db658734df","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0002","classname":"Book","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"]},{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"distributionlocation":"","hostedby":{"key":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","value":"VTT Research Information System"},"instancetype":{"classid":"0002","classname":"Book","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"]}, {"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1991-01-01"},"distributionlocation":"","hostedby":{"key":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","value":"VTT Research Information System"},"instancetype":{"classid":"0002","classname":"Book","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505716994,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2020-12-21T07:05:54Z","harvestDate":"2022-02-28T12:33:57.005Z","identifier":"oai:cris.vtt.fi:publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::74009c567c81b4aa55c813db658734df","oai:cris.vtt.fi:publications/bb17c77a-f574-4921-a5cb-32dc1f283fa3"],"pid":[],"publisher":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"VTT Technical Research Centre of Finland"},"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Veijalainen , J & Wolski , A 1991 , Prepare and commit certification for decentralized transaction management in rigorous multidatabases : Research Report No. J-1 . VTT Technical Research Centre of Finland ."}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Prepare and commit certification for decentralized transaction management in rigorous multidatabases:Research Report No. J-1"}]} {"author":[{"fullname":"Hanhijärvi, Antti","name":"Antti","pid":[],"rank":1,"surname":"Hanhijärvi"},{"fullname":"Hukka, A.","name":"A.","pid":[],"rank":2,"surname":"Hukka"},{"fullname":"Paajanen, T.","name":"T.","pid":[],"rank":3,"surname":"Paajanen"},{"fullname":"Pulkkinen, P.","name":"P.","pid":[],"rank":4,"surname":"Pulkkinen"},{"fullname":"Sundman, S.","name":"S.","pid":[],"rank":5,"surname":"Sundman"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:32:33.974Z","dateoftransformation":"2022-02-28T17:38:24.191Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"The paper presents experimental results of jet drying tests of birch (Betula pendula) and spruce (Picea abies) veneers at temperatures of 140°C and 190°C. Results include drying rates for 1.5 mm thick birch veneers as well as 1.5 mm and 2.6 mm thick spruce veneers of both heartwood and sapwood. Based on the test results, material parameter values for a simplified drying model are assessed. The model is based on the use of an effective diffusion coefficient and an effective surface emission coefficient, which values are calibrated to fit to the experimental results. It is observed, that separate model parameter sets are needed for the two different species but also for occurrence of heartwood or sapwood (spruce) and different thickness values of veneers."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|dedup_wf_001::08d6f2001319c86d0e69b0f83ad75df2","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1007/s00107-003-0379-4"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8cc91100-904f-43c5-bb3d-1cc3e0f4a4b5"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505760180,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-31T21:18:51Z","harvestDate":"2022-02-28T12:32:33.974Z","identifier":"oai:cris.vtt.fi:publications/8cc91100-904f-43c5-bb3d-1cc3e0f4a4b5","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::9cb10895b4a92b0215b85acb2c3268b9","oai:cris.vtt.fi:publications/8cc91100-904f-43c5-bb3d-1cc3e0f4a4b5"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Hanhijärvi , A , Hukka , A , Paajanen , T , Pulkkinen , P & Sundman , S 2003 , ' Experimental investigation of jet drying of birch and spruce veneers and modelling with a simplified approach ' , Holz als Roh- und Werkstoff , vol. 61 , no. 2 , pp. 83-88 . https://doi.org/10.1007/s00107-003-0379-4"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"jet drying"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"drying"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"veneers"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"birch"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"spruce"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"heartwood"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"sapwood"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Experimental investigation of jet drying of birch and spruce veneers and modelling with a simplified approach"}]} -{"author":[{"fullname":"Vainonen-Ahlgren, Elizaveta","name":"Elizaveta","pid":[],"rank":1,"surname":"Vainonen-Ahlgren"},{"fullname":"Likonen, Jari","name":"Jari","pid":[],"rank":2,"surname":"Likonen"},{"fullname":"Renvall,","pid":[],"rank":3},{"fullname":"Rohde, V.","name":"V.","pid":[],"rank":4,"surname":"Rohde"},{"fullname":"Mayer, M.","name":"M.","pid":[],"rank":5,"surname":"Mayer"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"dateofcollection":"2022-02-28T12:32:50.667Z","dateoftransformation":"2022-02-28T17:49:49.964Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"To investigate material transport in scrape-off layer plasma and long term deposition in divertor, 13CH4 was puffed at the end of 2004 and 2005 experimental campaigns into ASDEX Upgrade from the outer mid-plane.
Ex situ analyses of the tiles were performed by secondary ion mass spectrometry.
The peaks of 13C were detected below the bottom inner strike point and at the horizontal tile at the outer lower divertor. It was detected ∼21% of the total puffed 13C amount.
The deposition rate for carbon by plasma was also calculated in long term experiment. It was obtained to be 22 × 10−3 and 8.7 × 10−3 g/s for the upper (campaign 2004) and lower (campaign 2003) divertors, respectively."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1016/j.jnucmat.2007.01.026"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Patent","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505766149,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-02-01T02:35:05Z","harvestDate":"2022-02-28T12:32:50.667Z","identifier":"oai:cris.vtt.fi:publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::a29614444f5030f11e75c6c27264d272","oai:cris.vtt.fi:publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Vainonen-Ahlgren , E , Likonen , J , Renvall , Rohde , V & Mayer , M 2007 , ' Migration of 13C and deposition at ASDEX Upgrade ' , Journal of Nuclear Materials , vol. 363-365 , pp. 270-275 . https://doi.org/10.1016/j.jnucmat.2007.01.026"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"ASDEX upgrade"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"divertor"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"carbon based materials"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"erosion"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"deposition"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"ITER"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"JET"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"plasma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fusion energy"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Migration of 13C and deposition at ASDEX Upgrade"}]} +{"author":[{"fullname":"Vainonen-Ahlgren, Elizaveta","name":"Elizaveta","pid":[],"rank":1,"surname":"Vainonen-Ahlgren"},{"fullname":"Likonen, Jari","name":"Jari","pid":[],"rank":2,"surname":"Likonen"},{"fullname":"Renvall,","pid":[],"rank":3},{"fullname":"Rohde, V.","name":"V.","pid":[],"rank":4,"surname":"Rohde"},{"fullname":"Mayer, M.","name":"M.","pid":[],"rank":5,"surname":"Mayer"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"dateofcollection":"2022-02-28T12:32:50.667Z","dateoftransformation":"2022-02-28T17:49:49.964Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"To investigate material transport in scrape-off layer plasma and long term deposition in divertor, 13CH4 was puffed at the end of 2004 and 2005 experimental campaigns into ASDEX Upgrade from the outer mid-plane.
Ex situ analyses of the tiles were performed by secondary ion mass spectrometry.
The peaks of 13C were detected below the bottom inner strike point and at the horizontal tile at the outer lower divertor. It was detected ∼21% of the total puffed 13C amount.
The deposition rate for carbon by plasma was also calculated in long term experiment. It was obtained to be 22 × 10−3 and 8.7 × 10−3 g/s for the upper (campaign 2004) and lower (campaign 2003) divertors, respectively."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1016/j.jnucmat.2007.01.026"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2007-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Patent","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505766149,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-02-01T02:35:05Z","harvestDate":"2022-02-28T12:32:50.667Z","identifier":"oai:cris.vtt.fi:publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::a29614444f5030f11e75c6c27264d272","oai:cris.vtt.fi:publications/2472b21e-1fdc-4121-946e-e9c8fae6d02d"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Vainonen-Ahlgren , E , Likonen , J , Renvall , Rohde , V & Mayer , M 2007 , ' Migration of 13C and deposition at ASDEX Upgrade ' , Journal of Nuclear Materials , vol. 363-365 , pp. 270-275 . https://doi.org/10.1016/j.jnucmat.2007.01.026"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"ASDEX upgrade"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"divertor"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"carbon based materials"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"erosion"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"deposition"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"ITER"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"JET"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"plasma"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fusion energy"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Migration of 13C and deposition at ASDEX Upgrade"}]} {"author":[{"fullname":"Aalto, Timo","name":"Timo","pid":[],"rank":1,"surname":"Aalto"},{"fullname":"Harjanne, Mikko","name":"Mikko","pid":[],"rank":2,"surname":"Harjanne"},{"fullname":"Kapulainen, Markku","name":"Markku","pid":[],"rank":3,"surname":"Kapulainen"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[{"dataInfo": null,"id": "beopen"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:32:37.581Z","dateoftransformation":"2022-02-28T19:39:18.717Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"A novel method has been developed for measuring the rotational angle of a fiber's or a waveguide's polarization axis with respect to a reference angle. The reference angle is the polarization axis of the measuring device. The method also gives the true polarization extinction ratio of the measured fiber or waveguide. The method is suitable for the characterization and rotational alignment of polarization-maintaining waveguides and fibers. In particular, the method can be used to rotationally align the fiber-waveguide interconnections during waveguide characterization. The measuring device is either a linear polarizer or a polarization splitter that is accurately rotated with respect to the device under test. According to the experiments with a polarization-maintaining fiber, the method is very easy and inexpensive to implement, and the angular accuracy can be better than 0.2 deg."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|openorgs____::64badd35233ba2cd4946368ef2f4cf57","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1117/1.1600730"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8cd538fb-6484-4655-81dd-47348d358fd4"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505829230,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-31T21:47:37Z","harvestDate":"2022-02-28T12:32:37.581Z","identifier":"oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::df0143af011fd82af8ac2d07b03ee8cd","oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Aalto , T , Harjanne , M & Kapulainen , M 2003 , ' Method for the rotational alignment of polarization-maintaining optical fibers and waveguides ' , Optical Engineering , vol. 42 , no. 10 , pp. 2861-2867 . https://doi.org/10.1117/1.1600730"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"optical waveguide"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization-maintaining fiber"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"characterization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fiber-waveguide coupling"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization axis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarizer"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization splitter"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Method for the rotational alignment of polarization-maintaining optical fibers and waveguides"}]} {"author":[{"fullname":"Aalto, Timo","name":"Timo","pid":[],"rank":1,"surname":"Aalto"},{"fullname":"Harjanne, Mikko","name":"Mikko","pid":[],"rank":2,"surname":"Harjanne"},{"fullname":"Kapulainen, Markku","name":"Markku","pid":[],"rank":3,"surname":"Kapulainen"}],"bestaccessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[{"dataInfo": null,"id": "beopen"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"dateofcollection":"2022-02-28T12:32:37.581Z","dateoftransformation":"2022-02-28T19:39:18.717Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"A novel method has been developed for measuring the rotational angle of a fiber's or a waveguide's polarization axis with respect to a reference angle. The reference angle is the polarization axis of the measuring device. The method also gives the true polarization extinction ratio of the measured fiber or waveguide. The method is suitable for the characterization and rotational alignment of polarization-maintaining waveguides and fibers. In particular, the method can be used to rotationally align the fiber-waveguide interconnections during waveguide characterization. The measuring device is either a linear polarizer or a polarization splitter that is accurately rotated with respect to the device under test. According to the experiments with a polarization-maintaining fiber, the method is very easy and inexpensive to implement, and the angular accuracy can be better than 0.2 deg."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|openorgs____::64badd35233ba2cd4946368ef2f4cf57","instance":[{"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.1117/1.1600730"}],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2003-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/8cd538fb-6484-4655-81dd-47348d358fd4"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505829230,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-01-31T21:47:37Z","harvestDate":"2022-02-28T12:32:37.581Z","identifier":"oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::df0143af011fd82af8ac2d07b03ee8cd","oai:cris.vtt.fi:publications/8cd538fb-6484-4655-81dd-47348d358fd4"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Aalto , T , Harjanne , M & Kapulainen , M 2003 , ' Method for the rotational alignment of polarization-maintaining optical fibers and waveguides ' , Optical Engineering , vol. 42 , no. 10 , pp. 2861-2867 . https://doi.org/10.1117/1.1600730"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"optical waveguide"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization-maintaining fiber"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"characterization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"fiber-waveguide coupling"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization axis"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarizer"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"value":"polarization splitter"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Method for the rotational alignment of polarization-maintaining optical fibers and waveguides"}]} {"author":[{"fullname":"Penttilä, Raimo","name":"Raimo","pid":[],"rank":1,"surname":"Penttilä"},{"fullname":"Vanttaja, Ilkka","name":"Ilkka","pid":[],"rank":2,"surname":"Vanttaja"},{"fullname":"Haapamäki, Petteri","name":"Petteri","pid":[],"rank":3,"surname":"Haapamäki"},{"fullname":"Kujanpää, Veli","name":"Veli","pid":[],"rank":4,"surname":"Kujanpää"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","value":"VTT Research Information System"}],"context": [{"dataInfo":null, "id":"dh-ch"}],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1994-01-01"},"dateofcollection":"2022-02-28T12:35:26.769Z","dateoftransformation":"2022-02-28T19:54:10.494Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1994-01-01"},"distributionlocation":"","hostedby":{"key":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","value":"VTT Research Information System"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/ddbd28ea-5fbf-43e1-896f-e69856870c26"]}],"language":{"classid":"fin","classname":"Finnish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1646505838552,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2019-08-08T07:09:42Z","harvestDate":"2022-02-28T12:35:26.769Z","identifier":"oai:cris.vtt.fi:publications/ddbd28ea-5fbf-43e1-896f-e69856870c26","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["oai:cris.vtt.fi:publications/ddbd28ea-5fbf-43e1-896f-e69856870c26","50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"UNKNOWN","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Penttilä , R , Vanttaja , I , Haapamäki , P & Kujanpää , V 1994 , ' Liimauksen ja puristusliittämisen yhdistämisellä lisää lujuutta, jäykkyyttä ja tiiveyttä ' , Ohutlevyuutiset , no. 2 , pp. 17-19 ."}],"subject":[],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Liimauksen ja puristusliittämisen yhdistämisellä lisää lujuutta, jäykkyyttä ja tiiveyttä"}]} \ No newline at end of file From e4b814b3f1bdb3ab52a439aff82aa56359a80cb7 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 6 Dec 2024 13:58:39 +0100 Subject: [PATCH 238/239] code formatting --- .../dhp/collection/CollectorWorker.java | 2 +- .../collection/crossref/Crossref2Oaf.scala | 13 ++-- .../crossref/CrossrefMappingTest.scala | 18 +++-- .../PrepareResultCommunitySetStep1.java | 76 ++++++++++--------- .../PrepareResultCommunitySetStep2.java | 2 +- .../ResultToCommunityJobTest.java | 70 ++++++++--------- 6 files changed, 97 insertions(+), 84 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java index 4c6d0653e..67966d523 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java @@ -7,7 +7,6 @@ import java.io.IOException; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; -import eu.dnetlib.dhp.collection.plugin.zenodo.CollectZenodoDumpCollectorPlugin; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; @@ -29,6 +28,7 @@ import eu.dnetlib.dhp.collection.plugin.mongodb.MongoDbDumpCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.oai.OaiCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.osf.OsfPreprintsCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.rest.RestCollectorPlugin; +import eu.dnetlib.dhp.collection.plugin.zenodo.CollectZenodoDumpCollectorPlugin; import eu.dnetlib.dhp.common.aggregation.AggregatorReport; import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala index c72b366a0..ea2177497 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala @@ -503,7 +503,6 @@ case object Crossref2Oaf { ) } - if (doi.startsWith("10.3410") || doi.startsWith("10.12703")) instance.setHostedby( OafMapperUtils.keyValue(OafMapperUtils.createOpenaireId(10, "openaire____::H1Connect", true), "H1Connect") @@ -556,14 +555,18 @@ case object Crossref2Oaf { result } - - def generateIdentifier(oaf: Result, doi: String): String = { val id = DHPUtils.md5(doi.toLowerCase) s"50|doiboost____|$id" } - private def generateAuthor(given: String, family: String, orcid: String, index: Int, affiliation: Option[List[mappingAffiliation]]): Author = { + private def generateAuthor( + given: String, + family: String, + orcid: String, + index: Int, + affiliation: Option[List[mappingAffiliation]] + ): Author = { val a = new Author a.setName(given) a.setSurname(family) @@ -700,7 +703,6 @@ case object Crossref2Oaf { if (objectType == null) return resultList - // If the item has a relations is-review-of, then we force it to a peer-review val is_review = json \ "relation" \ "is-review-of" \ "id" var force_to_review = false @@ -713,7 +715,6 @@ case object Crossref2Oaf { if (typology == null) return List() - val result = generateItemFromType(typology._2) if (result == null) return List() diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala index 12ca14ba1..ebe247d8a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala @@ -28,17 +28,21 @@ class CrossrefMappingTest extends AbstractVocabularyTest { val input = IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/issn_pub.json"), "utf-8") - Crossref2Oaf.convert(input, vocabularies, TransformationType.All).foreach(record => { - Assertions.assertNotNull(record) - }) + Crossref2Oaf + .convert(input, vocabularies, TransformationType.All) + .foreach(record => { + Assertions.assertNotNull(record) + }) } - @Test def mappingAffiliation(): Unit = { val input = - IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/affiliationTest.json"), "utf-8") + IOUtils.toString( + getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/affiliationTest.json"), + "utf-8" + ) val data = Crossref2Oaf.convert(input, vocabularies, TransformationType.OnlyResult) data.foreach(record => { Assertions.assertNotNull(record) @@ -46,10 +50,10 @@ class CrossrefMappingTest extends AbstractVocabularyTest { val publication = record.asInstanceOf[Publication] publication.getAuthor.asScala.foreach(author => { Assertions.assertNotNull(author.getRawAffiliationString) - Assertions.assertTrue(author.getRawAffiliationString.size()>0) + Assertions.assertTrue(author.getRawAffiliationString.size() > 0) - }) }) + }) println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(data.head)) } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index 764390442..ecb7cc827 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -1,16 +1,16 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static java.lang.String.join; + import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import static java.lang.String.join; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; -import eu.dnetlib.dhp.schema.common.ModelConstants; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; @@ -22,6 +22,7 @@ import com.google.gson.Gson; import eu.dnetlib.dhp.api.Utils; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.utils.ISLookupClientFactory; @@ -37,8 +38,7 @@ public class PrepareResultCommunitySetStep1 { * relation */ // TODO - private static final String RESULT_CONTEXT_QUERY_TEMPLATE = - "select target resultId, community_context " + private static final String RESULT_CONTEXT_QUERY_TEMPLATE = "select target resultId, community_context " + "from (select id, collect_set(co.id) community_context " + " from result " + " lateral view explode (context) c as co " @@ -60,26 +60,26 @@ public class PrepareResultCommunitySetStep1 { + "where length(co) > 0 " + "group by resultId"; - private static final String RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO = - "select target as resultId, community_context " + - "from resultWithContext rwc " + - "join relatedToRelations r " + - "join patents p " + - "on rwc.id = r.source and r.target = p.id"; + private static final String RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO = "select target as resultId, community_context " + + + "from resultWithContext rwc " + + "join relatedToRelations r " + + "join patents p " + + "on rwc.id = r.source and r.target = p.id"; private static final String RESULT_WITH_CONTEXT = "select id, collect_set(co.id) community_context \n" + - " from result " + - " lateral view explode (context) c as co " + - " where lower(co.id) IN %s" + - " group by id"; + " from result " + + " lateral view explode (context) c as co " + + " where lower(co.id) IN %s" + + " group by id"; private static final String RESULT_PATENT = "select id " + - " from result " + - " where array_contains(instance.instancetype.classname, 'Patent')"; + " from result " + + " where array_contains(instance.instancetype.classname, 'Patent')"; private static final String IS_RELATED_TO_RELATIONS = "select source, target " + - " from relation " + - " where lower(relClass) = 'isrelatedto' and datainfo.deletedbyinference = false"; + " from relation " + + " where lower(relClass) = 'isrelatedto' and datainfo.deletedbyinference = false"; public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -107,17 +107,25 @@ public class PrepareResultCommunitySetStep1 { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final String allowedsemrel ="(" + join(",", - Arrays.asList(parser.get("allowedsemrels").split(";")).stream().map(value -> "'" + value.toLowerCase() + "'") - .toArray(String[]::new)) + ")"; + final String allowedsemrel = "(" + join( + ",", + Arrays + .asList(parser.get("allowedsemrels").split(";")) + .stream() + .map(value -> "'" + value.toLowerCase() + "'") + .toArray(String[]::new)) + + ")"; log.info("allowedSemRel: {}", allowedsemrel); final String baseURL = parser.get("baseURL"); log.info("baseURL: {}", baseURL); - final String communityIdList = "(" + join(",", getCommunityList(baseURL).stream() + final String communityIdList = "(" + join( + ",", getCommunityList(baseURL) + .stream() .map(value -> "'" + value.toLowerCase() + "'") - .toArray(String[]::new)) + ")"; + .toArray(String[]::new)) + + ")"; final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); @@ -161,18 +169,17 @@ public class PrepareResultCommunitySetStep1 { relation.createOrReplaceTempView("relation"); Dataset result = readPath(spark, inputResultPath, resultClazz) - .where("datainfo.deletedbyinference != true AND datainfo.invisible != true"); + .where("datainfo.deletedbyinference != true AND datainfo.invisible != true"); result.createOrReplaceTempView("result"); final String outputResultPath = outputPath + "/" + resultType; log.info("writing output results to: {}", outputResultPath); - String resultContextQuery = String .format( RESULT_CONTEXT_QUERY_TEMPLATE, - "AND lower(co.id) IN " + communityIdList, - "AND lower(relClass) IN " + allowedsemrel); + "AND lower(co.id) IN " + communityIdList, + "AND lower(relClass) IN " + allowedsemrel); Dataset result_context = spark.sql(resultContextQuery); Dataset rwc = spark.sql(String.format(RESULT_WITH_CONTEXT, communityIdList)); @@ -183,18 +190,17 @@ public class PrepareResultCommunitySetStep1 { patents.createOrReplaceTempView("patents"); relatedToRelations.createOrReplaceTempView("relatedTorelations"); - - result_context = result_context.unionAll( spark.sql(RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO)); + result_context = result_context.unionAll(spark.sql(RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO)); result_context.createOrReplaceTempView("result_context"); spark - .sql(RESULT_COMMUNITY_LIST_QUERY) - .as(Encoders.bean(ResultCommunityList.class)) - .write() - .option("compression", "gzip") - .mode(SaveMode.Append) - .json(outputResultPath); + .sql(RESULT_COMMUNITY_LIST_QUERY) + .as(Encoders.bean(ResultCommunityList.class)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Append) + .json(outputResultPath); } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java index 9bebc36e5..9801b1bf6 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java @@ -77,7 +77,7 @@ public class PrepareResultCommunitySetStep2 { if (b == null) { return a; } - Set community_set = new HashSet<>(a.getCommunityList()); + Set community_set = new HashSet<>(a.getCommunityList()); community_set.addAll(b.getCommunityList()); a.setCommunityList(new ArrayList<>(community_set)); return a; diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index c1fcff4d9..2b52c91de 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -10,7 +10,6 @@ import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; -import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -27,6 +26,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import eu.dnetlib.dhp.schema.oaf.Dataset; import scala.collection.Seq; @@ -279,53 +279,55 @@ public class ResultToCommunityJobTest { @Test public void prepareStep1Test() throws Exception { /* - - - final String allowedsemrel = join(",", Arrays.stream(parser.get("allowedsemrels").split(";")) - .map(value -> "'" + value.toLowerCase() + "'") - .toArray(String[]::new)); - - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - - final String baseURL = parser.get("baseURL"); - log.info("baseURL: {}", baseURL); + * final String allowedsemrel = join(",", Arrays.stream(parser.get("allowedsemrels").split(";")) .map(value -> + * "'" + value.toLowerCase() + "'") .toArray(String[]::new)); log.info("allowedSemRel: {}", new + * Gson().toJson(allowedsemrel)); final String baseURL = parser.get("baseURL"); log.info("baseURL: {}", + * baseURL); */ PrepareResultCommunitySetStep1 - .main( - new String[] { - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass() - .getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph") - .getPath(), - "-hive_metastore_uris", "", - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", - "-outputPath", workingDir.toString() + "/preparedInfo", - "-allowedsemrels","issupplementto;issupplementedby", - "-baseURL","https://dev-openaire.d4science.org/openaire/community/" - }); + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph") + .getPath(), + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", + "-outputPath", workingDir.toString() + "/preparedInfo", + "-allowedsemrels", "issupplementto;issupplementedby", + "-baseURL", "https://dev-openaire.d4science.org/openaire/community/" + }); - - org.apache.spark.sql.Dataset resultCommunityList = spark.read().schema(Encoders.bean(ResultCommunityList.class).schema()) - .json(workingDir.toString() + "/preparedInfo/publication") - .as(Encoders.bean(ResultCommunityList.class)); + org.apache.spark.sql.Dataset resultCommunityList = spark + .read() + .schema(Encoders.bean(ResultCommunityList.class).schema()) + .json(workingDir.toString() + "/preparedInfo/publication") + .as(Encoders.bean(ResultCommunityList.class)); Assertions.assertEquals(2, resultCommunityList.count()); - Assertions.assertEquals(1,resultCommunityList.filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'").count()); - Assertions.assertEquals(1,resultCommunityList.filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'").count()); + Assertions + .assertEquals( + 1, + resultCommunityList.filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'").count()); + Assertions + .assertEquals( + 1, + resultCommunityList.filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'").count()); ArrayList communities = resultCommunityList - .filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'") - .first().getCommunityList(); + .filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'") + .first() + .getCommunityList(); Assertions.assertEquals(2, communities.size()); Assertions.assertTrue(communities.stream().anyMatch(cid -> "beopen".equals(cid))); Assertions.assertTrue(communities.stream().anyMatch(cid -> "dh-ch".equals(cid))); communities = resultCommunityList - .filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'") - .first().getCommunityList(); + .filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'") + .first() + .getCommunityList(); Assertions.assertEquals(1, communities.size()); Assertions.assertEquals("dh-ch", communities.get(0)); } - } From 0d050061146fc8fb7dfb1f619601719b3892da82 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Fri, 6 Dec 2024 14:23:47 +0100 Subject: [PATCH 239/239] code formatted --- .../java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java index 8172456bb..1adad104e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java @@ -160,7 +160,7 @@ public class ORCIDExtractor extends Thread { } } finally { for (SequenceFile.Writer k : fileMap.values()) { - log.info("Thread {}: Completed processed {} items", id, extractedItem); + log.info("Thread {}: Completed processed {} items", id, extractedItem); k.hflush(); k.close(); }