diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/SparkDownloadUpdateDatacite.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/SparkDownloadUpdateDatacite.scala new file mode 100644 index 000000000..0b459fcf6 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/SparkDownloadUpdateDatacite.scala @@ -0,0 +1,53 @@ +package eu.dnetlib.dhp.actionmanager.datacite + + +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.{Oaf, Result} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.LocalFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.apache.spark.sql.functions.max +import org.slf4j.{Logger, LoggerFactory} + +import java.text.SimpleDateFormat +import java.util.{Date, Locale} +import scala.io.Source + +object SparkDownloadUpdateDatacite { + val log: Logger = LoggerFactory.getLogger(getClass) + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf + val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/generate_dataset_params.json")).mkString) + parser.parseArgument(args) + val master = parser.get("master") + val sourcePath = parser.get("sourcePath") + val workingPath = parser.get("workingPath") + + val hdfsuri = parser.get("namenode") + log.info(s"namenode is $hdfsuri") + + + val spark: SparkSession = SparkSession.builder().config(conf) + .appName(getClass.getSimpleName) + .master(master) + .getOrCreate() + + implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val resEncoder: Encoder[Result] = Encoders.kryo[Result] + + import spark.implicits._ + + + val maxDate:String = spark.read.load(workingPath).as[Oaf].filter(s => s.isInstanceOf[Result]).map(r => r.asInstanceOf[Result].getDateofcollection).select(max("value")).first().getString(0) + val ISO8601FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.US) + val string_to_date =ISO8601FORMAT.parse(maxDate) + val ts = string_to_date.getTime + + + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java new file mode 100644 index 000000000..7d91e47cc --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java @@ -0,0 +1,154 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.io.IOException; +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.MapDocumentUtil; +import scala.Tuple2; + +public class SparkWhitelistSimRels extends AbstractSparkAction { + + private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); + + private static final String WHITELIST_SEPARATOR = "####"; + + public SparkWhitelistSimRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + new SparkWhitelistSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException, SAXException { + + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + final String whiteListPath = parser.get("whiteListPath"); + + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + log.info("whiteListPath: '{}'", whiteListPath); + + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + // file format: source####target + Dataset> whiteListRels = spark + .createDataset( + sc + .textFile(whiteListPath) + // check if the line is in the correct format: id1####id2 + .filter(s -> s.contains(WHITELIST_SEPARATOR) && s.split(WHITELIST_SEPARATOR).length == 2) + .map(s -> new Tuple2<>(s.split(WHITELIST_SEPARATOR)[0], s.split(WHITELIST_SEPARATOR)[1])) + .rdd(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + // for each dedup configuration + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + + final String entity = dedupConf.getWf().getEntityType(); + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Adding whitelist simrels for: '{}'", subEntity); + + final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); + + Dataset> entities = spark + .createDataset( + sc + .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) + .repartition(numPartitions) + .mapToPair( + (PairFunction) s -> { + MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); + return new Tuple2<>(d.getIdentifier(), "present"); + }) + .rdd(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + Dataset> whiteListRels1 = whiteListRels + .joinWith(entities, whiteListRels.col("_1").equalTo(entities.col("_1")), "inner") + .map( + (MapFunction, Tuple2>, Tuple2>) Tuple2::_1, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + Dataset> whiteListRels2 = whiteListRels1 + .joinWith(entities, whiteListRels1.col("_2").equalTo(entities.col("_1")), "inner") + .map( + (MapFunction, Tuple2>, Tuple2>) Tuple2::_1, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + Dataset whiteListSimRels = whiteListRels2 + .map( + (MapFunction, Relation>) r -> createSimRel(r._1(), r._2(), entity), + Encoders.bean(Relation.class)); + + saveParquet(whiteListSimRels, outputPath, SaveMode.Append); + } + } + + private Relation createSimRel(String source, String target, String entity) { + final Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setSubRelType("dedupSimilarity"); + r.setRelClass("isSimilarTo"); + r.setDataInfo(new DataInfo()); + + switch (entity) { + case "result": + r.setRelType("resultResult"); + break; + case "organization": + r.setRelType("organizationOrganization"); + break; + default: + throw new IllegalArgumentException("unmanaged entity type: " + entity); + } + return r; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/UpdateOpenorgsJob.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/UpdateOpenorgsJob.java new file mode 100644 index 000000000..d094fb72b --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/UpdateOpenorgsJob.java @@ -0,0 +1,117 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.IOUtils; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; + +public class UpdateOpenorgsJob { + + private static final Logger log = LoggerFactory.getLogger(UpdateOpenorgsJob.class); + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/updateOpenorgsJob_parameters.json"))); + parser.parseArgument(args); + + final String apiUrl = parser.get("apiUrl"); + final int delay = Integer.parseInt(parser.get("delay")); + + log.info("apiUrl: '{}'", apiUrl); + log.info("delay: '{}'", delay); + + APIResponse res = httpCall(apiUrl); + while (res != null && res.getStatus().equals(ImportStatus.RUNNING)) { + TimeUnit.MINUTES.sleep(delay); + res = httpCall(apiUrl + "/status"); + } + + if (res == null) { + log.error("Openorgs Update FAILED: No response"); + throw new RuntimeException("Openorgs Update FAILED: No response"); + } + + if (res.getStatus() == null || !res.getStatus().equals(ImportStatus.SUCCESS)) { + log.error("Openorgs Update FAILED: '{}' - '{}'", res.getStatus(), res.getMessage()); + throw new RuntimeException(res.getMessage()); + } + + } + + private static APIResponse httpCall(final String url) throws Exception { + final HttpGet req = new HttpGet(url); + + try (final CloseableHttpClient client = HttpClients.createDefault()) { + try (final CloseableHttpResponse response = client.execute(req)) { + final String s = IOUtils.toString(response.getEntity().getContent()); + return (new ObjectMapper()).readValue(s, APIResponse.class); + } + } + } + +} + +class APIResponse { + private String id; + private Long dateStart; + private Long dateEnd; + private ImportStatus status; + private String message; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public Long getDateStart() { + return dateStart; + } + + public void setDateStart(Long dateStart) { + this.dateStart = dateStart; + } + + public Long getDateEnd() { + return dateEnd; + } + + public void setDateEnd(Long dateEnd) { + this.dateEnd = dateEnd; + } + + public ImportStatus getStatus() { + return status; + } + + public void setStatus(ImportStatus status) { + this.status = status; + } + + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } +} + +enum ImportStatus { + SUCCESS, FAILED, RUNNING, NOT_LAUNCHED, NOT_YET_STARTED +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index 30442406c..6947019e8 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -28,6 +28,11 @@ dbPwd password to access the OpenOrgs database + + dbConnections + 10 + number of connections to the postgres db + workingPath path for the working directory @@ -223,7 +228,7 @@ --dbTable${dbTable} --dbUser${dbUser} --dbPwd${dbPwd} - --numConnections20 + --numConnections${dbConnections} @@ -254,19 +259,24 @@ --dbTable${dbTable} --dbUser${dbUser} --dbPwd${dbPwd} - --numConnections20 + --numConnections${dbConnections} - - ${jobTracker} - ${nameNode} - /usr/bin/curl - ${apiUrl} - + + + + oozie.launcher.mapreduce.user.classpath.first + true + + + eu.dnetlib.dhp.oa.dedup.UpdateOpenorgsJob + --apiUrl${apiUrl} + --delay5 + diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml index 342d83e8e..02fdd8431 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml @@ -20,6 +20,10 @@ workingPath path for the working directory + + whiteListPath + path for the whitelist of similarity relations + dedupGraphPath path for the output graph @@ -130,6 +134,34 @@ --workingPath${workingPath} --numPartitions8000 + + + + + + + yarn + cluster + Add Whitelist Similarity Relations + eu.dnetlib.dhp.oa.dedup.SparkWhitelistSimRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --workingPath${workingPath} + --whiteListPath${whiteListPath} + --numPartitions8000 + diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateOpenorgsJob_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateOpenorgsJob_parameters.json new file mode 100644 index 000000000..5ca4a3dba --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateOpenorgsJob_parameters.json @@ -0,0 +1,14 @@ +[ + { + "paramName": "api", + "paramLongName": "apiUrl", + "paramDescription": "the url of the API", + "paramRequired": true + }, + { + "paramName": "d", + "paramLongName": "delay", + "paramDescription": "delay for the HTTP call in minutes", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json new file mode 100644 index 000000000..0a5cad7c4 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json @@ -0,0 +1,38 @@ +[ + { + "paramName": "la", + "paramLongName": "isLookUpUrl", + "paramDescription": "address for the LookUp", + "paramRequired": true + }, + { + "paramName": "asi", + "paramLongName": "actionSetId", + "paramDescription": "action set identifier (name of the orchestrator)", + "paramRequired": true + }, + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of the raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "path of the working directory", + "paramRequired": true + }, + { + "paramName": "np", + "paramLongName": "numPartitions", + "paramDescription": "number of partitions for the similarity relations intermediate phases", + "paramRequired": false + }, + { + "paramName": "wl", + "paramLongName": "whiteListPath", + "paramDescription": "whitelist file path for the addition of custom simrels", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java index 2f992bd78..549988767 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java @@ -5,13 +5,16 @@ import static java.nio.file.Files.createTempDirectory; import static org.apache.spark.sql.functions.count; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.lenient; import java.io.File; +import java.io.FileReader; import java.io.IOException; import java.io.Serializable; import java.net.URISyntaxException; import java.nio.file.Paths; +import java.util.List; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -55,6 +58,10 @@ public class SparkDedupTest implements Serializable { private static String testOutputBasePath; private static String testDedupGraphBasePath; private static final String testActionSetId = "test-orchestrator"; + private static String whitelistPath; + private static List whiteList; + + private static String WHITELIST_SEPARATOR = "####"; @BeforeAll public static void cleanUp() throws IOException, URISyntaxException { @@ -71,6 +78,12 @@ public class SparkDedupTest implements Serializable { .toAbsolutePath() .toString(); + whitelistPath = Paths + .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/whitelist.simrels.txt").toURI()) + .toFile() + .getAbsolutePath(); + whiteList = IOUtils.readLines(new FileReader(whitelistPath)); + FileUtils.deleteDirectory(new File(testOutputBasePath)); FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); @@ -202,6 +215,84 @@ public class SparkDedupTest implements Serializable { @Test @Order(2) + void whitelistSimRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkWhitelistSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json"))); + + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-la", "lookupurl", + "-w", testOutputBasePath, + "-np", "50", + "-wl", whitelistPath + }); + + new SparkWhitelistSimRels(parser, spark).run(isLookUpService); + + long orgs_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) + .count(); + + long pubs_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "publication")) + .count(); + + long ds_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "dataset")) + .count(); + + long orp_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "otherresearchproduct")) + .count(); + + // entities simrels supposed to be equal to the number of previous step (no rels in whitelist) + assertEquals(3082, orgs_simrel); + assertEquals(7036, pubs_simrel); + assertEquals(442, ds_simrel); + assertEquals(6750, orp_simrel); + + // entities simrels to be different from the number of previous step (new simrels in the whitelist) + Dataset sw_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "software")); + + // check if the first relation in the whitelist exists + assertTrue( + sw_simrel + .as(Encoders.bean(Relation.class)) + .toJavaRDD() + .filter( + rel -> rel.getSource().equalsIgnoreCase(whiteList.get(0).split(WHITELIST_SEPARATOR)[0]) + && rel.getTarget().equalsIgnoreCase(whiteList.get(0).split(WHITELIST_SEPARATOR)[1])) + .count() > 0); + // check if the second relation in the whitelist exists + assertTrue( + sw_simrel + .as(Encoders.bean(Relation.class)) + .toJavaRDD() + .filter( + rel -> rel.getSource().equalsIgnoreCase(whiteList.get(1).split(WHITELIST_SEPARATOR)[0]) + && rel.getTarget().equalsIgnoreCase(whiteList.get(1).split(WHITELIST_SEPARATOR)[1])) + .count() > 0); + + assertEquals(338, sw_simrel.count()); + + } + + @Test + @Order(3) void cutMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -297,7 +388,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(3) + @Order(4) void createMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -353,7 +444,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(4) + @Order(5) void createDedupRecordTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -394,13 +485,13 @@ public class SparkDedupTest implements Serializable { assertEquals(85, orgs_deduprecord); assertEquals(65, pubs_deduprecord); - assertEquals(51, sw_deduprecord); + assertEquals(49, sw_deduprecord); assertEquals(97, ds_deduprecord); assertEquals(89, orp_deduprecord); } @Test - @Order(5) + @Order(6) void updateEntityTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -479,7 +570,7 @@ public class SparkDedupTest implements Serializable { assertEquals(838, organizations); assertEquals(100, projects); assertEquals(100, datasource); - assertEquals(200, softwares); + assertEquals(198, softwares); assertEquals(389, dataset); assertEquals(517, otherresearchproduct); @@ -516,7 +607,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(6) + @Order(7) void propagateRelationTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -566,7 +657,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(7) + @Order(8) void testRelations() throws Exception { testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10); testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/whitelist.simrels.txt b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/whitelist.simrels.txt new file mode 100644 index 000000000..862ca466d --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/whitelist.simrels.txt @@ -0,0 +1,2 @@ +50|r37b0ad08687::f645b9729d1e1025a72c57883f0f2cac####50|r37b0ad08687::4c55b436743b5c49fa32cd582fd9e1aa +50|datacite____::a90f49f9fde5393c00633bea6e4e374a####50|datacite____::5f55cdee77303ba8a2bf9996c32a330c \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala index b77de13b9..e501b4823 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala @@ -13,10 +13,30 @@ import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} import org.slf4j.{Logger, LoggerFactory} import scala.collection.JavaConverters._ +import org.json4s.DefaultFormats +import org.json4s.JsonAST.{JField, JObject, JString,JArray} +import org.json4s.jackson.JsonMethods.parse object SparkGenerateDoiBoost { + def extractIdGRID(input:String):List[(String,String)] = { + implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats + lazy val json: org.json4s.JValue = parse(input) + + val id:String = (json \ "id").extract[String] + + val grids:List[String] = for { + + JObject(pid) <- json \ "pid" + JField("qualifier", JObject(qualifier)) <- pid + JField("classid", JString(classid)) <-qualifier + JField("value", JString(vl)) <- pid + if classid == "GRID" + } yield vl + grids.map(g => (id, s"unresolved::grid::${g.toLowerCase}"))(collection.breakOut) + } + def main(args: Array[String]): Unit = { @@ -36,6 +56,7 @@ object SparkGenerateDoiBoost { val hostedByMapPath = parser.get("hostedByMapPath") val workingDirPath = parser.get("workingPath") + val openaireOrganizationPath = parser.get("openaireOrganizationPath") val crossrefAggregator = new Aggregator[(String, Publication), Publication, Publication] with Serializable { override def zero: Publication = new Publication @@ -156,7 +177,7 @@ object SparkGenerateDoiBoost { magPubs.joinWith(a,magPubs("_1").equalTo(a("PaperId"))).flatMap(item => { val pub:Publication = item._1._2 val affiliation = item._2 - val affId:String = if (affiliation.GridId.isDefined) DoiBoostMappingUtil.generateGridAffiliationId(affiliation.GridId.get) else DoiBoostMappingUtil.generateMAGAffiliationId(affiliation.AffiliationId.toString) + val affId:String = if (affiliation.GridId.isDefined) s"unresolved::grid::${affiliation.GridId.get.toLowerCase}" else DoiBoostMappingUtil.generateMAGAffiliationId(affiliation.AffiliationId.toString) val r:Relation = new Relation r.setSource(pub.getId) r.setTarget(affId) @@ -174,9 +195,35 @@ object SparkGenerateDoiBoost { r1.setDataInfo(pub.getDataInfo) r1.setCollectedfrom(List(DoiBoostMappingUtil.createMAGCollectedFrom()).asJava) List(r, r1) - })(mapEncoderRel).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/doiBoostPublicationAffiliation") + })(mapEncoderRel).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/doiBoostPublicationAffiliation_unresolved") + + + val unresolvedRels:Dataset[(String, Relation)] = spark.read.load(s"$workingDirPath/doiBoostPublicationAffiliation_unresolved").as[Relation].map(r => { + + if (r.getSource.startsWith("unresolved")) + (r.getSource, r) + else if (r.getTarget.startsWith("unresolved")) + (r.getTarget,r) + else + ("resolved", r) + }) + + val openaireOrganization:Dataset[(String,String)] = spark.read.text(openaireOrganizationPath).as[String].flatMap(s => extractIdGRID(s)).groupByKey(_._2).reduceGroups((x,y) => if (x != null) x else y ).map(_._2) + + unresolvedRels.joinWith(openaireOrganization,unresolvedRels("_1").equalTo(openaireOrganization("_2"))) + .map { x => + val currentRels = x._1._2 + val currentOrgs = x._2 + if (currentOrgs!= null) + if(currentRels.getSource.startsWith("unresolved")) + currentRels.setSource(currentOrgs._1) + else + currentRels.setTarget(currentOrgs._1) + currentRels + }.write.save(s"$workingDirPath/doiBoostPublicationAffiliation") + magPubs.joinWith(a,magPubs("_1").equalTo(a("PaperId"))).map( item => { val affiliation = item._2 if (affiliation.GridId.isEmpty) { diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/generate_doiboost_params.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/generate_doiboost_params.json index 1ff63dd0e..39455fb67 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/generate_doiboost_params.json +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/generate_doiboost_params.json @@ -1,7 +1,8 @@ [ {"paramName": "m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}, {"paramName": "hb", "paramLongName":"hostedByMapPath", "paramDescription": "the hosted By Map Path", "paramRequired": true}, + {"paramName": "oo", "paramLongName":"openaireOrganizationPath", "paramDescription": "the openaire Organization Path", "paramRequired": true}, {"paramName": "ap", "paramLongName":"affiliationPath", "paramDescription": "the Affliation Path", "paramRequired": true}, {"paramName": "pa", "paramLongName":"paperAffiliationPath", "paramDescription": "the paperAffiliation Path", "paramRequired": true}, - {"paramName": "w", "paramLongName":"workingPath", "paramDescription": "the Working Path", "paramRequired": true} + {"paramName": "w", "paramLongName":"workingPath", "paramDescription": "the Working Path", "paramRequired": true} ] diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml index f5596b60e..eb82c3a7d 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml @@ -27,6 +27,12 @@ hostedByMapPath the hostedByMap Path + + openaireOrganizationPath + the OpenAire Organizations Path + + + outputPath the Path of the sequence file action set @@ -214,6 +220,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --hostedByMapPath${hostedByMapPath} + --openaireOrganizationPath${openaireOrganizationPath} --affiliationPath${inputPathMAG}/dataset/Affiliations --paperAffiliationPath${inputPathMAG}/dataset/PaperAuthorAffiliations --workingPath${workingPath} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkCreateBaselineDataFrame.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkCreateBaselineDataFrame.scala index 26efd723f..ee76cf8ad 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkCreateBaselineDataFrame.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkCreateBaselineDataFrame.scala @@ -6,18 +6,130 @@ import eu.dnetlib.dhp.schema.oaf.Result import eu.dnetlib.dhp.sx.graph.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf} import eu.dnetlib.dhp.utils.ISLookupClientFactory import org.apache.commons.io.IOUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem, Path} +import org.apache.http.client.config.RequestConfig +import org.apache.http.client.methods.HttpGet +import org.apache.http.impl.client.HttpClientBuilder import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql._ import org.slf4j.{Logger, LoggerFactory} +import java.io.InputStream import scala.io.Source import scala.xml.pull.XMLEventReader object SparkCreateBaselineDataFrame { + def requestBaseLineUpdatePage(maxFile:String):List[(String,String)] = { + val data =requestPage("https://ftp.ncbi.nlm.nih.gov/pubmed/updatefiles/") + + val result =data.lines.filter(l => l.startsWith("") + val start = l.indexOf("= 0 && end >start) + l.substring(start+9, (end-start)) + else + "" + }.filter(s =>s.endsWith(".gz") ).filter(s => s > maxFile).map(s => (s,s"https://ftp.ncbi.nlm.nih.gov/pubmed/updatefiles/$s")).toList + + result + } + + + def downloadBaselinePart(url:String):InputStream = { + val r = new HttpGet(url) + val timeout = 60; // seconds + val config = RequestConfig.custom() + .setConnectTimeout(timeout * 1000) + .setConnectionRequestTimeout(timeout * 1000) + .setSocketTimeout(timeout * 1000).build() + val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build() + val response = client.execute(r) + println(s"get response with status${response.getStatusLine.getStatusCode}") + response.getEntity.getContent + + } + + def requestPage(url:String):String = { + val r = new HttpGet(url) + val timeout = 60; // seconds + val config = RequestConfig.custom() + .setConnectTimeout(timeout * 1000) + .setConnectionRequestTimeout(timeout * 1000) + .setSocketTimeout(timeout * 1000).build() + val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build() + try { + var tries = 4 + while (tries > 0) { + println(s"requesting ${r.getURI}") + try { + val response = client.execute(r) + println(s"get response with status${response.getStatusLine.getStatusCode}") + if (response.getStatusLine.getStatusCode > 400) { + tries -= 1 + } + else + return IOUtils.toString(response.getEntity.getContent) + } catch { + case e: Throwable => + println(s"Error on requesting ${r.getURI}") + e.printStackTrace() + tries -= 1 + } + } + "" + } finally { + if (client != null) + client.close() + } + } + + + + + + + def downloadBaseLineUpdate(baselinePath:String, hdfsServerUri:String ):Unit = { + + + val conf = new Configuration + conf.set("fs.defaultFS", hdfsServerUri) + val fs = FileSystem.get(conf) + val p = new Path(baselinePath) + val files = fs.listFiles(p,false) + var max_file = "" + while (files.hasNext) { + val c = files.next() + val data = c.getPath.toString + val fileName = data.substring(data.lastIndexOf("/")+1) + + if (fileName> max_file) + max_file = fileName + } + + val files_to_download = requestBaseLineUpdatePage(max_file) + + files_to_download.foreach { u => + val hdfsWritePath: Path = new Path(s"$baselinePath/${u._1}") + val fsDataOutputStream: FSDataOutputStream = fs.create(hdfsWritePath, true) + val i = downloadBaselinePart(u._2) + val buffer = Array.fill[Byte](1024)(0) + while(i.read(buffer)>0) { + fsDataOutputStream.write(buffer) + } + i.close() + println(s"Downloaded ${u._2} into $baselinePath/${u._1}") + fsDataOutputStream.close() + } + + } + + val pmArticleAggregator: Aggregator[(String, PMArticle), PMArticle, PMArticle] = new Aggregator[(String, PMArticle), PMArticle, PMArticle] with Serializable { override def zero: PMArticle = new PMArticle @@ -51,6 +163,10 @@ object SparkCreateBaselineDataFrame { val targetPath = parser.get("targetPath") log.info("targetPath: {}", targetPath) + val hdfsServerUri = parser.get("hdfsServerUri") + log.info("hdfsServerUri: {}", targetPath) + + val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl) val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService) val spark: SparkSession = @@ -61,16 +177,15 @@ object SparkCreateBaselineDataFrame { .master(parser.get("master")).getOrCreate() import spark.implicits._ - val sc = spark.sparkContext - - implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle]) implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal]) implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor]) implicit val resultEncoder: Encoder[Result] = Encoders.kryo(classOf[Result]) + downloadBaseLineUpdate(s"$workingPath/baseline", hdfsServerUri) + val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline",2000) val ds:Dataset[PMArticle] = spark.createDataset(k.filter(i => i._1.endsWith(".gz")).flatMap(i =>{ val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes())) @@ -87,7 +202,5 @@ object SparkCreateBaselineDataFrame { .map(a => PubMedToOaf.convert(a, vocabularies)).as[Result] .filter(p => p!= null) .write.mode(SaveMode.Overwrite).save(targetPath) - - //s"$workingPath/oaf/baseline_oaf" } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkDownloadEBILinks.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkDownloadEBILinks.scala index 08e060459..e940fdff0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkDownloadEBILinks.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkDownloadEBILinks.scala @@ -5,17 +5,15 @@ import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF.EBILinkItem import eu.dnetlib.dhp.sx.graph.bio.pubmed.{PMArticle, PMAuthor, PMJournal} import org.apache.commons.io.IOUtils import org.apache.http.client.config.RequestConfig -import org.apache.http.client.methods.{HttpGet, HttpUriRequest} +import org.apache.http.client.methods.HttpGet import org.apache.http.impl.client.HttpClientBuilder import org.apache.spark.SparkConf -import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.functions.max -import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} +import org.apache.spark.sql._ import org.slf4j.{Logger, LoggerFactory} object SparkDownloadEBILinks { - def createEBILinks(pmid:Long):EBILinkItem = { val res = requestLinks(pmid) @@ -24,39 +22,42 @@ object SparkDownloadEBILinks { null } + def requestPage(url:String):String = { + val r = new HttpGet(url) + val timeout = 60; // seconds + val config = RequestConfig.custom() + .setConnectTimeout(timeout * 1000) + .setConnectionRequestTimeout(timeout * 1000) + .setSocketTimeout(timeout * 1000).build() + val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build() + try { + var tries = 4 + while (tries > 0) { + println(s"requesting ${r.getURI}") + try { + val response = client.execute(r) + println(s"get response with status${response.getStatusLine.getStatusCode}") + if (response.getStatusLine.getStatusCode > 400) { + tries -= 1 + } + else + return IOUtils.toString(response.getEntity.getContent) + } catch { + case e: Throwable => + println(s"Error on requesting ${r.getURI}") + e.printStackTrace() + tries -= 1 + } + } + "" + } finally { + if (client != null) + client.close() + } + } def requestLinks(PMID:Long):String = { - val r = new HttpGet(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json") - val timeout = 60; // seconds - val config = RequestConfig.custom() - .setConnectTimeout(timeout * 1000) - .setConnectionRequestTimeout(timeout * 1000) - .setSocketTimeout(timeout * 1000).build() - val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build() - try { - var tries = 4 - while (tries > 0) { - println(s"requesting ${r.getURI}") - try { - val response = client.execute(r) - println(s"get response with status${response.getStatusLine.getStatusCode}") - if (response.getStatusLine.getStatusCode > 400) { - tries -= 1 - } - else - return IOUtils.toString(response.getEntity.getContent) - } catch { - case e: Throwable => - println(s"Error on requesting ${r.getURI}") - e.printStackTrace() - tries -= 1 - } - } - "" - } finally { - if (client != null) - client.close() - } + requestPage(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json") } def main(args: Array[String]): Unit = { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkEBILinksToOaf.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkEBILinksToOaf.scala index f14e5f264..1924d919e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkEBILinksToOaf.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ebi/SparkEBILinksToOaf.scala @@ -32,14 +32,9 @@ object SparkEBILinksToOaf { import spark.implicits._ implicit val PMEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf]) - val ebi_rdd:Dataset[EBILinkItem] = spark.createDataset(spark.sparkContext.textFile(sourcePath).map(s => BioDBToOAF.extractEBILinksFromDump(s))).as[EBILinkItem] - - ebi_rdd.write.mode(SaveMode.Overwrite).save(s"${sourcePath}_dataset") - val ebLinks:Dataset[EBILinkItem] = spark.read.load(s"${sourcePath}_dataset").as[EBILinkItem].filter(l => l.links!= null) ebLinks.flatMap(j =>BioDBToOAF.parse_ebi_links(j.links)) - .repartition(4000) .filter(p => BioDBToOAF.EBITargetLinksFilter(p)) .flatMap(p => BioDBToOAF.convertEBILinksToOaf(p)) .write.mode(SaveMode.Overwrite).save(targetPath) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql index ea483a4a7..46e0eb5e1 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql @@ -1,10 +1,10 @@ DROP VIEW IF EXISTS ${hiveDbName}.result; CREATE VIEW IF NOT EXISTS ${hiveDbName}.result as - select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.publication p + select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.publication p union all - select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.dataset d + select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.dataset d union all - select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.software s + select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.software s union all - select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.otherresearchproduct o; + select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.otherresearchproduct o; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/baseline_to_oaf_params.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/baseline_to_oaf_params.json index 38eb50094..4bee770bd 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/baseline_to_oaf_params.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/baseline_to_oaf_params.json @@ -1,6 +1,7 @@ [ - {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"i", "paramLongName":"isLookupUrl","paramDescription": "isLookupUrl", "paramRequired": true}, - {"paramName":"w", "paramLongName":"workingPath","paramDescription": "the path of the sequencial file to read", "paramRequired": true}, - {"paramName":"t", "paramLongName":"targetPath","paramDescription": "the oaf path ", "paramRequired": true} + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true}, + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the oaf path ", "paramRequired": true}, + {"paramName":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json index 0ae19234a..0860ed558 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json @@ -1,5 +1,5 @@ [ - {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"s", "paramLongName":"sourcePath","paramDescription": "the source Path", "paramRequired": true}, - {"paramName":"w", "paramLongName":"workingPath","paramDescription": "the working path ", "paramRequired": true} + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source Path", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path ", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/oozie_app/workflow.xml index 3f442c5c6..7612321c0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/oozie_app/workflow.xml @@ -25,7 +25,6 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - yarn-cluster @@ -43,6 +42,7 @@ --workingPath${workingPath} --masteryarn + --hdfsServerUri${nameNode} @@ -74,7 +74,7 @@ yarn-cluster cluster - Create Baselnie DataSet + Create Baseline DataSet eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates dhp-graph-mapper-${projectVersion}.jar diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/update/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/update/oozie_app/workflow.xml index 1b738caed..cd3bb8c71 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/update/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/ebi/update/oozie_app/workflow.xml @@ -1,59 +1,67 @@ - - - - sourcePath - the Working Path - - - workingPath - the Working Path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - + + + + sourcePath + the Working Path + + + workingPath + the Working Path + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + - + - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + - - - yarn-cluster - cluster - Incremental Download EBI Links - eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.shuffle.partitions=2000 - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath} - --workingPath${workingPath} - --masteryarn - - - - - - - \ No newline at end of file + + + yarn-cluster + cluster + Incremental Download EBI Links + eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.shuffle.partitions=2000 + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --workingPath${workingPath} + --masteryarn + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/BioScholixTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/BioScholixTest.scala index 8e063db7c..87279eb21 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/BioScholixTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/BioScholixTest.scala @@ -7,6 +7,7 @@ import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result} import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF.ScholixResolved import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF import eu.dnetlib.dhp.sx.graph.bio.pubmed.PubMedToOaf.dataInfo +import eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks import org.json4s.DefaultFormats import org.json4s.JsonAST.{JField, JObject, JString} import org.json4s.jackson.JsonMethods.parse @@ -50,6 +51,8 @@ class BioScholixTest extends AbstractVocabularyTest{ } + + @Test def testEBIData() = { val inputXML = Source.fromInputStream(getClass.getResourceAsStream("pubmed.xml")).mkString diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/pubmed/ls_result b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/pubmed/ls_result new file mode 100644 index 000000000..98a0841c4 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/graph/bio/pubmed/ls_result @@ -0,0 +1,1433 @@ +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0001.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0002.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0003.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0004.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0005.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0006.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0007.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0008.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0009.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0010.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0011.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0012.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0013.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0014.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0015.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0016.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0017.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0018.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0019.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0020.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0021.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0022.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0023.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0024.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0025.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0026.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0027.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0028.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0029.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0030.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0031.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0032.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0033.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0034.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0035.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0036.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0037.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0038.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0039.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0040.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0041.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0042.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0043.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0044.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0045.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0046.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0047.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0048.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0049.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0050.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0051.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0052.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0053.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0054.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0055.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0056.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0057.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0058.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0059.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0060.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0061.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0062.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0063.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0064.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0065.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0066.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0067.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0068.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0069.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0070.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0071.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0072.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0073.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0074.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0075.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0076.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0077.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0078.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0079.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0080.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0081.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0082.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0083.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0084.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0085.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0086.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0087.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0088.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0089.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0090.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0091.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0092.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0093.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0094.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0095.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0096.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0097.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0098.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0099.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0100.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0101.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0102.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0103.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0104.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0105.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0106.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0107.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0108.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0109.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0110.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0111.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0112.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0113.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0114.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0115.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0116.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0117.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0118.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0119.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0120.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0121.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0122.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0123.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0124.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0125.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0126.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0127.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0128.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0129.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0130.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0131.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0132.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0133.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0134.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0135.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0136.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0137.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0138.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0139.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0140.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0141.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0142.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0143.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0144.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0145.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0146.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0147.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0148.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0149.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0150.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0151.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0152.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0153.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0154.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0155.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0156.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0157.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0158.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0159.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0160.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0161.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0162.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0163.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0164.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0165.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0166.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0167.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0168.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0169.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0170.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0171.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0172.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0173.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0174.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0175.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0176.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0177.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0178.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0179.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0180.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0181.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0182.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0183.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0184.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0185.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0186.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0187.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0188.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0189.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0190.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0191.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0192.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0193.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0194.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0195.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0196.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0197.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0198.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0199.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0200.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0201.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0202.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0203.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0204.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0205.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0206.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0207.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0208.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0209.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0210.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0211.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0212.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0213.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0214.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0215.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0216.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0217.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0218.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0219.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0220.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0221.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0222.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0223.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0224.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0225.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0226.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0227.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0228.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0229.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0230.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0231.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0232.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0233.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0234.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0235.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0236.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0237.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0238.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0239.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0240.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0241.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0242.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0243.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0244.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0245.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0246.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0247.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0248.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0249.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0250.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0251.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0252.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0253.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0254.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0255.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0256.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0257.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0258.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0259.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0260.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0261.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0262.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0263.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0264.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0265.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0266.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0267.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0268.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0269.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0270.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0271.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0272.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0273.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0274.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0275.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0276.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0277.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0278.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0279.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0280.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0281.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0282.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0283.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0284.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0285.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0286.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0287.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0288.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0289.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0290.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0291.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0292.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0293.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0294.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0295.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0296.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0297.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0298.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0299.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0300.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0301.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0302.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0303.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0304.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0305.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0306.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0307.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0308.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0309.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0310.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0311.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0312.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0313.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0314.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0315.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0316.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0317.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0318.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0319.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0320.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0321.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0322.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0323.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0324.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0325.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0326.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0327.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0328.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0329.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0330.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0331.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0332.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0333.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0334.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0335.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0336.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0337.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0338.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0339.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0340.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0341.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0342.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0343.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0344.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0345.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0346.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0347.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0348.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0349.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0350.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0351.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0352.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0353.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0354.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0355.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0356.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0357.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0358.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0359.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0360.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0361.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0362.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0363.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0364.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0365.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0366.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0367.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0368.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0369.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0370.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0371.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0372.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0373.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0374.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0375.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0376.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0377.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0378.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0379.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0380.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0381.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0382.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0383.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0384.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0385.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0386.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0387.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0388.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0389.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0390.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0391.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0392.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0393.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0394.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0395.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0396.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0397.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0398.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0399.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0400.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0401.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0402.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0403.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0404.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0405.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0406.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0407.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0408.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0409.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0410.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0411.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0412.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0413.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0414.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0415.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0416.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0417.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0418.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0419.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0420.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0421.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0422.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0423.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0424.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0425.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0426.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0427.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0428.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0429.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0430.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0431.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0432.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0433.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0434.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0435.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0436.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0437.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0438.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0439.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0440.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0441.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0442.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0443.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0444.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0445.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0446.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0447.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0448.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0449.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0450.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0451.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0452.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0453.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0454.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0455.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0456.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0457.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0458.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0459.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0460.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0461.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0462.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0463.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0464.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0465.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0466.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0467.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0468.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0469.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0470.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0471.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0472.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0473.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0474.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0475.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0476.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0477.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0478.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0479.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0480.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0481.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0482.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0483.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0484.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0485.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0486.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0487.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0488.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0489.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0490.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0491.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0492.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0493.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0494.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0495.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0496.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0497.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0498.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0499.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0500.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0501.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0502.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0503.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0504.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0505.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0506.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0507.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0508.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0509.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0510.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0511.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0512.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0513.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0514.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0515.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0516.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0517.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0518.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0519.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0520.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0521.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0522.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0523.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0524.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0525.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0526.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0527.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0528.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0529.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0530.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0531.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0532.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0533.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0534.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0535.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0536.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0537.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0538.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0539.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0540.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0541.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0542.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0543.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0544.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0545.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0546.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0547.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0548.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0549.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0550.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0551.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0552.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0553.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0554.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0555.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0556.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0557.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0558.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0559.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0560.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0561.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0562.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0563.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0564.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0565.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0566.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0567.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0568.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0569.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0570.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0571.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0572.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0573.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0574.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0575.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0576.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0577.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0578.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0579.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0580.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0581.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0582.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0583.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0584.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0585.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0586.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0587.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0588.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0589.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0590.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0591.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0592.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0593.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0594.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0595.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0596.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0597.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0598.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0599.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0600.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0601.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0602.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0603.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0604.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0605.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0606.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0607.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0608.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0609.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0610.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0611.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0612.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0613.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0614.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0615.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0616.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0617.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0618.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0619.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0620.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0621.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0622.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0623.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0624.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0625.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0626.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0627.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0628.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0629.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0630.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0631.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0632.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0633.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0634.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0635.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0636.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0637.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0638.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0639.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0640.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0641.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0642.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0643.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0644.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0645.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0646.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0647.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0648.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0649.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0650.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0651.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0652.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0653.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0654.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0655.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0656.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0657.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0658.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0659.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0660.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0661.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0662.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0663.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0664.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0665.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0666.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0667.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0668.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0669.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0670.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0671.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0672.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0673.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0674.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0675.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0676.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0677.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0678.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0679.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0680.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0681.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0682.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0683.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0684.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0685.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0686.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0687.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0688.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0689.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0690.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0691.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0692.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0693.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0694.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0695.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0696.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0697.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0698.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0699.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0700.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0701.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0702.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0703.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0704.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0705.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0706.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0707.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0708.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0709.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0710.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0711.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0712.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0713.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0714.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0715.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0716.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0717.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0718.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0719.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0720.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0721.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0722.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0723.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0724.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0725.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0726.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0727.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0728.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0729.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0730.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0731.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0732.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0733.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0734.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0735.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0736.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0737.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0738.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0739.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0740.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0741.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0742.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0743.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0744.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0745.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0746.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0747.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0748.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0749.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0750.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0751.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0752.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0753.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0754.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0755.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0756.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0757.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0758.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0759.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0760.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0761.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0762.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0763.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0764.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0765.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0766.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0767.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0768.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0769.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0770.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0771.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0772.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0773.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0774.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0775.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0776.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0777.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0778.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0779.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0780.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0781.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0782.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0783.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0784.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0785.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0786.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0787.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0788.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0789.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0790.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0791.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0792.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0793.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0794.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0795.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0796.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0797.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0798.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0799.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0800.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0801.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0802.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0803.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0804.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0805.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0806.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0807.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0808.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0809.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0810.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0811.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0812.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0813.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0814.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0815.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0816.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0817.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0818.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0819.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0820.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0821.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0822.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0823.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0824.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0825.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0826.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0827.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0828.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0829.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0830.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0831.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0832.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0833.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0834.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0835.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0836.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0837.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0838.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0839.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0840.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0841.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0842.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0843.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0844.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0845.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0846.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0847.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0848.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0849.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0850.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0851.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0852.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0853.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0854.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0855.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0856.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0857.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0858.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0859.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0860.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0861.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0862.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0863.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0864.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0865.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0866.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0867.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0868.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0869.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0870.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0871.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0872.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0873.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0874.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0875.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0876.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0877.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0878.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0879.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0880.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0881.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0882.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0883.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0884.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0885.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0886.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0887.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0888.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0889.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0890.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0891.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0892.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0893.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0894.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0895.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0896.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0897.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0898.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0899.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0900.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0901.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0902.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0903.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0904.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0905.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0906.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0907.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0908.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0909.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0910.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0911.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0912.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0913.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0914.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0915.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0916.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0917.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0918.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0919.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0920.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0921.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0922.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0923.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0924.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0925.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0926.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0927.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0928.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0929.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0930.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0931.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0932.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0933.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0934.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0935.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0936.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0937.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0938.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0939.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0940.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0941.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0942.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0943.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0944.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0945.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0946.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0947.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0948.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0949.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0950.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0951.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0952.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0953.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0954.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0955.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0956.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0957.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0958.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0959.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0960.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0961.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0962.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0963.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0964.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0965.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0966.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0967.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0968.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0969.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0970.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0971.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0972.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0973.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0974.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0975.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0976.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0977.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0978.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0979.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0980.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0981.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0982.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0983.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0984.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0985.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0986.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0987.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0988.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0989.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0990.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0991.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0992.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0993.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0994.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0995.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0996.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0997.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0998.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n0999.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1000.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1001.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1002.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1003.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1004.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1005.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1006.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1007.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1008.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1009.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1010.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1011.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1012.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1013.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1014.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1015.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1016.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1017.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1018.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1019.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1020.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1021.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1022.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1023.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1024.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1025.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1026.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1027.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1028.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1029.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1030.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1031.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1032.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1033.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1034.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1035.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1036.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1037.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1038.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1039.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1040.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1041.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1042.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1043.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1044.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1045.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1046.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1047.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1048.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1049.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1050.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1051.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1052.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1053.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1054.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1055.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1056.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1057.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1058.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1059.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1060.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1061.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1062.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1063.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1064.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1065.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1066.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1067.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1068.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1069.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1070.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1071.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1072.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1073.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1074.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1075.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1076.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1077.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1078.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1079.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1080.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1081.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1082.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1083.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1084.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1085.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1086.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1087.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1088.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1089.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1090.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1091.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1092.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1093.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1094.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1095.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1096.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1097.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1098.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1099.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1100.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1101.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1102.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1103.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1104.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1105.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1106.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1107.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1108.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1109.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1110.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1111.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1112.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1113.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1114.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1115.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1116.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1117.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1118.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1119.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1120.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1121.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1122.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1123.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1124.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1125.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1126.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1127.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1128.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1129.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1130.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1131.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1132.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1133.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1134.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1135.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1136.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1137.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1138.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1139.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1140.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1141.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1142.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1143.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1144.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1145.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1146.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1147.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1148.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1149.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1150.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1151.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1152.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1153.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1154.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1155.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1156.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1157.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1158.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1159.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1160.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1161.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1162.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1163.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1164.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1165.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1166.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1167.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1168.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1169.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1170.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1171.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1172.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1173.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1174.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1175.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1176.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1177.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1178.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1179.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1180.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1181.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1182.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1183.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1184.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1185.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1186.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1187.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1188.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1189.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1190.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1191.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1192.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1193.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1194.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1195.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1196.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1197.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1198.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1199.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1200.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1201.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1202.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1203.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1204.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1205.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1206.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1207.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1208.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1209.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1210.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1211.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1212.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1213.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1214.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1215.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1216.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1217.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1218.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1219.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1220.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1221.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1222.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1223.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1224.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1225.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1226.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1227.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1228.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1229.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1230.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1231.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1232.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1233.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1234.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1235.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1236.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1237.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1238.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1239.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1240.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1241.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1242.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1243.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1244.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1245.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1246.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1247.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1248.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1249.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1250.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1251.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1252.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1253.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1254.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1255.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1256.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1257.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1258.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1259.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1260.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1261.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1262.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1263.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1264.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1265.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1266.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1267.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1268.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1269.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1270.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1271.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1272.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1273.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1274.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1275.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1276.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1277.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1278.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1279.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1280.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1281.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1282.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1283.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1284.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1285.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1286.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1287.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1288.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1289.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1290.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1291.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1292.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1293.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1294.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1295.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1296.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1297.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1298.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1299.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1300.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1301.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1302.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1303.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1304.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1305.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1306.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1307.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1308.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1309.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1310.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1311.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1312.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1313.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1314.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1315.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1316.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1317.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1318.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1319.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1320.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1321.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1322.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1323.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1324.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1325.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1326.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1327.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1328.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1329.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1330.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1331.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1332.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1333.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1334.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1335.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1336.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1337.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1338.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1339.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1340.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1341.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1342.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1343.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1344.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1345.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1346.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1347.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1348.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1349.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1350.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1351.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1352.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1353.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1354.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1355.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1356.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1357.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1358.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1359.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1360.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1361.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1362.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1363.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1364.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1365.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1366.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1367.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1368.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1369.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1370.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1371.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1372.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1373.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1374.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1375.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1376.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1377.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1378.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1379.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1380.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1381.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1382.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1383.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1384.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1385.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1386.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1387.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1388.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1389.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1390.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1391.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1392.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1393.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1394.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1395.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1396.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1397.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1398.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1399.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1400.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1401.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1402.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1403.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1404.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1405.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1406.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1407.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1408.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1409.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1410.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1411.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1412.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1413.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1414.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1415.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1416.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1417.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1418.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1419.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1420.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1421.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1422.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1423.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1424.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1425.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1426.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1427.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1428.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1429.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1430.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1431.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1432.xml.gz +hdfs://nameservice1/data/scholix/input/baseline/pubmed21n1433.xml.gz \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/contexts.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/contexts.sh index 6c5823b0c..6d42ab13d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/contexts.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/contexts.sh @@ -16,7 +16,7 @@ curl -L ${CONTEXT_API}/contexts/?type=ri,community -H "accept: application/json" cat contexts.csv | cut -d , -f1 | xargs -I {} curl -L ${CONTEXT_API}/context/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split(":")[0]),\(.id),\(.label)"' > categories.csv cat categories.csv | cut -d , -f2 | sed 's/:/%3A/g'| xargs -I {} curl -L ${CONTEXT_API}/context/category/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split("::")[0])::\(.id|split("::")[1]),\(.id),\(.label)"' > concepts.csv cat contexts.csv | sed 's/^\(.*\),\(.*\)/\1,\1::other,\2/' >> categories.csv -cat categories.csv | grep -v ::other | sed 's/^.*,\(.*\),\(.*\)/\1,\1::other,\2/' >> concepts.csv +cat categories.csv | sed 's/^.*,\(.*\),\(.*\)/\1,\1::other,\2/' >> concepts.csv echo "uploading context data to hdfs" hdfs dfs -mkdir ${TMP} diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/indicators.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/indicators.sh index fb944f4ff..93faa43d6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/indicators.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/indicators.sh @@ -15,5 +15,5 @@ hdfs dfs -copyToLocal $SCRIPT_PATH echo "Creating indicators" impala-shell -q "invalidate metadata" impala-shell -d ${TARGET} -q "show tables" --delimited | sed "s/^\(.*\)/compute stats ${TARGET}.\1;/" | impala-shell -c -f - -cat step16_7-createIndicatorsTables.sql | impala-shell -d $TARGET -f - +cat step16-createIndicatorsTables.sql | impala-shell -d $TARGET -f - echo "Indicators created" \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory-post.sh similarity index 58% rename from dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory.sh rename to dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory-post.sh index ff03bca03..db8d39af2 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory-post.sh @@ -9,16 +9,9 @@ fi export SOURCE=$1 export TARGET=$2 export SHADOW=$3 -export SCRIPT_PATH=$4 -echo "Getting file from " $4 -hdfs dfs -copyToLocal $4 - -echo "Creating observatory database" -impala-shell -q "drop database if exists ${TARGET} cascade" -impala-shell -q "create database if not exists ${TARGET}" -impala-shell -d ${SOURCE} -q "show tables" --delimited | sed "s/\(.*\)/create view ${TARGET}.\1 as select * from ${SOURCE}.\1;/" | impala-shell -f - -cat step21-createObservatoryDB.sql | sed s/SOURCE/$1/g | sed s/TARGET/$2/g1 | impala-shell -f - +impala-shell -q "invalidate metadata;" +impala-shell -d ${TARGET} -q "show tables" --delimited | sed "s/\(.*\)/compute stats ${TARGET}.\1;/" | impala-shell -f - echo "Impala shell finished" echo "Updating shadow observatory database" diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory-pre.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory-pre.sh new file mode 100644 index 000000000..55a308c50 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/observatory-pre.sh @@ -0,0 +1,16 @@ +export PYTHON_EGG_CACHE=/home/$(whoami)/.python-eggs +export link_folder=/tmp/impala-shell-python-egg-cache-$(whoami) +if ! [ -L $link_folder ] +then + rm -Rf "$link_folder" + ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} +fi + +export SOURCE=$1 +export TARGET=$2 +export SHADOW=$3 + +echo "Creating observatory database" +impala-shell -q "drop database if exists ${TARGET} cascade" +impala-shell -q "create database if not exists ${TARGET}" +impala-shell -d ${SOURCE} -q "show tables" --delimited | grep -iv roar | sed "s/\(.*\)/create view ${TARGET}.\1 as select * from ${SOURCE}.\1;/" | impala-shell -f - diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index 77fbd3b18..fc0162a9c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -23,6 +23,11 @@ CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * FROM ${external_stats_db_name}.rndexpediture; +CREATE OR REPLACE VIEW ${stats_db_name}.licenses_normalized AS +SELECT * +FROM ${external_stats_db_name}.licenses_normalized; + + ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -- Creation date of the database diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql similarity index 100% rename from dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_6.sql rename to dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_7-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql similarity index 83% rename from dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_7-createIndicatorsTables.sql rename to dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 020787039..0ea4a5adc 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_7-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -233,4 +233,50 @@ on p.id= tmp.id; create table indi_pub_has_abstract stored as parquet as select distinct publication.id, coalesce(abstract, 1) has_abstract -from publication; \ No newline at end of file +from publication; + +create table indi_with_orcid stored as parquet as +select distinct r.id, coalesce(has_orcid, 0) as has_orcid +from result r +left outer join (select id, 1 as has_orcid from result_orcid) tmp +on r.id= tmp.id + +create table indi_funded_result_with_fundref stored as parquet as +select distinct r.id, coalesce(fundref, 0) as fundref +from project_results r +left outer join (select distinct id, 1 as fundref from project_results +where provenance='Harvested') tmp +on r.id= tmp.id + +create table indi_result_org_country_collab stored as parquet as +with tmp as +(select o.id as id, o.country , ro.id as result,r.type from organization o +join result_organization ro on o.id=ro.organization +join result r on r.id=ro.id where o.country <> 'UNKNOWN') +select o1.id org1,o2.country country2, o1.type, count(distinct o1.result) as collaborations +from tmp as o1 +join tmp as o2 on o1.result=o2.result +where o1.id<>o2.id and o1.country<>o2.country +group by o1.id, o1.type,o2.country + +create table indi_result_org_collab stored as parquet as +with tmp as +(select o.id, ro.id as result,r.type from organization o +join result_organization ro on o.id=ro.organization +join result r on r.id=ro.id) +select o1.id org1,o2.id org2, o1.type, count(distinct o1.result) as collaborations +from tmp as o1 +join tmp as o2 on o1.result=o2.result +where o1.id<>o2.id +group by o1.id, o2.id, o1.type + +create table indi_result_org_country_collab stored as parquet as +with tmp as +(select o.id as id, o.country , ro.id as result,r.type from organization o +join result_organization ro on o.id=ro.organization +join result r on r.id=ro.id where o.country <> 'UNKNOWN') +select o1.id org1,o2.country country2, o1.type, count(distinct o1.result) as collaborations +from tmp as o1 +join tmp as o2 on o1.result=o2.result +where o1.id<>o2.id and o1.country<>o2.country +group by o1.id, o1.type,o2.country diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql deleted file mode 100644 index 481fd9e8c..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql +++ /dev/null @@ -1,62 +0,0 @@ ----------------------------------------------------- --- Shortcuts for various definitions in stats db --- ----------------------------------------------------- - --- Peer reviewed: --- Results that have been collected from Crossref -create table ${stats_db_name}.result_peerreviewed as -with peer_reviewed as ( - select distinct r.id as id - from ${stats_db_name}.result r - join ${stats_db_name}.result_sources rs on rs.id=r.id - join ${stats_db_name}.datasource d on d.id=rs.datasource - where d.name='Crossref') -select distinct peer_reviewed.id as id, true as peer_reviewed -from peer_reviewed -union all -select distinct r.id as id, false as peer_reviewed -from ${stats_db_name}.result r -left outer join peer_reviewed pr on pr.id=r.id -where pr.id is null; - --- Green OA: --- OA results that are hosted by an Institutional repository and have NOT been harvested from a DOAJ journal. -create table ${stats_db_name}.result_greenoa as -with result_green as ( - select distinct r.id as id - from ${stats_db_name}.result r - join ${stats_db_name}.result_datasources rd on rd.id=r.id - join ${stats_db_name}.datasource d on d.id=rd.datasource - left outer join ( - select rd.id from ${stats_db_name}.result_datasources rd - join ${stats_db_name}.datasource d on rd.datasource=d.id - join ${stats_db_name}.datasource_sources sds on sds.id=d.id - join ${stats_db_name}.datasource sd on sd.id=sds.datasource - where sd.name='DOAJ-ARTICLES' - ) as doaj on doaj.id=r.id - where r.bestlicence in ('Open Access', 'Open Source') and d.type='Institutional Repository' and doaj.id is null) -select distinct result_green.id, true as green -from result_green -union all -select distinct r.id as id, false as green -from ${stats_db_name}.result r -left outer join result_green rg on rg.id=r.id -where rg.id is null; - --- GOLD OA: --- OA results that have been harvested from a DOAJ journal. -create table ${stats_db_name}.result_gold as -with result_gold as ( - select distinct r.id as id - from ${stats_db_name}.result r - join ${stats_db_name}.result_datasources rd on rd.id=r.id - join ${stats_db_name}.datasource d on d.id=rd.datasource - join ${stats_db_name}.datasource_sources sds on sds.id=d.id - join ${stats_db_name}.datasource sd on sd.id=sds.datasource - where r.type='publication' and r.bestlicence='Open Access' and sd.name='DOAJ-Articles') -select distinct result_gold.id, true as gold -from result_gold -union all -select distinct r.id, false as gold -from ${stats_db_name}.result r -where r.id not in (select id from result_gold); \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql new file mode 100644 index 000000000..6b4d9b1b0 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql @@ -0,0 +1,22 @@ +---------------------------------------------------- +-- Shortcuts for various definitions in stats db --- +---------------------------------------------------- + +-- Peer reviewed: +create table ${stats_db_name}.result_peerreviewed as +select r.id as id, case when doi.doi_from_crossref=1 and grey.grey_lit=0 then true else false end as peer_reviewed +from ${stats_db_name}.result r +left outer join ${stats_db_name}.indi_pub_doi_from_crossref doi on doi.id=r.id +left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id; + +-- Green OA: +create table ${stats_db_name}.result_greenoa as +select r.id, case when green.green_oa=1 then true else false end as green +from ${stats_db_name}.result r +left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id; + +-- GOLD OA: +create table ${stats_db_name}.result_gold as +select r.id, case when gold.gold_oa=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 diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql index 5da028304..9ea50d488 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql @@ -104,25 +104,42 @@ create table TARGET.project_results as select id as result, project as id from T compute stats TARGET.project_results; -- indicators -create table TARGET.indi_pub_green_oa as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); -compute stats TARGET.indi_pub_green_oa; - -create table TARGET.indi_pub_grey_lit as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id); -compute stats TARGET.indi_pub_grey_lit; - -create table TARGET.indi_pub_doi_from_crossref as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id); -compute stats TARGET.indi_pub_doi_from_crossref; - -create table TARGET.indi_pub_gold_oa as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); -compute stats TARGET.indi_pub_gold_oa; - +create view TARGET.indi_dataset_avg_year_content_oa as select * from SOURCE.indi_dataset_avg_year_content_oa orig; +create view TARGET.indi_dataset_avg_year_context_oa as select * from SOURCE.indi_dataset_avg_year_context_oa orig; create view TARGET.indi_dataset_avg_year_country_oa as select * from SOURCE.indi_dataset_avg_year_country_oa orig; + +create view TARGET.indi_other_avg_year_content_oa as select * from SOURCE.indi_other_avg_year_content_oa orig; +create view TARGET.indi_other_avg_year_context_oa as select * from SOURCE.indi_other_avg_year_context_oa orig; +create view TARGET.indi_other_avg_year_country_oa as select * from SOURCE.indi_other_avg_year_country_oa orig; + create view TARGET.indi_project_datasets_count as select * from SOURCE.indi_project_datasets_count orig; create view TARGET.indi_project_otherresearch_count as select * from SOURCE.indi_project_otherresearch_count orig; create view TARGET.indi_project_pubs_count as select * from SOURCE.indi_project_pubs_count orig; create view TARGET.indi_project_software_count as select * from SOURCE.indi_project_software_count orig; + +create view TARGET.indi_pub_avg_year_content_oa as select * from SOURCE.indi_pub_avg_year_content_oa orig; +create view TARGET.indi_pub_avg_year_context_oa as select * from SOURCE.indi_pub_avg_year_context_oa orig; create view TARGET.indi_pub_avg_year_country_oa as select * from SOURCE.indi_pub_avg_year_country_oa orig; +create table TARGET.indi_pub_green_oa as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_green_oa; +create table TARGET.indi_pub_grey_lit as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_grey_lit; +create table TARGET.indi_pub_doi_from_crossref as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_doi_from_crossref; +create table TARGET.indi_pub_gold_oa as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_gold_oa; +create table TARGET.indi_pub_has_abstract as select * from SOURCE.indi_pub_has_abstract orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_has_abstract; +create table TARGET.indi_pub_has_cc_licence as select * from SOURCE.indi_pub_has_cc_licence orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_has_cc_licence; +create table TARGET.indi_pub_has_cc_licence_url as select * from SOURCE.indi_pub_has_cc_licence_url orig where exists (select 1 from TARGET.result r where r.id=orig.id); +compute stats TARGET.indi_pub_has_cc_licence_url; + +create view TARGET.indi_software_avg_year_content_oa as select * from SOURCE.indi_software_avg_year_content_oa orig; +create view TARGET.indi_software_avg_year_context_oa as select * from SOURCE.indi_software_avg_year_context_oa orig; +create view TARGET.indi_software_avg_year_country_oa as select * from SOURCE.indi_software_avg_year_country_oa orig; + --denorm alter table TARGET.result rename to TARGET.res_tmp; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql index 40cdf3f6d..e24370e7d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql @@ -1,259 +1,561 @@ -create table TARGET.result_affiliated_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, c.code, c.name; +create table ${observatory_db_name}.result_cc_licence stored as parquet as +select r.id, coalesce(rln.count, 0) > 0 as cc_licence +from ${stats_db_name}.result r + left outer join ( + select rl.id, sum(case when lower(rln.normalized) like 'cc-%' then 1 else 0 end) as count + from ${stats_db_name}.result_licenses rl + left outer join ${stats_db_name}.licenses_normalized rln on rl.type=rln.license + group by rl.id +) rln on rln.id=r.id; -create table TARGET.result_affiliated_year stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, r.year -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year; +create table ${observatory_db_name}.result_affiliated_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; -create table TARGET.result_affiliated_year_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, r.year, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year, c.code, c.name; +create table ${observatory_db_name}.result_affiliated_year stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + r.year +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; -create table TARGET.result_affiliated_datasource stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, d.name as dname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_datasources rd on rd.id=r.id -left outer join SOURCE.datasource d on d.id=rd.datasource -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name; +create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + r.year, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; -create table TARGET.result_affiliated_datasource_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, d.name as dname, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_datasources rd on rd.id=r.id -left outer join SOURCE.datasource d on d.id=rd.datasource -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name, c.code, c.name; +create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + d.name as dname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_datasources rd on rd.id=r.id + left outer join ${stats_db_name}.datasource d on d.id=rd.datasource + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; -create table TARGET.result_affiliated_organization stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, o.name as oname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name; +create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + d.name as dname, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_datasources rd on rd.id=r.id + left outer join ${stats_db_name}.datasource d on d.id=rd.datasource + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; -create table TARGET.result_affiliated_organization_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, o.name as oname, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name, c.code, c.name; +create table ${observatory_db_name}.result_affiliated_organization stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + o.name as oname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; -create table TARGET.result_affiliated_funder stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, p.funder as pfunder -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -join SOURCE.result_projects rp on rp.id=r.id -join SOURCE.project p on p.id=rp.project -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder; +create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + o.name as oname, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; -create table TARGET.result_affiliated_funder_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, p.funder as pfunder, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_organization ro on ro.id=r.id -join SOURCE.organization o on o.id=ro.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -join SOURCE.result_projects rp on rp.id=r.id -join SOURCE.project p on p.id=rp.project -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder, c.code, c.name; +create table ${observatory_db_name}.result_affiliated_funder stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + p.funder as pfunder +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + join ${stats_db_name}.result_projects rp on rp.id=r.id + join ${stats_db_name}.project p on p.id=rp.project + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; -create table TARGET.result_deposited_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, c.code, c.name; +create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + p.funder as pfunder, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.organization o on o.id=ro.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + join ${stats_db_name}.result_projects rp on rp.id=r.id + join ${stats_db_name}.project p on p.id=rp.project + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; -create table TARGET.result_deposited_year stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, r.year -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year; +create table ${observatory_db_name}.result_deposited_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; -create table TARGET.result_deposited_year_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, r.year, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year, c.code, c.name; +create table ${observatory_db_name}.result_deposited_year stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + r.year +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; -create table TARGET.result_deposited_datasource stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, d.name as dname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name; +create table ${observatory_db_name}.result_deposited_year_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + r.year, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; -create table TARGET.result_deposited_datasource_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, d.name as dname, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name, c.code, c.name; +create table ${observatory_db_name}.result_deposited_datasource stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + d.name as dname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; -create table TARGET.result_deposited_organization stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, o.name as oname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name; +create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + d.name as dname, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; -create table TARGET.result_deposited_organization_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, o.name as oname, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name, c.code, c.name; +create table ${observatory_db_name}.result_deposited_organization stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + o.name as oname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; -create table TARGET.result_deposited_funder stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, p.funder as pfunder -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -join SOURCE.result_projects rp on rp.id=r.id -join SOURCE.project p on p.id=rp.project -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder; +create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + o.name as oname, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; -create table TARGET.result_deposited_funder_country stored as parquet as -select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, - case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, - r.peer_reviewed, r.type, p.funder as pfunder, c.code as ccode, c.name as cname -from SOURCE.result r -join SOURCE.result_datasources rd on rd.id=r.id -join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') -join SOURCE.datasource_organizations dor on dor.id=d.id -join SOURCE.organization o on o.id=dor.organization -join SOURCE.country c on c.code=o.country and c.continent_name='Europe' -join SOURCE.result_projects rp on rp.id=r.id -join SOURCE.project p on p.id=rp.project -left outer join SOURCE.result_licenses rl on rl.id=r.id -left outer join SOURCE.result_pids pids on pids.id=r.id -group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder, c.code, c.name; +create table ${observatory_db_name}.result_deposited_funder stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + p.funder as pfunder +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + join ${stats_db_name}.result_projects rp on rp.id=r.id + join ${stats_db_name}.project p on p.id=rp.project + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; -compute stats TARGET.result_affiliated_country; -compute stats TARGET.result_affiliated_year; -compute stats TARGET.result_affiliated_year_country; -compute stats TARGET.result_affiliated_datasource; -compute stats TARGET.result_affiliated_datasource_country; -compute stats TARGET.result_affiliated_organization; -compute stats TARGET.result_affiliated_organization_country; -compute stats TARGET.result_affiliated_funder; -compute stats TARGET.result_affiliated_funder_country; -compute stats TARGET.result_deposited_country; -compute stats TARGET.result_deposited_year; -compute stats TARGET.result_deposited_year_country; -compute stats TARGET.result_deposited_datasource; -compute stats TARGET.result_deposited_datasource_country; -compute stats TARGET.result_deposited_organization; -compute stats TARGET.result_deposited_organization_country; -compute stats TARGET.result_deposited_funder; -compute stats TARGET.result_deposited_funder_country; +create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as +select + count(distinct r.id) as total, + r.green, + r.gold, + case when rl.type is not null then true else false end as licence, + case when pids.pid is not null then true else false end as pid, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, + r.peer_reviewed, + rln.cc_licence, + r.abstract as abstract, + r.authors > 1 as multiple_authors, + rpc.count > 1 as multiple_projects, + rfc.count > 1 as multiple_funders, + r.type, + p.funder as pfunder, c.code as ccode, c.name as cname +from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') + join ${stats_db_name}.datasource_organizations dor on dor.id=d.id + join ${stats_db_name}.organization o on o.id=dor.organization + join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe' + join ${stats_db_name}.result_projects rp on rp.id=r.id + join ${stats_db_name}.project p on p.id=rp.project + left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id + left outer join ${stats_db_name}.result_pids pids on pids.id=r.id + left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id + left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id + left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id +group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, + case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, + cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index a329ca4bf..08d33f4e8 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 @@ -239,14 +239,51 @@ stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} - + - + ${hive_jdbc_url} - + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + + + + + + + + ${jobTracker} + ${nameNode} + contexts.sh + ${context_api_url} + ${stats_db_name} + contexts.sh + + + + + + + + ${jobTracker} + ${nameNode} + indicators.sh + ${stats_db_name} + ${wf:appPath()}/scripts/step16-createIndicatorsTables.sql + indicators.sh + + + + + + + + ${hive_jdbc_url} + stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} @@ -261,48 +298,11 @@ stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} - - - - - - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${jobTracker} - ${nameNode} - indicators.sh - ${stats_db_name} - ${wf:appPath()}/scripts/step16_7-createIndicatorsTables.sql - indicators.sh - - - - - - - - ${jobTracker} - ${nameNode} - contexts.sh - ${context_api_url} - ${stats_db_name} - contexts.sh - - + - + ${jobTracker} ${nameNode} @@ -326,20 +326,44 @@ ${wf:appPath()}/scripts/step20-createMonitorDB.sql monitor.sh + + + + + + + ${jobTracker} + ${nameNode} + observatory-pre.sh + ${stats_db_name} + ${observatory_db_name} + ${observatory_db_shadow_name} + observatory-pre.sh + + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + observatory_db_name=${observatory_db_name} + + + + + + ${jobTracker} ${nameNode} - observatory.sh + observatory-post.sh ${stats_db_name} ${observatory_db_name} ${observatory_db_shadow_name} - ${wf:appPath()}/scripts/step21-createObservatoryDB.sql - observatory.sh + observatory-post.sh