From 1f5aba12faefdfa5d56d38b58deea15b46b60ea9 Mon Sep 17 00:00:00 2001 From: antleb Date: Wed, 17 Apr 2024 23:54:23 +0300 Subject: [PATCH 01/26] slight optimization in indi_pub_gold_oa definition --- .../stats/oozie_app/scripts/step16-createIndicatorsTables.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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..0845387d3 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 @@ -242,7 +242,7 @@ create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet a select id, issn_online as issn from ${stats_db_name}.datasource d join gold_oa on gold_oa.issn=d.issn_online) foo ) SELECT DISTINCT pd.id, coalesce(is_gold, 0) as is_gold - FROM ${stats_db_name}.publication_datasources pd + FROM ${stats_db_name}.publication pd left outer join ( select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd From 27d22bd8f945db559392fc1eabcfe185d4183aac Mon Sep 17 00:00:00 2001 From: antleb Date: Wed, 17 Apr 2024 23:59:52 +0300 Subject: [PATCH 02/26] slight optimization in indi_pub_gold_oa definition --- .../stats/oozie_app/scripts/step16-createIndicatorsTables.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0845387d3..455c173ef 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 @@ -246,7 +246,7 @@ create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet a left outer join ( select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd - join dd on dd.id=pd.datasource + left semi join dd on dd.id=pd.datasource left outer join ${stats_db_name}.result_accessroute ra on ra.id = pd.id where ra.accessroute = 'gold') tmp on tmp.id=pd.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc purge; /*EOS*/ From 308ae580a97afba2cf19bf79d8022dbac33fc1e1 Mon Sep 17 00:00:00 2001 From: antleb Date: Thu, 18 Apr 2024 10:57:52 +0300 Subject: [PATCH 03/26] slight optimization in indi_pub_gold_oa definition --- .../scripts/step16-createIndicatorsTables.sql | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) 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 455c173ef..18d66c6db 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 @@ -282,14 +282,17 @@ 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 pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication pd +select distinct p.id, coalesce(is_hybrid, 0) is_hybrid +from ${stats_db_name}.publication p left outer join ( - select pd.id, 1 as is_hybrid from ${stats_db_name}.publication pd - join ${stats_db_name}.result_instance ri on ri.id=pd.id - join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id - join ${stats_db_name}.result_accessroute ra on ra.id=pd.id + select p.id, 1 as is_hybrid + from ${stats_db_name}.publication p + join ${stats_db_name}.result_instance ri on ri.id=p.id join ${stats_db_name}.datasource d on d.id=ri.hostedby - where indi_gold.is_gold=0 and ((d.type like '%Journal%' and ri.accessright!='Closed Access' and ri.accessright!='Restricted' and ri.license is not null) or ra.accessroute='hybrid')) tmp on pd.id=tmp.id; /*EOS*/ + join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=p.id + left outer join ${stats_db_name}.result_accessroute ra on ra.id=p.id + where indi_gold.is_gold=0 and + ((d.type like '%Journal%' and ri.accessright not in ('Closed Access', 'Restricted', 'Not Available') and ri.license is not null) or ra.accessroute='hybrid')) tmp on pd.i=tmp.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_org_fairness purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_fairness stored as parquet as From e728a0897c88c4496f4533f713714387b9a1c25f Mon Sep 17 00:00:00 2001 From: antleb Date: Thu, 18 Apr 2024 11:07:55 +0300 Subject: [PATCH 04/26] fixed the definition of indi_pub_bronze_oa --- .../scripts/step16-createIndicatorsTables.sql | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) 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 18d66c6db..ac14e2904 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 @@ -664,17 +664,18 @@ 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 pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication pd -left outer join (select pd.id, 1 as is_bronze_oa from ${stats_db_name}.publication pd -join ${stats_db_name}.result_instance ri on ri.id=pd.id -join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id -join ${stats_db_name}.indi_pub_hybrid indi_hybrid on indi_hybrid.id=pd.id -join ${stats_db_name}.result_accessroute ra on ra.id=pd.id -join ${stats_db_name}.datasource d on d.id=ri.hostedby -where indi_gold.is_gold=0 and indi_hybrid.is_hybrid=0 -and ((d.type like '%Journal%' and ri.accessright!='Closed Access' -and ri.accessright!='Restricted' and ri.license is null) or ra.accessroute='bronze')) tmp -on pd.id=tmp.id; /*EOS*/ +select 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 + from ${stats_db_name}.publication p + join ${stats_db_name}.result_instance ri on ri.id=p.id + join ${stats_db_name}.datasource d on d.id=ri.hostedby + join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=p.id + join ${stats_db_name}.indi_pub_hybrid indi_hybrid on indi_hybrid.id=p.id + left outer join ${stats_db_name}.result_accessroute ra on ra.id=p.id + where indi_gold.is_gold=0 and indi_hybrid.is_hybrid=0 + and ((d.type like '%Journal%' and ri.accessright not in ('Closed Access', 'Restricted', 'Not Available') and ri.license is null) or ra.accessroute='bronze')) tmp on p.id=tmp.id; /*EOS*/ CREATE TEMPORARY VIEW project_year_result_year as select p.id project_id, acronym, r.id result_id, r.year, p.end_year From 43d05dbebb4d0a8760dee242a1da0146b3698689 Mon Sep 17 00:00:00 2001 From: antleb Date: Thu, 18 Apr 2024 11:53:50 +0300 Subject: [PATCH 05/26] fixed the definition of result_country --- .../scripts/step16-createIndicatorsTables.sql | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) 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 ac14e2904..9ea84023a 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 @@ -1004,13 +1004,18 @@ 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 ro.id, coalesce(o.country, f.country) -from ${stats_db_name}.result_organization ro -left outer join ${stats_db_name}.organization o on o.id=ro.organization -left outer join ${stats_db_name}.result_projects rp on rp.id=ro.id -left outer join ${stats_db_name}.project p on p.id=rp.project -left outer join ${stats_db_name}.funder f on f.name=p.funder -where coalesce(o.country, f.country) IS NOT NULL; +select distinct * +from ( + select ro.id, o.country + from ${stats_db_name}.result_organization ro + left outer join ${stats_db_name}.organization o on o.id=ro.organization + union all + select rp.id, f.country + from ${stats_db_name}.result_projects + left outer join ${stats_db_name}.project p on p.id=rp.project + left outer join ${stats_db_name}.funder f on f.name=p.funder + ) rc +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 From 0c71c58df69a23968b942fcc62d7d63e4cd3d551 Mon Sep 17 00:00:00 2001 From: antleb Date: Thu, 18 Apr 2024 12:01:27 +0300 Subject: [PATCH 06/26] fixed the definition of gold_oa --- .../stats/oozie_app/scripts/step16-createIndicatorsTables.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 9ea84023a..65193a50c 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 @@ -247,7 +247,9 @@ create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet a select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd left semi join dd on dd.id=pd.datasource - left outer join ${stats_db_name}.result_accessroute ra on ra.id = pd.id where ra.accessroute = 'gold') tmp on tmp.id=pd.id; /*EOS*/ + union all + select ra.id, 1 as is_gold + from ${stats_db_name}.result_accessroute ra on ra.id = pd.id where ra.accessroute = 'gold') tmp on tmp.id=pd.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc stored as parquet as From c3fe9662b22e1a80ccb56479639338f79f8d1832 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Fri, 19 Apr 2024 12:45:36 +0300 Subject: [PATCH 07/26] all indicator tables are now stored as parquet --- .../oozie_app/scripts/step16-createIndicatorsTables.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 65193a50c..1a4002bcf 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 @@ -380,7 +380,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 as +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 from allresults ar join result_fair rf on rf.organization=ar.organization; /*EOS*/ @@ -639,7 +639,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 as +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 from ${stats_db_name}.result p left outer join ( @@ -653,7 +653,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 as +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) as is_interdisciplinary from pub_fos_totals p From d2649a1429ffcb7355d41696bd7abd2744d0a81b Mon Sep 17 00:00:00 2001 From: antleb Date: Tue, 23 Apr 2024 16:03:16 +0300 Subject: [PATCH 08/26] increased the jvm ram --- .../dnetlib/dhp/oa/graph/usagerawdata/oozie_app/workflow.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dhp-workflows/dhp-usage-raw-data-update/src/main/resources/eu/dnetlib/dhp/oa/graph/usagerawdata/oozie_app/workflow.xml b/dhp-workflows/dhp-usage-raw-data-update/src/main/resources/eu/dnetlib/dhp/oa/graph/usagerawdata/oozie_app/workflow.xml index 022a107ab..b684b5e24 100644 --- a/dhp-workflows/dhp-usage-raw-data-update/src/main/resources/eu/dnetlib/dhp/oa/graph/usagerawdata/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-usage-raw-data-update/src/main/resources/eu/dnetlib/dhp/oa/graph/usagerawdata/oozie_app/workflow.xml @@ -30,6 +30,10 @@ oozie.launcher.mapred.job.queue.name ${oozieLauncherQueueName} + + mapred.child.java.opts + -Xmx16g + From 49af2e574088d24a799d4c22741a8f0e03455826 Mon Sep 17 00:00:00 2001 From: LSmyrnaios Date: Tue, 23 Apr 2024 17:15:04 +0300 Subject: [PATCH 09/26] Miscellaneous updates to the copying operation to Impala Cluster: - Update the algorithm for creating views that depend on other views; overcome some bash-instabilities. - Upon any error, fail the whole process, not just the current DB-creation, as those errors usually indicate a bug in the initial DB-creation, that should be fixed immediately. - Enhance parallel-copy of large files by "hadoop distcp" command. - Reduce the "invalidate metadata" commands to just the current DB's tables, in order to eliminate the general overhead on Impala. - Show the number of tables and views in the logs. - Fix some log-messages. --- .../oozie_app/copyDataToImpalaCluster.sh | 71 +++++++++---------- .../oozie_app/copyDataToImpalaCluster.sh | 71 +++++++++---------- .../oozie_app/copyDataToImpalaCluster.sh | 71 +++++++++---------- .../oozie_app/copyDataToImpalaCluster.sh | 71 +++++++++---------- 4 files changed, 132 insertions(+), 152 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 3d9986b64..059fb9089 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 @@ -67,24 +67,21 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" rm -f error.log - return 1 + exit 2 fi - # Make Impala aware of the deletion of the old DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" - # Using max-bandwidth of: 50 * 100 Mb/s = 5 Gb/s - # Using max memory of: 50 * 6144 = 300 Gb + # 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. - # The " -Ddistcp.dynamic.recordsPerChunk=50" arg is not available in our version of hadoop + # The " -Ddistcp.dynamic.recordsPerChunk=N" arg is not available in our version of hadoop # The "ug" args cannot be used as we get a "User does not belong to hive" error. # The "p" argument cannot be used, as it blocks the files from being used, giving a "sticky bit"-error, even after applying chmod and chown onm the files. hadoop distcp -Dmapreduce.map.memory.mb=6144 -m 70 -bandwidth 150 \ -numListstatusThreads 40 \ -copybuffersize 1048576 \ -strategy dynamic \ + -blocksperchunk 8 \ -pb \ ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} @@ -92,9 +89,9 @@ function copydb() { if [ $? -eq 0 ]; then echo -e "\nSuccessfully copied the files of '${db}'.\n" else - echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT WITH EXIT STATUS: $?\n\n" + echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log - return 2 + exit 3 fi # In case we ever use this script for a writable DB (using inserts/updates), we should perform the following costly operation as well.. @@ -105,14 +102,11 @@ function copydb() { # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" - # Make Impala aware of the creation of the new DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 # 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. 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. for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elemetns are single-words. @@ -129,9 +123,11 @@ function copydb() { all_create_view_statements+=("$create_view_statement") else echo -e "\n'${i}' is a table, so we will check for its parquet files and create the table on Impala cluster.\n" + ((num_tables++)) CURRENT_PRQ_FILE=`hdfs dfs -conf ${IMPALA_CONFIG_FILE} -ls -C "${IMPALA_HDFS_DB_BASE_PATH}/${db}.db/${i}/" | grep -v 'Found' | grep -v '_impala_insert_staging' | head -1` if [ -z "$CURRENT_PRQ_FILE" ]; then # If there is not parquet-file inside. echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" + exit 4 # Comment out when testing a DB which has such a table, just for performing this exact test-check. 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"` @@ -142,74 +138,73 @@ function copydb() { fi done - echo -e "\nAll tables have been created, going to create the views..\n" + previous_num_of_views_to_retry=${#all_create_view_statements[@]} + if [[ $num_tables -gt 0 ]]; then + echo -e "\nAll ${num_tables} tables have been created, for db '${db}', going to create the ${previous_num_of_views_to_retry} views..\n" + else + echo -e "\nDB '${db}' does not have any tables, moving on to create the ${previous_num_of_views_to_retry} views..\n" + fi - # Time to loop through the views and create them. - # At this point all table-schemas should have been created. - - previous_num_of_views_to_retry=${#all_create_view_statements} if [[ $previous_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nAll_create_view_statements:\n\n${all_create_view_statements[@]}\n" # DEBUG - # Make Impala aware of the new tables, so it knows them when creating the views. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 + echo -e "\nAll_create_view_statements (${previous_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" # DEBUG else echo -e "\nDB '${db}' does not contain any views.\n" fi level_counter=0 - while [[ ${#all_create_view_statements[@]} -gt 0 ]]; do + while [[ $previous_num_of_views_to_retry -gt 0 ]]; do ((level_counter++)) # The only accepted reason for a view to not be created, is if it depends on another view, which has not been created yet. # In this case, we should retry creating this particular view again. - should_retry_create_view_statements=() + 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 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 "CREATE VIEW " | sed 's/CREATE VIEW //g' | sed 's/ as select .*//g')' failed to be created, possibly because it depends on another view.\n" - should_retry_create_view_statements+=("$create_view_statement") + 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" + ((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. sleep 1 # Wait a bit for Impala to register that the view was created, before possibly referencing it by another view. fi done - new_num_of_views_to_retry=${#should_retry_create_view_statements} + all_create_view_statements=("$(echo "${all_create_view_statements[@]}" | grep -v '^[\s]*$')") # Re-index the array, filtering-out any empty elements. + # 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" - return 3 + exit 5 elif [[ $new_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nTo be retried \"create_view_statements\":\n\n${should_retry_create_view_statements[@]}\n" - previous_num_of_views_to_retry=$new_num_of_views_to_retry + echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" else echo -e "\nFinished creating views, for db: '${db}', in level-${level_counter}.\n" fi - all_create_view_statements=("${should_retry_create_view_statement[@]}") # This is needed in any case to either move forward with the rest of the views or stop at 0 remaining views. + previous_num_of_views_to_retry=$new_num_of_views_to_retry done - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 - 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. 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}" + sleep 1 impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; fi done + # 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" 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 - return 4 + exit 6 fi rm -f error.log 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 2711d6e12..1130a684d 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 @@ -66,24 +66,21 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" rm -f error.log - return 1 + exit 2 fi - # Make Impala aware of the deletion of the old DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" - # Using max-bandwidth of: 50 * 100 Mb/s = 5 Gb/s - # Using max memory of: 50 * 6144 = 300 Gb + # 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. - # The " -Ddistcp.dynamic.recordsPerChunk=50" arg is not available in our version of hadoop + # The " -Ddistcp.dynamic.recordsPerChunk=N" arg is not available in our version of hadoop # The "ug" args cannot be used as we get a "User does not belong to hive" error. # The "p" argument cannot be used, as it blocks the files from being used, giving a "sticky bit"-error, even after applying chmod and chown onm the files. hadoop distcp -Dmapreduce.map.memory.mb=6144 -m 70 -bandwidth 150 \ -numListstatusThreads 40 \ -copybuffersize 1048576 \ -strategy dynamic \ + -blocksperchunk 8 \ -pb \ ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} @@ -91,9 +88,9 @@ function copydb() { if [ $? -eq 0 ]; then echo -e "\nSuccessfully copied the files of '${db}'.\n" else - echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT WITH EXIT STATUS: $?\n\n" + echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log - return 2 + exit 3 fi # In case we ever use this script for a writable DB (using inserts/updates), we should perform the following costly operation as well.. @@ -104,14 +101,11 @@ function copydb() { # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" - # Make Impala aware of the creation of the new DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 # 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. 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. for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elemetns are single-words. @@ -128,9 +122,11 @@ function copydb() { all_create_view_statements+=("$create_view_statement") else echo -e "\n'${i}' is a table, so we will check for its parquet files and create the table on Impala cluster.\n" + ((num_tables++)) CURRENT_PRQ_FILE=`hdfs dfs -conf ${IMPALA_CONFIG_FILE} -ls -C "${IMPALA_HDFS_DB_BASE_PATH}/${db}.db/${i}/" | grep -v 'Found' | grep -v '_impala_insert_staging' | head -1` if [ -z "$CURRENT_PRQ_FILE" ]; then # If there is not parquet-file inside. echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" + exit 4 # Comment out when testing a DB which has such a table, just for performing this exact test-check. 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"` @@ -141,74 +137,73 @@ function copydb() { fi done - echo -e "\nAll tables have been created, going to create the views..\n" + previous_num_of_views_to_retry=${#all_create_view_statements[@]} + if [[ $num_tables -gt 0 ]]; then + echo -e "\nAll ${num_tables} tables have been created, for db '${db}', going to create the ${previous_num_of_views_to_retry} views..\n" + else + echo -e "\nDB '${db}' does not have any tables, moving on to create the ${previous_num_of_views_to_retry} views..\n" + fi - # Time to loop through the views and create them. - # At this point all table-schemas should have been created. - - previous_num_of_views_to_retry=${#all_create_view_statements} if [[ $previous_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nAll_create_view_statements:\n\n${all_create_view_statements[@]}\n" # DEBUG - # Make Impala aware of the new tables, so it knows them when creating the views. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 + echo -e "\nAll_create_view_statements (${previous_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" # DEBUG else echo -e "\nDB '${db}' does not contain any views.\n" fi level_counter=0 - while [[ ${#all_create_view_statements[@]} -gt 0 ]]; do + while [[ $previous_num_of_views_to_retry -gt 0 ]]; do ((level_counter++)) # The only accepted reason for a view to not be created, is if it depends on another view, which has not been created yet. # In this case, we should retry creating this particular view again. - should_retry_create_view_statements=() + 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 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 "CREATE VIEW " | sed 's/CREATE VIEW //g' | sed 's/ as select .*//g')' failed to be created, possibly because it depends on another view.\n" - should_retry_create_view_statements+=("$create_view_statement") + 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" + ((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. sleep 1 # Wait a bit for Impala to register that the view was created, before possibly referencing it by another view. fi done - new_num_of_views_to_retry=${#should_retry_create_view_statements} + all_create_view_statements=("$(echo "${all_create_view_statements[@]}" | grep -v '^[\s]*$')") # Re-index the array, filtering-out any empty elements. + # 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" - return 3 + exit 5 elif [[ $new_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nTo be retried \"create_view_statements\":\n\n${should_retry_create_view_statements[@]}\n" - previous_num_of_views_to_retry=$new_num_of_views_to_retry + echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" else echo -e "\nFinished creating views, for db: '${db}', in level-${level_counter}.\n" fi - all_create_view_statements=("${should_retry_create_view_statement[@]}") # This is needed in any case to either move forward with the rest of the views or stop at 0 remaining views. + previous_num_of_views_to_retry=$new_num_of_views_to_retry done - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 - 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. 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}" + sleep 1 impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; fi done + # 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" 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 - return 4 + exit 6 fi rm -f error.log 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 5ad9df762..de275145b 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 @@ -66,24 +66,21 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" rm -f error.log - return 1 + exit 2 fi - # Make Impala aware of the deletion of the old DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" - # Using max-bandwidth of: 50 * 100 Mb/s = 5 Gb/s - # Using max memory of: 50 * 6144 = 300 Gb + # 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. - # The " -Ddistcp.dynamic.recordsPerChunk=50" arg is not available in our version of hadoop + # The " -Ddistcp.dynamic.recordsPerChunk=N" arg is not available in our version of hadoop # The "ug" args cannot be used as we get a "User does not belong to hive" error. # The "p" argument cannot be used, as it blocks the files from being used, giving a "sticky bit"-error, even after applying chmod and chown onm the files. hadoop distcp -Dmapreduce.map.memory.mb=6144 -m 70 -bandwidth 150 \ -numListstatusThreads 40 \ -copybuffersize 1048576 \ -strategy dynamic \ + -blocksperchunk 8 \ -pb \ ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} @@ -91,9 +88,9 @@ function copydb() { if [ $? -eq 0 ]; then echo -e "\nSuccessfully copied the files of '${db}'.\n" else - echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT WITH EXIT STATUS: $?\n\n" + echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log - return 2 + exit 3 fi # In case we ever use this script for a writable DB (using inserts/updates), we should perform the following costly operation as well.. @@ -104,14 +101,11 @@ function copydb() { # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" - # Make Impala aware of the creation of the new DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 # 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. 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. for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elemetns are single-words. @@ -128,9 +122,11 @@ function copydb() { all_create_view_statements+=("$create_view_statement") else echo -e "\n'${i}' is a table, so we will check for its parquet files and create the table on Impala cluster.\n" + ((num_tables++)) CURRENT_PRQ_FILE=`hdfs dfs -conf ${IMPALA_CONFIG_FILE} -ls -C "${IMPALA_HDFS_DB_BASE_PATH}/${db}.db/${i}/" | grep -v 'Found' | grep -v '_impala_insert_staging' | head -1` if [ -z "$CURRENT_PRQ_FILE" ]; then # If there is not parquet-file inside. echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" + exit 4 # Comment out when testing a DB which has such a table, just for performing this exact test-check. 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"` @@ -141,74 +137,73 @@ function copydb() { fi done - echo -e "\nAll tables have been created, going to create the views..\n" + previous_num_of_views_to_retry=${#all_create_view_statements[@]} + if [[ $num_tables -gt 0 ]]; then + echo -e "\nAll ${num_tables} tables have been created, for db '${db}', going to create the ${previous_num_of_views_to_retry} views..\n" + else + echo -e "\nDB '${db}' does not have any tables, moving on to create the ${previous_num_of_views_to_retry} views..\n" + fi - # Time to loop through the views and create them. - # At this point all table-schemas should have been created. - - previous_num_of_views_to_retry=${#all_create_view_statements} if [[ $previous_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nAll_create_view_statements:\n\n${all_create_view_statements[@]}\n" # DEBUG - # Make Impala aware of the new tables, so it knows them when creating the views. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 + echo -e "\nAll_create_view_statements (${previous_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" # DEBUG else echo -e "\nDB '${db}' does not contain any views.\n" fi level_counter=0 - while [[ ${#all_create_view_statements[@]} -gt 0 ]]; do + while [[ $previous_num_of_views_to_retry -gt 0 ]]; do ((level_counter++)) # The only accepted reason for a view to not be created, is if it depends on another view, which has not been created yet. # In this case, we should retry creating this particular view again. - should_retry_create_view_statements=() + 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 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 "CREATE VIEW " | sed 's/CREATE VIEW //g' | sed 's/ as select .*//g')' failed to be created, possibly because it depends on another view.\n" - should_retry_create_view_statements+=("$create_view_statement") + 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" + ((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. sleep 1 # Wait a bit for Impala to register that the view was created, before possibly referencing it by another view. fi done - new_num_of_views_to_retry=${#should_retry_create_view_statements} + all_create_view_statements=("$(echo "${all_create_view_statements[@]}" | grep -v '^[\s]*$')") # Re-index the array, filtering-out any empty elements. + # 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" - return 3 + exit 5 elif [[ $new_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nTo be retried \"create_view_statements\":\n\n${should_retry_create_view_statements[@]}\n" - previous_num_of_views_to_retry=$new_num_of_views_to_retry + echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" else echo -e "\nFinished creating views, for db: '${db}', in level-${level_counter}.\n" fi - all_create_view_statements=("${should_retry_create_view_statement[@]}") # This is needed in any case to either move forward with the rest of the views or stop at 0 remaining views. + previous_num_of_views_to_retry=$new_num_of_views_to_retry done - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 - 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. 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}" + sleep 1 impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; fi done + # 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" 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 - return 4 + exit 6 fi rm -f error.log 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 c2324b912..6fc0aa745 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 @@ -68,24 +68,21 @@ function copydb() { if [ -n "$log_errors" ]; then echo -e "\n\nERROR: THERE WAS A PROBLEM WHEN DROPPING THE OLD DATABASE! EXITING...\n\n" rm -f error.log - return 1 + exit 2 fi - # Make Impala aware of the deletion of the old DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - echo -e "\n\nCopying files of '${db}', from Ocean to Impala cluster..\n" - # Using max-bandwidth of: 50 * 100 Mb/s = 5 Gb/s - # Using max memory of: 50 * 6144 = 300 Gb + # 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. - # The " -Ddistcp.dynamic.recordsPerChunk=50" arg is not available in our version of hadoop + # The " -Ddistcp.dynamic.recordsPerChunk=N" arg is not available in our version of hadoop # The "ug" args cannot be used as we get a "User does not belong to hive" error. # The "p" argument cannot be used, as it blocks the files from being used, giving a "sticky bit"-error, even after applying chmod and chown onm the files. hadoop distcp -Dmapreduce.map.memory.mb=6144 -m 70 -bandwidth 150 \ -numListstatusThreads 40 \ -copybuffersize 1048576 \ -strategy dynamic \ + -blocksperchunk 8 \ -pb \ ${OCEAN_HDFS_NODE}/user/hive/warehouse/${db}.db ${IMPALA_HDFS_DB_BASE_PATH} @@ -93,9 +90,9 @@ function copydb() { if [ $? -eq 0 ]; then echo -e "\nSuccessfully copied the files of '${db}'.\n" else - echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT WITH EXIT STATUS: $?\n\n" + echo -e "\n\nERROR: FAILED TO TRANSFER THE FILES OF '${db}', WITH 'hadoop distcp'. GOT EXIT STATUS: $?\n\n" rm -f error.log - return 2 + exit 3 fi # In case we ever use this script for a writable DB (using inserts/updates), we should perform the following costly operation as well.. @@ -106,14 +103,11 @@ function copydb() { # create the new database (with the same name) impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "create database ${db}" - # Make Impala aware of the creation of the new DB immediately. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 # 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. 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. for i in ${entities_on_ocean[@]}; do # Use un-quoted values, as the elemetns are single-words. @@ -130,9 +124,11 @@ function copydb() { all_create_view_statements+=("$create_view_statement") else echo -e "\n'${i}' is a table, so we will check for its parquet files and create the table on Impala cluster.\n" + ((num_tables++)) CURRENT_PRQ_FILE=`hdfs dfs -conf ${IMPALA_CONFIG_FILE} -ls -C "${IMPALA_HDFS_DB_BASE_PATH}/${db}.db/${i}/" | grep -v 'Found' | grep -v '_impala_insert_staging' | head -1` if [ -z "$CURRENT_PRQ_FILE" ]; then # If there is not parquet-file inside. echo -e "\nERROR: THE TABLE \"${i}\" HAD NO FILES TO GET THE SCHEMA FROM! IT'S EMPTY!\n\n" + exit 4 # Comment out when testing a DB which has such a table, just for performing this exact test-check. 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"` @@ -143,74 +139,73 @@ function copydb() { fi done - echo -e "\nAll tables have been created, going to create the views..\n" + previous_num_of_views_to_retry=${#all_create_view_statements[@]} + if [[ $num_tables -gt 0 ]]; then + echo -e "\nAll ${num_tables} tables have been created, for db '${db}', going to create the ${previous_num_of_views_to_retry} views..\n" + else + echo -e "\nDB '${db}' does not have any tables, moving on to create the ${previous_num_of_views_to_retry} views..\n" + fi - # Time to loop through the views and create them. - # At this point all table-schemas should have been created. - - previous_num_of_views_to_retry=${#all_create_view_statements} if [[ $previous_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nAll_create_view_statements:\n\n${all_create_view_statements[@]}\n" # DEBUG - # Make Impala aware of the new tables, so it knows them when creating the views. - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 + echo -e "\nAll_create_view_statements (${previous_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" # DEBUG else echo -e "\nDB '${db}' does not contain any views.\n" fi level_counter=0 - while [[ ${#all_create_view_statements[@]} -gt 0 ]]; do + while [[ $previous_num_of_views_to_retry -gt 0 ]]; do ((level_counter++)) # The only accepted reason for a view to not be created, is if it depends on another view, which has not been created yet. # In this case, we should retry creating this particular view again. - should_retry_create_view_statements=() + 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 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 "CREATE VIEW " | sed 's/CREATE VIEW //g' | sed 's/ as select .*//g')' failed to be created, possibly because it depends on another view.\n" - should_retry_create_view_statements+=("$create_view_statement") + 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" + ((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. sleep 1 # Wait a bit for Impala to register that the view was created, before possibly referencing it by another view. fi done - new_num_of_views_to_retry=${#should_retry_create_view_statements} + all_create_view_statements=("$(echo "${all_create_view_statements[@]}" | grep -v '^[\s]*$')") # Re-index the array, filtering-out any empty elements. + # 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" - return 3 + exit 5 elif [[ $new_num_of_views_to_retry -gt 0 ]]; then - echo -e "\nTo be retried \"create_view_statements\":\n\n${should_retry_create_view_statements[@]}\n" - previous_num_of_views_to_retry=$new_num_of_views_to_retry + echo -e "\nTo be retried \"create_view_statements\" (${new_num_of_views_to_retry}):\n\n${all_create_view_statements[@]}\n" else echo -e "\nFinished creating views, for db: '${db}', in level-${level_counter}.\n" fi - all_create_view_statements=("${should_retry_create_view_statement[@]}") # This is needed in any case to either move forward with the rest of the views or stop at 0 remaining views. + previous_num_of_views_to_retry=$new_num_of_views_to_retry done - sleep 1 - impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "INVALIDATE METADATA" - sleep 1 - 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. 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}" + sleep 1 impala-shell --user ${HADOOP_USER_NAME} -i ${IMPALA_HOSTNAME} -q "compute stats ${db}.${i}"; fi done + # 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" 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 - return 4 + exit 6 fi rm -f error.log From 2615136efc0a86ceb92f82f2380e68230330ef83 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 30 Apr 2024 11:58:42 +0200 Subject: [PATCH 10/26] added a retry mechanism --- .../collection/plugin/rest/RestIterator.java | 379 +++++++++--------- 1 file changed, 200 insertions(+), 179 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 1107bcf46..c13f29806 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 @@ -18,7 +18,11 @@ 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.*; +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 org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; @@ -35,7 +39,7 @@ import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; /** - * log.info(...) equal to log.trace(...) in the application-logs + * log.info(...) equal to log.trace(...) in the application-logs *

* known bug: at resumptionType 'discover' if the (resultTotal % resultSizeValue) == 0 the collecting fails -> change the resultSizeValue * @@ -47,6 +51,7 @@ 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; private final HttpClientParams clientParams; @@ -60,8 +65,8 @@ public class RestIterator implements Iterator { private final int resultSizeValue; private int resumptionInt = 0; // integer resumption token (first record to harvest) private int resultTotal = -1; - private String resumptionStr = Integer.toString(resumptionInt); // string resumption token (first record to harvest - // or token scanned from results) + 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; @@ -73,75 +78,75 @@ public class RestIterator implements Iterator { private final String querySize; private final String authMethod; private final String authToken; - private final Queue recordQueue = new PriorityBlockingQueue(); + private final Queue recordQueue = new PriorityBlockingQueue<>(); private int discoverResultSize = 0; private int pagination = 1; /* - * While resultFormatValue is added to the request parameter, this is used to say that the results are retrieved in - * json. useful for cases when the target API expects a resultFormatValue != json, but the results are returned in - * json. An example is the EU Open Data Portal API: resultFormatValue=standard, results are in json format. + * While resultFormatValue is added to the request parameter, this is used to say that the results are retrieved in json. useful for + * cases when the target API expects a resultFormatValue != json, but the results are returned in json. An example is the EU Open Data + * Portal API: resultFormatValue=standard, results are in json format. */ private final String resultOutputFormat; - /** RestIterator class - * compatible to version 1.3.33 + /** + * RestIterator class compatible to version 1.3.33 */ public RestIterator( - final HttpClientParams clientParams, - final String baseUrl, - final String resumptionType, - final String resumptionParam, - final String resumptionXpath, - final String resultTotalXpath, - final String resultFormatParam, - final String resultFormatValue, - final String resultSizeParam, - final String resultSizeValueStr, - final String queryParams, - final String entityXpath, - final String authMethod, - final String authToken, - final String resultOutputFormat) { + final HttpClientParams clientParams, + final String baseUrl, + final String resumptionType, + final String resumptionParam, + final String resumptionXpath, + final String resultTotalXpath, + final String resultFormatParam, + final String resultFormatValue, + final String resultSizeParam, + final String resultSizeValueStr, + final String queryParams, + final String entityXpath, + final String authMethod, + final String authToken, + final String resultOutputFormat) { this.clientParams = clientParams; this.baseUrl = baseUrl; this.resumptionType = resumptionType; this.resumptionParam = resumptionParam; this.resultFormatValue = resultFormatValue; - this.resultSizeValue = Integer.valueOf(resultSizeValueStr); + this.resultSizeValue = Integer.parseInt(resultSizeValueStr); this.queryParams = queryParams; this.authMethod = authMethod; this.authToken = authToken; this.resultOutputFormat = resultOutputFormat; - queryFormat = StringUtils.isNotBlank(resultFormatParam) ? "&" + resultFormatParam + "=" + resultFormatValue - : ""; - querySize = StringUtils.isNotBlank(resultSizeParam) ? "&" + resultSizeParam + "=" + resultSizeValueStr : ""; + this.queryFormat = StringUtils.isNotBlank(resultFormatParam) ? "&" + resultFormatParam + "=" + resultFormatValue + : ""; + this.querySize = StringUtils.isNotBlank(resultSizeParam) ? "&" + resultSizeParam + "=" + resultSizeValueStr : ""; try { initXmlTransformation(resultTotalXpath, resumptionXpath, entityXpath); - } catch (Exception e) { + } catch (final Exception e) { throw new IllegalStateException("xml transformation init failed: " + e.getMessage()); } initQueue(); } - private void initXmlTransformation(String resultTotalXpath, String resumptionXpath, String entityXpath) - throws TransformerConfigurationException, XPathExpressionException { + private void initXmlTransformation(final String resultTotalXpath, final String resumptionXpath, final String entityXpath) + throws TransformerConfigurationException, XPathExpressionException { final TransformerFactory factory = TransformerFactory.newInstance(); - transformer = factory.newTransformer(); - transformer.setOutputProperty(OutputKeys.INDENT, "yes"); - transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "3"); - xpath = XPathFactory.newInstance().newXPath(); - xprResultTotalPath = xpath.compile(resultTotalXpath); - xprResumptionPath = xpath.compile(StringUtils.isBlank(resumptionXpath) ? "/" : resumptionXpath); - xprEntity = xpath.compile(entityXpath); + 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(resultTotalXpath); + this.xprResumptionPath = this.xpath.compile(StringUtils.isBlank(resumptionXpath) ? "/" : resumptionXpath); + this.xprEntity = this.xpath.compile(entityXpath); } private void initQueue() { - query = baseUrl + "?" + queryParams + querySize + queryFormat; - log.info("REST calls starting with {}", query); + this.query = this.baseUrl + "?" + this.queryParams + this.querySize + this.queryFormat; + log.info("REST calls starting with {}", this.query); } private void disconnect() { @@ -150,127 +155,140 @@ public class RestIterator implements Iterator { /* * (non-Javadoc) + * * @see java.util.Iterator#hasNext() */ @Override public boolean hasNext() { - if (recordQueue.isEmpty() && query.isEmpty()) { + if (this.recordQueue.isEmpty() && this.query.isEmpty()) { disconnect(); return false; - } else { - return true; } + return true; } /* * (non-Javadoc) + * * @see java.util.Iterator#next() */ @Override public String next() { - synchronized (recordQueue) { - while (recordQueue.isEmpty() && !query.isEmpty()) { + synchronized (this.recordQueue) { + while (this.recordQueue.isEmpty() && !this.query.isEmpty()) { try { - query = downloadPage(query); - } catch (CollectorException e) { + this.query = downloadPage(this.query, 0); + } catch (final CollectorException e) { log.debug("CollectorPlugin.next()-Exception: {}", e); throw new RuntimeException(e); } } - return recordQueue.poll(); + return this.recordQueue.poll(); } } /* - * download page and return nextQuery + * download page and return nextQuery (with number of attempt) */ - private String downloadPage(String query) throws CollectorException { - String resultJson; - String resultXml = ""; - String nextQuery = ""; - String emptyXml = resultXml + "<" + JsonUtils.XML_WRAP_TAG + ">"; - Node resultNode = null; - NodeList nodeList = null; - String qUrlArgument = ""; - int urlOldResumptionSize = 0; - InputStream theHttpInputStream; + private String downloadPage(String query, final int attempt) throws CollectorException { - // check if cursor=* is initial set otherwise add it to the queryParam URL - if (resumptionType.equalsIgnoreCase("deep-cursor")) { - log.debug("check resumptionType deep-cursor and check cursor=*?{}", query); - if (!query.contains("&cursor=")) { - query += "&cursor=*"; + 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 { - log.info("requestig URL [{}]", query); + String resultJson; + String resultXml = ""; + String nextQuery = ""; + final String emptyXml = resultXml + "<" + JsonUtils.XML_WRAP_TAG + ">"; + Node resultNode = null; + NodeList nodeList = null; + String qUrlArgument = ""; + int urlOldResumptionSize = 0; + InputStream theHttpInputStream; - URL qUrl = new URL(query); - log.debug("authMethod: {}", authMethod); - if ("bearer".equalsIgnoreCase(this.authMethod)) { - log.trace("authMethod before inputStream: {}", resultXml); - HttpURLConnection conn = (HttpURLConnection) qUrl.openConnection(); - conn.setRequestProperty(HttpHeaders.AUTHORIZATION, "Bearer " + authToken); - conn.setRequestProperty(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); - conn.setRequestMethod("GET"); - theHttpInputStream = conn.getInputStream(); - } else if (BASIC.equalsIgnoreCase(this.authMethod)) { - log.trace("authMethod before inputStream: {}", resultXml); - HttpURLConnection conn = (HttpURLConnection) qUrl.openConnection(); - conn.setRequestProperty(HttpHeaders.AUTHORIZATION, "Basic " + authToken); - conn.setRequestProperty(HttpHeaders.ACCEPT, ContentType.APPLICATION_XML.getMimeType()); - conn.setRequestMethod("GET"); - theHttpInputStream = conn.getInputStream(); - } else { - theHttpInputStream = qUrl.openStream(); - } - - resultStream = theHttpInputStream; - if ("json".equals(resultOutputFormat)) { - resultJson = IOUtils.toString(resultStream, StandardCharsets.UTF_8); - resultXml = JsonUtils.convertToXML(resultJson); - resultStream = IOUtils.toInputStream(resultXml, UTF_8); - } - - if (!(emptyXml).equalsIgnoreCase(resultXml)) { - resultNode = (Node) xpath.evaluate("/", new InputSource(resultStream), XPathConstants.NODE); - nodeList = (NodeList) xprEntity.evaluate(resultNode, XPathConstants.NODESET); - log.debug("nodeList.length: {}", nodeList.getLength()); - for (int i = 0; i < nodeList.getLength(); i++) { - StringWriter sw = new StringWriter(); - transformer.transform(new DOMSource(nodeList.item(i)), new StreamResult(sw)); - String toEnqueue = sw.toString(); - if (toEnqueue == null || StringUtils.isBlank(toEnqueue) || emptyXml.equalsIgnoreCase(toEnqueue)) { - log.warn("The following record resulted in empty item for the feeding queue: {}", resultXml); - } else { - recordQueue.add(sw.toString()); - } + // check if cursor=* is initial set otherwise add it to the queryParam URL + if ("deep-cursor".equalsIgnoreCase(this.resumptionType)) { + log.debug("check resumptionType deep-cursor and check cursor=*?{}", query); + if (!query.contains("&cursor=")) { + query += "&cursor=*"; } - } else { - log.warn("resultXml is equal with emptyXml"); } - resumptionInt += resultSizeValue; + try { + log.info("requesting URL [{}]", query); - switch (resumptionType.toLowerCase()) { + final URL qUrl = new URL(query); + log.debug("authMethod: {}", this.authMethod); + if ("bearer".equalsIgnoreCase(this.authMethod)) { + log.trace("authMethod before inputStream: {}", resultXml); + final HttpURLConnection conn = (HttpURLConnection) qUrl.openConnection(); + conn.setRequestProperty(HttpHeaders.AUTHORIZATION, "Bearer " + this.authToken); + conn.setRequestProperty(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); + conn.setRequestMethod("GET"); + theHttpInputStream = conn.getInputStream(); + } else if (this.BASIC.equalsIgnoreCase(this.authMethod)) { + log.trace("authMethod before inputStream: {}", resultXml); + final HttpURLConnection conn = (HttpURLConnection) qUrl.openConnection(); + conn.setRequestProperty(HttpHeaders.AUTHORIZATION, "Basic " + this.authToken); + conn.setRequestProperty(HttpHeaders.ACCEPT, ContentType.APPLICATION_XML.getMimeType()); + conn.setRequestMethod("GET"); + theHttpInputStream = conn.getInputStream(); + } else { + theHttpInputStream = qUrl.openStream(); + } + + this.resultStream = theHttpInputStream; + if ("json".equals(this.resultOutputFormat)) { + resultJson = IOUtils.toString(this.resultStream, StandardCharsets.UTF_8); + resultXml = JsonUtils.convertToXML(resultJson); + this.resultStream = IOUtils.toInputStream(resultXml, UTF_8); + } + + if (!(emptyXml).equalsIgnoreCase(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) || emptyXml.equalsIgnoreCase(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.resultSizeValue; + + switch (this.resumptionType.toLowerCase()) { case "scan": // read of resumptionToken , evaluate next results, e.g. OAI, iterate over items - resumptionStr = xprResumptionPath.evaluate(resultNode); + this.resumptionStr = this.xprResumptionPath.evaluate(resultNode); break; case "count": // begin at one step for all records, iterate over items - resumptionStr = Integer.toString(resumptionInt); + this.resumptionStr = Integer.toString(this.resumptionInt); break; case "discover": // size of result items unknown, iterate over items (for openDOAR - 201808) - if (resultSizeValue < 2) { - throw new CollectorException("Mode: discover, Param 'resultSizeValue' is less than 2"); - } + if (this.resultSizeValue < 2) { throw new CollectorException("Mode: discover, Param 'resultSizeValue' is less than 2"); } qUrlArgument = qUrl.getQuery(); - String[] arrayQUrlArgument = qUrlArgument.split("&"); - for (String arrayUrlArgStr : arrayQUrlArgument) { - if (arrayUrlArgStr.startsWith(resumptionParam)) { - String[] resumptionKeyValue = arrayUrlArgStr.split("="); + final String[] arrayQUrlArgument = qUrlArgument.split("&"); + for (final String arrayUrlArgStr : arrayQUrlArgument) { + if (arrayUrlArgStr.startsWith(this.resumptionParam)) { + final String[] resumptionKeyValue = arrayUrlArgStr.split("="); if (isInteger(resumptionKeyValue[1])) { urlOldResumptionSize = Integer.parseInt(resumptionKeyValue[1]); log.debug("discover OldResumptionSize from Url (int): {}", urlOldResumptionSize); @@ -281,101 +299,104 @@ public class RestIterator implements Iterator { } if (((emptyXml).equalsIgnoreCase(resultXml)) - || ((nodeList != null) && (nodeList.getLength() < resultSizeValue))) { + || ((nodeList != null) && (nodeList.getLength() < this.resultSizeValue))) { // resumptionStr = ""; if (nodeList != null) { - discoverResultSize += nodeList.getLength(); + this.discoverResultSize += nodeList.getLength(); } - resultTotal = discoverResultSize; + this.resultTotal = this.discoverResultSize; } else { - resumptionStr = Integer.toString(resumptionInt); - resultTotal = resumptionInt + 1; + this.resumptionStr = Integer.toString(this.resumptionInt); + this.resultTotal = this.resumptionInt + 1; if (nodeList != null) { - discoverResultSize += nodeList.getLength(); + this.discoverResultSize += nodeList.getLength(); } } - log.info("discoverResultSize: {}", discoverResultSize); + log.info("discoverResultSize: {}", this.discoverResultSize); break; case "pagination": case "page": // pagination, iterate over page numbers - pagination += 1; + this.pagination += 1; if (nodeList != null) { - discoverResultSize += nodeList.getLength(); + this.discoverResultSize += nodeList.getLength(); } else { - resultTotal = discoverResultSize; - pagination = discoverResultSize; + this.resultTotal = this.discoverResultSize; + this.pagination = this.discoverResultSize; } - resumptionInt = pagination; - resumptionStr = Integer.toString(resumptionInt); + this.resumptionInt = this.pagination; + this.resumptionStr = Integer.toString(this.resumptionInt); break; case "deep-cursor": // size of result items unknown, iterate over items (for supporting deep cursor in - // solr) + // solr) // isn't relevant -- if (resultSizeValue < 2) {throw new CollectorServiceException("Mode: // deep-cursor, Param 'resultSizeValue' is less than 2");} - resumptionStr = encodeValue(xprResumptionPath.evaluate(resultNode)); - queryParams = queryParams.replace("&cursor=*", ""); + this.resumptionStr = encodeValue(this.xprResumptionPath.evaluate(resultNode)); + this.queryParams = this.queryParams.replace("&cursor=*", ""); // terminating if length of nodeList is 0 - if ((nodeList != null) && (nodeList.getLength() < discoverResultSize)) { - resumptionInt += (nodeList.getLength() + 1 - resultSizeValue); + if ((nodeList != null) && (nodeList.getLength() < this.discoverResultSize)) { + this.resumptionInt += ((nodeList.getLength() + 1) - this.resultSizeValue); } else { - resumptionInt += (nodeList.getLength() - resultSizeValue); // subtract the resultSizeValue - // because the iteration is over - // real length and the - // resultSizeValue is added before - // the switch() + this.resumptionInt += (nodeList.getLength() - this.resultSizeValue); // subtract the resultSizeValue + // because the iteration is over + // real length and the + // resultSizeValue is added before + // the switch() } - discoverResultSize = nodeList.getLength(); + this.discoverResultSize = nodeList.getLength(); log - .debug( - "downloadPage().deep-cursor: resumptionStr=" + resumptionStr + " ; queryParams=" - + queryParams + " resumptionLengthIncreased: " + resumptionInt); + .debug("downloadPage().deep-cursor: resumptionStr=" + this.resumptionStr + " ; queryParams=" + + this.queryParams + " resumptionLengthIncreased: " + this.resumptionInt); break; default: // otherwise: abort // resultTotal = resumptionInt; break; + } + + } catch (final Exception e) { + log.error(e.getMessage(), e); + throw new IllegalStateException("collection failed: " + e.getMessage()); } - } catch (Exception e) { - log.error(e.getMessage(), e); - throw new IllegalStateException("collection failed: " + e.getMessage()); - } - - try { - if (resultTotal == -1) { - resultTotal = Integer.parseInt(xprResultTotalPath.evaluate(resultNode)); - if (resumptionType.equalsIgnoreCase("page") && !BASIC.equalsIgnoreCase(authMethod)) { - resultTotal += 1; - } // to correct the upper bound - log.info("resultTotal was -1 is now: " + resultTotal); + try { + if (this.resultTotal == -1) { + this.resultTotal = Integer.parseInt(this.xprResultTotalPath.evaluate(resultNode)); + if ("page".equalsIgnoreCase(this.resumptionType) && !this.BASIC.equalsIgnoreCase(this.authMethod)) { + this.resultTotal += 1; + } // to correct the upper bound + 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()); } - } catch (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 + "?" + this.queryParams + this.querySize + "&" + this.resumptionParam + "=" + this.resumptionStr + + this.queryFormat; + } 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); } - log.debug("resultTotal: " + resultTotal); - log.debug("resInt: " + resumptionInt); - if (resumptionInt <= resultTotal) { - nextQuery = baseUrl + "?" + queryParams + querySize + "&" + resumptionParam + "=" + resumptionStr - + queryFormat; - } 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; } - private boolean isInteger(String s) { + private boolean isInteger(final String s) { boolean isValidInteger = false; try { Integer.parseInt(s); @@ -383,7 +404,7 @@ public class RestIterator implements Iterator { // s is a valid integer isValidInteger = true; - } catch (NumberFormatException ex) { + } catch (final NumberFormatException ex) { // s is not an integer } @@ -391,20 +412,20 @@ public class RestIterator implements Iterator { } // Method to encode a string value using `UTF-8` encoding scheme - private String encodeValue(String value) { + private String encodeValue(final String value) { try { return URLEncoder.encode(value, StandardCharsets.UTF_8.toString()); - } catch (UnsupportedEncodingException ex) { + } catch (final UnsupportedEncodingException ex) { throw new RuntimeException(ex.getCause()); } } public String getResultFormatValue() { - return resultFormatValue; + return this.resultFormatValue; } public String getResultOutputFormat() { - return resultOutputFormat; + return this.resultOutputFormat; } } From f4068de298af90e8d74463449d0df4ff2d0af55a Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 2 May 2024 09:51:33 +0200 Subject: [PATCH 11/26] code reindent + tests --- .../collection/plugin/rest/RestIterator.java | 211 ++++++++++-------- .../plugin/rest/OsfPreprintCollectorTest.java | 22 +- 2 files changed, 133 insertions(+), 100 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 c13f29806..76af6cff1 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 @@ -65,7 +65,8 @@ public class RestIterator implements Iterator { private final int resultSizeValue; 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 + 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; @@ -82,9 +83,9 @@ public class RestIterator implements Iterator { private int discoverResultSize = 0; private int pagination = 1; /* - * While resultFormatValue is added to the request parameter, this is used to say that the results are retrieved in json. useful for - * cases when the target API expects a resultFormatValue != json, but the results are returned in json. An example is the EU Open Data - * Portal API: resultFormatValue=standard, results are in json format. + * While resultFormatValue is added to the request parameter, this is used to say that the results are retrieved in + * json. useful for cases when the target API expects a resultFormatValue != json, but the results are returned in + * json. An example is the EU Open Data Portal API: resultFormatValue=standard, results are in json format. */ private final String resultOutputFormat; @@ -92,21 +93,21 @@ public class RestIterator implements Iterator { * RestIterator class compatible to version 1.3.33 */ public RestIterator( - final HttpClientParams clientParams, - final String baseUrl, - final String resumptionType, - final String resumptionParam, - final String resumptionXpath, - final String resultTotalXpath, - final String resultFormatParam, - final String resultFormatValue, - final String resultSizeParam, - final String resultSizeValueStr, - final String queryParams, - final String entityXpath, - final String authMethod, - final String authToken, - final String resultOutputFormat) { + final HttpClientParams clientParams, + final String baseUrl, + final String resumptionType, + final String resumptionParam, + final String resumptionXpath, + final String resultTotalXpath, + final String resultFormatParam, + final String resultFormatValue, + final String resultSizeParam, + final String resultSizeValueStr, + final String queryParams, + final String entityXpath, + final String authMethod, + final String authToken, + final String resultOutputFormat) { this.clientParams = clientParams; this.baseUrl = baseUrl; @@ -120,8 +121,9 @@ public class RestIterator implements Iterator { this.resultOutputFormat = resultOutputFormat; this.queryFormat = StringUtils.isNotBlank(resultFormatParam) ? "&" + resultFormatParam + "=" + resultFormatValue - : ""; - this.querySize = StringUtils.isNotBlank(resultSizeParam) ? "&" + resultSizeParam + "=" + resultSizeValueStr : ""; + : ""; + this.querySize = StringUtils.isNotBlank(resultSizeParam) ? "&" + resultSizeParam + "=" + resultSizeValueStr + : ""; try { initXmlTransformation(resultTotalXpath, resumptionXpath, entityXpath); @@ -132,8 +134,9 @@ public class RestIterator implements Iterator { initQueue(); } - private void initXmlTransformation(final String resultTotalXpath, final String resumptionXpath, final String entityXpath) - throws TransformerConfigurationException, XPathExpressionException { + private void initXmlTransformation(final String resultTotalXpath, final String resumptionXpath, + final String entityXpath) + throws TransformerConfigurationException, XPathExpressionException { final TransformerFactory factory = TransformerFactory.newInstance(); this.transformer = factory.newTransformer(); this.transformer.setOutputProperty(OutputKeys.INDENT, "yes"); @@ -155,7 +158,6 @@ public class RestIterator implements Iterator { /* * (non-Javadoc) - * * @see java.util.Iterator#hasNext() */ @Override @@ -169,7 +171,6 @@ public class RestIterator implements Iterator { /* * (non-Javadoc) - * * @see java.util.Iterator#next() */ @Override @@ -192,7 +193,9 @@ public class RestIterator implements Iterator { */ private String downloadPage(String query, 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, query:" + query); + } if (attempt > 0) { final int delay = (attempt * 5000); @@ -254,15 +257,19 @@ public class RestIterator implements Iterator { } if (!(emptyXml).equalsIgnoreCase(resultXml)) { - resultNode = (Node) this.xpath.evaluate("/", new InputSource(this.resultStream), XPathConstants.NODE); + 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) || emptyXml.equalsIgnoreCase(toEnqueue)) { - log.warn("The following record resulted in empty item for the feeding queue: {}", resultXml); + if ((toEnqueue == null) || StringUtils.isBlank(toEnqueue) + || emptyXml.equalsIgnoreCase(toEnqueue)) { + log + .warn( + "The following record resulted in empty item for the feeding queue: {}", resultXml); } else { this.recordQueue.add(sw.toString()); } @@ -274,90 +281,95 @@ public class RestIterator implements Iterator { this.resumptionInt += this.resultSizeValue; switch (this.resumptionType.toLowerCase()) { - case "scan": // read of resumptionToken , evaluate next results, e.g. OAI, iterate over items - this.resumptionStr = this.xprResumptionPath.evaluate(resultNode); - break; + case "scan": // read of resumptionToken , evaluate next results, e.g. OAI, iterate over items + this.resumptionStr = this.xprResumptionPath.evaluate(resultNode); + break; - case "count": // begin at one step for all records, iterate over items - this.resumptionStr = Integer.toString(this.resumptionInt); - break; + case "count": // begin at one step for all records, iterate over items + this.resumptionStr = Integer.toString(this.resumptionInt); + break; - case "discover": // size of result items unknown, iterate over items (for openDOAR - 201808) - if (this.resultSizeValue < 2) { 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)) { - final String[] resumptionKeyValue = arrayUrlArgStr.split("="); - if (isInteger(resumptionKeyValue[1])) { - urlOldResumptionSize = Integer.parseInt(resumptionKeyValue[1]); - log.debug("discover OldResumptionSize from Url (int): {}", urlOldResumptionSize); - } else { - log.debug("discover OldResumptionSize from Url (str): {}", resumptionKeyValue[1]); + case "discover": // size of result items unknown, iterate over items (for openDOAR - 201808) + if (this.resultSizeValue < 2) { + 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)) { + final String[] resumptionKeyValue = arrayUrlArgStr.split("="); + if (isInteger(resumptionKeyValue[1])) { + urlOldResumptionSize = Integer.parseInt(resumptionKeyValue[1]); + log.debug("discover OldResumptionSize from Url (int): {}", urlOldResumptionSize); + } else { + log.debug("discover OldResumptionSize from Url (str): {}", resumptionKeyValue[1]); + } } } - } - if (((emptyXml).equalsIgnoreCase(resultXml)) + if (((emptyXml).equalsIgnoreCase(resultXml)) || ((nodeList != null) && (nodeList.getLength() < this.resultSizeValue))) { - // resumptionStr = ""; + // resumptionStr = ""; + if (nodeList != null) { + this.discoverResultSize += nodeList.getLength(); + } + this.resultTotal = this.discoverResultSize; + } else { + this.resumptionStr = Integer.toString(this.resumptionInt); + this.resultTotal = this.resumptionInt + 1; + if (nodeList != null) { + this.discoverResultSize += nodeList.getLength(); + } + } + log.info("discoverResultSize: {}", this.discoverResultSize); + break; + + case "pagination": + case "page": // pagination, iterate over page numbers + this.pagination += 1; if (nodeList != null) { this.discoverResultSize += nodeList.getLength(); + } else { + this.resultTotal = this.discoverResultSize; + this.pagination = this.discoverResultSize; } - this.resultTotal = this.discoverResultSize; - } else { + this.resumptionInt = this.pagination; this.resumptionStr = Integer.toString(this.resumptionInt); - this.resultTotal = this.resumptionInt + 1; - if (nodeList != null) { - this.discoverResultSize += nodeList.getLength(); + break; + + case "deep-cursor": // size of result items unknown, iterate over items (for supporting deep cursor + // in + // solr) + // isn't relevant -- if (resultSizeValue < 2) {throw new CollectorServiceException("Mode: + // deep-cursor, Param 'resultSizeValue' is less than 2");} + + this.resumptionStr = encodeValue(this.xprResumptionPath.evaluate(resultNode)); + this.queryParams = this.queryParams.replace("&cursor=*", ""); + + // terminating if length of nodeList is 0 + if ((nodeList != null) && (nodeList.getLength() < this.discoverResultSize)) { + this.resumptionInt += ((nodeList.getLength() + 1) - this.resultSizeValue); + } else { + this.resumptionInt += (nodeList.getLength() - this.resultSizeValue); // subtract the + // resultSizeValue + // because the iteration is over + // real length and the + // resultSizeValue is added before + // the switch() } - } - log.info("discoverResultSize: {}", this.discoverResultSize); - break; - case "pagination": - case "page": // pagination, iterate over page numbers - this.pagination += 1; - if (nodeList != null) { - this.discoverResultSize += nodeList.getLength(); - } else { - this.resultTotal = this.discoverResultSize; - this.pagination = this.discoverResultSize; - } - this.resumptionInt = this.pagination; - this.resumptionStr = Integer.toString(this.resumptionInt); - break; + this.discoverResultSize = nodeList.getLength(); - case "deep-cursor": // size of result items unknown, iterate over items (for supporting deep cursor in - // solr) - // isn't relevant -- if (resultSizeValue < 2) {throw new CollectorServiceException("Mode: - // deep-cursor, Param 'resultSizeValue' is less than 2");} - - this.resumptionStr = encodeValue(this.xprResumptionPath.evaluate(resultNode)); - this.queryParams = this.queryParams.replace("&cursor=*", ""); - - // terminating if length of nodeList is 0 - if ((nodeList != null) && (nodeList.getLength() < this.discoverResultSize)) { - this.resumptionInt += ((nodeList.getLength() + 1) - this.resultSizeValue); - } else { - this.resumptionInt += (nodeList.getLength() - this.resultSizeValue); // subtract the resultSizeValue - // because the iteration is over - // real length and the - // resultSizeValue is added before - // the switch() - } - - this.discoverResultSize = nodeList.getLength(); - - log - .debug("downloadPage().deep-cursor: resumptionStr=" + this.resumptionStr + " ; queryParams=" + log + .debug( + "downloadPage().deep-cursor: resumptionStr=" + this.resumptionStr + " ; queryParams=" + this.queryParams + " resumptionLengthIncreased: " + this.resumptionInt); - break; + break; - default: // otherwise: abort - // resultTotal = resumptionInt; - break; + default: // otherwise: abort + // resultTotal = resumptionInt; + break; } } catch (final Exception e) { @@ -380,8 +392,9 @@ public class RestIterator implements Iterator { log.debug("resultTotal: " + this.resultTotal); log.debug("resInt: " + this.resumptionInt); if (this.resumptionInt <= this.resultTotal) { - nextQuery = this.baseUrl + "?" + this.queryParams + this.querySize + "&" + this.resumptionParam + "=" + this.resumptionStr - + this.queryFormat; + nextQuery = this.baseUrl + "?" + this.queryParams + this.querySize + "&" + this.resumptionParam + "=" + + this.resumptionStr + + this.queryFormat; } else { nextQuery = ""; // if (resumptionType.toLowerCase().equals("deep-cursor")) { resumptionInt -= 1; } // correct the 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 bc2d12661..90f4c7f25 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 @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.collection.plugin.rest; import java.util.HashMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Stream; import org.junit.jupiter.api.Assertions; @@ -69,7 +70,7 @@ public class OsfPreprintCollectorTest { @Test @Disabled - void test() throws CollectorException { + void test_limited() throws CollectorException { final AtomicInteger i = new AtomicInteger(0); final Stream stream = this.rcp.collect(this.api, new AggregatorReport()); @@ -82,4 +83,23 @@ public class OsfPreprintCollectorTest { log.info("{}", i.intValue()); Assertions.assertTrue(i.intValue() > 0); } + + @Test + @Disabled + void test_all() throws CollectorException { + final AtomicLong i = new AtomicLong(0); + final Stream stream = this.rcp.collect(this.api, new AggregatorReport()); + + stream.forEach(s -> { + Assertions.assertTrue(s.length() > 0); + if ((i.incrementAndGet() % 1000) == 0) { + log.info("COLLECTED: {}", i.get()); + } + + }); + + log.info("TOTAL: {}", i.get()); + Assertions.assertTrue(i.get() > 0); + } + } From 69c5efbd8b2015f993a04205e117cbb4b204f0e2 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Fri, 3 May 2024 13:57:56 +0200 Subject: [PATCH 12/26] Fix: when applying enrichments with no instance information the resulting merge entity was generated with no instance instead of keeping the original information --- .../java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 9eb1ec01d..28db94766 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 @@ -874,9 +874,11 @@ public class MergeUtils { if (toEnrichInstances == null) { return enrichmentResult; } - if (enrichmentInstances == null) { - return enrichmentResult; + + if (enrichmentInstances == null || enrichmentInstances.isEmpty()) { + return toEnrichInstances; } + Map ri = toInstanceMap(enrichmentInstances); toEnrichInstances.forEach(i -> { From e1a0fb89334da1f6f8944c1138f3f9ba841e6493 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 3 May 2024 14:14:18 +0200 Subject: [PATCH 13/26] fixed id prefix creation for the fosnodoi records --- .../createunresolvedentities/PrepareFOSSparkJob.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 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 ffcaedda7..dd85f6a4e 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 @@ -80,9 +80,10 @@ public class PrepareFOSSparkJob implements Serializable { fosDataset .groupByKey((MapFunction) v -> v.getOaid().toLowerCase(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, it) -> { - return getResult(ModelSupport.getIdPrefix(Result.class) + "|" + k, it); - }, Encoders.bean(Result.class)) + .mapGroups( + (MapGroupsFunction) (k, + it) -> getResult(ModelSupport.entityIdPrefix.get(Result.class.getSimpleName()) + "|" + k, it), + Encoders.bean(Result.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") From a5d13d5d2777f36124a86a563a18052d3b41c2a2 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 3 May 2024 14:14:34 +0200 Subject: [PATCH 14/26] code formatting --- .../eu/dnetlib/pace/common/PaceCommonUtils.java | 15 ++++++++------- .../main/java/eu/dnetlib/pace/model/Person.java | 11 ++++++----- .../java/eu/dnetlib/pace/util/Capitalise.java | 3 ++- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/pace/common/PaceCommonUtils.java b/dhp-common/src/main/java/eu/dnetlib/pace/common/PaceCommonUtils.java index a279271b5..61fbc2470 100644 --- a/dhp-common/src/main/java/eu/dnetlib/pace/common/PaceCommonUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/pace/common/PaceCommonUtils.java @@ -1,19 +1,20 @@ package eu.dnetlib.pace.common; -import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; -import com.ibm.icu.text.Transliterator; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; - import java.nio.charset.StandardCharsets; import java.text.Normalizer; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; + +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import com.ibm.icu.text.Transliterator; + /** * Set of common functions for the framework * diff --git a/dhp-common/src/main/java/eu/dnetlib/pace/model/Person.java b/dhp-common/src/main/java/eu/dnetlib/pace/model/Person.java index c95c9d823..6a1957183 100644 --- a/dhp-common/src/main/java/eu/dnetlib/pace/model/Person.java +++ b/dhp-common/src/main/java/eu/dnetlib/pace/model/Person.java @@ -1,20 +1,21 @@ package eu.dnetlib.pace.model; +import java.nio.charset.Charset; +import java.text.Normalizer; +import java.util.List; +import java.util.Set; + import com.google.common.base.Joiner; import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.hash.Hashing; + import eu.dnetlib.pace.common.PaceCommonUtils; import eu.dnetlib.pace.util.Capitalise; import eu.dnetlib.pace.util.DotAbbreviations; -import java.nio.charset.Charset; -import java.text.Normalizer; -import java.util.List; -import java.util.Set; - public class Person { private static final String UTF8 = "UTF-8"; diff --git a/dhp-common/src/main/java/eu/dnetlib/pace/util/Capitalise.java b/dhp-common/src/main/java/eu/dnetlib/pace/util/Capitalise.java index 015386423..671320c71 100644 --- a/dhp-common/src/main/java/eu/dnetlib/pace/util/Capitalise.java +++ b/dhp-common/src/main/java/eu/dnetlib/pace/util/Capitalise.java @@ -1,9 +1,10 @@ package eu.dnetlib.pace.util; -import com.google.common.base.Function; import org.apache.commons.lang3.text.WordUtils; +import com.google.common.base.Function; + public class Capitalise implements Function { private final char[] DELIM = { From 04862271850f22c92145e878005b62217af8d1d2 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 3 May 2024 14:31:12 +0200 Subject: [PATCH 15/26] [cleaning] deactivating the cleaning of FOS subjects found in the metadata provided by repositories --- .../dhp/oa/graph/clean/CleaningRuleMap.java | 38 ++++++++++++++----- 1 file changed, 28 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningRuleMap.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningRuleMap.java index 807055adb..732471f99 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningRuleMap.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningRuleMap.java @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.graph.clean; import java.io.Serializable; import java.util.HashMap; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.lang3.SerializationUtils; @@ -29,7 +30,10 @@ public class CleaningRuleMap extends HashMap, SerializableConsumer cleanQualifier(vocabularies, (AccessRight) o)); mapping.put(Country.class, o -> cleanCountry(vocabularies, (Country) o)); mapping.put(Relation.class, o -> cleanRelation(vocabularies, (Relation) o)); - mapping.put(Subject.class, o -> cleanSubject(vocabularies, (Subject) o)); + + // commenting out the subject cleaning until we decide if we want to it or not and the implementation will + // be completed. At the moment it is not capable of expanding the whole hierarchy. + // mapping.put(Subject.class, o -> cleanSubject(vocabularies, (Subject) o)); return mapping; } @@ -38,8 +42,15 @@ public class CleaningRuleMap extends HashMap, SerializableConsumer { if (ModelConstants.DNET_SUBJECT_KEYWORD.equalsIgnoreCase(subject.getQualifier().getClassid())) { @@ -49,14 +60,21 @@ public class CleaningRuleMap extends HashMap, SerializableConsumer Date: Fri, 3 May 2024 15:53:52 +0200 Subject: [PATCH 16/26] fixed id prefix creation for the fosnodoi records, again --- .../createunresolvedentities/PrepareFOSSparkJob.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 dd85f6a4e..c248423d4 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 @@ -82,7 +82,8 @@ public class PrepareFOSSparkJob implements Serializable { .groupByKey((MapFunction) v -> v.getOaid().toLowerCase(), Encoders.STRING()) .mapGroups( (MapGroupsFunction) (k, - it) -> getResult(ModelSupport.entityIdPrefix.get(Result.class.getSimpleName()) + "|" + k, it), + it) -> getResult( + ModelSupport.entityIdPrefix.get(Result.class.getSimpleName().toLowerCase()) + "|" + k, it), Encoders.bean(Result.class)) .write() .mode(SaveMode.Overwrite) From 711048ceedc99383c291bc532373e09294fe0815 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Tue, 7 May 2024 15:44:33 +0200 Subject: [PATCH 17/26] PrepareRelationsJob rewritten to use Spark Dataframe API and Windowing functions --- .../dhp/oa/provision/PrepareRelationsJob.java | 190 ++++-------------- 1 file changed, 38 insertions(+), 152 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java index fdf397ad7..c2eb8c408 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java @@ -1,43 +1,31 @@ package eu.dnetlib.dhp.oa.provision; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.util.HashSet; -import java.util.Optional; -import java.util.PriorityQueue; -import java.util.Set; -import java.util.stream.Collectors; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; -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.api.java.function.FlatMapFunction; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.sql.Encoder; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.expressions.Aggregator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; import com.google.common.collect.Sets; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; -import eu.dnetlib.dhp.oa.provision.model.SortableRelationKey; -import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner; import eu.dnetlib.dhp.schema.oaf.Relation; -import scala.Tuple2; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.expressions.Window; +import org.apache.spark.sql.expressions.WindowSpec; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static org.apache.spark.sql.functions.col; /** * PrepareRelationsJob prunes the relationships: only consider relationships that are not virtually deleted @@ -130,130 +118,28 @@ public class PrepareRelationsJob { private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) { - JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath) - .filter(rel -> !(rel.getSource().startsWith("unresolved") || rel.getTarget().startsWith("unresolved"))) - .filter(rel -> !rel.getDataInfo().getDeletedbyinference()) - .filter(rel -> !relationFilter.contains(StringUtils.lowerCase(rel.getRelClass()))); + WindowSpec source_w = Window + .partitionBy("source", "subRelType") + .orderBy(col("target").desc_nulls_last()); - JavaRDD pruned = pruneRels( - pruneRels( - rels, - sourceMaxRelations, relPartitions, (Function) Relation::getSource), - targetMaxRelations, relPartitions, (Function) Relation::getTarget); - spark - .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + WindowSpec target_w = Window + .partitionBy("target", "subRelType") + .orderBy(col("source").desc_nulls_last()); - private static JavaRDD pruneRels(JavaRDD rels, int maxRelations, - int relPartitions, Function idFn) { - return rels - .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) - .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) - .groupBy(Tuple2::_1) - .map(Tuple2::_2) - .map(t -> Iterables.limit(t, maxRelations)) - .flatMap(Iterable::iterator) - .map(Tuple2::_2); - } - - // experimental - private static void prepareRelationsDataset( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, - int relPartitions) { - spark - .read() - .textFile(inputRelationsPath) - .repartition(relPartitions) - .map( - (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), - Encoders.kryo(Relation.class)) - .filter((FilterFunction) rel -> !rel.getDataInfo().getDeletedbyinference()) - .filter((FilterFunction) rel -> !relationFilter.contains(rel.getRelClass())) - .groupByKey( - (MapFunction) Relation::getSource, - Encoders.STRING()) - .agg(new RelationAggregator(maxRelations).toColumn()) - .flatMap( - (FlatMapFunction, Relation>) t -> Iterables - .limit(t._2().getRelations(), maxRelations) - .iterator(), - Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } - - public static class RelationAggregator - extends Aggregator { - - private final int maxRelations; - - public RelationAggregator(int maxRelations) { - this.maxRelations = maxRelations; - } - - @Override - public RelationList zero() { - return new RelationList(); - } - - @Override - public RelationList reduce(RelationList b, Relation a) { - b.getRelations().add(a); - return getSortableRelationList(b); - } - - @Override - public RelationList merge(RelationList b1, RelationList b2) { - b1.getRelations().addAll(b2.getRelations()); - return getSortableRelationList(b1); - } - - @Override - public RelationList finish(RelationList r) { - return getSortableRelationList(r); - } - - private RelationList getSortableRelationList(RelationList b1) { - RelationList sr = new RelationList(); - sr - .setRelations( - b1 - .getRelations() - .stream() - .limit(maxRelations) - .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); - return sr; - } - - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(RelationList.class); - } - - @Override - public Encoder outputEncoder() { - return Encoders.kryo(RelationList.class); - } - } - - /** - * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text - * file, - * - * @param spark - * @param inputPath - * @return the JavaRDD containing all the relationships - */ - private static JavaRDD readPathRelationRDD( - SparkSession spark, final String inputPath) { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class)); + spark.read().schema(Encoders.bean(Relation.class).schema()).json(inputRelationsPath) + .where("source NOT LIKE 'unresolved%' AND target NOT LIKE 'unresolved%'") + .where("datainfo.deletedbyinference != true") + .where(relationFilter.isEmpty() ? "" : "lower(relClass) NOT IN ("+ Joiner.on(',').join(relationFilter) +")") + .withColumn("source_w_pos", functions.row_number().over(source_w)) + .where("source_w_pos < " + sourceMaxRelations ) + .drop("source_w_pos") + .withColumn("target_w_pos", functions.row_number().over(target_w)) + .where("target_w_pos < " + targetMaxRelations) + .drop( "target_w_pos") + .coalesce(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); } private static void removeOutputDir(SparkSession spark, String path) { From b4e33894322d1693460be2cfcf0afb23d3b9135f Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 7 May 2024 16:25:17 +0200 Subject: [PATCH 18/26] fixed property mapping creating the RelatedEntity transient objects. spark cores & memory adjustments. Code formatting --- .../CreateRelatedEntitiesJob_phase1.java | 9 ++- .../dhp/oa/provision/PrepareRelationsJob.java | 72 +++++++++++-------- .../dhp/oa/provision/oozie_app/workflow.xml | 10 +-- 3 files changed, 54 insertions(+), 37 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 da80deee0..63f3c2ead 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 @@ -153,10 +153,15 @@ public class CreateRelatedEntitiesJob_phase1 { result .getTitle() .stream() + .filter(t -> StringUtils.isNotBlank(t.getValue())) .findFirst() - .map(StructuredProperty::getValue) .ifPresent( - title -> re.getTitle().setValue(StringUtils.left(title, ModelHardLimits.MAX_TITLE_LENGTH))); + title -> { + re.setTitle(title); + re + .getTitle() + .setValue(StringUtils.left(title.getValue(), ModelHardLimits.MAX_TITLE_LENGTH)); + }); } if (Objects.nonNull(result.getDescription()) && !result.getDescription().isEmpty()) { result diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java index c2eb8c408..f50c7774b 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java @@ -1,14 +1,15 @@ package eu.dnetlib.dhp.oa.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; -import com.google.common.collect.Sets; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; -import eu.dnetlib.dhp.schema.oaf.Relation; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static org.apache.spark.sql.functions.col; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +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.sql.Encoders; @@ -20,12 +21,15 @@ import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashSet; -import java.util.Optional; -import java.util.Set; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import com.google.common.collect.Sets; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import static org.apache.spark.sql.functions.col; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; +import eu.dnetlib.dhp.schema.oaf.Relation; /** * PrepareRelationsJob prunes the relationships: only consider relationships that are not virtually deleted @@ -119,27 +123,33 @@ public class PrepareRelationsJob { Set relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) { WindowSpec source_w = Window - .partitionBy("source", "subRelType") - .orderBy(col("target").desc_nulls_last()); + .partitionBy("source", "subRelType") + .orderBy(col("target").desc_nulls_last()); WindowSpec target_w = Window - .partitionBy("target", "subRelType") - .orderBy(col("source").desc_nulls_last()); + .partitionBy("target", "subRelType") + .orderBy(col("source").desc_nulls_last()); - spark.read().schema(Encoders.bean(Relation.class).schema()).json(inputRelationsPath) - .where("source NOT LIKE 'unresolved%' AND target NOT LIKE 'unresolved%'") - .where("datainfo.deletedbyinference != true") - .where(relationFilter.isEmpty() ? "" : "lower(relClass) NOT IN ("+ Joiner.on(',').join(relationFilter) +")") - .withColumn("source_w_pos", functions.row_number().over(source_w)) - .where("source_w_pos < " + sourceMaxRelations ) - .drop("source_w_pos") - .withColumn("target_w_pos", functions.row_number().over(target_w)) - .where("target_w_pos < " + targetMaxRelations) - .drop( "target_w_pos") - .coalesce(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); + spark + .read() + .schema(Encoders.bean(Relation.class).schema()) + .json(inputRelationsPath) + .where("source NOT LIKE 'unresolved%' AND target NOT LIKE 'unresolved%'") + .where("datainfo.deletedbyinference != true") + .where( + relationFilter.isEmpty() ? "" + : "lower(relClass) NOT IN (" + + relationFilter.stream().map(s -> "'" + s + "'").collect(Collectors.joining(",")) + ")") + .withColumn("source_w_pos", functions.row_number().over(source_w)) + .where("source_w_pos < " + sourceMaxRelations) + .drop("source_w_pos") + .withColumn("target_w_pos", functions.row_number().over(target_w)) + .where("target_w_pos < " + targetMaxRelations) + .drop("target_w_pos") + .coalesce(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); } private static void removeOutputDir(SparkSession spark, String path) { 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 eb446ddd8..434b4c9af 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 @@ -144,21 +144,23 @@ eu.dnetlib.dhp.oa.provision.PrepareRelationsJob dhp-graph-provision-${projectVersion}.jar - --executor-cores=${sparkExecutorCoresForJoining} - --executor-memory=${sparkExecutorMemoryForJoining} + --executor-cores=4 + --executor-memory=6G --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.executor.memoryOverhead=6G --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=15000 + --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${inputGraphRootPath}/relation --outputPath${workingDir}/relation --sourceMaxRelations${sourceMaxRelations} --targetMaxRelations${targetMaxRelations} --relationFilter${relationFilter} - --relPartitions5000 + --relPartitions15000 From 18aa323ee972c8b0565273ada553892f0568f83e Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 8 May 2024 11:36:46 +0200 Subject: [PATCH 19/26] cleanup unused classes, adjustments in the oozie wf definition --- .../dhp/oa/provision/RelationComparator.java | 44 ---------- .../dhp/oa/provision/RelationList.java | 25 ------ .../dhp/oa/provision/SortableRelation.java | 81 ------------------- .../model/ProvisionModelSupport.java | 10 +-- .../dhp/oa/provision/oozie_app/workflow.xml | 11 +-- 5 files changed, 7 insertions(+), 164 deletions(-) delete mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java delete mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java delete mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java deleted file mode 100644 index e13bc60eb..000000000 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java +++ /dev/null @@ -1,44 +0,0 @@ - -package eu.dnetlib.dhp.oa.provision; - -import java.util.Comparator; -import java.util.Map; -import java.util.Optional; - -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Maps; - -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Relation; - -public class RelationComparator implements Comparator { - - private static final Map weights = Maps.newHashMap(); - - static { - weights.put(ModelConstants.OUTCOME, 0); - weights.put(ModelConstants.SUPPLEMENT, 1); - weights.put(ModelConstants.REVIEW, 2); - weights.put(ModelConstants.CITATION, 3); - weights.put(ModelConstants.AFFILIATION, 4); - weights.put(ModelConstants.RELATIONSHIP, 5); - weights.put(ModelConstants.PUBLICATION_DATASET, 6); - weights.put(ModelConstants.SIMILARITY, 7); - - weights.put(ModelConstants.PROVISION, 8); - weights.put(ModelConstants.PARTICIPATION, 9); - weights.put(ModelConstants.DEDUP, 10); - } - - private Integer getWeight(Relation o) { - return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE); - } - - @Override - public int compare(Relation o1, Relation o2) { - return ComparisonChain - .start() - .compare(getWeight(o1), getWeight(o2)) - .result(); - } -} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java deleted file mode 100644 index 6e5fd7dba..000000000 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java +++ /dev/null @@ -1,25 +0,0 @@ - -package eu.dnetlib.dhp.oa.provision; - -import java.io.Serializable; -import java.util.PriorityQueue; -import java.util.Queue; - -import eu.dnetlib.dhp.schema.oaf.Relation; - -public class RelationList implements Serializable { - - private Queue relations; - - public RelationList() { - this.relations = new PriorityQueue<>(new RelationComparator()); - } - - public Queue getRelations() { - return relations; - } - - public void setRelations(Queue relations) { - this.relations = relations; - } -} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java deleted file mode 100644 index 8740b47fc..000000000 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java +++ /dev/null @@ -1,81 +0,0 @@ - -package eu.dnetlib.dhp.oa.provision; - -import java.io.Serializable; -import java.util.Map; -import java.util.Optional; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Maps; - -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Relation; - -public class SortableRelation extends Relation implements Comparable, Serializable { - - private static final Map weights = Maps.newHashMap(); - - static { - weights.put(ModelConstants.OUTCOME, 0); - weights.put(ModelConstants.SUPPLEMENT, 1); - weights.put(ModelConstants.REVIEW, 2); - weights.put(ModelConstants.CITATION, 3); - weights.put(ModelConstants.AFFILIATION, 4); - weights.put(ModelConstants.RELATIONSHIP, 5); - weights.put(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID, 6); - weights.put(ModelConstants.SIMILARITY, 7); - - weights.put(ModelConstants.PROVISION, 8); - weights.put(ModelConstants.PARTICIPATION, 9); - weights.put(ModelConstants.DEDUP, 10); - } - - private static final long serialVersionUID = 34753984579L; - - private String groupingKey; - - public static SortableRelation create(Relation r, String groupingKey) { - SortableRelation sr = new SortableRelation(); - sr.setGroupingKey(groupingKey); - sr.setSource(r.getSource()); - sr.setTarget(r.getTarget()); - sr.setRelType(r.getRelType()); - sr.setSubRelType(r.getSubRelType()); - sr.setRelClass(r.getRelClass()); - sr.setDataInfo(r.getDataInfo()); - sr.setCollectedfrom(r.getCollectedfrom()); - sr.setLastupdatetimestamp(r.getLastupdatetimestamp()); - sr.setProperties(r.getProperties()); - sr.setValidated(r.getValidated()); - sr.setValidationDate(r.getValidationDate()); - - return sr; - } - - @JsonIgnore - public Relation asRelation() { - return this; - } - - @Override - public int compareTo(SortableRelation o) { - return ComparisonChain - .start() - .compare(getGroupingKey(), o.getGroupingKey()) - .compare(getWeight(this), getWeight(o)) - .result(); - } - - private Integer getWeight(SortableRelation o) { - return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE); - } - - public String getGroupingKey() { - return groupingKey; - } - - public void setGroupingKey(String groupingKey) { - this.groupingKey = groupingKey; - } -} 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 0e6e95de5..10a99704c 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 @@ -1,8 +1,6 @@ package eu.dnetlib.dhp.oa.provision.model; -import static org.apache.commons.lang3.StringUtils.substringBefore; - import java.io.StringReader; import java.util.*; import java.util.stream.Collectors; @@ -16,12 +14,9 @@ import org.jetbrains.annotations.Nullable; import com.google.common.base.Splitter; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.common.vocabulary.VocabularyTerm; -import eu.dnetlib.dhp.oa.provision.RelationList; -import eu.dnetlib.dhp.oa.provision.SortableRelation; import eu.dnetlib.dhp.oa.provision.utils.ContextDef; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.schema.common.ModelSupport; @@ -55,10 +50,7 @@ public class ProvisionModelSupport { .newArrayList( RelatedEntityWrapper.class, JoinedEntity.class, - RelatedEntity.class, - SortableRelationKey.class, - SortableRelation.class, - RelationList.class)); + RelatedEntity.class)); return modelClasses.toArray(new Class[] {}); } 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 434b4c9af..1fc28e7ca 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 @@ -125,7 +125,7 @@ ${wf:conf('resumeFrom') eq 'prepare_relations'} ${wf:conf('resumeFrom') eq 'fork_join_related_entities'} ${wf:conf('resumeFrom') eq 'fork_join_all_entities'} - ${wf:conf('resumeFrom') eq 'convert_to_xml'} + ${wf:conf('resumeFrom') eq 'create_payloads'} ${wf:conf('resumeFrom') eq 'drop_solr_collection'} ${wf:conf('resumeFrom') eq 'to_solr_index'} @@ -587,19 +587,20 @@ - + - + yarn cluster - convert_to_xml + create_payloads eu.dnetlib.dhp.oa.provision.XmlConverterJob dhp-graph-provision-${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} @@ -607,7 +608,7 @@ --conf spark.sql.shuffle.partitions=3840 --conf spark.network.timeout=${sparkNetworkTimeout} - --inputPath${workingDir}/join_entities + --inputPath/user/claudio.atzori/data/beta_provision/join_entities --outputPath${workingDir}/xml_json --contextApiBaseUrl${contextApiBaseUrl} --isLookupUrl${isLookupUrl} From 90a4fb3547af243dd2960127d23ac28dd32bcfb9 Mon Sep 17 00:00:00 2001 From: Antonis Lempesis Date: Wed, 8 May 2024 13:17:58 +0300 Subject: [PATCH 20/26] fixed typos --- .../oozie_app/scripts/step16-createIndicatorsTables.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 78bea9126..f5b950fe8 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 @@ -249,7 +249,7 @@ create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet a left semi join dd on dd.id=pd.datasource union all select ra.id, 1 as is_gold - from ${stats_db_name}.result_accessroute ra on ra.id = pd.id where ra.accessroute = 'gold') tmp on tmp.id=pd.id; /*EOS*/ + from ${stats_db_name}.result_accessroute ra where ra.accessroute = 'gold') tmp on tmp.id=pd.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc stored as parquet as @@ -294,7 +294,7 @@ left outer join ( join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=p.id left outer join ${stats_db_name}.result_accessroute ra on ra.id=p.id where indi_gold.is_gold=0 and - ((d.type like '%Journal%' and ri.accessright not in ('Closed Access', 'Restricted', 'Not Available') and ri.license is not null) or ra.accessroute='hybrid')) tmp on pd.i=tmp.id; /*EOS*/ + ((d.type like '%Journal%' and ri.accessright not in ('Closed Access', 'Restricted', 'Not Available') and ri.license is not null) or ra.accessroute='hybrid')) tmp on p.id=tmp.id; /*EOS*/ drop table if exists ${stats_db_name}.indi_org_fairness purge; /*EOS*/ create table if not exists ${stats_db_name}.indi_org_fairness stored as parquet as @@ -1006,14 +1006,14 @@ 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 * +select distinct id, country from ( select ro.id, o.country from ${stats_db_name}.result_organization ro left outer join ${stats_db_name}.organization o on o.id=ro.organization union all select rp.id, f.country - from ${stats_db_name}.result_projects + from ${stats_db_name}.result_projects rp left outer join ${stats_db_name}.project p on p.id=rp.project left outer join ${stats_db_name}.funder f on f.name=p.funder ) rc From 0cada3cc8f502b4528bb6fdef06e7a5c032dbc68 Mon Sep 17 00:00:00 2001 From: antleb Date: Wed, 8 May 2024 13:42:53 +0300 Subject: [PATCH 21/26] every step is run in the analytics queue. Hardcoded for now, will make a parameter later --- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql | 1 + .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql | 4 +++- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql | 4 +++- .../oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql | 4 +++- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql | 4 +++- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql | 4 +++- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml | 3 ++- 7 files changed, 18 insertions(+), 6 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 f50c13521..7bad34e86 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,3 +1,4 @@ +set mapred.job.queue.name=analytics; ------------------------------------------------------ ------------------------------------------------------ -- 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 ce6b6cc2f..65a5d789f 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,3 +1,5 @@ +set mapred.job.queue.name=analytics; + ------------------------------------------------------ ------------------------------------------------------ -- Additional relations @@ -104,4 +106,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; 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..e3d910454 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,3 +1,5 @@ +set mapred.job.queue.name=analytics; + ------------------------------------------- --- Extra tables, mostly used by indicators @@ -63,4 +65,4 @@ from ( join ${stats_db_name}.result res on res.id=r.id where r.amount is not null; -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; 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..c837ea579 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,3 +1,5 @@ +set mapred.job.queue.name=analytics; + ---------------------------------------------------- -- Shortcuts for various definitions in stats db --- ---------------------------------------------------- @@ -25,4 +27,4 @@ drop table if exists ${stats_db_name}.result_gold purge; 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; 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..fe3bb6799 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,3 +1,5 @@ +set mapred.job.queue.name=analytics; + -- 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; @@ -53,4 +55,4 @@ LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; 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 ${stats_db_name}.result_tmp; 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..4f7247e14 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,3 +1,5 @@ +set mapred.job.queue.name=analytics; + -------------------------------------------------------------- -------------------------------------------------------------- -- Publication table/view and Publication related tables/views @@ -111,4 +113,4 @@ SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type= 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; 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..d5f9ae886 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 @@ -368,6 +368,7 @@ ${sparkClusterOpts} ${sparkResourceOpts} ${sparkApplicationOpts} + --queue analytics --hiveMetastoreUris${hive_metastore_uris} --sqleu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -551,4 +552,4 @@ - \ No newline at end of file + From 39a2afe8b538c45b1e4d20ed31d3eee1c9dbdd7b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 9 May 2024 13:54:42 +0200 Subject: [PATCH 22/26] [graph provision] fixed XML serialization of the usage counts measures, renamed workflow actions to better reflect their role --- ...erterJob.java => PayloadConverterJob.java} | 16 +-- .../model/ProvisionModelSupport.java | 11 +- .../oa/provision/utils/XmlRecordFactory.java | 110 ++++++++++-------- .../utils/XmlSerializationUtils.java | 33 ++++++ ...on => input_params_payload_converter.json} | 0 .../dhp/oa/provision/oozie_app/workflow.xml | 2 +- .../dhp/oa/provision/EOSCFuture_Test.java | 2 +- .../provision/IndexRecordTransformerTest.java | 6 +- .../oa/provision/XmlRecordFactoryTest.java | 14 +-- 9 files changed, 120 insertions(+), 74 deletions(-) rename dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/{XmlConverterJob.java => PayloadConverterJob.java} (92%) rename dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/{input_params_xml_converter.json => input_params_payload_converter.json} (100%) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java similarity index 92% rename from dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java rename to dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java index 4353e863f..f34caad75 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PayloadConverterJob.java @@ -3,24 +3,16 @@ package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static eu.dnetlib.dhp.utils.DHPUtils.toSeq; -import static org.apache.spark.sql.functions.*; import java.util.List; import java.util.Map; import java.util.Optional; 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.SparkContext; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.*; -import org.apache.spark.sql.expressions.UserDefinedFunction; -import org.apache.spark.sql.types.DataTypes; import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,9 +37,9 @@ import scala.Tuple2; /** * XmlConverterJob converts the JoinedEntities as XML records */ -public class XmlConverterJob { +public class PayloadConverterJob { - private static final Logger log = LoggerFactory.getLogger(XmlConverterJob.class); + private static final Logger log = LoggerFactory.getLogger(PayloadConverterJob.class); public static final String schemaLocation = "https://www.openaire.eu/schema/1.0/oaf-1.0.xsd"; @@ -56,8 +48,8 @@ public class XmlConverterJob { final ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - XmlConverterJob.class - .getResourceAsStream("/eu/dnetlib/dhp/oa/provision/input_params_xml_converter.json"))); + PayloadConverterJob.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json"))); parser.parseArgument(args); final Boolean isSparkSessionManaged = Optional 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 10a99704c..a085a72e0 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 @@ -19,8 +19,10 @@ import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.common.vocabulary.VocabularyTerm; import eu.dnetlib.dhp.oa.provision.utils.ContextDef; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; +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.solr.*; import eu.dnetlib.dhp.schema.solr.AccessRight; import eu.dnetlib.dhp.schema.solr.Author; @@ -66,7 +68,11 @@ public class ProvisionModelSupport { .setHeader( SolrRecordHeader .newInstance( - e.getId(), e.getOriginalId(), type, deletedbyinference)); + StringUtils + .substringAfter( + e.getId(), + IdentifierFactory.ID_PREFIX_SEPARATOR), + e.getOriginalId(), type, deletedbyinference)); r.setCollectedfrom(asProvenance(e.getCollectedfrom())); r.setContext(asContext(e.getContext(), contextMapper)); r.setPid(asPid(e.getPid())); @@ -106,7 +112,8 @@ public class ProvisionModelSupport { .newInstance( relation.getRelType(), relation.getRelClass(), - relation.getTarget(), relatedRecordType)); + StringUtils.substringAfter(relation.getTarget(), IdentifierFactory.ID_PREFIX_SEPARATOR), + relatedRecordType)); rr.setAcronym(re.getAcronym()); rr.setCode(re.getCode()); 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 63597c61e..65fa122c8 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 @@ -1,25 +1,23 @@ package eu.dnetlib.dhp.oa.provision.utils; -import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.authorPidTypes; -import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.getRelDescriptor; -import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.commons.lang3.StringUtils.substringBefore; - -import java.io.IOException; -import java.io.Serializable; -import java.io.StringReader; -import java.io.StringWriter; -import java.net.MalformedURLException; -import java.net.URL; -import java.util.*; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import javax.xml.transform.*; -import javax.xml.transform.dom.DOMSource; -import javax.xml.transform.stream.StreamResult; - +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +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.Result; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -31,27 +29,26 @@ import org.dom4j.Node; import org.dom4j.io.OutputFormat; import org.dom4j.io.SAXReader; import org.dom4j.io.XMLWriter; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -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; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; -import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; import scala.Tuple2; +import javax.xml.transform.*; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; +import java.io.IOException; +import java.io.Serializable; +import java.io.StringReader; +import java.io.StringWriter; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.authorPidTypes; +import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.getRelDescriptor; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.commons.lang3.StringUtils.substringBefore; + public class XmlRecordFactory implements Serializable { /** @@ -93,10 +90,13 @@ public class XmlRecordFactory implements Serializable { } public String build(final JoinedEntity je) { + return build(je, false); + } + + public String build(final JoinedEntity je, final Boolean validate) { final Set contexts = Sets.newHashSet(); - // final OafEntity entity = toOafEntity(je.getEntity()); final OafEntity entity = je.getEntity(); final TemplateFactory templateFactory = new TemplateFactory(); try { @@ -122,8 +122,14 @@ public class XmlRecordFactory implements Serializable { .buildBody( mainType, metadata, relations, listChildren(entity, je, templateFactory), listExtraInfo(entity)); - return templateFactory.buildRecord(entity, schemaLocation, body); - // return printXML(templateFactory.buildRecord(entity, schemaLocation, body), indent); + String xmlRecord = templateFactory.buildRecord(entity, schemaLocation, body); + + if (Boolean.TRUE.equals(validate)) { + // rise an exception when an invalid record was built + new SAXReader().read(new StringReader(xmlRecord)); + } + return xmlRecord; + // return printXML(templateFactory.buildRecord(entity, schemaLocation, body), indent); } catch (final Throwable e) { throw new RuntimeException(String.format("error building record '%s'", entity.getId()), e); } @@ -1038,13 +1044,21 @@ public class XmlRecordFactory implements Serializable { } private List measuresAsXml(List measures) { - return measures - .stream() - .map(m -> { - List> l = Lists.newArrayList(new Tuple2<>("id", m.getId())); - m.getUnit().forEach(kv -> l.add(new Tuple2<>(kv.getKey(), kv.getValue()))); - return XmlSerializationUtils.asXmlElement("measure", l); - }) + return Stream + .concat( + measures + .stream() + .filter(m -> !"downloads".equals(m.getId()) && !"views".equals(m.getId())) + .map(m -> { + List> l = Lists.newArrayList(new Tuple2<>("id", m.getId())); + m.getUnit().forEach(kv -> l.add(new Tuple2<>(kv.getKey(), kv.getValue()))); + return XmlSerializationUtils.asXmlElement("measure", l); + }), + measures + .stream() + .filter(m -> "downloads".equals(m.getId()) || "views".equals(m.getId())) + .filter(m -> m.getUnit().stream().anyMatch(u -> Integer.parseInt(u.getValue()) > 0)) + .map(m -> XmlSerializationUtils.usageMeasureAsXmlElement("measure", m))) .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 deacac3ad..31763ace3 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,7 +5,11 @@ 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; @@ -166,6 +170,35 @@ public class XmlSerializationUtils { return sb.toString(); } + // infrastruct_::f66f1bd369679b5b077dcdf006089556||OpenAIRE + public static String usageMeasureAsXmlElement(String name, Measure measure) { + HashSet dsIds = Optional + .ofNullable(measure.getUnit()) + .map( + m -> m + .stream() + .map(KeyValue::getKey) + .collect(Collectors.toCollection(HashSet::new))) + .orElse(new HashSet<>()); + + StringBuilder sb = new StringBuilder(); + dsIds.forEach(dsId -> { + sb + .append("<") + .append(name); + for (KeyValue kv : measure.getUnit()) { + sb.append(" ").append(attr(measure.getId(), kv.getValue())); + } + sb + .append(">") + .append(dsId) + .append(""); + }); + return sb.toString(); + } + public static String mapEoscIf(EoscIfGuidelines e) { return asXmlElement( "eoscifguidelines", Lists diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_xml_converter.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json similarity index 100% rename from dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_xml_converter.json rename to dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json 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 1fc28e7ca..59058d467 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 @@ -594,7 +594,7 @@ yarn cluster create_payloads - eu.dnetlib.dhp.oa.provision.XmlConverterJob + eu.dnetlib.dhp.oa.provision.PayloadConverterJob dhp-graph-provision-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/EOSCFuture_Test.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/EOSCFuture_Test.java index 1a982ca39..4c43de25c 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/EOSCFuture_Test.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/EOSCFuture_Test.java @@ -50,7 +50,7 @@ public class EOSCFuture_Test { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final OtherResearchProduct p = OBJECT_MAPPER .readValue( 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 8d5aa3f3a..718b43f03 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 @@ -57,7 +57,7 @@ public class IndexRecordTransformerTest { public void testPublicationRecordTransformation() throws IOException, TransformerException { final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = load("publication.json", Publication.class); final Project pj = load("project.json", Project.class); @@ -82,7 +82,7 @@ public class IndexRecordTransformerTest { void testPeerReviewed() throws IOException, TransformerException { final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = load("publication.json", Publication.class); @@ -98,7 +98,7 @@ public class IndexRecordTransformerTest { public void testRiunet() throws IOException, TransformerException { final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = load("riunet.json", Publication.class); 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 f26c384d2..d617991a1 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 @@ -37,7 +37,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = OBJECT_MAPPER .readValue(IOUtils.toString(getClass().getResourceAsStream("publication.json")), Publication.class); @@ -105,7 +105,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = OBJECT_MAPPER .readValue(IOUtils.toString(getClass().getResourceAsStream("publication.json")), Publication.class); @@ -136,7 +136,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = OBJECT_MAPPER .readValue(IOUtils.toString(getClass().getResourceAsStream("publication.json")), Publication.class); @@ -166,7 +166,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Datasource d = OBJECT_MAPPER .readValue(IOUtils.toString(getClass().getResourceAsStream("datasource.json")), Datasource.class); @@ -203,7 +203,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final OtherResearchProduct p = OBJECT_MAPPER .readValue( @@ -226,7 +226,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final OtherResearchProduct p = OBJECT_MAPPER .readValue( @@ -249,7 +249,7 @@ public class XmlRecordFactoryTest { final ContextMapper contextMapper = new ContextMapper(); final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, - XmlConverterJob.schemaLocation); + PayloadConverterJob.schemaLocation); final Publication p = OBJECT_MAPPER .readValue( From 55f39f785094f6500171d06945b3e5fcfc479a4c Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 9 May 2024 14:06:04 +0200 Subject: [PATCH 23/26] [graph provision] adds the possibility to validate the XML records before storing them via the validateXML parameter --- .../dhp/oa/provision/PayloadConverterJob.java | 17 ++++++++++++----- .../input_params_payload_converter.json | 6 ++++++ .../dhp/oa/provision/oozie_app/workflow.xml | 6 ++++++ 3 files changed, 24 insertions(+), 5 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 f34caad75..d7e22e557 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 @@ -64,6 +64,12 @@ public class PayloadConverterJob { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); + final Boolean validateXML = Optional + .ofNullable(parser.get("validateXML")) + .map(Boolean::valueOf) + .orElse(Boolean.FALSE); + log.info("validateXML: {}", validateXML); + final String contextApiBaseUrl = parser.get("contextApiBaseUrl"); log.info("contextApiBaseUrl: {}", contextApiBaseUrl); @@ -78,18 +84,19 @@ public class PayloadConverterJob { runWithSparkSession(conf, isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - convertToXml( + createPayloads( spark, inputPath, outputPath, ContextMapper.fromAPI(contextApiBaseUrl), - VocabularyGroup.loadVocsFromIS(isLookup)); + VocabularyGroup.loadVocsFromIS(isLookup), validateXML); }); } - private static void convertToXml( + private static void createPayloads( final SparkSession spark, final String inputPath, final String outputPath, final ContextMapper contextMapper, - final VocabularyGroup vocabularies) { + final VocabularyGroup vocabularies, + final Boolean validateXML) { final XmlRecordFactory recordFactory = new XmlRecordFactory( prepareAccumulators(spark.sparkContext()), @@ -110,7 +117,7 @@ public class PayloadConverterJob { .as(Encoders.kryo(JoinedEntity.class)) .map( (MapFunction>) je -> new Tuple2<>( - recordFactory.build(je), + recordFactory.build(je, validateXML), ProvisionModelSupport.transform(je, contextMapper, vocabularies)), Encoders.tuple(Encoders.STRING(), Encoders.bean(SolrRecord.class))) .map( diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json index 4509eb9de..1b43ca5fd 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_payload_converter.json @@ -22,5 +22,11 @@ "paramLongName": "isLookupUrl", "paramDescription": "URL of the context ISLookup Service", "paramRequired": true + }, + { + "paramName": "val", + "paramLongName": "validateXML", + "paramDescription": "should the process check the XML validity", + "paramRequired": false } ] 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 59058d467..1682f2ed5 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 @@ -13,6 +13,11 @@ contextApiBaseUrl context API URL + + validateXML + should the payload converter validate the XMLs + false + relPartitions number or partitions for the relations Dataset @@ -610,6 +615,7 @@ --inputPath/user/claudio.atzori/data/beta_provision/join_entities --outputPath${workingDir}/xml_json + --validateXML${validateXML} --contextApiBaseUrl${contextApiBaseUrl} --isLookupUrl${isLookupUrl} From 1efe7f7e39ea10d9c010cdefd40e1439b5bb52dd Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 14 May 2024 12:39:31 +0200 Subject: [PATCH 24/26] [graph provision] upgrade to dhp-schema:6.1.2, included project.oamandatepublications in the JSON payload mapping, fixed serialisation of the usageCounts measures --- .../dhp/oa/provision/PayloadConverterJob.java | 6 +- .../model/ProvisionModelSupport.java | 1 + .../oa/provision/utils/XmlRecordFactory.java | 79 ++++++++++--------- .../utils/XmlSerializationUtils.java | 8 +- pom.xml | 2 +- 5 files changed, 49 insertions(+), 47 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 d7e22e557..d46ab1404 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 @@ -65,9 +65,9 @@ public class PayloadConverterJob { log.info("outputPath: {}", outputPath); final Boolean validateXML = Optional - .ofNullable(parser.get("validateXML")) - .map(Boolean::valueOf) - .orElse(Boolean.FALSE); + .ofNullable(parser.get("validateXML")) + .map(Boolean::valueOf) + .orElse(Boolean.FALSE); log.info("validateXML: {}", validateXML); final String contextApiBaseUrl = parser.get("contextApiBaseUrl"); 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 a085a72e0..48e6b3ec9 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 @@ -146,6 +146,7 @@ public class ProvisionModelSupport { ps.setContracttype(mapCodeLabel(p.getContracttype())); ps.setCurrency(mapField(p.getCurrency())); ps.setDuration(mapField(p.getDuration())); + ps.setOamandatepublications(mapField(p.getOamandatepublications())); ps.setCallidentifier(mapField(p.getCallidentifier())); ps.setEcarticle29_3(mapField(p.getEcarticle29_3())); ps.setEnddate(mapField(p.getEnddate())); 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 65fa122c8..ec322dbd4 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 @@ -1,23 +1,25 @@ package eu.dnetlib.dhp.oa.provision.utils; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -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.Result; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; -import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; +import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.authorPidTypes; +import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.getRelDescriptor; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.commons.lang3.StringUtils.substringBefore; + +import java.io.IOException; +import java.io.Serializable; +import java.io.StringReader; +import java.io.StringWriter; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import javax.xml.transform.*; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; + import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -29,26 +31,27 @@ import org.dom4j.Node; import org.dom4j.io.OutputFormat; import org.dom4j.io.SAXReader; import org.dom4j.io.XMLWriter; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +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; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; import scala.Tuple2; -import javax.xml.transform.*; -import javax.xml.transform.dom.DOMSource; -import javax.xml.transform.stream.StreamResult; -import java.io.IOException; -import java.io.Serializable; -import java.io.StringReader; -import java.io.StringWriter; -import java.net.MalformedURLException; -import java.net.URL; -import java.util.*; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.authorPidTypes; -import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.getRelDescriptor; -import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.commons.lang3.StringUtils.substringBefore; - public class XmlRecordFactory implements Serializable { /** @@ -127,9 +130,9 @@ public class XmlRecordFactory implements Serializable { if (Boolean.TRUE.equals(validate)) { // rise an exception when an invalid record was built new SAXReader().read(new StringReader(xmlRecord)); - } - return xmlRecord; - // return printXML(templateFactory.buildRecord(entity, schemaLocation, body), indent); + } + return xmlRecord; + // return printXML(templateFactory.buildRecord(entity, schemaLocation, body), indent); } catch (final Throwable e) { throw new RuntimeException(String.format("error building record '%s'", entity.getId()), e); } 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 31763ace3..b4d021b68 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 @@ -190,11 +190,9 @@ public class XmlSerializationUtils { sb.append(" ").append(attr(measure.getId(), kv.getValue())); } sb - .append(">") - .append(dsId) - .append(""); + .append(" ") + .append(attr("datasource", dsId)) + .append("/>"); }); return sb.toString(); } diff --git a/pom.xml b/pom.xml index 892382b9d..bd19bda49 100644 --- a/pom.xml +++ b/pom.xml @@ -888,7 +888,7 @@ 3.3.3 3.4.2 [2.12,3.0) - [6.1.1] + [6.1.2] [4.0.3] [6.0.5] [3.1.6] From 0611c81a2fcdb769974dc35c412774c76a1921bb Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 May 2024 15:33:10 +0200 Subject: [PATCH 25/26] [graph provision] using Qualifier.classNames to populate the correponsing fields in the JSON payload --- .../provision/model/ProvisionModelSupport.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 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 48e6b3ec9..f46aebdcf 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 @@ -387,7 +387,7 @@ public class ProvisionModelSupport { .equals( Optional .ofNullable(t.getQualifier()) - .map(Qualifier::getClassid) + .map(Qualifier::getClassname) .orElse(null))) .map(StructuredProperty::getValue) .collect(Collectors.toList())) @@ -405,7 +405,7 @@ public class ProvisionModelSupport { .equals( Optional .ofNullable(t.getQualifier()) - .map(Qualifier::getClassid) + .map(Qualifier::getClassname) .orElse(null))) .map(StructuredProperty::getValue) .findFirst()) @@ -472,7 +472,7 @@ public class ProvisionModelSupport { } private static String mapQualifier(eu.dnetlib.dhp.schema.oaf.Qualifier q) { - return Optional.ofNullable(q).map(Qualifier::getClassid).orElse(null); + return Optional.ofNullable(q).map(Qualifier::getClassname).orElse(null); } private static Journal mapJournal(eu.dnetlib.dhp.schema.oaf.Journal joaf) { @@ -581,7 +581,7 @@ public class ProvisionModelSupport { .map( pids -> pids .stream() - .map(p -> Pid.newInstance(p.getQualifier().getClassid(), p.getValue())) + .map(p -> Pid.newInstance(p.getQualifier().getClassname(), p.getValue())) .collect(Collectors.toList())) .orElse(null); } @@ -606,8 +606,8 @@ public class ProvisionModelSupport { subjects -> subjects .stream() .filter(s -> Objects.nonNull(s.getQualifier())) - .filter(s -> Objects.nonNull(s.getQualifier().getClassid())) - .map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassid())) + .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) + .map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassname())) .collect(Collectors.toList())) .orElse(null); } @@ -619,8 +619,8 @@ public class ProvisionModelSupport { subjects -> subjects .stream() .filter(s -> Objects.nonNull(s.getQualifier())) - .filter(s -> Objects.nonNull(s.getQualifier().getClassid())) - .map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassid())) + .filter(s -> Objects.nonNull(s.getQualifier().getClassname())) + .map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassname())) .collect(Collectors.toList())) .orElse(null); } From 92f018d1962c964f4c15ac18a9d33b2fe6ae5301 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 May 2024 15:39:18 +0200 Subject: [PATCH 26/26] [graph provision] fixed path pointing to an intermediate data store in the working directory --- .../eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1682f2ed5..50acb4526 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 @@ -613,7 +613,7 @@ --conf spark.sql.shuffle.partitions=3840 --conf spark.network.timeout=${sparkNetworkTimeout} - --inputPath/user/claudio.atzori/data/beta_provision/join_entities + --inputPath${workingDir}/join_entities --outputPath${workingDir}/xml_json --validateXML${validateXML} --contextApiBaseUrl${contextApiBaseUrl}