diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index 0b602b774..7a8e55a6e 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -67,6 +67,7 @@ public class AuthorMerger { a -> a .getPid() .stream() + .filter(Objects::nonNull) .map(p -> new Tuple2<>(pidToComparableString(p), a))) .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); @@ -78,6 +79,7 @@ public class AuthorMerger { a -> a .getPid() .stream() + .filter(Objects::nonNull) .filter(p -> !basePidAuthorMap.containsKey(pidToComparableString(p))) .map(p -> new Tuple2<>(p, a))) .collect(Collectors.toList()); @@ -150,7 +152,7 @@ public class AuthorMerger { } private static boolean hasPid(Author a) { - if (a == null || a.getPid() == null || a.getPid().size() == 0) + if (a == null || a.getPid() == null || a.getPid().isEmpty()) return false; return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); } @@ -159,7 +161,10 @@ public class AuthorMerger { if (StringUtils.isNotBlank(author.getSurname())) { return new Person(author.getSurname() + ", " + author.getName(), false); } else { - return new Person(author.getFullname(), false); + if (StringUtils.isNotBlank(author.getFullname())) + return new Person(author.getFullname(), false); + else + return new Person("", false); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/AbstractRestClient.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/AbstractRestClient.scala index 8df203283..92a870e37 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/AbstractRestClient.scala +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/AbstractRestClient.scala @@ -1,9 +1,10 @@ package eu.dnetlib.dhp.actionmanager.datacite import org.apache.commons.io.IOUtils +import org.apache.http.client.config.RequestConfig import org.apache.http.client.methods.{HttpGet, HttpPost, HttpRequestBase, HttpUriRequest} import org.apache.http.entity.StringEntity -import org.apache.http.impl.client.HttpClients +import org.apache.http.impl.client.{HttpClientBuilder, HttpClients} import java.io.IOException @@ -56,31 +57,31 @@ abstract class AbstractRestClient extends Iterator[String]{ private def doHTTPRequest[A <: HttpUriRequest](r: A) :String ={ - val client = HttpClients.createDefault + 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() var tries = 4 - try { - while (tries > 0) { - + while (tries > 0) { println(s"requesting ${r.getURI}") - val response = client.execute(r) - println(s"get response with status${response.getStatusLine.getStatusCode}") - if (response.getStatusLine.getStatusCode > 400) { - tries -= 1 + 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 } - else - return IOUtils.toString(response.getEntity.getContent) } "" - } catch { - case e: Throwable => - throw new RuntimeException("Error on executing request ", e) - } finally try client.close() - catch { - case e: IOException => - throw new RuntimeException("Unable to close client ", e) - } - } - + } getBufferData() - } \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala index 045927bed..cfdd98d30 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala @@ -367,7 +367,7 @@ object DataciteToOAFTransformation { result.setDateofcollection(ISO8601FORMAT.format(d)) - result.setDateoftransformation(ISO8601FORMAT.format(ts)) + result.setDateoftransformation(ISO8601FORMAT.format(d)) result.setDataInfo(dataInfo) val creators = (json \\ "creators").extractOrElse[List[CreatorType]](List()) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/ImportDatacite.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/ImportDatacite.scala index 931ac06f6..2b73d2955 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/ImportDatacite.scala +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/ImportDatacite.scala @@ -140,7 +140,7 @@ object ImportDatacite { private def writeSequenceFile(hdfsTargetPath: Path, timestamp: Long, conf: Configuration, bs:Int): Long = { var from:Long = timestamp * 1000 - val delta:Long = 50000000L + val delta:Long = 100000000L var client: DataciteAPIImporter = null val now :Long =System.currentTimeMillis() var i = 0 diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/scholix/SparkCreateActionset.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/scholix/SparkCreateActionset.scala new file mode 100644 index 000000000..b78f411ee --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/scholix/SparkCreateActionset.scala @@ -0,0 +1,73 @@ +package eu.dnetlib.dhp.actionmanager.scholix + +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result} +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.slf4j.{Logger, LoggerFactory} + +import scala.io.Source + +object SparkCreateActionset { + + def main(args: Array[String]): Unit = { + val log: Logger = LoggerFactory.getLogger(getClass) + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/actionset/generate_actionset.json")).mkString) + parser.parseArgument(args) + + + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + + val sourcePath = parser.get("sourcePath") + log.info(s"sourcePath -> $sourcePath") + + val targetPath = parser.get("targetPath") + log.info(s"targetPath -> $targetPath") + + val workingDirFolder = parser.get("workingDirFolder") + log.info(s"workingDirFolder -> $workingDirFolder") + + implicit val oafEncoders: Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val resultEncoders: Encoder[Result] = Encoders.kryo[Result] + implicit val relationEncoders: Encoder[Relation] = Encoders.kryo[Relation] + + import spark.implicits._ + + val relation = spark.read.load(s"$sourcePath/relation").as[Relation] + + relation.filter(r => (r.getDataInfo == null || r.getDataInfo.getDeletedbyinference == false) && !r.getRelClass.toLowerCase.contains("merge")) + .flatMap(r => List(r.getSource, r.getTarget)).distinct().write.mode(SaveMode.Overwrite).save(s"$workingDirFolder/id_relation") + + + val idRelation = spark.read.load(s"$workingDirFolder/id_relation").as[String] + + log.info("extract source and target Identifier involved in relations") + + + log.info("save relation filtered") + + relation.filter(r => (r.getDataInfo == null || r.getDataInfo.getDeletedbyinference == false) && !r.getRelClass.toLowerCase.contains("merge")) + .write.mode(SaveMode.Overwrite).save(s"$workingDirFolder/actionSetOaf") + + log.info("saving entities") + + val entities: Dataset[(String, Result)] = spark.read.load(s"$sourcePath/entities/*").as[Result].map(p => (p.getId, p))(Encoders.tuple(Encoders.STRING, resultEncoders)) + + + entities.filter(r => r.isInstanceOf[Result]).map(r => r.asInstanceOf[Result]) + entities + .joinWith(idRelation, entities("_1").equalTo(idRelation("value"))) + .map(p => p._1._2) + .write.mode(SaveMode.Append).save(s"$workingDirFolder/actionSetOaf") + + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/scholix/SparkSaveActionSet.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/scholix/SparkSaveActionSet.scala new file mode 100644 index 000000000..1df7ea3fb --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/scholix/SparkSaveActionSet.scala @@ -0,0 +1,86 @@ +package eu.dnetlib.dhp.actionmanager.scholix + +import com.fasterxml.jackson.databind.ObjectMapper +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.action.AtomicAction +import eu.dnetlib.dhp.schema.oaf.{Oaf, Dataset => OafDataset,Publication, Software, OtherResearchProduct, Relation} +import org.apache.hadoop.io.Text +import org.apache.hadoop.io.compress.GzipCodec +import org.apache.hadoop.mapred.SequenceFileOutputFormat +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +import scala.io.Source + +object SparkSaveActionSet { + + + def toActionSet(item: Oaf): (String, String) = { + val mapper = new ObjectMapper() + + item match { + case dataset: OafDataset => + val a: AtomicAction[OafDataset] = new AtomicAction[OafDataset] + a.setClazz(classOf[OafDataset]) + a.setPayload(dataset) + (dataset.getClass.getCanonicalName, mapper.writeValueAsString(a)) + case publication: Publication => + val a: AtomicAction[Publication] = new AtomicAction[Publication] + a.setClazz(classOf[Publication]) + a.setPayload(publication) + (publication.getClass.getCanonicalName, mapper.writeValueAsString(a)) + case software: Software => + val a: AtomicAction[Software] = new AtomicAction[Software] + a.setClazz(classOf[Software]) + a.setPayload(software) + (software.getClass.getCanonicalName, mapper.writeValueAsString(a)) + case orp: OtherResearchProduct => + val a: AtomicAction[OtherResearchProduct] = new AtomicAction[OtherResearchProduct] + a.setClazz(classOf[OtherResearchProduct]) + a.setPayload(orp) + (orp.getClass.getCanonicalName, mapper.writeValueAsString(a)) + + case relation: Relation => + val a: AtomicAction[Relation] = new AtomicAction[Relation] + a.setClazz(classOf[Relation]) + a.setPayload(relation) + (relation.getClass.getCanonicalName, mapper.writeValueAsString(a)) + case _ => + null + } + + } + + def main(args: Array[String]): Unit = { + val log: Logger = LoggerFactory.getLogger(getClass) + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/actionset/save_actionset.json")).mkString) + parser.parseArgument(args) + + + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + + val sourcePath = parser.get("sourcePath") + log.info(s"sourcePath -> $sourcePath") + + val targetPath = parser.get("targetPath") + log.info(s"targetPath -> $targetPath") + + implicit val oafEncoders: Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val tEncoder: Encoder[(String, String)] = Encoders.tuple(Encoders.STRING, Encoders.STRING) + + spark.read.load(sourcePath).as[Oaf] + .map(o => toActionSet(o)) + .filter(o => o != null) + .rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$targetPath", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text, Text]], classOf[GzipCodec]) + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml index 69fc95957..021704f54 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml @@ -4,10 +4,6 @@ mainPath the working path of Datacite stores - - oafTargetPath - the target path where the OAF records are stored - isLookupUrl The IS lookUp service endopoint @@ -17,26 +13,15 @@ 100 The request block size - - exportLinks - false - instructs the transformation phase to produce the links or not - - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - ${wf:conf('resumeFrom') eq 'TransformDatacite'} - - - @@ -60,11 +45,12 @@ --masteryarn-cluster --blocksize${blocksize} - + - + + yarn-cluster cluster @@ -82,9 +68,9 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${mainPath}/datacite_dump - --targetPath${oafTargetPath} + --targetPath${mainPath}/datacite_oaf --isLookupUrl${isLookupUrl} - --exportLinks${exportLinks} + --exportLinkstrue --masteryarn-cluster diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/generate_actionset.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/generate_actionset.json new file mode 100644 index 000000000..0563808ea --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/generate_actionset.json @@ -0,0 +1,6 @@ +[ + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath","paramDescription": "source path", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingDirFolder","paramDescription": "the working Dir Folder", "paramRequired": true}, + {"paramName":"t", "paramLongName":"targetPath","paramDescription": "the target path ", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/oozie_app/config-default.xml new file mode 100644 index 000000000..dd3c32c62 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/oozie_app/config-default.xml @@ -0,0 +1,23 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + + oozie.launcher.mapreduce.user.classpath.first + true + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/oozie_app/workflow.xml new file mode 100644 index 000000000..2d97b5163 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/oozie_app/workflow.xml @@ -0,0 +1,76 @@ + + + + sourcePath + the path of the consistent graph + + + workingDirFolder + the path of working dir ActionSet + + + outputPath + the path of Scholexplorer ActionSet + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn-cluster + cluster + Create Action Set + eu.dnetlib.dhp.actionmanager.scholix.SparkCreateActionset + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=3840 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --targetPath${outputPath} + --workingDirFolder${workingDirFolder} + --masteryarn-cluster + + + + + + + + + yarn-cluster + cluster + Save Action Set + eu.dnetlib.dhp.actionmanager.scholix.SparkSaveActionSet + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=3840 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${workingDirFolder}/actionSetOaf + --targetPath${outputPath} + --masteryarn-cluster + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/save_actionset.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/save_actionset.json new file mode 100644 index 000000000..0264c825f --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/actionset/save_actionset.json @@ -0,0 +1,5 @@ +[ + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath","paramDescription": "source path", "paramRequired": true}, + {"paramName":"t", "paramLongName":"targetPath","paramDescription": "the target path ", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTest.scala b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTest.scala index 0d10c41dc..a795a910d 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTest.scala @@ -3,13 +3,14 @@ package eu.dnetlib.dhp.actionmanager.datacite import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.databind.SerializationFeature - import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest import eu.dnetlib.dhp.schema.oaf.Oaf import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.junit.jupiter.MockitoExtension +import java.text.SimpleDateFormat +import java.util.Locale import scala.io.Source @ExtendWith(Array(classOf[MockitoExtension])) @@ -22,6 +23,18 @@ class DataciteToOAFTest extends AbstractVocabularyTest{ super.setUpVocabulary() } + + @Test + def testDateMapping:Unit = { + val inputDate = "2021-07-14T11:52:54+0000" + val ISO8601FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.US) + val dt = ISO8601FORMAT.parse(inputDate) + println(dt.getTime) + + + } + + @Test def testMapping() :Unit = { val record =Source.fromInputStream(getClass.getResourceAsStream("record.json")).mkString diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/record.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/record.json index 3ae10be73..f5aa65940 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/record.json +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/record.json @@ -1 +1 @@ -{"id":"10.7282/t3-sjyd-1r46","type":"dois","attributes":{"doi":"10.7282/t3-sjyd-1r46","identifiers":[],"creators":[{"name":"Huang, Xu","nameType":"Personal","givenName":"Xu","familyName":"Huang","affiliation":[],"nameIdentifiers":[]},{"name":"Guo, Zhixiong","nameType":"Personal","givenName":"Zhixiong","familyName":"Guo","nameIdentifiers":[{"schemeUri":"https://orcid.org","nameIdentifier":"https://orcid.org/0000-0003-0481-2738","nameIdentifierScheme":"ORCID"}],"affiliation":[]}],"titles":[{"title":"High thermal conductance across c-BN/diamond interface"}],"publisher":"Rutgers University","container":{},"publicationYear":2099,"subjects":[{"subject":"Diamond"},{"subject":"Cubic boron nitride"},{"subject":"Thermal conductivity"},{"subject":"Interface"},{"subject":"Phonon"},{"subject":"Thermal conductance"}],"contributors":[],"dates":[{"date":"2099-12-31","dateType":"Accepted"},{"date":"2099","dateType":"Issued"}],"language":"en","types":{"ris":"RPRT","bibtex":"article","citeproc":"article-journal","schemaOrg":"ScholarlyArticle","resourceType":"Accepted manuscript","resourceTypeGeneral":"Text"},"relatedIdentifiers":[],"sizes":[],"formats":["application/pdf"],"version":null,"rightsList":[{"rights":"Embargo"}],"descriptions":[{"description":"High thermal conductivity electronic components with low interfacial thermal resistance are of technological importance and fundamental interest of research. Diamond, a superhard material with ultrahigh thermal conductivity at room temperature, is desirable for microelectronics thermal management. Cubic polymorph of boron nitride (c-BN) is a promising material due to wide bandgap and diamond like structure and properties. To understand the nature in thermal transport of diamond, c-BN and the most commonly used silicon (Si) semiconductor, ab initio phonon Boltzmann transport equations are employed to investigate lattice vibrational properties of these three materials. At 300 K, the predicted thermal conductivity of Si, diamond and c-BN reached 142, 2112, and 736 W/(m��K), respectively. What's more, heat transport phenomena across the interfaces of Si/diamond, c-BN/diamond and Si/c-BN are unfolded. In comparison, the interfacial thermal conductance of c-BN/diamond is ten-fold of Si/diamond; besides, the thermal conductance across Si/c-BN interface is 20.2% larger than that of Si/diamond at 300 K and 18.9% larger at 340 K. These findings provide us new vision and potential solution to heat dissipation of high-local-power density devices, shedding light on future thermal management of c-BN and diamond related electronics.","descriptionType":"Abstract"}],"geoLocations":[],"fundingReferences":[],"url":"https://scholarship.libraries.rutgers.edu/discovery/fulldisplay/alma991031549917804646/01RUT_INST:ResearchRepository","contentUrl":null,"metadataVersion":1,"schemaVersion":"http://datacite.org/schema/kernel-4","source":"mds","isActive":true,"state":"findable","reason":null,"viewCount":0,"downloadCount":0,"referenceCount":0,"citationCount":0,"partCount":0,"partOfCount":0,"versionCount":0,"versionOfCount":0,"created":"2020-06-30T21:12:19Z","registered":"2020-07-02T16:45:07Z","published":null,"updated":"2021-01-14T18:24:19Z"},"relationships":{"client":{"data":{"id":"rutgers.lib","type":"clients"}}}} \ No newline at end of file +{"id":"10.5517/ccdc.csd.cc25rpzm","type":"dois","attributes":{"doi":"10.5517/ccdc.csd.cc25rpzm","prefix":"10.5517","suffix":"ccdc.csd.cc25rpzm","identifiers":[{"identifier":"2018781","identifierType":"CCDC"}],"alternateIdentifiers":[{"alternateIdentifierType":"CCDC","alternateIdentifier":"2018781"}],"creators":[{"name":"Ling, Irene","affiliation":[],"nameIdentifiers":[]},{"name":"Sobolev, Alexandre N.","affiliation":[],"nameIdentifiers":[]},{"name":"Raston, Colin L.","affiliation":[],"nameIdentifiers":[]}],"titles":[{"title":"CCDC 2018781: Experimental Crystal Structure Determination"}],"publisher":"Cambridge Crystallographic Data Centre","container":{},"publicationYear":2021,"subjects":[{"subject":"Crystal Structure"},{"subject":"Experimental 3D Coordinates"},{"subject":"Crystal System"},{"subject":"Space Group"},{"subject":"Cell Parameters"},{"subject":"Crystallography"},{"subject":"bis[penta-aqua-copper(ii)] bis(mu-5,11,17,23-tetra-sulfonato-25,26,27,28-tetrahydroxycalix(4)arene)-dodeca-aqua-tri-copper(ii) bis(nitrate) heptahydrate"}],"contributors":[],"dates":[],"language":"en","types":{"ris":"DATA","bibtex":"misc","citeproc":"dataset","schemaOrg":"Dataset","resourceTypeGeneral":"Dataset"},"relatedIdentifiers":[{"relationType":"IsSupplementTo","relatedIdentifier":"10.1080/00958972.2020.1849642","relatedIdentifierType":"DOI"}],"sizes":[],"formats":["CIF"],"version":null,"rightsList":[],"descriptions":[{"description":"Related Article: Irene Ling, Alexandre N. Sobolev, Colin L. Raston|2021|J.Coord.Chem.|74|40|doi:10.1080/00958972.2020.1849642","descriptionType":"Other"}],"geoLocations":[],"fundingReferences":[],"xml":"PD94bWwgdmVyc2lvbj0iMS4wIiBlbmNvZGluZz0iVVRGLTgiPz4KPHJlc291cmNlIHhtbG5zOnhzaT0iaHR0cDovL3d3dy53My5vcmcvMjAwMS9YTUxTY2hlbWEtaW5zdGFuY2UiIHhtbG5zPSJodHRwOi8vZGF0YWNpdGUub3JnL3NjaGVtYS9rZXJuZWwtNCIgeHNpOnNjaGVtYUxvY2F0aW9uPSJodHRwOi8vZGF0YWNpdGUub3JnL3NjaGVtYS9rZXJuZWwtNCBodHRwOi8vc2NoZW1hLmRhdGFjaXRlLm9yZy9tZXRhL2tlcm5lbC00L21ldGFkYXRhLnhzZCI+CiAgPGlkZW50aWZpZXIgaWRlbnRpZmllclR5cGU9IkRPSSI+MTAuNTUxNy9DQ0RDLkNTRC5DQzI1UlBaTTwvaWRlbnRpZmllcj4KICA8Y3JlYXRvcnM+CiAgICA8Y3JlYXRvcj4KICAgICAgPGNyZWF0b3JOYW1lPkxpbmcsIElyZW5lPC9jcmVhdG9yTmFtZT4KICAgIDwvY3JlYXRvcj4KICAgIDxjcmVhdG9yPgogICAgICA8Y3JlYXRvck5hbWU+U29ib2xldiwgQWxleGFuZHJlIE4uPC9jcmVhdG9yTmFtZT4KICAgIDwvY3JlYXRvcj4KICAgIDxjcmVhdG9yPgogICAgICA8Y3JlYXRvck5hbWU+UmFzdG9uLCBDb2xpbiBMLjwvY3JlYXRvck5hbWU+CiAgICA8L2NyZWF0b3I+CiAgPC9jcmVhdG9ycz4KICA8dGl0bGVzPgogICAgPHRpdGxlPkNDREMgMjAxODc4MTogRXhwZXJpbWVudGFsIENyeXN0YWwgU3RydWN0dXJlIERldGVybWluYXRpb248L3RpdGxlPgogIDwvdGl0bGVzPgogIDxwdWJsaXNoZXI+Q2FtYnJpZGdlIENyeXN0YWxsb2dyYXBoaWMgRGF0YSBDZW50cmU8L3B1Ymxpc2hlcj4KICA8cHVibGljYXRpb25ZZWFyPjIwMjE8L3B1YmxpY2F0aW9uWWVhcj4KICA8cmVzb3VyY2VUeXBlIHJlc291cmNlVHlwZUdlbmVyYWw9IkRhdGFzZXQiLz4KICA8c3ViamVjdHM+CiAgICA8c3ViamVjdD5DcnlzdGFsIFN0cnVjdHVyZTwvc3ViamVjdD4KICAgIDxzdWJqZWN0PkV4cGVyaW1lbnRhbCAzRCBDb29yZGluYXRlczwvc3ViamVjdD4KICAgIDxzdWJqZWN0PkNyeXN0YWwgU3lzdGVtPC9zdWJqZWN0PgogICAgPHN1YmplY3Q+U3BhY2UgR3JvdXA8L3N1YmplY3Q+CiAgICA8c3ViamVjdD5DZWxsIFBhcmFtZXRlcnM8L3N1YmplY3Q+CiAgICA8c3ViamVjdD5DcnlzdGFsbG9ncmFwaHk8L3N1YmplY3Q+CiAgICA8c3ViamVjdD5iaXNbcGVudGEtYXF1YS1jb3BwZXIoaWkpXSBiaXMobXUtNSwxMSwxNywyMy10ZXRyYS1zdWxmb25hdG8tMjUsMjYsMjcsMjgtdGV0cmFoeWRyb3h5Y2FsaXgoNClhcmVuZSktZG9kZWNhLWFxdWEtdHJpLWNvcHBlcihpaSkgYmlzKG5pdHJhdGUpIGhlcHRhaHlkcmF0ZTwvc3ViamVjdD4KICA8L3N1YmplY3RzPgogIDxsYW5ndWFnZT5lbmc8L2xhbmd1YWdlPgogIDxhbHRlcm5hdGVJZGVudGlmaWVycz4KICAgIDxhbHRlcm5hdGVJZGVudGlmaWVyIGFsdGVybmF0ZUlkZW50aWZpZXJUeXBlPSJDQ0RDIj4yMDE4NzgxPC9hbHRlcm5hdGVJZGVudGlmaWVyPgogIDwvYWx0ZXJuYXRlSWRlbnRpZmllcnM+CiAgPHJlbGF0ZWRJZGVudGlmaWVycz4KICAgIDxyZWxhdGVkSWRlbnRpZmllciByZWxhdGVkSWRlbnRpZmllclR5cGU9IkRPSSIgcmVsYXRpb25UeXBlPSJJc1N1cHBsZW1lbnRUbyI+MTAuMTA4MC8wMDk1ODk3Mi4yMDIwLjE4NDk2NDI8L3JlbGF0ZWRJZGVudGlmaWVyPgogIDwvcmVsYXRlZElkZW50aWZpZXJzPgogIDxzaXplcy8+CiAgPGZvcm1hdHM+CiAgICA8Zm9ybWF0PkNJRjwvZm9ybWF0PgogIDwvZm9ybWF0cz4KICA8dmVyc2lvbi8+CiAgPGRlc2NyaXB0aW9ucz4KICAgIDxkZXNjcmlwdGlvbiBkZXNjcmlwdGlvblR5cGU9Ik90aGVyIj5SZWxhdGVkIEFydGljbGU6IElyZW5lIExpbmcsICBBbGV4YW5kcmUgTi4gU29ib2xldiwgIENvbGluIEwuIFJhc3RvbnwyMDIxfEouQ29vcmQuQ2hlbS58NzR8NDB8ZG9pOjEwLjEwODAvMDA5NTg5NzIuMjAyMC4xODQ5NjQyPC9kZXNjcmlwdGlvbj4KICA8L2Rlc2NyaXB0aW9ucz4KPC9yZXNvdXJjZT4K","url":"http://www.ccdc.cam.ac.uk/services/structure_request?id=doi:10.5517/ccdc.csd.cc25rpzm&sid=DataCite","contentUrl":null,"metadataVersion":3,"schemaVersion":"http://datacite.org/schema/kernel-4","source":"api","isActive":true,"state":"findable","reason":null,"viewCount":0,"viewsOverTime":[],"downloadCount":0,"downloadsOverTime":[],"referenceCount":0,"citationCount":0,"citationsOverTime":[],"partCount":0,"partOfCount":0,"versionCount":0,"versionOfCount":0,"created":"2021-03-09T13:25:35.000Z","registered":"2021-03-09T13:25:36.000Z","published":"2021","updated":"2021-03-31T21:49:56.000Z"},"relationships":{"client":{"data":{"id":"ccdc.csd","type":"clients"}},"provider":{"data":{"id":"ccdc","type":"providers"}},"media":{"data":{"id":"10.5517/ccdc.csd.cc25rpzm","type":"media"}},"references":{"data":[]},"citations":{"data":[]},"parts":{"data":[]},"partOf":{"data":[]},"versions":{"data":[]},"versionOf":{"data":[]}}} \ No newline at end of file diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsReferencedBy.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsReferencedBy.java index 21786687e..bcbcf755f 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsReferencedBy.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsReferencedBy.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingDatasetIsReferencedBy extends AbstractEnrichMissingDataset { @@ -11,7 +12,7 @@ public class EnrichMissingDatasetIsReferencedBy extends AbstractEnrichMissingDat @Override protected boolean filterByType(final String relType) { - return relType.equals("isReferencedBy"); + return relType.equals(ModelConstants.IS_REFERENCED_BY); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsRelatedTo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsRelatedTo.java index 0f3739434..4125974ce 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsRelatedTo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsRelatedTo.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingDatasetIsRelatedTo extends AbstractEnrichMissingDataset { @@ -11,7 +12,7 @@ public class EnrichMissingDatasetIsRelatedTo extends AbstractEnrichMissingDatase @Override protected boolean filterByType(final String relType) { - return relType.equals("isRelatedTo"); + return relType.equals(ModelConstants.IS_RELATED_TO); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedBy.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedBy.java index cde227fee..480daf666 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedBy.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedBy.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingDatasetIsSupplementedBy extends AbstractEnrichMissingDataset { @@ -11,7 +12,7 @@ public class EnrichMissingDatasetIsSupplementedBy extends AbstractEnrichMissingD @Override protected boolean filterByType(final String relType) { - return relType.equals("isSupplementedBy"); + return relType.equals(ModelConstants.IS_SUPPLEMENTED_BY); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedTo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedTo.java index 750165ff5..97b1eb8bd 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedTo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetIsSupplementedTo.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingDatasetIsSupplementedTo extends AbstractEnrichMissingDataset { @@ -11,7 +12,7 @@ public class EnrichMissingDatasetIsSupplementedTo extends AbstractEnrichMissingD @Override protected boolean filterByType(final String relType) { - return relType.equals("isSupplementedTo"); + return relType.equals(ModelConstants.IS_SUPPLEMENT_TO); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetReferences.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetReferences.java index b1c0afe16..0978486a3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetReferences.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/EnrichMissingDatasetReferences.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingDatasetReferences extends AbstractEnrichMissingDataset { @@ -11,7 +12,7 @@ public class EnrichMissingDatasetReferences extends AbstractEnrichMissingDataset @Override protected boolean filterByType(final String relType) { - return relType.equals("references"); + return relType.equals(ModelConstants.REFERENCES); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsReferencedBy.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsReferencedBy.java index eebb5c1a6..ff9155c9d 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsReferencedBy.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsReferencedBy.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingPublicationIsReferencedBy extends AbstractEnrichMissingPublication { @@ -11,6 +12,6 @@ public class EnrichMissingPublicationIsReferencedBy extends AbstractEnrichMissin @Override protected boolean filterByType(final String relType) { - return relType.equals("isReferencedBy"); + return relType.equals(ModelConstants.IS_REFERENCED_BY); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsRelatedTo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsRelatedTo.java index a8aa550d4..1051559c9 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsRelatedTo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsRelatedTo.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingPublicationIsRelatedTo extends AbstractEnrichMissingPublication { @@ -11,7 +12,7 @@ public class EnrichMissingPublicationIsRelatedTo extends AbstractEnrichMissingPu @Override protected boolean filterByType(final String relType) { - return relType.equals("isRelatedTo"); + return relType.equals(ModelConstants.IS_RELATED_TO); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedBy.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedBy.java index 762ac942e..d97f46f09 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedBy.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedBy.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingPublicationIsSupplementedBy extends AbstractEnrichMissingPublication { @@ -11,6 +12,6 @@ public class EnrichMissingPublicationIsSupplementedBy extends AbstractEnrichMiss @Override protected boolean filterByType(final String relType) { - return relType.equals("isSupplementedBy"); + return relType.equals(ModelConstants.IS_SUPPLEMENTED_BY); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedTo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedTo.java index fc7196a01..b33b340e3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedTo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationIsSupplementedTo.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingPublicationIsSupplementedTo extends AbstractEnrichMissingPublication { @@ -11,7 +12,7 @@ public class EnrichMissingPublicationIsSupplementedTo extends AbstractEnrichMiss @Override protected boolean filterByType(final String relType) { - return relType.equals("isSupplementedTo"); + return relType.equals(ModelConstants.IS_SUPPLEMENT_TO); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationReferences.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationReferences.java index da1994454..fe0f96b6e 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationReferences.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/EnrichMissingPublicationReferences.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; import eu.dnetlib.dhp.broker.model.Topic; +import eu.dnetlib.dhp.schema.common.ModelConstants; public class EnrichMissingPublicationReferences extends AbstractEnrichMissingPublication { @@ -11,7 +12,7 @@ public class EnrichMissingPublicationReferences extends AbstractEnrichMissingPub @Override protected boolean filterByType(final String relType) { - return relType.equals("references"); + return relType.equals(ModelConstants.REFERENCES); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java index c7be633a9..7c4ca1d22 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java @@ -17,6 +17,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; public class ClusterUtils { @@ -52,15 +53,15 @@ public class ClusterUtils { } public static boolean isDedupRoot(final String id) { - return id.contains("dedup_wf_"); + return id.contains("dedup"); } public static final boolean isValidResultResultClass(final String s) { - return s.equals("isReferencedBy") - || s.equals("isRelatedTo") - || s.equals("references") - || s.equals("isSupplementedBy") - || s.equals("isSupplementedTo"); + return s.equals(ModelConstants.IS_REFERENCED_BY) + || s.equals(ModelConstants.IS_RELATED_TO) + || s.equals(ModelConstants.REFERENCES) + || s.equals(ModelConstants.IS_SUPPLEMENTED_BY) + || s.equals(ModelConstants.IS_SUPPLEMENT_TO); } public static T incrementAccumulator(final T o, final LongAccumulator acc) { diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java index 03709c8fe..fdef7f77d 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java @@ -23,6 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; @@ -77,48 +78,54 @@ public class SparkUpdateEntity extends AbstractSparkAction { (type, clazz) -> { final String outputPath = dedupGraphPath + "/" + type; removeOutputDir(spark, outputPath); + final String ip = DedupUtility.createEntityPath(graphBasePath, type.toString()); + if (HdfsSupport.exists(ip, sc.hadoopConfiguration())) { + JavaRDD sourceEntity = sc + .textFile(DedupUtility.createEntityPath(graphBasePath, type.toString())); - JavaRDD sourceEntity = sc - .textFile(DedupUtility.createEntityPath(graphBasePath, type.toString())); + if (mergeRelExists(workingPath, type.toString())) { - if (mergeRelExists(workingPath, type.toString())) { + final String mergeRelPath = DedupUtility + .createMergeRelPath(workingPath, "*", type.toString()); + final String dedupRecordPath = DedupUtility + .createDedupRecordPath(workingPath, "*", type.toString()); - final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, "*", type.toString()); - final String dedupRecordPath = DedupUtility - .createDedupRecordPath(workingPath, "*", type.toString()); + final Dataset rel = spark + .read() + .load(mergeRelPath) + .as(Encoders.bean(Relation.class)); - final Dataset rel = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class)); + final JavaPairRDD mergedIds = rel + .where("relClass == 'merges'") + .where("source != target") + .select(rel.col("target")) + .distinct() + .toJavaRDD() + .mapToPair( + (PairFunction) r -> new Tuple2<>(r.getString(0), "d")); - final JavaPairRDD mergedIds = rel - .where("relClass == 'merges'") - .where("source != target") - .select(rel.col("target")) - .distinct() - .toJavaRDD() - .mapToPair( - (PairFunction) r -> new Tuple2<>(r.getString(0), "d")); + JavaPairRDD entitiesWithId = sourceEntity + .mapToPair( + (PairFunction) s -> new Tuple2<>( + MapDocumentUtil.getJPathString(IDJSONPATH, s), s)); + if (type == EntityType.organization) // exclude root records from organizations + entitiesWithId = excludeRootOrgs(entitiesWithId, rel); - JavaPairRDD entitiesWithId = sourceEntity - .mapToPair( - (PairFunction) s -> new Tuple2<>( - MapDocumentUtil.getJPathString(IDJSONPATH, s), s)); - if (type == EntityType.organization) // exclude root records from organizations - entitiesWithId = excludeRootOrgs(entitiesWithId, rel); + JavaRDD map = entitiesWithId + .leftOuterJoin(mergedIds) + .map(k -> { + if (k._2()._2().isPresent()) { + return updateDeletedByInference(k._2()._1(), clazz); + } + return k._2()._1(); + }); - JavaRDD map = entitiesWithId - .leftOuterJoin(mergedIds) - .map(k -> { - if (k._2()._2().isPresent()) { - return updateDeletedByInference(k._2()._1(), clazz); - } - return k._2()._1(); - }); + sourceEntity = map.union(sc.textFile(dedupRecordPath)); - sourceEntity = map.union(sc.textFile(dedupRecordPath)); + } + sourceEntity.saveAsTextFile(outputPath, GzipCodec.class); } - - sourceEntity.saveAsTextFile(outputPath, GzipCodec.class); }); } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/DoiBoostMappingUtil.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/DoiBoostMappingUtil.scala index c0939fec1..e68880433 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/DoiBoostMappingUtil.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/DoiBoostMappingUtil.scala @@ -1,12 +1,16 @@ package eu.dnetlib.doiboost +import java.time.LocalDate +import java.time.format.DateTimeFormatter + import eu.dnetlib.dhp.schema.action.AtomicAction -import eu.dnetlib.dhp.schema.oaf.{AccessRight, DataInfo, Dataset, Field, Instance, KeyValue, Oaf, Organization, Publication, Qualifier, Relation, Result, StructuredProperty} +import eu.dnetlib.dhp.schema.oaf.{AccessRight, DataInfo, Dataset, Field, Instance, KeyValue, Oaf, OpenAccessRoute, Organization, Publication, Qualifier, Relation, Result, StructuredProperty} import eu.dnetlib.dhp.utils.DHPUtils import org.apache.commons.lang3.StringUtils import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.schema.common.ModelConstants import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils +import eu.dnetlib.doiboost.DoiBoostMappingUtil.{getClosedAccessQualifier, getEmbargoedAccessQualifier, getUnknownQualifier} import org.json4s import org.json4s.DefaultFormats import org.json4s.jackson.JsonMethods.parse @@ -118,14 +122,92 @@ object DoiBoostMappingUtil { } + def decideAccessRight(lic : Field[String], date:String) : AccessRight = { + if(lic == null){ + //Default value Unknown + return getUnknownQualifier() + } + val license : String = lic.getValue + //CC licenses + if(license.startsWith("cc") || + license.startsWith("http://creativecommons.org/licenses") || + license.startsWith("https://creativecommons.org/licenses") || + + //ACS Publications Author choice licenses (considered OPEN also by Unpaywall) + license.equals("http://pubs.acs.org/page/policy/authorchoice_ccby_termsofuse.html") || + license.equals("http://pubs.acs.org/page/policy/authorchoice_termsofuse.html") || + license.equals("http://pubs.acs.org/page/policy/authorchoice_ccbyncnd_termsofuse.html") || + + //APA (considered OPEN also by Unpaywall) + license.equals("http://www.apa.org/pubs/journals/resources/open-access.aspx")){ + + val oaq : AccessRight = getOpenAccessQualifier() + oaq.setOpenAccessRoute(OpenAccessRoute.hybrid) + return oaq + } + + //OUP (BUT ONLY AFTER 12 MONTHS FROM THE PUBLICATION DATE, OTHERWISE THEY ARE EMBARGOED) + if(license.equals("https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model")){ + val now = java.time.LocalDate.now + + try{ + val pub_date = LocalDate.parse(date, DateTimeFormatter.ofPattern("yyyy-MM-dd")) + if (((now.toEpochDay - pub_date.toEpochDay)/365.0) > 1){ + val oaq : AccessRight = getOpenAccessQualifier() + oaq.setOpenAccessRoute(OpenAccessRoute.hybrid) + return oaq + } + else{ + return getEmbargoedAccessQualifier() + } + }catch { + case e: Exception => { + try{ + val pub_date = LocalDate.parse(date, DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'")) + if (((now.toEpochDay - pub_date.toEpochDay)/365.0) > 1){ + val oaq : AccessRight = getOpenAccessQualifier() + oaq.setOpenAccessRoute(OpenAccessRoute.hybrid) + return oaq + } + else{ + return getEmbargoedAccessQualifier() + } + }catch{ + case ex: Exception => return getClosedAccessQualifier() + } + } + + } + + } + + return getClosedAccessQualifier() + + } + def getOpenAccessQualifier():AccessRight = { - OafMapperUtils.accessRight("OPEN","Open Access", ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) + + OafMapperUtils.accessRight(ModelConstants.ACCESS_RIGHT_OPEN,"Open Access", ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) } def getRestrictedQualifier():AccessRight = { - OafMapperUtils.accessRight("RESTRICTED","Restricted",ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) + OafMapperUtils.accessRight( "RESTRICTED","Restricted",ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) + } + + + def getUnknownQualifier():AccessRight = { + OafMapperUtils.accessRight(ModelConstants.UNKNOWN, ModelConstants.NOT_AVAILABLE,ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) + } + + + def getEmbargoedAccessQualifier():AccessRight = { + OafMapperUtils.accessRight("EMBARGO","Embargo",ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) + } + + def getClosedAccessQualifier():AccessRight = { + OafMapperUtils.accessRight("CLOSED","Closed Access", ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES) } @@ -150,10 +232,11 @@ object DoiBoostMappingUtil { if (item != null) { hb.setValue(item.officialname) hb.setKey(generateDSId(item.id)) - if (item.openAccess) + if (item.openAccess) { i.setAccessright(getOpenAccessQualifier()) - val ar = getOpenAccessQualifier() - publication.setBestaccessright(OafMapperUtils.qualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename)) + i.getAccessright.setOpenAccessRoute(OpenAccessRoute.gold) + } + } else { hb = ModelConstants.UNKNOWN_REPOSITORY @@ -161,17 +244,8 @@ object DoiBoostMappingUtil { i.setHostedby(hb) }) - val ar = publication.getInstance().asScala.filter(i => i.getInstancetype != null && i.getAccessright!= null && i.getAccessright.getClassid!= null).map(f=> f.getAccessright.getClassid) - if (ar.nonEmpty) { - if(ar.contains(ModelConstants.ACCESS_RIGHT_OPEN)){ - val ar = getOpenAccessQualifier() - publication.setBestaccessright(OafMapperUtils.qualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename)) - } - else { - val ar = getRestrictedQualifier() - publication.setBestaccessright(OafMapperUtils.qualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename)) - } - } + publication.setBestaccessright(OafMapperUtils.createBestAccessRights(publication.getInstance())) + publication } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index fb96717d9..25f0ff381 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -4,7 +4,7 @@ import eu.dnetlib.dhp.schema.common.ModelConstants import eu.dnetlib.dhp.schema.oaf._ import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, OafMapperUtils} import eu.dnetlib.dhp.utils.DHPUtils -import eu.dnetlib.doiboost.DoiBoostMappingUtil._ +import eu.dnetlib.doiboost.DoiBoostMappingUtil.{decideAccessRight, _} import org.apache.commons.lang.StringUtils import org.json4s import org.json4s.DefaultFormats @@ -168,12 +168,22 @@ case object Crossref2Oaf { // Mapping instance val instance = new Instance() val license = for { - JString(lic) <- json \ "license" \ "URL" - } yield asField(lic) - val l = license.filter(d => StringUtils.isNotBlank(d.getValue)) - if (l.nonEmpty) - instance.setLicense(l.head) - + JObject(license) <- json \ "license" + JField("URL", JString(lic)) <- license + JField("content-version", JString(content_version)) <- license + } yield (asField(lic), content_version) + val l = license.filter(d => StringUtils.isNotBlank(d._1.getValue)) + if (l.nonEmpty){ + if (l exists (d => d._2.equals("vor"))){ + for(d <- l){ + if (d._2.equals("vor")){ + instance.setLicense(d._1) + } + } + } + else{ + instance.setLicense(l.head._1)} + } // Ticket #6281 added pid to Instance instance.setPid(result.getPid) @@ -185,7 +195,7 @@ case object Crossref2Oaf { OafMapperUtils.qualifier("0001", "peerReviewed", ModelConstants.DNET_REVIEW_LEVELS, ModelConstants.DNET_REVIEW_LEVELS)) } - instance.setAccessright(getRestrictedQualifier()) + instance.setAccessright(decideAccessRight(instance.getLicense, result.getDateofacceptance.getValue)) instance.setInstancetype(OafMapperUtils.qualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE)) result.setResourcetype(OafMapperUtils.qualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE)) diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/UnpayWallToOAF.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/UnpayWallToOAF.scala index cc758bcae..c8324cde1 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/UnpayWallToOAF.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/UnpayWallToOAF.scala @@ -11,6 +11,7 @@ import org.slf4j.{Logger, LoggerFactory} import scala.collection.JavaConverters._ import eu.dnetlib.doiboost.DoiBoostMappingUtil._ +import eu.dnetlib.doiboost.uw.UnpayWallToOAF.get_unpaywall_color @@ -23,6 +24,21 @@ case class OALocation(evidence:Option[String], host_type:Option[String], is_best object UnpayWallToOAF { val logger: Logger = LoggerFactory.getLogger(getClass) + + def get_unpaywall_color(input:String):Option[OpenAccessRoute] = { + if(input == null || input.equalsIgnoreCase("close")) + return None + if(input.equalsIgnoreCase("green")) + return Some(OpenAccessRoute.green) + if(input.equalsIgnoreCase("bronze")) + return Some(OpenAccessRoute.bronze) + if(input.equalsIgnoreCase("hybrid")) + return Some(OpenAccessRoute.hybrid) + else + return Some(OpenAccessRoute.gold) + + } + def get_color(is_oa:Boolean, location: OALocation, journal_is_oa:Boolean):Option[OpenAccessRoute] = { if (is_oa) { if (location.host_type.isDefined) { @@ -65,7 +81,7 @@ object UnpayWallToOAF { val oaLocation:OALocation = (json \ "best_oa_location").extractOrElse[OALocation](null) - val colour = get_color(is_oa, oaLocation, journal_is_oa) + val colour = get_unpaywall_color((json \ "oa_status").extractOrElse[String](null)) pub.setCollectedfrom(List(createUnpayWallCollectedFrom()).asJava) pub.setDataInfo(generateDataInfo()) diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml index 03f7b7566..52f958d4d 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + sparkDriverMemory 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 f845d97f3..d3009a570 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 @@ -1,4 +1,4 @@ - + sparkDriverMemory @@ -99,7 +99,7 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=7680 --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -124,7 +124,7 @@ --executor-memory=${sparkExecutorIntersectionMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=7680 --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala index 0fa34d88e..75fb3f787 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala @@ -492,6 +492,124 @@ class CrossrefMappingTest { } + @Test + def testLicenseVorClosed() :Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_vor.json")).mkString + assertNotNull(json) + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + + val item : Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result] + + mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT) + println(mapper.writeValueAsString(item)) + + assertTrue(item.getInstance().asScala exists (i => i.getLicense.getValue.equals("https://www.springer.com/vor"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("CLOSED"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == null)) + + + + + } + + @Test + def testLicenseOpen() :Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_open.json")).mkString + + + assertNotNull(json) + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + + val item : Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result] + + assertTrue(item.getInstance().asScala exists (i => i.getLicense.getValue.equals("http://pubs.acs.org/page/policy/authorchoice_ccby_termsofuse.html"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("OPEN"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == OpenAccessRoute.hybrid)) + mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT) + println(mapper.writeValueAsString(item)) + + } + + @Test + def testLicenseEmbargoOpen() :Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_embargo_open.json")).mkString + + + assertNotNull(json) + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + + val item : Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result] + + assertTrue(item.getInstance().asScala exists (i => i.getLicense.getValue.equals("https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("OPEN"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == OpenAccessRoute.hybrid)) + mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT) + println(mapper.writeValueAsString(item)) + + } + + @Test + def testLicenseEmbargo() :Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_embargo.json")).mkString + + + assertNotNull(json) + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + + val item : Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result] + + assertTrue(item.getInstance().asScala exists (i => i.getLicense.getValue.equals("https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("EMBARGO"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == null)) + mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT) + println(mapper.writeValueAsString(item)) + + } + + + @Test + def testLicenseEmbargoDateTime() :Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_embargo_datetime.json")).mkString + + + assertNotNull(json) + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + + val item : Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result] + + assertTrue(item.getInstance().asScala exists (i => i.getLicense.getValue.equals("https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("EMBARGO"))) + assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == null)) + mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT) + println(mapper.writeValueAsString(item)) + + } + } diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala index 9a08b703a..46d4ec08d 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala @@ -65,7 +65,6 @@ class MAGMappingTest { val conf = new SparkConf() conf.setMaster("local[*]") conf.set("spark.driver.host", "localhost") - val spark: SparkSession = SparkSession .builder() @@ -93,7 +92,6 @@ class MAGMappingTest { implicit val formats = DefaultFormats - val conf = new SparkConf() conf.setMaster("local[*]") conf.set("spark.driver.host", "localhost") @@ -103,7 +101,6 @@ class MAGMappingTest { .appName(getClass.getSimpleName) .config(conf) .getOrCreate() - val path = getClass.getResource("duplicatedMagPapers.json").getPath import org.apache.spark.sql.Encoders diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo.json new file mode 100644 index 000000000..47ca55f34 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo.json @@ -0,0 +1,1537 @@ +{ +"indexed": { +"date-parts": [ +[ +2021, +7, +2 +] +], +"date-time": "2021-07-02T07:30:10Z", +"timestamp": 1625211010708 +}, +"reference-count": 83, +"publisher": "Springer Science and Business Media LLC", +"issue": "5", +"license": [ +{ +"URL": "https://www.springer.com/tdm", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "tdm" +}, +{ +"URL": "https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "vor" +} +], +"content-domain": { +"domain": [ +"link.springer.com" +], +"crossmark-restriction": false +}, +"short-container-title": [ +"Nat Astron" +], +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"DOI": "10.1038/s41550-020-01295-8", +"type": "journal-article", +"created": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T17:03:42Z", +"timestamp": 1614013422000 +}, +"page": "510-518", +"update-policy": "http://dx.doi.org/10.1007/springer_crossmark_policy", +"source": "Crossref", +"is-referenced-by-count": 6, +"title": [ +"A tidal disruption event coincident with a high-energy neutrino" +], +"prefix": "10.1038", +"volume": "5", +"author": [ +{ +"ORCID": "http://orcid.org/0000-0003-2434-0387", +"authenticated-orcid": false, +"given": "Robert", +"family": "Stein", +"sequence": "first", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3859-8074", +"authenticated-orcid": false, +"given": "Sjoert van", +"family": "Velzen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8594-8666", +"authenticated-orcid": false, +"given": "Marek", +"family": "Kowalski", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Anna", +"family": "Franckowiak", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3703-5154", +"authenticated-orcid": false, +"given": "Suvi", +"family": "Gezari", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3124-2814", +"authenticated-orcid": false, +"given": "James C. A.", +"family": "Miller-Jones", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Sara", +"family": "Frederick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0466-3779", +"authenticated-orcid": false, +"given": "Itai", +"family": "Sfaradi", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael F.", +"family": "Bietenholz", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5936-1156", +"authenticated-orcid": false, +"given": "Assaf", +"family": "Horesh", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rob", +"family": "Fender", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2403-4582", +"authenticated-orcid": false, +"given": "Simone", +"family": "Garrappa", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-2184-6430", +"authenticated-orcid": false, +"given": "Tomás", +"family": "Ahumada", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Igor", +"family": "Andreoni", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Justin", +"family": "Belicki", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8018-5348", +"authenticated-orcid": false, +"given": "Eric C.", +"family": "Bellm", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Markus", +"family": "Böttcher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Valery", +"family": "Brinnel", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rick", +"family": "Burruss", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1673-970X", +"authenticated-orcid": false, +"given": "S. Bradley", +"family": "Cenko", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8262-2924", +"authenticated-orcid": false, +"given": "Michael W.", +"family": "Coughlin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2292-0441", +"authenticated-orcid": false, +"given": "Virginia", +"family": "Cunningham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Andrew", +"family": "Drake", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Glennys R.", +"family": "Farrar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael", +"family": "Feeney", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Ryan J.", +"family": "Foley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3653-5598", +"authenticated-orcid": false, +"given": "Avishay", +"family": "Gal-Yam", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "V. Zach", +"family": "Golkhou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-4163-4996", +"authenticated-orcid": false, +"given": "Ariel", +"family": "Goobar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3168-0139", +"authenticated-orcid": false, +"given": "Matthew J.", +"family": "Graham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Erica", +"family": "Hammerstein", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3367-3415", +"authenticated-orcid": false, +"given": "George", +"family": "Helou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-9878-7889", +"authenticated-orcid": false, +"given": "Tiara", +"family": "Hung", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Mansi M.", +"family": "Kasliwal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5740-7747", +"authenticated-orcid": false, +"given": "Charles D.", +"family": "Kilpatrick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5105-344X", +"authenticated-orcid": false, +"given": "Albert K. H.", +"family": "Kong", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-6540-1484", +"authenticated-orcid": false, +"given": "Thomas", +"family": "Kupfer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2451-5482", +"authenticated-orcid": false, +"given": "Russ R.", +"family": "Laher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2242-0244", +"authenticated-orcid": false, +"given": "Ashish A.", +"family": "Mahabal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8532-9395", +"authenticated-orcid": false, +"given": "Frank J.", +"family": "Masci", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0280-7484", +"authenticated-orcid": false, +"given": "Jannis", +"family": "Necker", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8342-6274", +"authenticated-orcid": false, +"given": "Jakob", +"family": "Nordin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel A.", +"family": "Perley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8121-2560", +"authenticated-orcid": false, +"given": "Mickael", +"family": "Rigault", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7788-628X", +"authenticated-orcid": false, +"given": "Simeon", +"family": "Reusch", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Hector", +"family": "Rodriguez", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7559-315X", +"authenticated-orcid": false, +"given": "César", +"family": "Rojas-Bravo", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-7648-4142", +"authenticated-orcid": false, +"given": "Ben", +"family": "Rusholme", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-4401-0430", +"authenticated-orcid": false, +"given": "David L.", +"family": "Shupe", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-9898-5597", +"authenticated-orcid": false, +"given": "Leo P.", +"family": "Singer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1546-6615", +"authenticated-orcid": false, +"given": "Jesper", +"family": "Sollerman", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Maayane T.", +"family": "Soumagnac", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel", +"family": "Stern", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Kirsty", +"family": "Taggart", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Jakob", +"family": "van Santen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Charlotte", +"family": "Ward", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Patrick", +"family": "Woudt", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-6747-8509", +"authenticated-orcid": false, +"given": "Yuhan", +"family": "Yao", +"sequence": "additional", +"affiliation": [ + +] +} +], +"member": "297", +"published-online": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"reference": [ +{ +"key": "1295_CR1", +"doi-asserted-by": "crossref", +"first-page": "P03012", +"DOI": "10.1088/1748-0221/12/03/P03012", +"volume": "12", +"author": "MG Aartsen", +"year": "2017", +"unstructured": "Aartsen, M. G. et al. The IceCube Neutrino Observatory: instrumentation and online systems. J. Instrum. 12, P03012 (2017).", +"journal-title": "J. Instrum." +}, +{ +"key": "1295_CR2", +"unstructured": "Stein, R. IceCube-191001A—IceCube observation of a high-energy neutrino candidate event. GCN Circ. 25913 (2019)." +}, +{ +"key": "1295_CR3", +"doi-asserted-by": "crossref", +"first-page": "018002", +"DOI": "10.1088/1538-3873/aaecbe", +"volume": "131", +"author": "EC Bellm", +"year": "2019", +"unstructured": "Bellm, E. C. et al. The Zwicky Transient Facility: system overview, performance, and first results. Publ. Astron. Soc. Pac. 131, 018002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR4", +"doi-asserted-by": "crossref", +"first-page": "533", +"DOI": "10.1016/j.astropartphys.2007.03.005", +"volume": "27", +"author": "M Kowalski", +"year": "2007", +"unstructured": "Kowalski, M. & Mohr, A. Detecting neutrino transients with optical follow-up observations. Astropart. Phys. 27, 533–538 (2007).", +"journal-title": "Astropart. Phys." +}, +{ +"key": "1295_CR5", +"doi-asserted-by": "crossref", +"first-page": "329", +"DOI": "10.1088/0004-637X/693/1/329", +"volume": "693", +"author": "GR Farrar", +"year": "2009", +"unstructured": "Farrar, G. R. & Gruzinov, A. Giant AGN flares and cosmic ray bursts. Astrophys. J. 693, 329–332 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR6", +"doi-asserted-by": "crossref", +"first-page": "1354", +"DOI": "10.1093/mnras/stx863", +"volume": "469", +"author": "L Dai", +"year": "2017", +"unstructured": "Dai, L. & Fang, K. Can tidal disruption events produce the IceCube neutrinos? Mon. Not. R. Astron. Soc. 469, 1354–1359 (2017).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR7", +"doi-asserted-by": "crossref", +"first-page": "114", +"DOI": "10.3847/1538-4357/ab44ca", +"volume": "886", +"author": "K Hayasaki", +"year": "2019", +"unstructured": "Hayasaki, K. & Yamazaki, R. Neutrino emissions from tidal disruption remnants. Astrophys. J. 886, 114 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR8", +"unstructured": "Farrar, G. R. & Piran, T. Tidal disruption jets as the source of Ultra-High Energy Cosmic Rays. Preprint at https://arxiv.org/abs/1411.0704 (2014)." +}, +{ +"key": "1295_CR9", +"doi-asserted-by": "crossref", +"first-page": "3", +"DOI": "10.3847/1538-4357/aa6344", +"volume": "838", +"author": "N Senno", +"year": "2017", +"unstructured": "Senno, N., Murase, K. & Mészáros, P. High-energy neutrino flares from X-ray bright and dark tidal disruption events. Astrophys. J. 838, 3 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR10", +"doi-asserted-by": "crossref", +"first-page": "083005", +"DOI": "10.1103/PhysRevD.93.083005", +"volume": "93", +"author": "XY Wang", +"year": "2016", +"unstructured": "Wang, X. Y. & Liu, R. Y. Tidal disruption jets of supermassive black holes as hidden sources of cosmic rays: explaining the IceCube TeV–PeV neutrinos. Phys. Rev. D 93, 083005 (2016).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR11", +"doi-asserted-by": "crossref", +"first-page": "123001", +"DOI": "10.1103/PhysRevD.95.123001", +"volume": "95", +"author": "C Lunardini", +"year": "2017", +"unstructured": "Lunardini, C. & Winter, W. High energy neutrinos from the tidal disruption of stars. Phys. Rev. D 95, 123001 (2017).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR12", +"unstructured": "Stein, R., Franckowiak, A., Necker, J., Gezari, S. & Velzen, S. V. Candidate counterparts to IceCube-191001A with ZTF. Astron. Telegr. 13160 (2019)." +}, +{ +"key": "1295_CR13", +"doi-asserted-by": "crossref", +"first-page": "078001", +"DOI": "10.1088/1538-3873/ab006c", +"volume": "131", +"author": "MJ Graham", +"year": "2019", +"unstructured": "Graham, M. J. et al. The Zwicky Transient Facility: science objectives. Publ. Astron. Soc. Pac. 131, 078001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR14", +"unstructured": "Nordin, J. et al. TNS Astronomical Transient Report 33340 (2019)." +}, +{ +"key": "1295_CR15", +"unstructured": "Nicholl, M. et al. ePESSTO+ classification of optical transients. Astron. Telegr. 12752 (2019)." +}, +{ +"key": "1295_CR16", +"unstructured": "van Velzen, S. et al. Seventeen tidal disruption events from the first half of ZTF survey observations: entering a new era of population studies. Preprint at https://arxiv.org/abs/2001.01409 (2020)." +}, +{ +"key": "1295_CR17", +"doi-asserted-by": "crossref", +"first-page": "82", +"DOI": "10.3847/1538-4357/ab1844", +"volume": "878", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. Late-time UV observations of tidal disruption flares reveal unobscured, compact accretion disks. Astrophys. J. 878, 82 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR18", +"doi-asserted-by": "crossref", +"first-page": "5655", +"DOI": "10.1093/mnras/staa192", +"volume": "492", +"author": "A Mummery", +"year": "2020", +"unstructured": "Mummery, A. & Balbus, S. A. The spectral evolution of disc dominated tidal disruption events. Mon. Not. R. Astron. Soc. 492, 5655–5674 (2020).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR19", +"doi-asserted-by": "crossref", +"first-page": "184", +"DOI": "10.1088/0004-637X/764/2/184", +"volume": "764", +"author": "NJ McConnell", +"year": "2013", +"unstructured": "McConnell, N. J. & Ma, C. P. Revisiting the scaling relations of black hole masses and host galaxy properties. Astrophys. J. 764, 184 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR20", +"doi-asserted-by": "crossref", +"first-page": "149", +"DOI": "10.3847/1538-4357/aa633b", +"volume": "838", +"author": "K Auchettl", +"year": "2017", +"unstructured": "Auchettl, K., Guillochon, J. & Ramirez-Ruiz, E. New physical insights about tidal disruption events from a comprehensive observational inventory at X-ray wavelengths. Astrophys. J. 838, 149 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR21", +"doi-asserted-by": "crossref", +"first-page": "4136", +"DOI": "10.1093/mnras/stz1602", +"volume": "487", +"author": "T Wevers", +"year": "2019", +"unstructured": "Wevers, T. et al. Black hole masses of tidal disruption event host galaxies II. Mon. Not. R. Astron. Soc. 487, 4136–4152 (2019).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR22", +"doi-asserted-by": "crossref", +"first-page": "198", +"DOI": "10.3847/1538-4357/aafe0c", +"volume": "872", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. The first tidal disruption flare in ZTF: from photometric selection to multi-wavelength characterization. Astrophys. J. 872, 198 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR23", +"doi-asserted-by": "crossref", +"first-page": "A81", +"DOI": "10.1051/0004-6361/201117855", +"volume": "538", +"author": "G Morlino", +"year": "2012", +"unstructured": "Morlino, G. & Caprioli, D. Strong evidence for hadron acceleration in Tycho’s supernova remnant. Astron. Astrophys. 538, A81 (2012).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR24", +"doi-asserted-by": "crossref", +"first-page": "86", +"DOI": "10.3847/1538-4357/aaa8e0", +"volume": "854", +"author": "T Eftekhari", +"year": "2018", +"unstructured": "Eftekhari, T., Berger, E., Zauderer, B. A., Margutti, R. & Alexander, K. D. Radio monitoring of the tidal disruption event Swift J164449.3+573451. III. Late-time jet energetics and a deviation from equipartition. Astrophys. J. 854, 86 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR25", +"doi-asserted-by": "crossref", +"first-page": "1258", +"DOI": "10.1093/mnras/stt1645", +"volume": "436", +"author": "A Horesh", +"year": "2013", +"unstructured": "Horesh, A. et al. An early and comprehensive millimetre and centimetre wave and X-ray study of SN 2011dh: a non-equipartition blast wave expanding into a massive stellar wind. Mon. Not. R. Astron. Soc. 436, 1258–1267 (2013).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR26", +"doi-asserted-by": "crossref", +"first-page": "78", +"DOI": "10.1088/0004-637X/772/1/78", +"volume": "772", +"author": "R Barniol Duran", +"year": "2013", +"unstructured": "Barniol Duran, R., Nakar, E. & Piran, T. Radius constraints and minimal equipartition energy of relativistically moving synchrotron sources. Astrophys. J. 772, 78 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR27", +"doi-asserted-by": "crossref", +"first-page": "69", +"DOI": "10.1071/AS02053", +"volume": "20", +"author": "AG Polatidis", +"year": "2003", +"unstructured": "Polatidis, A. G. & Conway, J. E. Proper motions in compact symmetric objects. Publ. Astron. Soc. Aust. 20, 69–74 (2003).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR28", +"doi-asserted-by": "crossref", +"first-page": "L25", +"DOI": "10.3847/2041-8205/819/2/L25", +"volume": "819", +"author": "KD Alexander", +"year": "2016", +"unstructured": "Alexander, K. D., Berger, E., Guillochon, J., Zauderer, B. A. & Williams, P. K. G. Discovery of an outflow from radio observations of the tidal disruption event ASASSN-14li. Astrophys. J. Lett. 819, L25 (2016).", +"journal-title": "Astrophys. J. Lett." +}, +{ +"key": "1295_CR29", +"doi-asserted-by": "crossref", +"first-page": "127", +"DOI": "10.3847/0004-637X/827/2/127", +"volume": "827", +"author": "J Krolik", +"year": "2016", +"unstructured": "Krolik, J., Piran, T., Svirski, G. & Cheng, R. M. ASASSN-14li: a model tidal disruption event. Astrophys. J. 827, 127 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR30", +"doi-asserted-by": "crossref", +"first-page": "1", +"DOI": "10.3847/1538-4357/aab361", +"volume": "856", +"author": "DR Pasham", +"year": "2018", +"unstructured": "Pasham, D. R. & van Velzen, S. Discovery of a time lag between the soft X-ray and radio emission of the tidal disruption flare ASASSN-14li: evidence for linear disk–jet coupling. Astrophys. J. 856, 1 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR31", +"doi-asserted-by": "crossref", +"first-page": "L9", +"DOI": "10.1051/0004-6361/201834750", +"volume": "622", +"author": "NL Strotjohann", +"year": "2019", +"unstructured": "Strotjohann, N. L., Kowalski, M. & Franckowiak, A. Eddington bias for cosmic neutrino sources. Astron. Astrophys. 622, L9 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR32", +"doi-asserted-by": "crossref", +"first-page": "425", +"DOI": "10.1146/annurev.aa.22.090184.002233", +"volume": "22", +"author": "AM Hillas", +"year": "1984", +"unstructured": "Hillas, A. M. The origin of ultra-high-energy cosmic rays. Annu. Rev. Astron. Astrophys. 22, 425–444 (1984).", +"journal-title": "Annu. Rev. Astron. Astrophys." +}, +{ +"key": "1295_CR33", +"doi-asserted-by": "crossref", +"first-page": "eaat1378", +"DOI": "10.1126/science.aat1378", +"volume": "361", +"author": "IceCube Collaboration", +"year": "2018", +"unstructured": "IceCube Collaboration et al. Multimessenger observations of a flaring blazar coincident with high-energy neutrino IceCube-170922A. Science 361, eaat1378 (2018).", +"journal-title": "Science" +}, +{ +"key": "1295_CR34", +"unstructured": "Blaufuss, E., Kintscher, T., Lu, L. & Tung, C. F. The next generation of IceCube real-time neutrino alerts. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1021 (PoS, 2019)." +}, +{ +"key": "1295_CR35", +"doi-asserted-by": "crossref", +"first-page": "071101", +"DOI": "10.1103/PhysRevLett.116.071101", +"volume": "116", +"author": "K Murase", +"year": "2016", +"unstructured": "Murase, K., Guetta, D. & Ahlers, M. Hidden cosmic-ray accelerators as an origin of TeV–PeV cosmic neutrinos. Phys. Rev. Lett. 116, 071101 (2016).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR36", +"unstructured": "Stein, R. Search for neutrinos from populations of optical transients. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1016 (PoS, 2019).", +"DOI": "10.22323/1.358.1016", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR37", +"doi-asserted-by": "crossref", +"first-page": "048001", +"DOI": "10.1088/1538-3873/aaff99", +"volume": "131", +"author": "MW Coughlin", +"year": "2019", +"unstructured": "Coughlin, M. W. et al. 2900 square degree search for the optical counterpart of short gamma-ray burst GRB 180523B with the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 048001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR38", +"unstructured": "Stein, R. IceCube-200107A: IceCube observation of a high-energy neutrino candidate event. GCN Circ. 26655 (2020)." +}, +{ +"key": "1295_CR39", +"doi-asserted-by": "crossref", +"first-page": "018003", +"DOI": "10.1088/1538-3873/aae8ac", +"volume": "131", +"author": "FJ Masci", +"year": "2019", +"unstructured": "Masci, F. J. et al. The Zwicky Transient Facility: data processing, products, and archive. Publ. Astron. Soc. Pac. 131, 018003 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR40", +"doi-asserted-by": "crossref", +"first-page": "018001", +"DOI": "10.1088/1538-3873/aae904", +"volume": "131", +"author": "MT Patterson", +"year": "2019", +"unstructured": "Patterson, M. T. et al. The Zwicky Transient Facility Alert Distribution System. Publ. Astron. Soc. Pac. 131, 018001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR41", +"unstructured": "Stein, R. & Reusch, S. robertdstein/ampel_followup_pipeline: V1.1 Release (Zenodo, 2020); https://doi.org/10.5281/zenodo.4048336", +"DOI": "10.5281/zenodo.4048336", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR42", +"doi-asserted-by": "crossref", +"first-page": "A147", +"DOI": "10.1051/0004-6361/201935634", +"volume": "631", +"author": "J Nordin", +"year": "2019", +"unstructured": "Nordin, J. et al. Transient processing and analysis using AMPEL: alert management, photometry, and evaluation of light curves. Astron. Astrophys. 631, A147 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR43", +"doi-asserted-by": "crossref", +"first-page": "038002", +"DOI": "10.1088/1538-3873/aaf3fa", +"volume": "131", +"author": "A Mahabal", +"year": "2019", +"unstructured": "Mahabal, A. et al. Machine learning for the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 038002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR44", +"doi-asserted-by": "crossref", +"first-page": "075002", +"DOI": "10.1088/1538-3873/aac410", +"volume": "130", +"author": "MT Soumagnac", +"year": "2018", +"unstructured": "Soumagnac, M. T. & Ofek, E. O. catsHTM: a tool for fast accessing and cross-matching large astronomical catalogs. Publ. Astron. Soc. Pac. 130, 075002 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR45", +"doi-asserted-by": "crossref", +"first-page": "A1", +"DOI": "10.1051/0004-6361/201833051", +"volume": "616", +"author": "Gaia Collaboration", +"year": "2018", +"unstructured": "Gaia Collaboration et al. Gaia Data Release 2. Summary of the contents and survey properties. Astron. Astrophys. 616, A1 (2018).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR46", +"doi-asserted-by": "crossref", +"first-page": "128001", +"DOI": "10.1088/1538-3873/aae3d9", +"volume": "130", +"author": "Y Tachibana", +"year": "2018", +"unstructured": "Tachibana, Y. & Miller, A. A. A morphological classification model to identify unresolved PanSTARRS1 sources: application in the ZTF real-time pipeline. Publ. Astron. Soc. Pac. 130, 128001 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR47", +"unstructured": "Chambers, K. C. et al. The Pan-STARRS1 Surveys. Preprint at https://arxiv.org/abs/1612.05560 (2016)." +}, +{ +"key": "1295_CR48", +"doi-asserted-by": "crossref", +"first-page": "1868", +"DOI": "10.1088/0004-6256/140/6/1868", +"volume": "140", +"author": "EL Wright", +"year": "2010", +"unstructured": "Wright, E. L. et al. The Wide-field Infrared Survey Explorer (WISE): mission description and initial on-orbit performance. Astron. J. 140, 1868–1881 (2010).", +"journal-title": "Astron. J." +}, +{ +"key": "1295_CR49", +"doi-asserted-by": "crossref", +"first-page": "051103", +"DOI": "10.1103/PhysRevLett.124.051103", +"volume": "124", +"author": "MG Aartsen", +"year": "2020", +"unstructured": "Aartsen, M. G. et al. Time-integrated neutrino source searches with 10 years of IceCube data. Phys. Rev. Lett. 124, 051103 (2020).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR50", +"unstructured": "Steele, I. A. et al. The Liverpool Telescope: performance and first results. Proc. SPIE 5489, https://doi.org/10.1117/12.551456 (2004).", +"DOI": "10.1117/12.551456", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR51", +"doi-asserted-by": "crossref", +"first-page": "035003", +"DOI": "10.1088/1538-3873/aaa53f", +"volume": "130", +"author": "N Blagorodnova", +"year": "2018", +"unstructured": "Blagorodnova, N. et al. The SED Machine: a robotic spectrograph for fast transient classification. Publ. Astron. Soc. Pac. 130, 035003 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR52", +"doi-asserted-by": "crossref", +"first-page": "A115", +"DOI": "10.1051/0004-6361/201935344", +"volume": "627", +"author": "M Rigault", +"year": "2019", +"unstructured": "Rigault, M. et al. Fully automated integral field spectrograph pipeline for the SEDMachine: pysedm. Astron. Astrophys. 627, A115 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR53", +"doi-asserted-by": "crossref", +"first-page": "A68", +"DOI": "10.1051/0004-6361/201628275", +"volume": "593", +"author": "C Fremling", +"year": "2016", +"unstructured": "Fremling, C. et al. PTF12os and iPTF13bvn. Two stripped-envelope supernovae from low-mass progenitors in NGC 5806. Astron. Astrophys. 593, A68 (2016).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR54", +"doi-asserted-by": "crossref", +"first-page": "72", +"DOI": "10.3847/1538-4357/aa998e", +"volume": "852", +"author": "S van Velzen", +"year": "2018", +"unstructured": "van Velzen, S. On the mass and luminosity functions of tidal disruption flares: rate suppression due to black hole event horizons. Astrophys. J. 852, 72 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR55", +"doi-asserted-by": "crossref", +"first-page": "95", +"DOI": "10.1007/s11214-005-5095-4", +"volume": "120", +"author": "PWA Roming", +"year": "2005", +"unstructured": "Roming, P. W. A. et al. The Swift Ultra-Violet/Optical Telescope. Space Sci. Rev. 120, 95–142 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR56", +"doi-asserted-by": "crossref", +"first-page": "1005", +"DOI": "10.1086/422091", +"volume": "611", +"author": "N Gehrels", +"year": "2004", +"unstructured": "Gehrels, N. et al. The Swift Gamma-Ray Burst Mission. Astrophys. J. 611, 1005–1020 (2004).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR57", +"doi-asserted-by": "crossref", +"first-page": "19", +"DOI": "10.3847/0004-637X/829/1/19", +"volume": "829", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S., Mendez, A. J., Krolik, J. H. & Gorjian, V. Discovery of transient infrared emission from dust heated by stellar tidal disruption flares. Astrophys. J. 829, 19 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR58", +"doi-asserted-by": "crossref", +"first-page": "575", +"DOI": "10.1093/mnras/stw307", +"volume": "458", +"author": "W Lu", +"year": "2016", +"unstructured": "Lu, W., Kumar, P. & Evans, N. J. Infrared emission from tidal disruption events—probing the pc-scale dust content around galactic nuclei. Mon. Not. R. Astron. Soc. 458, 575–581 (2016).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR59", +"unstructured": "Miller, J. S. & Stone, R. P. S. The Kast Double Spectrograph. Technical Report No. 66 (Lick Observatory, 1993)." +}, +{ +"key": "1295_CR60", +"doi-asserted-by": "crossref", +"first-page": "375", +"DOI": "10.1086/133562", +"volume": "107", +"author": "JB Oke", +"year": "1995", +"unstructured": "Oke, J. B. et al. The Keck Low-Resolution Imaging Spectrometer. Publ. Astron. Soc. Pac. 107, 375–385 (1995).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR61", +"doi-asserted-by": "crossref", +"first-page": "765", +"DOI": "10.1111/j.1365-2966.2005.08957.x", +"volume": "359", +"author": "A Garcia-Rissmann", +"year": "2005", +"unstructured": "Garcia-Rissmann, A. et al. An atlas of calcium triplet spectra of active galaxies. Mon. Not. R. Astron. Soc. 359, 765–780 (2005).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR62", +"doi-asserted-by": "crossref", +"first-page": "165", +"DOI": "10.1007/s11214-005-5097-2", +"volume": "120", +"author": "DN Burrows", +"year": "2005", +"unstructured": "Burrows, D. N. et al. The Swift X-Ray Telescope. Space Sci. Rev. 120, 165–195 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR63", +"doi-asserted-by": "crossref", +"first-page": "L1", +"DOI": "10.1051/0004-6361:20000036", +"volume": "365", +"author": "F Jansen", +"year": "2001", +"unstructured": "Jansen, F. et al. XMM-Newton Observatory. I. The spacecraft and operations. Astron. Astrophys. 365, L1–L6 (2001).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR64", +"unstructured": "HI4PI Collaboration et al. HI4PI: a full-sky H i survey based on EBHIS and GASS. Astron. Astrophys. 594, A116 (2016).", +"DOI": "10.1051/0004-6361/201629178", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR65", +"unstructured": "Arnaud, K. A. in Astronomical Data Analysis Software and Systems V (eds Jacoby, G. H. & Barnes, J.) 17 (Astronomical Society of the Pacific, 1996)." +}, +{ +"key": "1295_CR66", +"doi-asserted-by": "crossref", +"first-page": "1545", +"DOI": "10.1111/j.1365-2966.2008.13953.x", +"volume": "391", +"author": "JTL Zwart", +"year": "2008", +"unstructured": "Zwart, J. T. L. et al. The Arcminute Microkelvin Imager. Mon. Not. R. Astron. Soc. 391, 1545–1558 (2008).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR67", +"doi-asserted-by": "crossref", +"first-page": "5677", +"DOI": "10.1093/mnras/sty074", +"volume": "475", +"author": "J Hickish", +"year": "2018", +"unstructured": "Hickish, J. et al. A digital correlator upgrade for the Arcminute MicroKelvin Imager. Mon. Not. R. Astron. Soc. 475, 5677–5687 (2018).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR68", +"doi-asserted-by": "crossref", +"first-page": "1396", +"DOI": "10.1093/mnras/stv1728", +"volume": "453", +"author": "YC Perrott", +"year": "2015", +"unstructured": "Perrott, Y. C. et al. AMI galactic plane survey at 16 GHz—II. Full data release with extended coverage and improved processing. Mon. Not. R. Astron. Soc. 453, 1396–1403 (2015).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR69", +"unstructured": "McMullin, J. P., Waters, B., Schiebel, D., Young, W. & Golap, K. in Astronomical Data Analysis Software and Systems XVI (eds Shaw, R. A. et al.) 127 (Astronomical Society of the Pacific, 2007)." +}, +{ +"key": "1295_CR70", +"doi-asserted-by": "crossref", +"first-page": "1071", +"DOI": "10.1088/0004-637X/697/2/1071", +"volume": "697", +"author": "WB Atwood", +"year": "2009", +"unstructured": "Atwood, W. B. et al. The Large Area Telescope on the Fermi Gamma-ray Space Telescope mission. Astrophys. J. 697, 1071–1102 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR71", +"unstructured": "Wood, M. et al. Fermipy: an open-source Python package for analysis of Fermi-LAT Data. In Proc. 35th International Cosmic Ray Conference (ICRC2017) 824 (PoS, 2017).", +"DOI": "10.22323/1.301.0824", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR72", +"unstructured": "Garrappa, S. & Buson, S. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR73", +"unstructured": "The Fermi-LAT collaboration. Fermi Large Area Telescope Fourth Source Catalog. Astrophys. J. Suppl. Ser. 247, 33 (2020)." +}, +{ +"key": "1295_CR74", +"doi-asserted-by": "crossref", +"first-page": "14", +"DOI": "10.1088/0004-637X/767/1/14", +"volume": "767", +"author": "T Pursimo", +"year": "2013", +"unstructured": "Pursimo, T. et al. The Micro-Arcsecond Scintillation-Induced Variability (MASIV) survey. III. Optical identifications and new redshifts. Astrophys. J. 767, 14 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR75", +"unstructured": "Garrappa, S., Buson, S. & Fermi-LAT Collaboration. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR76", +"doi-asserted-by": "crossref", +"first-page": "133", +"DOI": "10.1088/0004-637X/802/2/133", +"volume": "802", +"author": "C Diltz", +"year": "2015", +"unstructured": "Diltz, C., Böttcher, M. & Fossati, G. Time dependent hadronic modeling of flat spectrum radio quasars. Astrophys. J. 802, 133 (2015).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR77", +"doi-asserted-by": "crossref", +"first-page": "88", +"DOI": "10.1038/s41550-018-0610-1", +"volume": "3", +"author": "S Gao", +"year": "2019", +"unstructured": "Gao, S., Fedynitch, A., Winter, W. & Pohl, M. Modelling the coincident observation of a high-energy neutrino and a bright blazar flare. Nat. Astron. 3, 88–92 (2019).", +"journal-title": "Nat. Astron." +}, +{ +"key": "1295_CR78", +"unstructured": "Ayala, H. IceCube-191001A: HAWC follow-up. GCN Circ. 25936 (2019)." +}, +{ +"key": "1295_CR79", +"doi-asserted-by": "crossref", +"first-page": "62", +"DOI": "10.1126/science.aad1182", +"volume": "351", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S. et al. A radio jet from the optical and x-ray bright stellar tidal disruption flare ASASSN-14li. Science 351, 62–65 (2016).", +"journal-title": "Science" +}, +{ +"key": "1295_CR80", +"doi-asserted-by": "crossref", +"first-page": "306", +"DOI": "10.1086/670067", +"volume": "125", +"author": "D Foreman-Mackey", +"year": "2013", +"unstructured": "Foreman-Mackey, D., Hogg, D. W., Lang, D. & Goodman, J. emcee: the MCMC Hammer. Publ. Astron. Soc. Pac. 125, 306 (2013).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR81", +"doi-asserted-by": "crossref", +"first-page": "6", +"DOI": "10.3847/1538-4365/aab761", +"volume": "236", +"author": "J Guillochon", +"year": "2018", +"unstructured": "Guillochon, J. et al. MOSFiT: Modular Open Source Fitter for Transients. Astrophys. J. Suppl. Ser. 236, 6 (2018).", +"journal-title": "Astrophys. J. Suppl. Ser." +}, +{ +"key": "1295_CR82", +"doi-asserted-by": "crossref", +"first-page": "e008", +"DOI": "10.1017/pasa.2013.44", +"volume": "31", +"author": "J Granot", +"year": "2014", +"unstructured": "Granot, J. & van der Horst, A. J. Gamma-ray burst jets and their radio observations. Publ. Astron. Soc. Aust. 31, e008 (2014).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR83", +"doi-asserted-by": "crossref", +"first-page": "102", +"DOI": "10.1088/0004-637X/815/2/102", +"volume": "815", +"author": "W Fong", +"year": "2015", +"unstructured": "Fong, W., Berger, E., Margutti, R. & Zauderer, B. A. A decade of short-duration gamma-ray burst broadband afterglows: energetics, circumburst densities, and jet opening angles. Astrophys. J. 815, 102 (2015).", +"journal-title": "Astrophys. J." +} +], +"container-title": [ +"Nature Astronomy" +], +"original-title": [ + +], +"language": "en", +"link": [ +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8", +"content-type": "text/html", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "similarity-checking" +} +], +"deposited": { +"date-parts": [ +[ +2021, +5, +17 +] +], +"date-time": "2021-05-17T15:08:12Z", +"timestamp": 1621264092000 +}, +"score": 1.0, +"subtitle": [ + +], +"short-title": [ + +], +"issued": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"references-count": 83, +"journal-issue": { +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"issue": "5" +}, +"alternative-id": [ +"1295" +], +"URL": "http://dx.doi.org/10.1038/s41550-020-01295-8", +"relation": { +"cites": [ + +] +}, +"ISSN": [ +"2397-3366" +], +"issn-type": [ +{ +"value": "2397-3366", +"type": "electronic" +} +], +"assertion": [ +{ +"value": "21 July 2020", +"order": 1, +"name": "received", +"label": "Received", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "16 December 2020", +"order": 2, +"name": "accepted", +"label": "Accepted", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "22 February 2021", +"order": 3, +"name": "first_online", +"label": "First Online", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "The authors declare no competing interests.", +"order": 1, +"name": "Ethics", +"group": { +"name": "EthicsHeading", +"label": "Competing interests" +} +} +] +} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo_datetime.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo_datetime.json new file mode 100644 index 000000000..c84e16350 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo_datetime.json @@ -0,0 +1,1538 @@ +{ +"indexed": { +"date-parts": [ +[ +2021, +7, +2 +] +], +"date-time": "2021-07-02T07:30:10Z", +"timestamp": 1625211010708 +}, +"reference-count": 83, +"publisher": "Springer Science and Business Media LLC", +"issue": "5", +"license": [ +{ +"URL": "https://www.springer.com/tdm", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "tdm" +}, +{ +"URL": "https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "vor" +} +], +"content-domain": { +"domain": [ +"link.springer.com" +], +"crossmark-restriction": false +}, +"short-container-title": [ +"Nat Astron" +], +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"DOI": "10.1038/s41550-020-01295-8", +"type": "journal-article", +"created": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T17:03:42Z", +"timestamp": 1614013422000 +}, +"page": "510-518", +"update-policy": "http://dx.doi.org/10.1007/springer_crossmark_policy", +"source": "Crossref", +"is-referenced-by-count": 6, +"title": [ +"A tidal disruption event coincident with a high-energy neutrino" +], +"prefix": "10.1038", +"volume": "5", +"author": [ +{ +"ORCID": "http://orcid.org/0000-0003-2434-0387", +"authenticated-orcid": false, +"given": "Robert", +"family": "Stein", +"sequence": "first", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3859-8074", +"authenticated-orcid": false, +"given": "Sjoert van", +"family": "Velzen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8594-8666", +"authenticated-orcid": false, +"given": "Marek", +"family": "Kowalski", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Anna", +"family": "Franckowiak", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3703-5154", +"authenticated-orcid": false, +"given": "Suvi", +"family": "Gezari", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3124-2814", +"authenticated-orcid": false, +"given": "James C. A.", +"family": "Miller-Jones", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Sara", +"family": "Frederick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0466-3779", +"authenticated-orcid": false, +"given": "Itai", +"family": "Sfaradi", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael F.", +"family": "Bietenholz", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5936-1156", +"authenticated-orcid": false, +"given": "Assaf", +"family": "Horesh", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rob", +"family": "Fender", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2403-4582", +"authenticated-orcid": false, +"given": "Simone", +"family": "Garrappa", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-2184-6430", +"authenticated-orcid": false, +"given": "Tomás", +"family": "Ahumada", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Igor", +"family": "Andreoni", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Justin", +"family": "Belicki", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8018-5348", +"authenticated-orcid": false, +"given": "Eric C.", +"family": "Bellm", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Markus", +"family": "Böttcher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Valery", +"family": "Brinnel", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rick", +"family": "Burruss", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1673-970X", +"authenticated-orcid": false, +"given": "S. Bradley", +"family": "Cenko", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8262-2924", +"authenticated-orcid": false, +"given": "Michael W.", +"family": "Coughlin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2292-0441", +"authenticated-orcid": false, +"given": "Virginia", +"family": "Cunningham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Andrew", +"family": "Drake", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Glennys R.", +"family": "Farrar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael", +"family": "Feeney", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Ryan J.", +"family": "Foley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3653-5598", +"authenticated-orcid": false, +"given": "Avishay", +"family": "Gal-Yam", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "V. Zach", +"family": "Golkhou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-4163-4996", +"authenticated-orcid": false, +"given": "Ariel", +"family": "Goobar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3168-0139", +"authenticated-orcid": false, +"given": "Matthew J.", +"family": "Graham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Erica", +"family": "Hammerstein", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3367-3415", +"authenticated-orcid": false, +"given": "George", +"family": "Helou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-9878-7889", +"authenticated-orcid": false, +"given": "Tiara", +"family": "Hung", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Mansi M.", +"family": "Kasliwal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5740-7747", +"authenticated-orcid": false, +"given": "Charles D.", +"family": "Kilpatrick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5105-344X", +"authenticated-orcid": false, +"given": "Albert K. H.", +"family": "Kong", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-6540-1484", +"authenticated-orcid": false, +"given": "Thomas", +"family": "Kupfer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2451-5482", +"authenticated-orcid": false, +"given": "Russ R.", +"family": "Laher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2242-0244", +"authenticated-orcid": false, +"given": "Ashish A.", +"family": "Mahabal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8532-9395", +"authenticated-orcid": false, +"given": "Frank J.", +"family": "Masci", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0280-7484", +"authenticated-orcid": false, +"given": "Jannis", +"family": "Necker", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8342-6274", +"authenticated-orcid": false, +"given": "Jakob", +"family": "Nordin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel A.", +"family": "Perley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8121-2560", +"authenticated-orcid": false, +"given": "Mickael", +"family": "Rigault", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7788-628X", +"authenticated-orcid": false, +"given": "Simeon", +"family": "Reusch", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Hector", +"family": "Rodriguez", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7559-315X", +"authenticated-orcid": false, +"given": "César", +"family": "Rojas-Bravo", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-7648-4142", +"authenticated-orcid": false, +"given": "Ben", +"family": "Rusholme", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-4401-0430", +"authenticated-orcid": false, +"given": "David L.", +"family": "Shupe", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-9898-5597", +"authenticated-orcid": false, +"given": "Leo P.", +"family": "Singer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1546-6615", +"authenticated-orcid": false, +"given": "Jesper", +"family": "Sollerman", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Maayane T.", +"family": "Soumagnac", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel", +"family": "Stern", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Kirsty", +"family": "Taggart", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Jakob", +"family": "van Santen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Charlotte", +"family": "Ward", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Patrick", +"family": "Woudt", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-6747-8509", +"authenticated-orcid": false, +"given": "Yuhan", +"family": "Yao", +"sequence": "additional", +"affiliation": [ + +] +} +], +"member": "297", +"published-online": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"reference": [ +{ +"key": "1295_CR1", +"doi-asserted-by": "crossref", +"first-page": "P03012", +"DOI": "10.1088/1748-0221/12/03/P03012", +"volume": "12", +"author": "MG Aartsen", +"year": "2017", +"unstructured": "Aartsen, M. G. et al. The IceCube Neutrino Observatory: instrumentation and online systems. J. Instrum. 12, P03012 (2017).", +"journal-title": "J. Instrum." +}, +{ +"key": "1295_CR2", +"unstructured": "Stein, R. IceCube-191001A—IceCube observation of a high-energy neutrino candidate event. GCN Circ. 25913 (2019)." +}, +{ +"key": "1295_CR3", +"doi-asserted-by": "crossref", +"first-page": "018002", +"DOI": "10.1088/1538-3873/aaecbe", +"volume": "131", +"author": "EC Bellm", +"year": "2019", +"unstructured": "Bellm, E. C. et al. The Zwicky Transient Facility: system overview, performance, and first results. Publ. Astron. Soc. Pac. 131, 018002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR4", +"doi-asserted-by": "crossref", +"first-page": "533", +"DOI": "10.1016/j.astropartphys.2007.03.005", +"volume": "27", +"author": "M Kowalski", +"year": "2007", +"unstructured": "Kowalski, M. & Mohr, A. Detecting neutrino transients with optical follow-up observations. Astropart. Phys. 27, 533–538 (2007).", +"journal-title": "Astropart. Phys." +}, +{ +"key": "1295_CR5", +"doi-asserted-by": "crossref", +"first-page": "329", +"DOI": "10.1088/0004-637X/693/1/329", +"volume": "693", +"author": "GR Farrar", +"year": "2009", +"unstructured": "Farrar, G. R. & Gruzinov, A. Giant AGN flares and cosmic ray bursts. Astrophys. J. 693, 329–332 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR6", +"doi-asserted-by": "crossref", +"first-page": "1354", +"DOI": "10.1093/mnras/stx863", +"volume": "469", +"author": "L Dai", +"year": "2017", +"unstructured": "Dai, L. & Fang, K. Can tidal disruption events produce the IceCube neutrinos? Mon. Not. R. Astron. Soc. 469, 1354–1359 (2017).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR7", +"doi-asserted-by": "crossref", +"first-page": "114", +"DOI": "10.3847/1538-4357/ab44ca", +"volume": "886", +"author": "K Hayasaki", +"year": "2019", +"unstructured": "Hayasaki, K. & Yamazaki, R. Neutrino emissions from tidal disruption remnants. Astrophys. J. 886, 114 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR8", +"unstructured": "Farrar, G. R. & Piran, T. Tidal disruption jets as the source of Ultra-High Energy Cosmic Rays. Preprint at https://arxiv.org/abs/1411.0704 (2014)." +}, +{ +"key": "1295_CR9", +"doi-asserted-by": "crossref", +"first-page": "3", +"DOI": "10.3847/1538-4357/aa6344", +"volume": "838", +"author": "N Senno", +"year": "2017", +"unstructured": "Senno, N., Murase, K. & Mészáros, P. High-energy neutrino flares from X-ray bright and dark tidal disruption events. Astrophys. J. 838, 3 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR10", +"doi-asserted-by": "crossref", +"first-page": "083005", +"DOI": "10.1103/PhysRevD.93.083005", +"volume": "93", +"author": "XY Wang", +"year": "2016", +"unstructured": "Wang, X. Y. & Liu, R. Y. Tidal disruption jets of supermassive black holes as hidden sources of cosmic rays: explaining the IceCube TeV–PeV neutrinos. Phys. Rev. D 93, 083005 (2016).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR11", +"doi-asserted-by": "crossref", +"first-page": "123001", +"DOI": "10.1103/PhysRevD.95.123001", +"volume": "95", +"author": "C Lunardini", +"year": "2017", +"unstructured": "Lunardini, C. & Winter, W. High energy neutrinos from the tidal disruption of stars. Phys. Rev. D 95, 123001 (2017).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR12", +"unstructured": "Stein, R., Franckowiak, A., Necker, J., Gezari, S. & Velzen, S. V. Candidate counterparts to IceCube-191001A with ZTF. Astron. Telegr. 13160 (2019)." +}, +{ +"key": "1295_CR13", +"doi-asserted-by": "crossref", +"first-page": "078001", +"DOI": "10.1088/1538-3873/ab006c", +"volume": "131", +"author": "MJ Graham", +"year": "2019", +"unstructured": "Graham, M. J. et al. The Zwicky Transient Facility: science objectives. Publ. Astron. Soc. Pac. 131, 078001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR14", +"unstructured": "Nordin, J. et al. TNS Astronomical Transient Report 33340 (2019)." +}, +{ +"key": "1295_CR15", +"unstructured": "Nicholl, M. et al. ePESSTO+ classification of optical transients. Astron. Telegr. 12752 (2019)." +}, +{ +"key": "1295_CR16", +"unstructured": "van Velzen, S. et al. Seventeen tidal disruption events from the first half of ZTF survey observations: entering a new era of population studies. Preprint at https://arxiv.org/abs/2001.01409 (2020)." +}, +{ +"key": "1295_CR17", +"doi-asserted-by": "crossref", +"first-page": "82", +"DOI": "10.3847/1538-4357/ab1844", +"volume": "878", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. Late-time UV observations of tidal disruption flares reveal unobscured, compact accretion disks. Astrophys. J. 878, 82 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR18", +"doi-asserted-by": "crossref", +"first-page": "5655", +"DOI": "10.1093/mnras/staa192", +"volume": "492", +"author": "A Mummery", +"year": "2020", +"unstructured": "Mummery, A. & Balbus, S. A. The spectral evolution of disc dominated tidal disruption events. Mon. Not. R. Astron. Soc. 492, 5655–5674 (2020).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR19", +"doi-asserted-by": "crossref", +"first-page": "184", +"DOI": "10.1088/0004-637X/764/2/184", +"volume": "764", +"author": "NJ McConnell", +"year": "2013", +"unstructured": "McConnell, N. J. & Ma, C. P. Revisiting the scaling relations of black hole masses and host galaxy properties. Astrophys. J. 764, 184 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR20", +"doi-asserted-by": "crossref", +"first-page": "149", +"DOI": "10.3847/1538-4357/aa633b", +"volume": "838", +"author": "K Auchettl", +"year": "2017", +"unstructured": "Auchettl, K., Guillochon, J. & Ramirez-Ruiz, E. New physical insights about tidal disruption events from a comprehensive observational inventory at X-ray wavelengths. Astrophys. J. 838, 149 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR21", +"doi-asserted-by": "crossref", +"first-page": "4136", +"DOI": "10.1093/mnras/stz1602", +"volume": "487", +"author": "T Wevers", +"year": "2019", +"unstructured": "Wevers, T. et al. Black hole masses of tidal disruption event host galaxies II. Mon. Not. R. Astron. Soc. 487, 4136–4152 (2019).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR22", +"doi-asserted-by": "crossref", +"first-page": "198", +"DOI": "10.3847/1538-4357/aafe0c", +"volume": "872", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. The first tidal disruption flare in ZTF: from photometric selection to multi-wavelength characterization. Astrophys. J. 872, 198 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR23", +"doi-asserted-by": "crossref", +"first-page": "A81", +"DOI": "10.1051/0004-6361/201117855", +"volume": "538", +"author": "G Morlino", +"year": "2012", +"unstructured": "Morlino, G. & Caprioli, D. Strong evidence for hadron acceleration in Tycho’s supernova remnant. Astron. Astrophys. 538, A81 (2012).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR24", +"doi-asserted-by": "crossref", +"first-page": "86", +"DOI": "10.3847/1538-4357/aaa8e0", +"volume": "854", +"author": "T Eftekhari", +"year": "2018", +"unstructured": "Eftekhari, T., Berger, E., Zauderer, B. A., Margutti, R. & Alexander, K. D. Radio monitoring of the tidal disruption event Swift J164449.3+573451. III. Late-time jet energetics and a deviation from equipartition. Astrophys. J. 854, 86 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR25", +"doi-asserted-by": "crossref", +"first-page": "1258", +"DOI": "10.1093/mnras/stt1645", +"volume": "436", +"author": "A Horesh", +"year": "2013", +"unstructured": "Horesh, A. et al. An early and comprehensive millimetre and centimetre wave and X-ray study of SN 2011dh: a non-equipartition blast wave expanding into a massive stellar wind. Mon. Not. R. Astron. Soc. 436, 1258–1267 (2013).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR26", +"doi-asserted-by": "crossref", +"first-page": "78", +"DOI": "10.1088/0004-637X/772/1/78", +"volume": "772", +"author": "R Barniol Duran", +"year": "2013", +"unstructured": "Barniol Duran, R., Nakar, E. & Piran, T. Radius constraints and minimal equipartition energy of relativistically moving synchrotron sources. Astrophys. J. 772, 78 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR27", +"doi-asserted-by": "crossref", +"first-page": "69", +"DOI": "10.1071/AS02053", +"volume": "20", +"author": "AG Polatidis", +"year": "2003", +"unstructured": "Polatidis, A. G. & Conway, J. E. Proper motions in compact symmetric objects. Publ. Astron. Soc. Aust. 20, 69–74 (2003).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR28", +"doi-asserted-by": "crossref", +"first-page": "L25", +"DOI": "10.3847/2041-8205/819/2/L25", +"volume": "819", +"author": "KD Alexander", +"year": "2016", +"unstructured": "Alexander, K. D., Berger, E., Guillochon, J., Zauderer, B. A. & Williams, P. K. G. Discovery of an outflow from radio observations of the tidal disruption event ASASSN-14li. Astrophys. J. Lett. 819, L25 (2016).", +"journal-title": "Astrophys. J. Lett." +}, +{ +"key": "1295_CR29", +"doi-asserted-by": "crossref", +"first-page": "127", +"DOI": "10.3847/0004-637X/827/2/127", +"volume": "827", +"author": "J Krolik", +"year": "2016", +"unstructured": "Krolik, J., Piran, T., Svirski, G. & Cheng, R. M. ASASSN-14li: a model tidal disruption event. Astrophys. J. 827, 127 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR30", +"doi-asserted-by": "crossref", +"first-page": "1", +"DOI": "10.3847/1538-4357/aab361", +"volume": "856", +"author": "DR Pasham", +"year": "2018", +"unstructured": "Pasham, D. R. & van Velzen, S. Discovery of a time lag between the soft X-ray and radio emission of the tidal disruption flare ASASSN-14li: evidence for linear disk–jet coupling. Astrophys. J. 856, 1 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR31", +"doi-asserted-by": "crossref", +"first-page": "L9", +"DOI": "10.1051/0004-6361/201834750", +"volume": "622", +"author": "NL Strotjohann", +"year": "2019", +"unstructured": "Strotjohann, N. L., Kowalski, M. & Franckowiak, A. Eddington bias for cosmic neutrino sources. Astron. Astrophys. 622, L9 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR32", +"doi-asserted-by": "crossref", +"first-page": "425", +"DOI": "10.1146/annurev.aa.22.090184.002233", +"volume": "22", +"author": "AM Hillas", +"year": "1984", +"unstructured": "Hillas, A. M. The origin of ultra-high-energy cosmic rays. Annu. Rev. Astron. Astrophys. 22, 425–444 (1984).", +"journal-title": "Annu. Rev. Astron. Astrophys." +}, +{ +"key": "1295_CR33", +"doi-asserted-by": "crossref", +"first-page": "eaat1378", +"DOI": "10.1126/science.aat1378", +"volume": "361", +"author": "IceCube Collaboration", +"year": "2018", +"unstructured": "IceCube Collaboration et al. Multimessenger observations of a flaring blazar coincident with high-energy neutrino IceCube-170922A. Science 361, eaat1378 (2018).", +"journal-title": "Science" +}, +{ +"key": "1295_CR34", +"unstructured": "Blaufuss, E., Kintscher, T., Lu, L. & Tung, C. F. The next generation of IceCube real-time neutrino alerts. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1021 (PoS, 2019)." +}, +{ +"key": "1295_CR35", +"doi-asserted-by": "crossref", +"first-page": "071101", +"DOI": "10.1103/PhysRevLett.116.071101", +"volume": "116", +"author": "K Murase", +"year": "2016", +"unstructured": "Murase, K., Guetta, D. & Ahlers, M. Hidden cosmic-ray accelerators as an origin of TeV–PeV cosmic neutrinos. Phys. Rev. Lett. 116, 071101 (2016).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR36", +"unstructured": "Stein, R. Search for neutrinos from populations of optical transients. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1016 (PoS, 2019).", +"DOI": "10.22323/1.358.1016", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR37", +"doi-asserted-by": "crossref", +"first-page": "048001", +"DOI": "10.1088/1538-3873/aaff99", +"volume": "131", +"author": "MW Coughlin", +"year": "2019", +"unstructured": "Coughlin, M. W. et al. 2900 square degree search for the optical counterpart of short gamma-ray burst GRB 180523B with the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 048001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR38", +"unstructured": "Stein, R. IceCube-200107A: IceCube observation of a high-energy neutrino candidate event. GCN Circ. 26655 (2020)." +}, +{ +"key": "1295_CR39", +"doi-asserted-by": "crossref", +"first-page": "018003", +"DOI": "10.1088/1538-3873/aae8ac", +"volume": "131", +"author": "FJ Masci", +"year": "2019", +"unstructured": "Masci, F. J. et al. The Zwicky Transient Facility: data processing, products, and archive. Publ. Astron. Soc. Pac. 131, 018003 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR40", +"doi-asserted-by": "crossref", +"first-page": "018001", +"DOI": "10.1088/1538-3873/aae904", +"volume": "131", +"author": "MT Patterson", +"year": "2019", +"unstructured": "Patterson, M. T. et al. The Zwicky Transient Facility Alert Distribution System. Publ. Astron. Soc. Pac. 131, 018001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR41", +"unstructured": "Stein, R. & Reusch, S. robertdstein/ampel_followup_pipeline: V1.1 Release (Zenodo, 2020); https://doi.org/10.5281/zenodo.4048336", +"DOI": "10.5281/zenodo.4048336", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR42", +"doi-asserted-by": "crossref", +"first-page": "A147", +"DOI": "10.1051/0004-6361/201935634", +"volume": "631", +"author": "J Nordin", +"year": "2019", +"unstructured": "Nordin, J. et al. Transient processing and analysis using AMPEL: alert management, photometry, and evaluation of light curves. Astron. Astrophys. 631, A147 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR43", +"doi-asserted-by": "crossref", +"first-page": "038002", +"DOI": "10.1088/1538-3873/aaf3fa", +"volume": "131", +"author": "A Mahabal", +"year": "2019", +"unstructured": "Mahabal, A. et al. Machine learning for the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 038002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR44", +"doi-asserted-by": "crossref", +"first-page": "075002", +"DOI": "10.1088/1538-3873/aac410", +"volume": "130", +"author": "MT Soumagnac", +"year": "2018", +"unstructured": "Soumagnac, M. T. & Ofek, E. O. catsHTM: a tool for fast accessing and cross-matching large astronomical catalogs. Publ. Astron. Soc. Pac. 130, 075002 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR45", +"doi-asserted-by": "crossref", +"first-page": "A1", +"DOI": "10.1051/0004-6361/201833051", +"volume": "616", +"author": "Gaia Collaboration", +"year": "2018", +"unstructured": "Gaia Collaboration et al. Gaia Data Release 2. Summary of the contents and survey properties. Astron. Astrophys. 616, A1 (2018).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR46", +"doi-asserted-by": "crossref", +"first-page": "128001", +"DOI": "10.1088/1538-3873/aae3d9", +"volume": "130", +"author": "Y Tachibana", +"year": "2018", +"unstructured": "Tachibana, Y. & Miller, A. A. A morphological classification model to identify unresolved PanSTARRS1 sources: application in the ZTF real-time pipeline. Publ. Astron. Soc. Pac. 130, 128001 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR47", +"unstructured": "Chambers, K. C. et al. The Pan-STARRS1 Surveys. Preprint at https://arxiv.org/abs/1612.05560 (2016)." +}, +{ +"key": "1295_CR48", +"doi-asserted-by": "crossref", +"first-page": "1868", +"DOI": "10.1088/0004-6256/140/6/1868", +"volume": "140", +"author": "EL Wright", +"year": "2010", +"unstructured": "Wright, E. L. et al. The Wide-field Infrared Survey Explorer (WISE): mission description and initial on-orbit performance. Astron. J. 140, 1868–1881 (2010).", +"journal-title": "Astron. J." +}, +{ +"key": "1295_CR49", +"doi-asserted-by": "crossref", +"first-page": "051103", +"DOI": "10.1103/PhysRevLett.124.051103", +"volume": "124", +"author": "MG Aartsen", +"year": "2020", +"unstructured": "Aartsen, M. G. et al. Time-integrated neutrino source searches with 10 years of IceCube data. Phys. Rev. Lett. 124, 051103 (2020).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR50", +"unstructured": "Steele, I. A. et al. The Liverpool Telescope: performance and first results. Proc. SPIE 5489, https://doi.org/10.1117/12.551456 (2004).", +"DOI": "10.1117/12.551456", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR51", +"doi-asserted-by": "crossref", +"first-page": "035003", +"DOI": "10.1088/1538-3873/aaa53f", +"volume": "130", +"author": "N Blagorodnova", +"year": "2018", +"unstructured": "Blagorodnova, N. et al. The SED Machine: a robotic spectrograph for fast transient classification. Publ. Astron. Soc. Pac. 130, 035003 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR52", +"doi-asserted-by": "crossref", +"first-page": "A115", +"DOI": "10.1051/0004-6361/201935344", +"volume": "627", +"author": "M Rigault", +"year": "2019", +"unstructured": "Rigault, M. et al. Fully automated integral field spectrograph pipeline for the SEDMachine: pysedm. Astron. Astrophys. 627, A115 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR53", +"doi-asserted-by": "crossref", +"first-page": "A68", +"DOI": "10.1051/0004-6361/201628275", +"volume": "593", +"author": "C Fremling", +"year": "2016", +"unstructured": "Fremling, C. et al. PTF12os and iPTF13bvn. Two stripped-envelope supernovae from low-mass progenitors in NGC 5806. Astron. Astrophys. 593, A68 (2016).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR54", +"doi-asserted-by": "crossref", +"first-page": "72", +"DOI": "10.3847/1538-4357/aa998e", +"volume": "852", +"author": "S van Velzen", +"year": "2018", +"unstructured": "van Velzen, S. On the mass and luminosity functions of tidal disruption flares: rate suppression due to black hole event horizons. Astrophys. J. 852, 72 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR55", +"doi-asserted-by": "crossref", +"first-page": "95", +"DOI": "10.1007/s11214-005-5095-4", +"volume": "120", +"author": "PWA Roming", +"year": "2005", +"unstructured": "Roming, P. W. A. et al. The Swift Ultra-Violet/Optical Telescope. Space Sci. Rev. 120, 95–142 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR56", +"doi-asserted-by": "crossref", +"first-page": "1005", +"DOI": "10.1086/422091", +"volume": "611", +"author": "N Gehrels", +"year": "2004", +"unstructured": "Gehrels, N. et al. The Swift Gamma-Ray Burst Mission. Astrophys. J. 611, 1005–1020 (2004).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR57", +"doi-asserted-by": "crossref", +"first-page": "19", +"DOI": "10.3847/0004-637X/829/1/19", +"volume": "829", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S., Mendez, A. J., Krolik, J. H. & Gorjian, V. Discovery of transient infrared emission from dust heated by stellar tidal disruption flares. Astrophys. J. 829, 19 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR58", +"doi-asserted-by": "crossref", +"first-page": "575", +"DOI": "10.1093/mnras/stw307", +"volume": "458", +"author": "W Lu", +"year": "2016", +"unstructured": "Lu, W., Kumar, P. & Evans, N. J. Infrared emission from tidal disruption events—probing the pc-scale dust content around galactic nuclei. Mon. Not. R. Astron. Soc. 458, 575–581 (2016).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR59", +"unstructured": "Miller, J. S. & Stone, R. P. S. The Kast Double Spectrograph. Technical Report No. 66 (Lick Observatory, 1993)." +}, +{ +"key": "1295_CR60", +"doi-asserted-by": "crossref", +"first-page": "375", +"DOI": "10.1086/133562", +"volume": "107", +"author": "JB Oke", +"year": "1995", +"unstructured": "Oke, J. B. et al. The Keck Low-Resolution Imaging Spectrometer. Publ. Astron. Soc. Pac. 107, 375–385 (1995).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR61", +"doi-asserted-by": "crossref", +"first-page": "765", +"DOI": "10.1111/j.1365-2966.2005.08957.x", +"volume": "359", +"author": "A Garcia-Rissmann", +"year": "2005", +"unstructured": "Garcia-Rissmann, A. et al. An atlas of calcium triplet spectra of active galaxies. Mon. Not. R. Astron. Soc. 359, 765–780 (2005).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR62", +"doi-asserted-by": "crossref", +"first-page": "165", +"DOI": "10.1007/s11214-005-5097-2", +"volume": "120", +"author": "DN Burrows", +"year": "2005", +"unstructured": "Burrows, D. N. et al. The Swift X-Ray Telescope. Space Sci. Rev. 120, 165–195 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR63", +"doi-asserted-by": "crossref", +"first-page": "L1", +"DOI": "10.1051/0004-6361:20000036", +"volume": "365", +"author": "F Jansen", +"year": "2001", +"unstructured": "Jansen, F. et al. XMM-Newton Observatory. I. The spacecraft and operations. Astron. Astrophys. 365, L1–L6 (2001).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR64", +"unstructured": "HI4PI Collaboration et al. HI4PI: a full-sky H i survey based on EBHIS and GASS. Astron. Astrophys. 594, A116 (2016).", +"DOI": "10.1051/0004-6361/201629178", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR65", +"unstructured": "Arnaud, K. A. in Astronomical Data Analysis Software and Systems V (eds Jacoby, G. H. & Barnes, J.) 17 (Astronomical Society of the Pacific, 1996)." +}, +{ +"key": "1295_CR66", +"doi-asserted-by": "crossref", +"first-page": "1545", +"DOI": "10.1111/j.1365-2966.2008.13953.x", +"volume": "391", +"author": "JTL Zwart", +"year": "2008", +"unstructured": "Zwart, J. T. L. et al. The Arcminute Microkelvin Imager. Mon. Not. R. Astron. Soc. 391, 1545–1558 (2008).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR67", +"doi-asserted-by": "crossref", +"first-page": "5677", +"DOI": "10.1093/mnras/sty074", +"volume": "475", +"author": "J Hickish", +"year": "2018", +"unstructured": "Hickish, J. et al. A digital correlator upgrade for the Arcminute MicroKelvin Imager. Mon. Not. R. Astron. Soc. 475, 5677–5687 (2018).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR68", +"doi-asserted-by": "crossref", +"first-page": "1396", +"DOI": "10.1093/mnras/stv1728", +"volume": "453", +"author": "YC Perrott", +"year": "2015", +"unstructured": "Perrott, Y. C. et al. AMI galactic plane survey at 16 GHz—II. Full data release with extended coverage and improved processing. Mon. Not. R. Astron. Soc. 453, 1396–1403 (2015).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR69", +"unstructured": "McMullin, J. P., Waters, B., Schiebel, D., Young, W. & Golap, K. in Astronomical Data Analysis Software and Systems XVI (eds Shaw, R. A. et al.) 127 (Astronomical Society of the Pacific, 2007)." +}, +{ +"key": "1295_CR70", +"doi-asserted-by": "crossref", +"first-page": "1071", +"DOI": "10.1088/0004-637X/697/2/1071", +"volume": "697", +"author": "WB Atwood", +"year": "2009", +"unstructured": "Atwood, W. B. et al. The Large Area Telescope on the Fermi Gamma-ray Space Telescope mission. Astrophys. J. 697, 1071–1102 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR71", +"unstructured": "Wood, M. et al. Fermipy: an open-source Python package for analysis of Fermi-LAT Data. In Proc. 35th International Cosmic Ray Conference (ICRC2017) 824 (PoS, 2017).", +"DOI": "10.22323/1.301.0824", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR72", +"unstructured": "Garrappa, S. & Buson, S. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR73", +"unstructured": "The Fermi-LAT collaboration. Fermi Large Area Telescope Fourth Source Catalog. Astrophys. J. Suppl. Ser. 247, 33 (2020)." +}, +{ +"key": "1295_CR74", +"doi-asserted-by": "crossref", +"first-page": "14", +"DOI": "10.1088/0004-637X/767/1/14", +"volume": "767", +"author": "T Pursimo", +"year": "2013", +"unstructured": "Pursimo, T. et al. The Micro-Arcsecond Scintillation-Induced Variability (MASIV) survey. III. Optical identifications and new redshifts. Astrophys. J. 767, 14 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR75", +"unstructured": "Garrappa, S., Buson, S. & Fermi-LAT Collaboration. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR76", +"doi-asserted-by": "crossref", +"first-page": "133", +"DOI": "10.1088/0004-637X/802/2/133", +"volume": "802", +"author": "C Diltz", +"year": "2015", +"unstructured": "Diltz, C., Böttcher, M. & Fossati, G. Time dependent hadronic modeling of flat spectrum radio quasars. Astrophys. J. 802, 133 (2015).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR77", +"doi-asserted-by": "crossref", +"first-page": "88", +"DOI": "10.1038/s41550-018-0610-1", +"volume": "3", +"author": "S Gao", +"year": "2019", +"unstructured": "Gao, S., Fedynitch, A., Winter, W. & Pohl, M. Modelling the coincident observation of a high-energy neutrino and a bright blazar flare. Nat. Astron. 3, 88–92 (2019).", +"journal-title": "Nat. Astron." +}, +{ +"key": "1295_CR78", +"unstructured": "Ayala, H. IceCube-191001A: HAWC follow-up. GCN Circ. 25936 (2019)." +}, +{ +"key": "1295_CR79", +"doi-asserted-by": "crossref", +"first-page": "62", +"DOI": "10.1126/science.aad1182", +"volume": "351", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S. et al. A radio jet from the optical and x-ray bright stellar tidal disruption flare ASASSN-14li. Science 351, 62–65 (2016).", +"journal-title": "Science" +}, +{ +"key": "1295_CR80", +"doi-asserted-by": "crossref", +"first-page": "306", +"DOI": "10.1086/670067", +"volume": "125", +"author": "D Foreman-Mackey", +"year": "2013", +"unstructured": "Foreman-Mackey, D., Hogg, D. W., Lang, D. & Goodman, J. emcee: the MCMC Hammer. Publ. Astron. Soc. Pac. 125, 306 (2013).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR81", +"doi-asserted-by": "crossref", +"first-page": "6", +"DOI": "10.3847/1538-4365/aab761", +"volume": "236", +"author": "J Guillochon", +"year": "2018", +"unstructured": "Guillochon, J. et al. MOSFiT: Modular Open Source Fitter for Transients. Astrophys. J. Suppl. Ser. 236, 6 (2018).", +"journal-title": "Astrophys. J. Suppl. Ser." +}, +{ +"key": "1295_CR82", +"doi-asserted-by": "crossref", +"first-page": "e008", +"DOI": "10.1017/pasa.2013.44", +"volume": "31", +"author": "J Granot", +"year": "2014", +"unstructured": "Granot, J. & van der Horst, A. J. Gamma-ray burst jets and their radio observations. Publ. Astron. Soc. Aust. 31, e008 (2014).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR83", +"doi-asserted-by": "crossref", +"first-page": "102", +"DOI": "10.1088/0004-637X/815/2/102", +"volume": "815", +"author": "W Fong", +"year": "2015", +"unstructured": "Fong, W., Berger, E., Margutti, R. & Zauderer, B. A. A decade of short-duration gamma-ray burst broadband afterglows: energetics, circumburst densities, and jet opening angles. Astrophys. J. 815, 102 (2015).", +"journal-title": "Astrophys. J." +} +], +"container-title": [ +"Nature Astronomy" +], +"original-title": [ + +], +"language": "en", +"link": [ +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8", +"content-type": "text/html", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "similarity-checking" +} +], +"deposited": { +"date-parts": [ +[ +2021, +5, +17 +] +], +"date-time": "2021-05-17T15:08:12Z", +"timestamp": 1621264092000 +}, +"score": 1.0, +"subtitle": [ + +], +"short-title": [ + +], +"issued": { +"date-parts": [ +[ +2021, +2, +22 +] +], + "date-time": "2021-05-17T15:08:12Z" +}, +"references-count": 83, +"journal-issue": { +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"issue": "5" +}, +"alternative-id": [ +"1295" +], +"URL": "http://dx.doi.org/10.1038/s41550-020-01295-8", +"relation": { +"cites": [ + +] +}, +"ISSN": [ +"2397-3366" +], +"issn-type": [ +{ +"value": "2397-3366", +"type": "electronic" +} +], +"assertion": [ +{ +"value": "21 July 2020", +"order": 1, +"name": "received", +"label": "Received", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "16 December 2020", +"order": 2, +"name": "accepted", +"label": "Accepted", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "22 February 2021", +"order": 3, +"name": "first_online", +"label": "First Online", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "The authors declare no competing interests.", +"order": 1, +"name": "Ethics", +"group": { +"name": "EthicsHeading", +"label": "Competing interests" +} +} +] +} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo_open.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo_open.json new file mode 100644 index 000000000..e667f3c7f --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_embargo_open.json @@ -0,0 +1,1537 @@ +{ +"indexed": { +"date-parts": [ +[ +2021, +7, +2 +] +], +"date-time": "2021-07-02T07:30:10Z", +"timestamp": 1625211010708 +}, +"reference-count": 83, +"publisher": "Springer Science and Business Media LLC", +"issue": "5", +"license": [ +{ +"URL": "https://www.springer.com/tdm", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "tdm" +}, +{ +"URL": "https://academic.oup.com/journals/pages/open_access/funder_policies/chorus/standard_publication_model", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "vor" +} +], +"content-domain": { +"domain": [ +"link.springer.com" +], +"crossmark-restriction": false +}, +"short-container-title": [ +"Nat Astron" +], +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"DOI": "10.1038/s41550-020-01295-8", +"type": "journal-article", +"created": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T17:03:42Z", +"timestamp": 1614013422000 +}, +"page": "510-518", +"update-policy": "http://dx.doi.org/10.1007/springer_crossmark_policy", +"source": "Crossref", +"is-referenced-by-count": 6, +"title": [ +"A tidal disruption event coincident with a high-energy neutrino" +], +"prefix": "10.1038", +"volume": "5", +"author": [ +{ +"ORCID": "http://orcid.org/0000-0003-2434-0387", +"authenticated-orcid": false, +"given": "Robert", +"family": "Stein", +"sequence": "first", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3859-8074", +"authenticated-orcid": false, +"given": "Sjoert van", +"family": "Velzen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8594-8666", +"authenticated-orcid": false, +"given": "Marek", +"family": "Kowalski", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Anna", +"family": "Franckowiak", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3703-5154", +"authenticated-orcid": false, +"given": "Suvi", +"family": "Gezari", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3124-2814", +"authenticated-orcid": false, +"given": "James C. A.", +"family": "Miller-Jones", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Sara", +"family": "Frederick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0466-3779", +"authenticated-orcid": false, +"given": "Itai", +"family": "Sfaradi", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael F.", +"family": "Bietenholz", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5936-1156", +"authenticated-orcid": false, +"given": "Assaf", +"family": "Horesh", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rob", +"family": "Fender", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2403-4582", +"authenticated-orcid": false, +"given": "Simone", +"family": "Garrappa", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-2184-6430", +"authenticated-orcid": false, +"given": "Tomás", +"family": "Ahumada", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Igor", +"family": "Andreoni", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Justin", +"family": "Belicki", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8018-5348", +"authenticated-orcid": false, +"given": "Eric C.", +"family": "Bellm", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Markus", +"family": "Böttcher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Valery", +"family": "Brinnel", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rick", +"family": "Burruss", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1673-970X", +"authenticated-orcid": false, +"given": "S. Bradley", +"family": "Cenko", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8262-2924", +"authenticated-orcid": false, +"given": "Michael W.", +"family": "Coughlin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2292-0441", +"authenticated-orcid": false, +"given": "Virginia", +"family": "Cunningham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Andrew", +"family": "Drake", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Glennys R.", +"family": "Farrar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael", +"family": "Feeney", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Ryan J.", +"family": "Foley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3653-5598", +"authenticated-orcid": false, +"given": "Avishay", +"family": "Gal-Yam", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "V. Zach", +"family": "Golkhou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-4163-4996", +"authenticated-orcid": false, +"given": "Ariel", +"family": "Goobar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3168-0139", +"authenticated-orcid": false, +"given": "Matthew J.", +"family": "Graham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Erica", +"family": "Hammerstein", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3367-3415", +"authenticated-orcid": false, +"given": "George", +"family": "Helou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-9878-7889", +"authenticated-orcid": false, +"given": "Tiara", +"family": "Hung", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Mansi M.", +"family": "Kasliwal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5740-7747", +"authenticated-orcid": false, +"given": "Charles D.", +"family": "Kilpatrick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5105-344X", +"authenticated-orcid": false, +"given": "Albert K. H.", +"family": "Kong", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-6540-1484", +"authenticated-orcid": false, +"given": "Thomas", +"family": "Kupfer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2451-5482", +"authenticated-orcid": false, +"given": "Russ R.", +"family": "Laher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2242-0244", +"authenticated-orcid": false, +"given": "Ashish A.", +"family": "Mahabal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8532-9395", +"authenticated-orcid": false, +"given": "Frank J.", +"family": "Masci", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0280-7484", +"authenticated-orcid": false, +"given": "Jannis", +"family": "Necker", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8342-6274", +"authenticated-orcid": false, +"given": "Jakob", +"family": "Nordin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel A.", +"family": "Perley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8121-2560", +"authenticated-orcid": false, +"given": "Mickael", +"family": "Rigault", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7788-628X", +"authenticated-orcid": false, +"given": "Simeon", +"family": "Reusch", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Hector", +"family": "Rodriguez", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7559-315X", +"authenticated-orcid": false, +"given": "César", +"family": "Rojas-Bravo", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-7648-4142", +"authenticated-orcid": false, +"given": "Ben", +"family": "Rusholme", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-4401-0430", +"authenticated-orcid": false, +"given": "David L.", +"family": "Shupe", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-9898-5597", +"authenticated-orcid": false, +"given": "Leo P.", +"family": "Singer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1546-6615", +"authenticated-orcid": false, +"given": "Jesper", +"family": "Sollerman", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Maayane T.", +"family": "Soumagnac", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel", +"family": "Stern", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Kirsty", +"family": "Taggart", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Jakob", +"family": "van Santen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Charlotte", +"family": "Ward", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Patrick", +"family": "Woudt", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-6747-8509", +"authenticated-orcid": false, +"given": "Yuhan", +"family": "Yao", +"sequence": "additional", +"affiliation": [ + +] +} +], +"member": "297", +"published-online": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"reference": [ +{ +"key": "1295_CR1", +"doi-asserted-by": "crossref", +"first-page": "P03012", +"DOI": "10.1088/1748-0221/12/03/P03012", +"volume": "12", +"author": "MG Aartsen", +"year": "2017", +"unstructured": "Aartsen, M. G. et al. The IceCube Neutrino Observatory: instrumentation and online systems. J. Instrum. 12, P03012 (2017).", +"journal-title": "J. Instrum." +}, +{ +"key": "1295_CR2", +"unstructured": "Stein, R. IceCube-191001A—IceCube observation of a high-energy neutrino candidate event. GCN Circ. 25913 (2019)." +}, +{ +"key": "1295_CR3", +"doi-asserted-by": "crossref", +"first-page": "018002", +"DOI": "10.1088/1538-3873/aaecbe", +"volume": "131", +"author": "EC Bellm", +"year": "2019", +"unstructured": "Bellm, E. C. et al. The Zwicky Transient Facility: system overview, performance, and first results. Publ. Astron. Soc. Pac. 131, 018002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR4", +"doi-asserted-by": "crossref", +"first-page": "533", +"DOI": "10.1016/j.astropartphys.2007.03.005", +"volume": "27", +"author": "M Kowalski", +"year": "2007", +"unstructured": "Kowalski, M. & Mohr, A. Detecting neutrino transients with optical follow-up observations. Astropart. Phys. 27, 533–538 (2007).", +"journal-title": "Astropart. Phys." +}, +{ +"key": "1295_CR5", +"doi-asserted-by": "crossref", +"first-page": "329", +"DOI": "10.1088/0004-637X/693/1/329", +"volume": "693", +"author": "GR Farrar", +"year": "2009", +"unstructured": "Farrar, G. R. & Gruzinov, A. Giant AGN flares and cosmic ray bursts. Astrophys. J. 693, 329–332 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR6", +"doi-asserted-by": "crossref", +"first-page": "1354", +"DOI": "10.1093/mnras/stx863", +"volume": "469", +"author": "L Dai", +"year": "2017", +"unstructured": "Dai, L. & Fang, K. Can tidal disruption events produce the IceCube neutrinos? Mon. Not. R. Astron. Soc. 469, 1354–1359 (2017).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR7", +"doi-asserted-by": "crossref", +"first-page": "114", +"DOI": "10.3847/1538-4357/ab44ca", +"volume": "886", +"author": "K Hayasaki", +"year": "2019", +"unstructured": "Hayasaki, K. & Yamazaki, R. Neutrino emissions from tidal disruption remnants. Astrophys. J. 886, 114 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR8", +"unstructured": "Farrar, G. R. & Piran, T. Tidal disruption jets as the source of Ultra-High Energy Cosmic Rays. Preprint at https://arxiv.org/abs/1411.0704 (2014)." +}, +{ +"key": "1295_CR9", +"doi-asserted-by": "crossref", +"first-page": "3", +"DOI": "10.3847/1538-4357/aa6344", +"volume": "838", +"author": "N Senno", +"year": "2017", +"unstructured": "Senno, N., Murase, K. & Mészáros, P. High-energy neutrino flares from X-ray bright and dark tidal disruption events. Astrophys. J. 838, 3 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR10", +"doi-asserted-by": "crossref", +"first-page": "083005", +"DOI": "10.1103/PhysRevD.93.083005", +"volume": "93", +"author": "XY Wang", +"year": "2016", +"unstructured": "Wang, X. Y. & Liu, R. Y. Tidal disruption jets of supermassive black holes as hidden sources of cosmic rays: explaining the IceCube TeV–PeV neutrinos. Phys. Rev. D 93, 083005 (2016).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR11", +"doi-asserted-by": "crossref", +"first-page": "123001", +"DOI": "10.1103/PhysRevD.95.123001", +"volume": "95", +"author": "C Lunardini", +"year": "2017", +"unstructured": "Lunardini, C. & Winter, W. High energy neutrinos from the tidal disruption of stars. Phys. Rev. D 95, 123001 (2017).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR12", +"unstructured": "Stein, R., Franckowiak, A., Necker, J., Gezari, S. & Velzen, S. V. Candidate counterparts to IceCube-191001A with ZTF. Astron. Telegr. 13160 (2019)." +}, +{ +"key": "1295_CR13", +"doi-asserted-by": "crossref", +"first-page": "078001", +"DOI": "10.1088/1538-3873/ab006c", +"volume": "131", +"author": "MJ Graham", +"year": "2019", +"unstructured": "Graham, M. J. et al. The Zwicky Transient Facility: science objectives. Publ. Astron. Soc. Pac. 131, 078001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR14", +"unstructured": "Nordin, J. et al. TNS Astronomical Transient Report 33340 (2019)." +}, +{ +"key": "1295_CR15", +"unstructured": "Nicholl, M. et al. ePESSTO+ classification of optical transients. Astron. Telegr. 12752 (2019)." +}, +{ +"key": "1295_CR16", +"unstructured": "van Velzen, S. et al. Seventeen tidal disruption events from the first half of ZTF survey observations: entering a new era of population studies. Preprint at https://arxiv.org/abs/2001.01409 (2020)." +}, +{ +"key": "1295_CR17", +"doi-asserted-by": "crossref", +"first-page": "82", +"DOI": "10.3847/1538-4357/ab1844", +"volume": "878", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. Late-time UV observations of tidal disruption flares reveal unobscured, compact accretion disks. Astrophys. J. 878, 82 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR18", +"doi-asserted-by": "crossref", +"first-page": "5655", +"DOI": "10.1093/mnras/staa192", +"volume": "492", +"author": "A Mummery", +"year": "2020", +"unstructured": "Mummery, A. & Balbus, S. A. The spectral evolution of disc dominated tidal disruption events. Mon. Not. R. Astron. Soc. 492, 5655–5674 (2020).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR19", +"doi-asserted-by": "crossref", +"first-page": "184", +"DOI": "10.1088/0004-637X/764/2/184", +"volume": "764", +"author": "NJ McConnell", +"year": "2013", +"unstructured": "McConnell, N. J. & Ma, C. P. Revisiting the scaling relations of black hole masses and host galaxy properties. Astrophys. J. 764, 184 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR20", +"doi-asserted-by": "crossref", +"first-page": "149", +"DOI": "10.3847/1538-4357/aa633b", +"volume": "838", +"author": "K Auchettl", +"year": "2017", +"unstructured": "Auchettl, K., Guillochon, J. & Ramirez-Ruiz, E. New physical insights about tidal disruption events from a comprehensive observational inventory at X-ray wavelengths. Astrophys. J. 838, 149 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR21", +"doi-asserted-by": "crossref", +"first-page": "4136", +"DOI": "10.1093/mnras/stz1602", +"volume": "487", +"author": "T Wevers", +"year": "2019", +"unstructured": "Wevers, T. et al. Black hole masses of tidal disruption event host galaxies II. Mon. Not. R. Astron. Soc. 487, 4136–4152 (2019).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR22", +"doi-asserted-by": "crossref", +"first-page": "198", +"DOI": "10.3847/1538-4357/aafe0c", +"volume": "872", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. The first tidal disruption flare in ZTF: from photometric selection to multi-wavelength characterization. Astrophys. J. 872, 198 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR23", +"doi-asserted-by": "crossref", +"first-page": "A81", +"DOI": "10.1051/0004-6361/201117855", +"volume": "538", +"author": "G Morlino", +"year": "2012", +"unstructured": "Morlino, G. & Caprioli, D. Strong evidence for hadron acceleration in Tycho’s supernova remnant. Astron. Astrophys. 538, A81 (2012).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR24", +"doi-asserted-by": "crossref", +"first-page": "86", +"DOI": "10.3847/1538-4357/aaa8e0", +"volume": "854", +"author": "T Eftekhari", +"year": "2018", +"unstructured": "Eftekhari, T., Berger, E., Zauderer, B. A., Margutti, R. & Alexander, K. D. Radio monitoring of the tidal disruption event Swift J164449.3+573451. III. Late-time jet energetics and a deviation from equipartition. Astrophys. J. 854, 86 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR25", +"doi-asserted-by": "crossref", +"first-page": "1258", +"DOI": "10.1093/mnras/stt1645", +"volume": "436", +"author": "A Horesh", +"year": "2013", +"unstructured": "Horesh, A. et al. An early and comprehensive millimetre and centimetre wave and X-ray study of SN 2011dh: a non-equipartition blast wave expanding into a massive stellar wind. Mon. Not. R. Astron. Soc. 436, 1258–1267 (2013).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR26", +"doi-asserted-by": "crossref", +"first-page": "78", +"DOI": "10.1088/0004-637X/772/1/78", +"volume": "772", +"author": "R Barniol Duran", +"year": "2013", +"unstructured": "Barniol Duran, R., Nakar, E. & Piran, T. Radius constraints and minimal equipartition energy of relativistically moving synchrotron sources. Astrophys. J. 772, 78 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR27", +"doi-asserted-by": "crossref", +"first-page": "69", +"DOI": "10.1071/AS02053", +"volume": "20", +"author": "AG Polatidis", +"year": "2003", +"unstructured": "Polatidis, A. G. & Conway, J. E. Proper motions in compact symmetric objects. Publ. Astron. Soc. Aust. 20, 69–74 (2003).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR28", +"doi-asserted-by": "crossref", +"first-page": "L25", +"DOI": "10.3847/2041-8205/819/2/L25", +"volume": "819", +"author": "KD Alexander", +"year": "2016", +"unstructured": "Alexander, K. D., Berger, E., Guillochon, J., Zauderer, B. A. & Williams, P. K. G. Discovery of an outflow from radio observations of the tidal disruption event ASASSN-14li. Astrophys. J. Lett. 819, L25 (2016).", +"journal-title": "Astrophys. J. Lett." +}, +{ +"key": "1295_CR29", +"doi-asserted-by": "crossref", +"first-page": "127", +"DOI": "10.3847/0004-637X/827/2/127", +"volume": "827", +"author": "J Krolik", +"year": "2016", +"unstructured": "Krolik, J., Piran, T., Svirski, G. & Cheng, R. M. ASASSN-14li: a model tidal disruption event. Astrophys. J. 827, 127 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR30", +"doi-asserted-by": "crossref", +"first-page": "1", +"DOI": "10.3847/1538-4357/aab361", +"volume": "856", +"author": "DR Pasham", +"year": "2018", +"unstructured": "Pasham, D. R. & van Velzen, S. Discovery of a time lag between the soft X-ray and radio emission of the tidal disruption flare ASASSN-14li: evidence for linear disk–jet coupling. Astrophys. J. 856, 1 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR31", +"doi-asserted-by": "crossref", +"first-page": "L9", +"DOI": "10.1051/0004-6361/201834750", +"volume": "622", +"author": "NL Strotjohann", +"year": "2019", +"unstructured": "Strotjohann, N. L., Kowalski, M. & Franckowiak, A. Eddington bias for cosmic neutrino sources. Astron. Astrophys. 622, L9 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR32", +"doi-asserted-by": "crossref", +"first-page": "425", +"DOI": "10.1146/annurev.aa.22.090184.002233", +"volume": "22", +"author": "AM Hillas", +"year": "1984", +"unstructured": "Hillas, A. M. The origin of ultra-high-energy cosmic rays. Annu. Rev. Astron. Astrophys. 22, 425–444 (1984).", +"journal-title": "Annu. Rev. Astron. Astrophys." +}, +{ +"key": "1295_CR33", +"doi-asserted-by": "crossref", +"first-page": "eaat1378", +"DOI": "10.1126/science.aat1378", +"volume": "361", +"author": "IceCube Collaboration", +"year": "2018", +"unstructured": "IceCube Collaboration et al. Multimessenger observations of a flaring blazar coincident with high-energy neutrino IceCube-170922A. Science 361, eaat1378 (2018).", +"journal-title": "Science" +}, +{ +"key": "1295_CR34", +"unstructured": "Blaufuss, E., Kintscher, T., Lu, L. & Tung, C. F. The next generation of IceCube real-time neutrino alerts. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1021 (PoS, 2019)." +}, +{ +"key": "1295_CR35", +"doi-asserted-by": "crossref", +"first-page": "071101", +"DOI": "10.1103/PhysRevLett.116.071101", +"volume": "116", +"author": "K Murase", +"year": "2016", +"unstructured": "Murase, K., Guetta, D. & Ahlers, M. Hidden cosmic-ray accelerators as an origin of TeV–PeV cosmic neutrinos. Phys. Rev. Lett. 116, 071101 (2016).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR36", +"unstructured": "Stein, R. Search for neutrinos from populations of optical transients. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1016 (PoS, 2019).", +"DOI": "10.22323/1.358.1016", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR37", +"doi-asserted-by": "crossref", +"first-page": "048001", +"DOI": "10.1088/1538-3873/aaff99", +"volume": "131", +"author": "MW Coughlin", +"year": "2019", +"unstructured": "Coughlin, M. W. et al. 2900 square degree search for the optical counterpart of short gamma-ray burst GRB 180523B with the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 048001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR38", +"unstructured": "Stein, R. IceCube-200107A: IceCube observation of a high-energy neutrino candidate event. GCN Circ. 26655 (2020)." +}, +{ +"key": "1295_CR39", +"doi-asserted-by": "crossref", +"first-page": "018003", +"DOI": "10.1088/1538-3873/aae8ac", +"volume": "131", +"author": "FJ Masci", +"year": "2019", +"unstructured": "Masci, F. J. et al. The Zwicky Transient Facility: data processing, products, and archive. Publ. Astron. Soc. Pac. 131, 018003 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR40", +"doi-asserted-by": "crossref", +"first-page": "018001", +"DOI": "10.1088/1538-3873/aae904", +"volume": "131", +"author": "MT Patterson", +"year": "2019", +"unstructured": "Patterson, M. T. et al. The Zwicky Transient Facility Alert Distribution System. Publ. Astron. Soc. Pac. 131, 018001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR41", +"unstructured": "Stein, R. & Reusch, S. robertdstein/ampel_followup_pipeline: V1.1 Release (Zenodo, 2020); https://doi.org/10.5281/zenodo.4048336", +"DOI": "10.5281/zenodo.4048336", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR42", +"doi-asserted-by": "crossref", +"first-page": "A147", +"DOI": "10.1051/0004-6361/201935634", +"volume": "631", +"author": "J Nordin", +"year": "2019", +"unstructured": "Nordin, J. et al. Transient processing and analysis using AMPEL: alert management, photometry, and evaluation of light curves. Astron. Astrophys. 631, A147 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR43", +"doi-asserted-by": "crossref", +"first-page": "038002", +"DOI": "10.1088/1538-3873/aaf3fa", +"volume": "131", +"author": "A Mahabal", +"year": "2019", +"unstructured": "Mahabal, A. et al. Machine learning for the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 038002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR44", +"doi-asserted-by": "crossref", +"first-page": "075002", +"DOI": "10.1088/1538-3873/aac410", +"volume": "130", +"author": "MT Soumagnac", +"year": "2018", +"unstructured": "Soumagnac, M. T. & Ofek, E. O. catsHTM: a tool for fast accessing and cross-matching large astronomical catalogs. Publ. Astron. Soc. Pac. 130, 075002 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR45", +"doi-asserted-by": "crossref", +"first-page": "A1", +"DOI": "10.1051/0004-6361/201833051", +"volume": "616", +"author": "Gaia Collaboration", +"year": "2018", +"unstructured": "Gaia Collaboration et al. Gaia Data Release 2. Summary of the contents and survey properties. Astron. Astrophys. 616, A1 (2018).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR46", +"doi-asserted-by": "crossref", +"first-page": "128001", +"DOI": "10.1088/1538-3873/aae3d9", +"volume": "130", +"author": "Y Tachibana", +"year": "2018", +"unstructured": "Tachibana, Y. & Miller, A. A. A morphological classification model to identify unresolved PanSTARRS1 sources: application in the ZTF real-time pipeline. Publ. Astron. Soc. Pac. 130, 128001 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR47", +"unstructured": "Chambers, K. C. et al. The Pan-STARRS1 Surveys. Preprint at https://arxiv.org/abs/1612.05560 (2016)." +}, +{ +"key": "1295_CR48", +"doi-asserted-by": "crossref", +"first-page": "1868", +"DOI": "10.1088/0004-6256/140/6/1868", +"volume": "140", +"author": "EL Wright", +"year": "2010", +"unstructured": "Wright, E. L. et al. The Wide-field Infrared Survey Explorer (WISE): mission description and initial on-orbit performance. Astron. J. 140, 1868–1881 (2010).", +"journal-title": "Astron. J." +}, +{ +"key": "1295_CR49", +"doi-asserted-by": "crossref", +"first-page": "051103", +"DOI": "10.1103/PhysRevLett.124.051103", +"volume": "124", +"author": "MG Aartsen", +"year": "2020", +"unstructured": "Aartsen, M. G. et al. Time-integrated neutrino source searches with 10 years of IceCube data. Phys. Rev. Lett. 124, 051103 (2020).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR50", +"unstructured": "Steele, I. A. et al. The Liverpool Telescope: performance and first results. Proc. SPIE 5489, https://doi.org/10.1117/12.551456 (2004).", +"DOI": "10.1117/12.551456", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR51", +"doi-asserted-by": "crossref", +"first-page": "035003", +"DOI": "10.1088/1538-3873/aaa53f", +"volume": "130", +"author": "N Blagorodnova", +"year": "2018", +"unstructured": "Blagorodnova, N. et al. The SED Machine: a robotic spectrograph for fast transient classification. Publ. Astron. Soc. Pac. 130, 035003 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR52", +"doi-asserted-by": "crossref", +"first-page": "A115", +"DOI": "10.1051/0004-6361/201935344", +"volume": "627", +"author": "M Rigault", +"year": "2019", +"unstructured": "Rigault, M. et al. Fully automated integral field spectrograph pipeline for the SEDMachine: pysedm. Astron. Astrophys. 627, A115 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR53", +"doi-asserted-by": "crossref", +"first-page": "A68", +"DOI": "10.1051/0004-6361/201628275", +"volume": "593", +"author": "C Fremling", +"year": "2016", +"unstructured": "Fremling, C. et al. PTF12os and iPTF13bvn. Two stripped-envelope supernovae from low-mass progenitors in NGC 5806. Astron. Astrophys. 593, A68 (2016).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR54", +"doi-asserted-by": "crossref", +"first-page": "72", +"DOI": "10.3847/1538-4357/aa998e", +"volume": "852", +"author": "S van Velzen", +"year": "2018", +"unstructured": "van Velzen, S. On the mass and luminosity functions of tidal disruption flares: rate suppression due to black hole event horizons. Astrophys. J. 852, 72 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR55", +"doi-asserted-by": "crossref", +"first-page": "95", +"DOI": "10.1007/s11214-005-5095-4", +"volume": "120", +"author": "PWA Roming", +"year": "2005", +"unstructured": "Roming, P. W. A. et al. The Swift Ultra-Violet/Optical Telescope. Space Sci. Rev. 120, 95–142 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR56", +"doi-asserted-by": "crossref", +"first-page": "1005", +"DOI": "10.1086/422091", +"volume": "611", +"author": "N Gehrels", +"year": "2004", +"unstructured": "Gehrels, N. et al. The Swift Gamma-Ray Burst Mission. Astrophys. J. 611, 1005–1020 (2004).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR57", +"doi-asserted-by": "crossref", +"first-page": "19", +"DOI": "10.3847/0004-637X/829/1/19", +"volume": "829", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S., Mendez, A. J., Krolik, J. H. & Gorjian, V. Discovery of transient infrared emission from dust heated by stellar tidal disruption flares. Astrophys. J. 829, 19 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR58", +"doi-asserted-by": "crossref", +"first-page": "575", +"DOI": "10.1093/mnras/stw307", +"volume": "458", +"author": "W Lu", +"year": "2016", +"unstructured": "Lu, W., Kumar, P. & Evans, N. J. Infrared emission from tidal disruption events—probing the pc-scale dust content around galactic nuclei. Mon. Not. R. Astron. Soc. 458, 575–581 (2016).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR59", +"unstructured": "Miller, J. S. & Stone, R. P. S. The Kast Double Spectrograph. Technical Report No. 66 (Lick Observatory, 1993)." +}, +{ +"key": "1295_CR60", +"doi-asserted-by": "crossref", +"first-page": "375", +"DOI": "10.1086/133562", +"volume": "107", +"author": "JB Oke", +"year": "1995", +"unstructured": "Oke, J. B. et al. The Keck Low-Resolution Imaging Spectrometer. Publ. Astron. Soc. Pac. 107, 375–385 (1995).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR61", +"doi-asserted-by": "crossref", +"first-page": "765", +"DOI": "10.1111/j.1365-2966.2005.08957.x", +"volume": "359", +"author": "A Garcia-Rissmann", +"year": "2005", +"unstructured": "Garcia-Rissmann, A. et al. An atlas of calcium triplet spectra of active galaxies. Mon. Not. R. Astron. Soc. 359, 765–780 (2005).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR62", +"doi-asserted-by": "crossref", +"first-page": "165", +"DOI": "10.1007/s11214-005-5097-2", +"volume": "120", +"author": "DN Burrows", +"year": "2005", +"unstructured": "Burrows, D. N. et al. The Swift X-Ray Telescope. Space Sci. Rev. 120, 165–195 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR63", +"doi-asserted-by": "crossref", +"first-page": "L1", +"DOI": "10.1051/0004-6361:20000036", +"volume": "365", +"author": "F Jansen", +"year": "2001", +"unstructured": "Jansen, F. et al. XMM-Newton Observatory. I. The spacecraft and operations. Astron. Astrophys. 365, L1–L6 (2001).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR64", +"unstructured": "HI4PI Collaboration et al. HI4PI: a full-sky H i survey based on EBHIS and GASS. Astron. Astrophys. 594, A116 (2016).", +"DOI": "10.1051/0004-6361/201629178", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR65", +"unstructured": "Arnaud, K. A. in Astronomical Data Analysis Software and Systems V (eds Jacoby, G. H. & Barnes, J.) 17 (Astronomical Society of the Pacific, 1996)." +}, +{ +"key": "1295_CR66", +"doi-asserted-by": "crossref", +"first-page": "1545", +"DOI": "10.1111/j.1365-2966.2008.13953.x", +"volume": "391", +"author": "JTL Zwart", +"year": "2008", +"unstructured": "Zwart, J. T. L. et al. The Arcminute Microkelvin Imager. Mon. Not. R. Astron. Soc. 391, 1545–1558 (2008).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR67", +"doi-asserted-by": "crossref", +"first-page": "5677", +"DOI": "10.1093/mnras/sty074", +"volume": "475", +"author": "J Hickish", +"year": "2018", +"unstructured": "Hickish, J. et al. A digital correlator upgrade for the Arcminute MicroKelvin Imager. Mon. Not. R. Astron. Soc. 475, 5677–5687 (2018).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR68", +"doi-asserted-by": "crossref", +"first-page": "1396", +"DOI": "10.1093/mnras/stv1728", +"volume": "453", +"author": "YC Perrott", +"year": "2015", +"unstructured": "Perrott, Y. C. et al. AMI galactic plane survey at 16 GHz—II. Full data release with extended coverage and improved processing. Mon. Not. R. Astron. Soc. 453, 1396–1403 (2015).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR69", +"unstructured": "McMullin, J. P., Waters, B., Schiebel, D., Young, W. & Golap, K. in Astronomical Data Analysis Software and Systems XVI (eds Shaw, R. A. et al.) 127 (Astronomical Society of the Pacific, 2007)." +}, +{ +"key": "1295_CR70", +"doi-asserted-by": "crossref", +"first-page": "1071", +"DOI": "10.1088/0004-637X/697/2/1071", +"volume": "697", +"author": "WB Atwood", +"year": "2009", +"unstructured": "Atwood, W. B. et al. The Large Area Telescope on the Fermi Gamma-ray Space Telescope mission. Astrophys. J. 697, 1071–1102 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR71", +"unstructured": "Wood, M. et al. Fermipy: an open-source Python package for analysis of Fermi-LAT Data. In Proc. 35th International Cosmic Ray Conference (ICRC2017) 824 (PoS, 2017).", +"DOI": "10.22323/1.301.0824", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR72", +"unstructured": "Garrappa, S. & Buson, S. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR73", +"unstructured": "The Fermi-LAT collaboration. Fermi Large Area Telescope Fourth Source Catalog. Astrophys. J. Suppl. Ser. 247, 33 (2020)." +}, +{ +"key": "1295_CR74", +"doi-asserted-by": "crossref", +"first-page": "14", +"DOI": "10.1088/0004-637X/767/1/14", +"volume": "767", +"author": "T Pursimo", +"year": "2013", +"unstructured": "Pursimo, T. et al. The Micro-Arcsecond Scintillation-Induced Variability (MASIV) survey. III. Optical identifications and new redshifts. Astrophys. J. 767, 14 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR75", +"unstructured": "Garrappa, S., Buson, S. & Fermi-LAT Collaboration. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR76", +"doi-asserted-by": "crossref", +"first-page": "133", +"DOI": "10.1088/0004-637X/802/2/133", +"volume": "802", +"author": "C Diltz", +"year": "2015", +"unstructured": "Diltz, C., Böttcher, M. & Fossati, G. Time dependent hadronic modeling of flat spectrum radio quasars. Astrophys. J. 802, 133 (2015).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR77", +"doi-asserted-by": "crossref", +"first-page": "88", +"DOI": "10.1038/s41550-018-0610-1", +"volume": "3", +"author": "S Gao", +"year": "2019", +"unstructured": "Gao, S., Fedynitch, A., Winter, W. & Pohl, M. Modelling the coincident observation of a high-energy neutrino and a bright blazar flare. Nat. Astron. 3, 88–92 (2019).", +"journal-title": "Nat. Astron." +}, +{ +"key": "1295_CR78", +"unstructured": "Ayala, H. IceCube-191001A: HAWC follow-up. GCN Circ. 25936 (2019)." +}, +{ +"key": "1295_CR79", +"doi-asserted-by": "crossref", +"first-page": "62", +"DOI": "10.1126/science.aad1182", +"volume": "351", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S. et al. A radio jet from the optical and x-ray bright stellar tidal disruption flare ASASSN-14li. Science 351, 62–65 (2016).", +"journal-title": "Science" +}, +{ +"key": "1295_CR80", +"doi-asserted-by": "crossref", +"first-page": "306", +"DOI": "10.1086/670067", +"volume": "125", +"author": "D Foreman-Mackey", +"year": "2013", +"unstructured": "Foreman-Mackey, D., Hogg, D. W., Lang, D. & Goodman, J. emcee: the MCMC Hammer. Publ. Astron. Soc. Pac. 125, 306 (2013).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR81", +"doi-asserted-by": "crossref", +"first-page": "6", +"DOI": "10.3847/1538-4365/aab761", +"volume": "236", +"author": "J Guillochon", +"year": "2018", +"unstructured": "Guillochon, J. et al. MOSFiT: Modular Open Source Fitter for Transients. Astrophys. J. Suppl. Ser. 236, 6 (2018).", +"journal-title": "Astrophys. J. Suppl. Ser." +}, +{ +"key": "1295_CR82", +"doi-asserted-by": "crossref", +"first-page": "e008", +"DOI": "10.1017/pasa.2013.44", +"volume": "31", +"author": "J Granot", +"year": "2014", +"unstructured": "Granot, J. & van der Horst, A. J. Gamma-ray burst jets and their radio observations. Publ. Astron. Soc. Aust. 31, e008 (2014).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR83", +"doi-asserted-by": "crossref", +"first-page": "102", +"DOI": "10.1088/0004-637X/815/2/102", +"volume": "815", +"author": "W Fong", +"year": "2015", +"unstructured": "Fong, W., Berger, E., Margutti, R. & Zauderer, B. A. A decade of short-duration gamma-ray burst broadband afterglows: energetics, circumburst densities, and jet opening angles. Astrophys. J. 815, 102 (2015).", +"journal-title": "Astrophys. J." +} +], +"container-title": [ +"Nature Astronomy" +], +"original-title": [ + +], +"language": "en", +"link": [ +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8", +"content-type": "text/html", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "similarity-checking" +} +], +"deposited": { +"date-parts": [ +[ +2021, +5, +17 +] +], +"date-time": "2021-05-17T15:08:12Z", +"timestamp": 1621264092000 +}, +"score": 1.0, +"subtitle": [ + +], +"short-title": [ + +], +"issued": { +"date-parts": [ +[ +2020, +2, +22 +] +] +}, +"references-count": 83, +"journal-issue": { +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"issue": "5" +}, +"alternative-id": [ +"1295" +], +"URL": "http://dx.doi.org/10.1038/s41550-020-01295-8", +"relation": { +"cites": [ + +] +}, +"ISSN": [ +"2397-3366" +], +"issn-type": [ +{ +"value": "2397-3366", +"type": "electronic" +} +], +"assertion": [ +{ +"value": "21 July 2020", +"order": 1, +"name": "received", +"label": "Received", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "16 December 2020", +"order": 2, +"name": "accepted", +"label": "Accepted", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "22 February 2021", +"order": 3, +"name": "first_online", +"label": "First Online", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "The authors declare no competing interests.", +"order": 1, +"name": "Ethics", +"group": { +"name": "EthicsHeading", +"label": "Competing interests" +} +} +] +} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_open.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_open.json new file mode 100644 index 000000000..225a36b1f --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_open.json @@ -0,0 +1,1537 @@ +{ +"indexed": { +"date-parts": [ +[ +2021, +7, +2 +] +], +"date-time": "2021-07-02T07:30:10Z", +"timestamp": 1625211010708 +}, +"reference-count": 83, +"publisher": "Springer Science and Business Media LLC", +"issue": "5", +"license": [ +{ +"URL": "https://www.springer.com/tdm", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "tdm" +}, +{ +"URL": "http://pubs.acs.org/page/policy/authorchoice_ccby_termsofuse.html", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "vor" +} +], +"content-domain": { +"domain": [ +"link.springer.com" +], +"crossmark-restriction": false +}, +"short-container-title": [ +"Nat Astron" +], +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"DOI": "10.1038/s41550-020-01295-8", +"type": "journal-article", +"created": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T17:03:42Z", +"timestamp": 1614013422000 +}, +"page": "510-518", +"update-policy": "http://dx.doi.org/10.1007/springer_crossmark_policy", +"source": "Crossref", +"is-referenced-by-count": 6, +"title": [ +"A tidal disruption event coincident with a high-energy neutrino" +], +"prefix": "10.1038", +"volume": "5", +"author": [ +{ +"ORCID": "http://orcid.org/0000-0003-2434-0387", +"authenticated-orcid": false, +"given": "Robert", +"family": "Stein", +"sequence": "first", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3859-8074", +"authenticated-orcid": false, +"given": "Sjoert van", +"family": "Velzen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8594-8666", +"authenticated-orcid": false, +"given": "Marek", +"family": "Kowalski", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Anna", +"family": "Franckowiak", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3703-5154", +"authenticated-orcid": false, +"given": "Suvi", +"family": "Gezari", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3124-2814", +"authenticated-orcid": false, +"given": "James C. A.", +"family": "Miller-Jones", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Sara", +"family": "Frederick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0466-3779", +"authenticated-orcid": false, +"given": "Itai", +"family": "Sfaradi", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael F.", +"family": "Bietenholz", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5936-1156", +"authenticated-orcid": false, +"given": "Assaf", +"family": "Horesh", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rob", +"family": "Fender", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2403-4582", +"authenticated-orcid": false, +"given": "Simone", +"family": "Garrappa", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-2184-6430", +"authenticated-orcid": false, +"given": "Tomás", +"family": "Ahumada", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Igor", +"family": "Andreoni", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Justin", +"family": "Belicki", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8018-5348", +"authenticated-orcid": false, +"given": "Eric C.", +"family": "Bellm", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Markus", +"family": "Böttcher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Valery", +"family": "Brinnel", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rick", +"family": "Burruss", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1673-970X", +"authenticated-orcid": false, +"given": "S. Bradley", +"family": "Cenko", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8262-2924", +"authenticated-orcid": false, +"given": "Michael W.", +"family": "Coughlin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2292-0441", +"authenticated-orcid": false, +"given": "Virginia", +"family": "Cunningham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Andrew", +"family": "Drake", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Glennys R.", +"family": "Farrar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael", +"family": "Feeney", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Ryan J.", +"family": "Foley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3653-5598", +"authenticated-orcid": false, +"given": "Avishay", +"family": "Gal-Yam", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "V. Zach", +"family": "Golkhou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-4163-4996", +"authenticated-orcid": false, +"given": "Ariel", +"family": "Goobar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3168-0139", +"authenticated-orcid": false, +"given": "Matthew J.", +"family": "Graham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Erica", +"family": "Hammerstein", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3367-3415", +"authenticated-orcid": false, +"given": "George", +"family": "Helou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-9878-7889", +"authenticated-orcid": false, +"given": "Tiara", +"family": "Hung", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Mansi M.", +"family": "Kasliwal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5740-7747", +"authenticated-orcid": false, +"given": "Charles D.", +"family": "Kilpatrick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5105-344X", +"authenticated-orcid": false, +"given": "Albert K. H.", +"family": "Kong", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-6540-1484", +"authenticated-orcid": false, +"given": "Thomas", +"family": "Kupfer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2451-5482", +"authenticated-orcid": false, +"given": "Russ R.", +"family": "Laher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2242-0244", +"authenticated-orcid": false, +"given": "Ashish A.", +"family": "Mahabal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8532-9395", +"authenticated-orcid": false, +"given": "Frank J.", +"family": "Masci", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0280-7484", +"authenticated-orcid": false, +"given": "Jannis", +"family": "Necker", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8342-6274", +"authenticated-orcid": false, +"given": "Jakob", +"family": "Nordin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel A.", +"family": "Perley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8121-2560", +"authenticated-orcid": false, +"given": "Mickael", +"family": "Rigault", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7788-628X", +"authenticated-orcid": false, +"given": "Simeon", +"family": "Reusch", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Hector", +"family": "Rodriguez", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7559-315X", +"authenticated-orcid": false, +"given": "César", +"family": "Rojas-Bravo", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-7648-4142", +"authenticated-orcid": false, +"given": "Ben", +"family": "Rusholme", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-4401-0430", +"authenticated-orcid": false, +"given": "David L.", +"family": "Shupe", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-9898-5597", +"authenticated-orcid": false, +"given": "Leo P.", +"family": "Singer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1546-6615", +"authenticated-orcid": false, +"given": "Jesper", +"family": "Sollerman", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Maayane T.", +"family": "Soumagnac", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel", +"family": "Stern", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Kirsty", +"family": "Taggart", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Jakob", +"family": "van Santen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Charlotte", +"family": "Ward", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Patrick", +"family": "Woudt", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-6747-8509", +"authenticated-orcid": false, +"given": "Yuhan", +"family": "Yao", +"sequence": "additional", +"affiliation": [ + +] +} +], +"member": "297", +"published-online": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"reference": [ +{ +"key": "1295_CR1", +"doi-asserted-by": "crossref", +"first-page": "P03012", +"DOI": "10.1088/1748-0221/12/03/P03012", +"volume": "12", +"author": "MG Aartsen", +"year": "2017", +"unstructured": "Aartsen, M. G. et al. The IceCube Neutrino Observatory: instrumentation and online systems. J. Instrum. 12, P03012 (2017).", +"journal-title": "J. Instrum." +}, +{ +"key": "1295_CR2", +"unstructured": "Stein, R. IceCube-191001A—IceCube observation of a high-energy neutrino candidate event. GCN Circ. 25913 (2019)." +}, +{ +"key": "1295_CR3", +"doi-asserted-by": "crossref", +"first-page": "018002", +"DOI": "10.1088/1538-3873/aaecbe", +"volume": "131", +"author": "EC Bellm", +"year": "2019", +"unstructured": "Bellm, E. C. et al. The Zwicky Transient Facility: system overview, performance, and first results. Publ. Astron. Soc. Pac. 131, 018002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR4", +"doi-asserted-by": "crossref", +"first-page": "533", +"DOI": "10.1016/j.astropartphys.2007.03.005", +"volume": "27", +"author": "M Kowalski", +"year": "2007", +"unstructured": "Kowalski, M. & Mohr, A. Detecting neutrino transients with optical follow-up observations. Astropart. Phys. 27, 533–538 (2007).", +"journal-title": "Astropart. Phys." +}, +{ +"key": "1295_CR5", +"doi-asserted-by": "crossref", +"first-page": "329", +"DOI": "10.1088/0004-637X/693/1/329", +"volume": "693", +"author": "GR Farrar", +"year": "2009", +"unstructured": "Farrar, G. R. & Gruzinov, A. Giant AGN flares and cosmic ray bursts. Astrophys. J. 693, 329–332 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR6", +"doi-asserted-by": "crossref", +"first-page": "1354", +"DOI": "10.1093/mnras/stx863", +"volume": "469", +"author": "L Dai", +"year": "2017", +"unstructured": "Dai, L. & Fang, K. Can tidal disruption events produce the IceCube neutrinos? Mon. Not. R. Astron. Soc. 469, 1354–1359 (2017).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR7", +"doi-asserted-by": "crossref", +"first-page": "114", +"DOI": "10.3847/1538-4357/ab44ca", +"volume": "886", +"author": "K Hayasaki", +"year": "2019", +"unstructured": "Hayasaki, K. & Yamazaki, R. Neutrino emissions from tidal disruption remnants. Astrophys. J. 886, 114 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR8", +"unstructured": "Farrar, G. R. & Piran, T. Tidal disruption jets as the source of Ultra-High Energy Cosmic Rays. Preprint at https://arxiv.org/abs/1411.0704 (2014)." +}, +{ +"key": "1295_CR9", +"doi-asserted-by": "crossref", +"first-page": "3", +"DOI": "10.3847/1538-4357/aa6344", +"volume": "838", +"author": "N Senno", +"year": "2017", +"unstructured": "Senno, N., Murase, K. & Mészáros, P. High-energy neutrino flares from X-ray bright and dark tidal disruption events. Astrophys. J. 838, 3 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR10", +"doi-asserted-by": "crossref", +"first-page": "083005", +"DOI": "10.1103/PhysRevD.93.083005", +"volume": "93", +"author": "XY Wang", +"year": "2016", +"unstructured": "Wang, X. Y. & Liu, R. Y. Tidal disruption jets of supermassive black holes as hidden sources of cosmic rays: explaining the IceCube TeV–PeV neutrinos. Phys. Rev. D 93, 083005 (2016).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR11", +"doi-asserted-by": "crossref", +"first-page": "123001", +"DOI": "10.1103/PhysRevD.95.123001", +"volume": "95", +"author": "C Lunardini", +"year": "2017", +"unstructured": "Lunardini, C. & Winter, W. High energy neutrinos from the tidal disruption of stars. Phys. Rev. D 95, 123001 (2017).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR12", +"unstructured": "Stein, R., Franckowiak, A., Necker, J., Gezari, S. & Velzen, S. V. Candidate counterparts to IceCube-191001A with ZTF. Astron. Telegr. 13160 (2019)." +}, +{ +"key": "1295_CR13", +"doi-asserted-by": "crossref", +"first-page": "078001", +"DOI": "10.1088/1538-3873/ab006c", +"volume": "131", +"author": "MJ Graham", +"year": "2019", +"unstructured": "Graham, M. J. et al. The Zwicky Transient Facility: science objectives. Publ. Astron. Soc. Pac. 131, 078001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR14", +"unstructured": "Nordin, J. et al. TNS Astronomical Transient Report 33340 (2019)." +}, +{ +"key": "1295_CR15", +"unstructured": "Nicholl, M. et al. ePESSTO+ classification of optical transients. Astron. Telegr. 12752 (2019)." +}, +{ +"key": "1295_CR16", +"unstructured": "van Velzen, S. et al. Seventeen tidal disruption events from the first half of ZTF survey observations: entering a new era of population studies. Preprint at https://arxiv.org/abs/2001.01409 (2020)." +}, +{ +"key": "1295_CR17", +"doi-asserted-by": "crossref", +"first-page": "82", +"DOI": "10.3847/1538-4357/ab1844", +"volume": "878", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. Late-time UV observations of tidal disruption flares reveal unobscured, compact accretion disks. Astrophys. J. 878, 82 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR18", +"doi-asserted-by": "crossref", +"first-page": "5655", +"DOI": "10.1093/mnras/staa192", +"volume": "492", +"author": "A Mummery", +"year": "2020", +"unstructured": "Mummery, A. & Balbus, S. A. The spectral evolution of disc dominated tidal disruption events. Mon. Not. R. Astron. Soc. 492, 5655–5674 (2020).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR19", +"doi-asserted-by": "crossref", +"first-page": "184", +"DOI": "10.1088/0004-637X/764/2/184", +"volume": "764", +"author": "NJ McConnell", +"year": "2013", +"unstructured": "McConnell, N. J. & Ma, C. P. Revisiting the scaling relations of black hole masses and host galaxy properties. Astrophys. J. 764, 184 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR20", +"doi-asserted-by": "crossref", +"first-page": "149", +"DOI": "10.3847/1538-4357/aa633b", +"volume": "838", +"author": "K Auchettl", +"year": "2017", +"unstructured": "Auchettl, K., Guillochon, J. & Ramirez-Ruiz, E. New physical insights about tidal disruption events from a comprehensive observational inventory at X-ray wavelengths. Astrophys. J. 838, 149 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR21", +"doi-asserted-by": "crossref", +"first-page": "4136", +"DOI": "10.1093/mnras/stz1602", +"volume": "487", +"author": "T Wevers", +"year": "2019", +"unstructured": "Wevers, T. et al. Black hole masses of tidal disruption event host galaxies II. Mon. Not. R. Astron. Soc. 487, 4136–4152 (2019).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR22", +"doi-asserted-by": "crossref", +"first-page": "198", +"DOI": "10.3847/1538-4357/aafe0c", +"volume": "872", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. The first tidal disruption flare in ZTF: from photometric selection to multi-wavelength characterization. Astrophys. J. 872, 198 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR23", +"doi-asserted-by": "crossref", +"first-page": "A81", +"DOI": "10.1051/0004-6361/201117855", +"volume": "538", +"author": "G Morlino", +"year": "2012", +"unstructured": "Morlino, G. & Caprioli, D. Strong evidence for hadron acceleration in Tycho’s supernova remnant. Astron. Astrophys. 538, A81 (2012).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR24", +"doi-asserted-by": "crossref", +"first-page": "86", +"DOI": "10.3847/1538-4357/aaa8e0", +"volume": "854", +"author": "T Eftekhari", +"year": "2018", +"unstructured": "Eftekhari, T., Berger, E., Zauderer, B. A., Margutti, R. & Alexander, K. D. Radio monitoring of the tidal disruption event Swift J164449.3+573451. III. Late-time jet energetics and a deviation from equipartition. Astrophys. J. 854, 86 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR25", +"doi-asserted-by": "crossref", +"first-page": "1258", +"DOI": "10.1093/mnras/stt1645", +"volume": "436", +"author": "A Horesh", +"year": "2013", +"unstructured": "Horesh, A. et al. An early and comprehensive millimetre and centimetre wave and X-ray study of SN 2011dh: a non-equipartition blast wave expanding into a massive stellar wind. Mon. Not. R. Astron. Soc. 436, 1258–1267 (2013).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR26", +"doi-asserted-by": "crossref", +"first-page": "78", +"DOI": "10.1088/0004-637X/772/1/78", +"volume": "772", +"author": "R Barniol Duran", +"year": "2013", +"unstructured": "Barniol Duran, R., Nakar, E. & Piran, T. Radius constraints and minimal equipartition energy of relativistically moving synchrotron sources. Astrophys. J. 772, 78 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR27", +"doi-asserted-by": "crossref", +"first-page": "69", +"DOI": "10.1071/AS02053", +"volume": "20", +"author": "AG Polatidis", +"year": "2003", +"unstructured": "Polatidis, A. G. & Conway, J. E. Proper motions in compact symmetric objects. Publ. Astron. Soc. Aust. 20, 69–74 (2003).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR28", +"doi-asserted-by": "crossref", +"first-page": "L25", +"DOI": "10.3847/2041-8205/819/2/L25", +"volume": "819", +"author": "KD Alexander", +"year": "2016", +"unstructured": "Alexander, K. D., Berger, E., Guillochon, J., Zauderer, B. A. & Williams, P. K. G. Discovery of an outflow from radio observations of the tidal disruption event ASASSN-14li. Astrophys. J. Lett. 819, L25 (2016).", +"journal-title": "Astrophys. J. Lett." +}, +{ +"key": "1295_CR29", +"doi-asserted-by": "crossref", +"first-page": "127", +"DOI": "10.3847/0004-637X/827/2/127", +"volume": "827", +"author": "J Krolik", +"year": "2016", +"unstructured": "Krolik, J., Piran, T., Svirski, G. & Cheng, R. M. ASASSN-14li: a model tidal disruption event. Astrophys. J. 827, 127 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR30", +"doi-asserted-by": "crossref", +"first-page": "1", +"DOI": "10.3847/1538-4357/aab361", +"volume": "856", +"author": "DR Pasham", +"year": "2018", +"unstructured": "Pasham, D. R. & van Velzen, S. Discovery of a time lag between the soft X-ray and radio emission of the tidal disruption flare ASASSN-14li: evidence for linear disk–jet coupling. Astrophys. J. 856, 1 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR31", +"doi-asserted-by": "crossref", +"first-page": "L9", +"DOI": "10.1051/0004-6361/201834750", +"volume": "622", +"author": "NL Strotjohann", +"year": "2019", +"unstructured": "Strotjohann, N. L., Kowalski, M. & Franckowiak, A. Eddington bias for cosmic neutrino sources. Astron. Astrophys. 622, L9 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR32", +"doi-asserted-by": "crossref", +"first-page": "425", +"DOI": "10.1146/annurev.aa.22.090184.002233", +"volume": "22", +"author": "AM Hillas", +"year": "1984", +"unstructured": "Hillas, A. M. The origin of ultra-high-energy cosmic rays. Annu. Rev. Astron. Astrophys. 22, 425–444 (1984).", +"journal-title": "Annu. Rev. Astron. Astrophys." +}, +{ +"key": "1295_CR33", +"doi-asserted-by": "crossref", +"first-page": "eaat1378", +"DOI": "10.1126/science.aat1378", +"volume": "361", +"author": "IceCube Collaboration", +"year": "2018", +"unstructured": "IceCube Collaboration et al. Multimessenger observations of a flaring blazar coincident with high-energy neutrino IceCube-170922A. Science 361, eaat1378 (2018).", +"journal-title": "Science" +}, +{ +"key": "1295_CR34", +"unstructured": "Blaufuss, E., Kintscher, T., Lu, L. & Tung, C. F. The next generation of IceCube real-time neutrino alerts. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1021 (PoS, 2019)." +}, +{ +"key": "1295_CR35", +"doi-asserted-by": "crossref", +"first-page": "071101", +"DOI": "10.1103/PhysRevLett.116.071101", +"volume": "116", +"author": "K Murase", +"year": "2016", +"unstructured": "Murase, K., Guetta, D. & Ahlers, M. Hidden cosmic-ray accelerators as an origin of TeV–PeV cosmic neutrinos. Phys. Rev. Lett. 116, 071101 (2016).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR36", +"unstructured": "Stein, R. Search for neutrinos from populations of optical transients. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1016 (PoS, 2019).", +"DOI": "10.22323/1.358.1016", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR37", +"doi-asserted-by": "crossref", +"first-page": "048001", +"DOI": "10.1088/1538-3873/aaff99", +"volume": "131", +"author": "MW Coughlin", +"year": "2019", +"unstructured": "Coughlin, M. W. et al. 2900 square degree search for the optical counterpart of short gamma-ray burst GRB 180523B with the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 048001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR38", +"unstructured": "Stein, R. IceCube-200107A: IceCube observation of a high-energy neutrino candidate event. GCN Circ. 26655 (2020)." +}, +{ +"key": "1295_CR39", +"doi-asserted-by": "crossref", +"first-page": "018003", +"DOI": "10.1088/1538-3873/aae8ac", +"volume": "131", +"author": "FJ Masci", +"year": "2019", +"unstructured": "Masci, F. J. et al. The Zwicky Transient Facility: data processing, products, and archive. Publ. Astron. Soc. Pac. 131, 018003 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR40", +"doi-asserted-by": "crossref", +"first-page": "018001", +"DOI": "10.1088/1538-3873/aae904", +"volume": "131", +"author": "MT Patterson", +"year": "2019", +"unstructured": "Patterson, M. T. et al. The Zwicky Transient Facility Alert Distribution System. Publ. Astron. Soc. Pac. 131, 018001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR41", +"unstructured": "Stein, R. & Reusch, S. robertdstein/ampel_followup_pipeline: V1.1 Release (Zenodo, 2020); https://doi.org/10.5281/zenodo.4048336", +"DOI": "10.5281/zenodo.4048336", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR42", +"doi-asserted-by": "crossref", +"first-page": "A147", +"DOI": "10.1051/0004-6361/201935634", +"volume": "631", +"author": "J Nordin", +"year": "2019", +"unstructured": "Nordin, J. et al. Transient processing and analysis using AMPEL: alert management, photometry, and evaluation of light curves. Astron. Astrophys. 631, A147 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR43", +"doi-asserted-by": "crossref", +"first-page": "038002", +"DOI": "10.1088/1538-3873/aaf3fa", +"volume": "131", +"author": "A Mahabal", +"year": "2019", +"unstructured": "Mahabal, A. et al. Machine learning for the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 038002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR44", +"doi-asserted-by": "crossref", +"first-page": "075002", +"DOI": "10.1088/1538-3873/aac410", +"volume": "130", +"author": "MT Soumagnac", +"year": "2018", +"unstructured": "Soumagnac, M. T. & Ofek, E. O. catsHTM: a tool for fast accessing and cross-matching large astronomical catalogs. Publ. Astron. Soc. Pac. 130, 075002 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR45", +"doi-asserted-by": "crossref", +"first-page": "A1", +"DOI": "10.1051/0004-6361/201833051", +"volume": "616", +"author": "Gaia Collaboration", +"year": "2018", +"unstructured": "Gaia Collaboration et al. Gaia Data Release 2. Summary of the contents and survey properties. Astron. Astrophys. 616, A1 (2018).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR46", +"doi-asserted-by": "crossref", +"first-page": "128001", +"DOI": "10.1088/1538-3873/aae3d9", +"volume": "130", +"author": "Y Tachibana", +"year": "2018", +"unstructured": "Tachibana, Y. & Miller, A. A. A morphological classification model to identify unresolved PanSTARRS1 sources: application in the ZTF real-time pipeline. Publ. Astron. Soc. Pac. 130, 128001 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR47", +"unstructured": "Chambers, K. C. et al. The Pan-STARRS1 Surveys. Preprint at https://arxiv.org/abs/1612.05560 (2016)." +}, +{ +"key": "1295_CR48", +"doi-asserted-by": "crossref", +"first-page": "1868", +"DOI": "10.1088/0004-6256/140/6/1868", +"volume": "140", +"author": "EL Wright", +"year": "2010", +"unstructured": "Wright, E. L. et al. The Wide-field Infrared Survey Explorer (WISE): mission description and initial on-orbit performance. Astron. J. 140, 1868–1881 (2010).", +"journal-title": "Astron. J." +}, +{ +"key": "1295_CR49", +"doi-asserted-by": "crossref", +"first-page": "051103", +"DOI": "10.1103/PhysRevLett.124.051103", +"volume": "124", +"author": "MG Aartsen", +"year": "2020", +"unstructured": "Aartsen, M. G. et al. Time-integrated neutrino source searches with 10 years of IceCube data. Phys. Rev. Lett. 124, 051103 (2020).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR50", +"unstructured": "Steele, I. A. et al. The Liverpool Telescope: performance and first results. Proc. SPIE 5489, https://doi.org/10.1117/12.551456 (2004).", +"DOI": "10.1117/12.551456", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR51", +"doi-asserted-by": "crossref", +"first-page": "035003", +"DOI": "10.1088/1538-3873/aaa53f", +"volume": "130", +"author": "N Blagorodnova", +"year": "2018", +"unstructured": "Blagorodnova, N. et al. The SED Machine: a robotic spectrograph for fast transient classification. Publ. Astron. Soc. Pac. 130, 035003 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR52", +"doi-asserted-by": "crossref", +"first-page": "A115", +"DOI": "10.1051/0004-6361/201935344", +"volume": "627", +"author": "M Rigault", +"year": "2019", +"unstructured": "Rigault, M. et al. Fully automated integral field spectrograph pipeline for the SEDMachine: pysedm. Astron. Astrophys. 627, A115 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR53", +"doi-asserted-by": "crossref", +"first-page": "A68", +"DOI": "10.1051/0004-6361/201628275", +"volume": "593", +"author": "C Fremling", +"year": "2016", +"unstructured": "Fremling, C. et al. PTF12os and iPTF13bvn. Two stripped-envelope supernovae from low-mass progenitors in NGC 5806. Astron. Astrophys. 593, A68 (2016).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR54", +"doi-asserted-by": "crossref", +"first-page": "72", +"DOI": "10.3847/1538-4357/aa998e", +"volume": "852", +"author": "S van Velzen", +"year": "2018", +"unstructured": "van Velzen, S. On the mass and luminosity functions of tidal disruption flares: rate suppression due to black hole event horizons. Astrophys. J. 852, 72 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR55", +"doi-asserted-by": "crossref", +"first-page": "95", +"DOI": "10.1007/s11214-005-5095-4", +"volume": "120", +"author": "PWA Roming", +"year": "2005", +"unstructured": "Roming, P. W. A. et al. The Swift Ultra-Violet/Optical Telescope. Space Sci. Rev. 120, 95–142 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR56", +"doi-asserted-by": "crossref", +"first-page": "1005", +"DOI": "10.1086/422091", +"volume": "611", +"author": "N Gehrels", +"year": "2004", +"unstructured": "Gehrels, N. et al. The Swift Gamma-Ray Burst Mission. Astrophys. J. 611, 1005–1020 (2004).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR57", +"doi-asserted-by": "crossref", +"first-page": "19", +"DOI": "10.3847/0004-637X/829/1/19", +"volume": "829", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S., Mendez, A. J., Krolik, J. H. & Gorjian, V. Discovery of transient infrared emission from dust heated by stellar tidal disruption flares. Astrophys. J. 829, 19 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR58", +"doi-asserted-by": "crossref", +"first-page": "575", +"DOI": "10.1093/mnras/stw307", +"volume": "458", +"author": "W Lu", +"year": "2016", +"unstructured": "Lu, W., Kumar, P. & Evans, N. J. Infrared emission from tidal disruption events—probing the pc-scale dust content around galactic nuclei. Mon. Not. R. Astron. Soc. 458, 575–581 (2016).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR59", +"unstructured": "Miller, J. S. & Stone, R. P. S. The Kast Double Spectrograph. Technical Report No. 66 (Lick Observatory, 1993)." +}, +{ +"key": "1295_CR60", +"doi-asserted-by": "crossref", +"first-page": "375", +"DOI": "10.1086/133562", +"volume": "107", +"author": "JB Oke", +"year": "1995", +"unstructured": "Oke, J. B. et al. The Keck Low-Resolution Imaging Spectrometer. Publ. Astron. Soc. Pac. 107, 375–385 (1995).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR61", +"doi-asserted-by": "crossref", +"first-page": "765", +"DOI": "10.1111/j.1365-2966.2005.08957.x", +"volume": "359", +"author": "A Garcia-Rissmann", +"year": "2005", +"unstructured": "Garcia-Rissmann, A. et al. An atlas of calcium triplet spectra of active galaxies. Mon. Not. R. Astron. Soc. 359, 765–780 (2005).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR62", +"doi-asserted-by": "crossref", +"first-page": "165", +"DOI": "10.1007/s11214-005-5097-2", +"volume": "120", +"author": "DN Burrows", +"year": "2005", +"unstructured": "Burrows, D. N. et al. The Swift X-Ray Telescope. Space Sci. Rev. 120, 165–195 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR63", +"doi-asserted-by": "crossref", +"first-page": "L1", +"DOI": "10.1051/0004-6361:20000036", +"volume": "365", +"author": "F Jansen", +"year": "2001", +"unstructured": "Jansen, F. et al. XMM-Newton Observatory. I. The spacecraft and operations. Astron. Astrophys. 365, L1–L6 (2001).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR64", +"unstructured": "HI4PI Collaboration et al. HI4PI: a full-sky H i survey based on EBHIS and GASS. Astron. Astrophys. 594, A116 (2016).", +"DOI": "10.1051/0004-6361/201629178", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR65", +"unstructured": "Arnaud, K. A. in Astronomical Data Analysis Software and Systems V (eds Jacoby, G. H. & Barnes, J.) 17 (Astronomical Society of the Pacific, 1996)." +}, +{ +"key": "1295_CR66", +"doi-asserted-by": "crossref", +"first-page": "1545", +"DOI": "10.1111/j.1365-2966.2008.13953.x", +"volume": "391", +"author": "JTL Zwart", +"year": "2008", +"unstructured": "Zwart, J. T. L. et al. The Arcminute Microkelvin Imager. Mon. Not. R. Astron. Soc. 391, 1545–1558 (2008).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR67", +"doi-asserted-by": "crossref", +"first-page": "5677", +"DOI": "10.1093/mnras/sty074", +"volume": "475", +"author": "J Hickish", +"year": "2018", +"unstructured": "Hickish, J. et al. A digital correlator upgrade for the Arcminute MicroKelvin Imager. Mon. Not. R. Astron. Soc. 475, 5677–5687 (2018).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR68", +"doi-asserted-by": "crossref", +"first-page": "1396", +"DOI": "10.1093/mnras/stv1728", +"volume": "453", +"author": "YC Perrott", +"year": "2015", +"unstructured": "Perrott, Y. C. et al. AMI galactic plane survey at 16 GHz—II. Full data release with extended coverage and improved processing. Mon. Not. R. Astron. Soc. 453, 1396–1403 (2015).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR69", +"unstructured": "McMullin, J. P., Waters, B., Schiebel, D., Young, W. & Golap, K. in Astronomical Data Analysis Software and Systems XVI (eds Shaw, R. A. et al.) 127 (Astronomical Society of the Pacific, 2007)." +}, +{ +"key": "1295_CR70", +"doi-asserted-by": "crossref", +"first-page": "1071", +"DOI": "10.1088/0004-637X/697/2/1071", +"volume": "697", +"author": "WB Atwood", +"year": "2009", +"unstructured": "Atwood, W. B. et al. The Large Area Telescope on the Fermi Gamma-ray Space Telescope mission. Astrophys. J. 697, 1071–1102 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR71", +"unstructured": "Wood, M. et al. Fermipy: an open-source Python package for analysis of Fermi-LAT Data. In Proc. 35th International Cosmic Ray Conference (ICRC2017) 824 (PoS, 2017).", +"DOI": "10.22323/1.301.0824", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR72", +"unstructured": "Garrappa, S. & Buson, S. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR73", +"unstructured": "The Fermi-LAT collaboration. Fermi Large Area Telescope Fourth Source Catalog. Astrophys. J. Suppl. Ser. 247, 33 (2020)." +}, +{ +"key": "1295_CR74", +"doi-asserted-by": "crossref", +"first-page": "14", +"DOI": "10.1088/0004-637X/767/1/14", +"volume": "767", +"author": "T Pursimo", +"year": "2013", +"unstructured": "Pursimo, T. et al. The Micro-Arcsecond Scintillation-Induced Variability (MASIV) survey. III. Optical identifications and new redshifts. Astrophys. J. 767, 14 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR75", +"unstructured": "Garrappa, S., Buson, S. & Fermi-LAT Collaboration. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR76", +"doi-asserted-by": "crossref", +"first-page": "133", +"DOI": "10.1088/0004-637X/802/2/133", +"volume": "802", +"author": "C Diltz", +"year": "2015", +"unstructured": "Diltz, C., Böttcher, M. & Fossati, G. Time dependent hadronic modeling of flat spectrum radio quasars. Astrophys. J. 802, 133 (2015).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR77", +"doi-asserted-by": "crossref", +"first-page": "88", +"DOI": "10.1038/s41550-018-0610-1", +"volume": "3", +"author": "S Gao", +"year": "2019", +"unstructured": "Gao, S., Fedynitch, A., Winter, W. & Pohl, M. Modelling the coincident observation of a high-energy neutrino and a bright blazar flare. Nat. Astron. 3, 88–92 (2019).", +"journal-title": "Nat. Astron." +}, +{ +"key": "1295_CR78", +"unstructured": "Ayala, H. IceCube-191001A: HAWC follow-up. GCN Circ. 25936 (2019)." +}, +{ +"key": "1295_CR79", +"doi-asserted-by": "crossref", +"first-page": "62", +"DOI": "10.1126/science.aad1182", +"volume": "351", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S. et al. A radio jet from the optical and x-ray bright stellar tidal disruption flare ASASSN-14li. Science 351, 62–65 (2016).", +"journal-title": "Science" +}, +{ +"key": "1295_CR80", +"doi-asserted-by": "crossref", +"first-page": "306", +"DOI": "10.1086/670067", +"volume": "125", +"author": "D Foreman-Mackey", +"year": "2013", +"unstructured": "Foreman-Mackey, D., Hogg, D. W., Lang, D. & Goodman, J. emcee: the MCMC Hammer. Publ. Astron. Soc. Pac. 125, 306 (2013).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR81", +"doi-asserted-by": "crossref", +"first-page": "6", +"DOI": "10.3847/1538-4365/aab761", +"volume": "236", +"author": "J Guillochon", +"year": "2018", +"unstructured": "Guillochon, J. et al. MOSFiT: Modular Open Source Fitter for Transients. Astrophys. J. Suppl. Ser. 236, 6 (2018).", +"journal-title": "Astrophys. J. Suppl. Ser." +}, +{ +"key": "1295_CR82", +"doi-asserted-by": "crossref", +"first-page": "e008", +"DOI": "10.1017/pasa.2013.44", +"volume": "31", +"author": "J Granot", +"year": "2014", +"unstructured": "Granot, J. & van der Horst, A. J. Gamma-ray burst jets and their radio observations. Publ. Astron. Soc. Aust. 31, e008 (2014).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR83", +"doi-asserted-by": "crossref", +"first-page": "102", +"DOI": "10.1088/0004-637X/815/2/102", +"volume": "815", +"author": "W Fong", +"year": "2015", +"unstructured": "Fong, W., Berger, E., Margutti, R. & Zauderer, B. A. A decade of short-duration gamma-ray burst broadband afterglows: energetics, circumburst densities, and jet opening angles. Astrophys. J. 815, 102 (2015).", +"journal-title": "Astrophys. J." +} +], +"container-title": [ +"Nature Astronomy" +], +"original-title": [ + +], +"language": "en", +"link": [ +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8", +"content-type": "text/html", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "similarity-checking" +} +], +"deposited": { +"date-parts": [ +[ +2021, +5, +17 +] +], +"date-time": "2021-05-17T15:08:12Z", +"timestamp": 1621264092000 +}, +"score": 1.0, +"subtitle": [ + +], +"short-title": [ + +], +"issued": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"references-count": 83, +"journal-issue": { +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"issue": "5" +}, +"alternative-id": [ +"1295" +], +"URL": "http://dx.doi.org/10.1038/s41550-020-01295-8", +"relation": { +"cites": [ + +] +}, +"ISSN": [ +"2397-3366" +], +"issn-type": [ +{ +"value": "2397-3366", +"type": "electronic" +} +], +"assertion": [ +{ +"value": "21 July 2020", +"order": 1, +"name": "received", +"label": "Received", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "16 December 2020", +"order": 2, +"name": "accepted", +"label": "Accepted", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "22 February 2021", +"order": 3, +"name": "first_online", +"label": "First Online", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "The authors declare no competing interests.", +"order": 1, +"name": "Ethics", +"group": { +"name": "EthicsHeading", +"label": "Competing interests" +} +} +] +} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_vor.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_vor.json new file mode 100644 index 000000000..f2e91a23f --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/crossref/publication_license_vor.json @@ -0,0 +1,1537 @@ +{ +"indexed": { +"date-parts": [ +[ +2021, +7, +2 +] +], +"date-time": "2021-07-02T07:30:10Z", +"timestamp": 1625211010708 +}, +"reference-count": 83, +"publisher": "Springer Science and Business Media LLC", +"issue": "5", +"license": [ +{ +"URL": "https://www.springer.com/tdm", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "tdm" +}, +{ +"URL": "https://www.springer.com/vor", +"start": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T00:00:00Z", +"timestamp": 1613952000000 +}, +"delay-in-days": 0, +"content-version": "vor" +} +], +"content-domain": { +"domain": [ +"link.springer.com" +], +"crossmark-restriction": false +}, +"short-container-title": [ +"Nat Astron" +], +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"DOI": "10.1038/s41550-020-01295-8", +"type": "journal-article", +"created": { +"date-parts": [ +[ +2021, +2, +22 +] +], +"date-time": "2021-02-22T17:03:42Z", +"timestamp": 1614013422000 +}, +"page": "510-518", +"update-policy": "http://dx.doi.org/10.1007/springer_crossmark_policy", +"source": "Crossref", +"is-referenced-by-count": 6, +"title": [ +"A tidal disruption event coincident with a high-energy neutrino" +], +"prefix": "10.1038", +"volume": "5", +"author": [ +{ +"ORCID": "http://orcid.org/0000-0003-2434-0387", +"authenticated-orcid": false, +"given": "Robert", +"family": "Stein", +"sequence": "first", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3859-8074", +"authenticated-orcid": false, +"given": "Sjoert van", +"family": "Velzen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8594-8666", +"authenticated-orcid": false, +"given": "Marek", +"family": "Kowalski", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Anna", +"family": "Franckowiak", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3703-5154", +"authenticated-orcid": false, +"given": "Suvi", +"family": "Gezari", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3124-2814", +"authenticated-orcid": false, +"given": "James C. A.", +"family": "Miller-Jones", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Sara", +"family": "Frederick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0466-3779", +"authenticated-orcid": false, +"given": "Itai", +"family": "Sfaradi", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael F.", +"family": "Bietenholz", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5936-1156", +"authenticated-orcid": false, +"given": "Assaf", +"family": "Horesh", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rob", +"family": "Fender", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2403-4582", +"authenticated-orcid": false, +"given": "Simone", +"family": "Garrappa", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-2184-6430", +"authenticated-orcid": false, +"given": "Tomás", +"family": "Ahumada", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Igor", +"family": "Andreoni", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Justin", +"family": "Belicki", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8018-5348", +"authenticated-orcid": false, +"given": "Eric C.", +"family": "Bellm", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Markus", +"family": "Böttcher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Valery", +"family": "Brinnel", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Rick", +"family": "Burruss", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1673-970X", +"authenticated-orcid": false, +"given": "S. Bradley", +"family": "Cenko", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8262-2924", +"authenticated-orcid": false, +"given": "Michael W.", +"family": "Coughlin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2292-0441", +"authenticated-orcid": false, +"given": "Virginia", +"family": "Cunningham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Andrew", +"family": "Drake", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Glennys R.", +"family": "Farrar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Michael", +"family": "Feeney", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Ryan J.", +"family": "Foley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3653-5598", +"authenticated-orcid": false, +"given": "Avishay", +"family": "Gal-Yam", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "V. Zach", +"family": "Golkhou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-4163-4996", +"authenticated-orcid": false, +"given": "Ariel", +"family": "Goobar", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-3168-0139", +"authenticated-orcid": false, +"given": "Matthew J.", +"family": "Graham", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Erica", +"family": "Hammerstein", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-3367-3415", +"authenticated-orcid": false, +"given": "George", +"family": "Helou", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-9878-7889", +"authenticated-orcid": false, +"given": "Tiara", +"family": "Hung", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Mansi M.", +"family": "Kasliwal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5740-7747", +"authenticated-orcid": false, +"given": "Charles D.", +"family": "Kilpatrick", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-5105-344X", +"authenticated-orcid": false, +"given": "Albert K. H.", +"family": "Kong", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-6540-1484", +"authenticated-orcid": false, +"given": "Thomas", +"family": "Kupfer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2451-5482", +"authenticated-orcid": false, +"given": "Russ R.", +"family": "Laher", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-2242-0244", +"authenticated-orcid": false, +"given": "Ashish A.", +"family": "Mahabal", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8532-9395", +"authenticated-orcid": false, +"given": "Frank J.", +"family": "Masci", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-0280-7484", +"authenticated-orcid": false, +"given": "Jannis", +"family": "Necker", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-8342-6274", +"authenticated-orcid": false, +"given": "Jakob", +"family": "Nordin", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel A.", +"family": "Perley", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-8121-2560", +"authenticated-orcid": false, +"given": "Mickael", +"family": "Rigault", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7788-628X", +"authenticated-orcid": false, +"given": "Simeon", +"family": "Reusch", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Hector", +"family": "Rodriguez", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0002-7559-315X", +"authenticated-orcid": false, +"given": "César", +"family": "Rojas-Bravo", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-7648-4142", +"authenticated-orcid": false, +"given": "Ben", +"family": "Rusholme", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-4401-0430", +"authenticated-orcid": false, +"given": "David L.", +"family": "Shupe", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-9898-5597", +"authenticated-orcid": false, +"given": "Leo P.", +"family": "Singer", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0003-1546-6615", +"authenticated-orcid": false, +"given": "Jesper", +"family": "Sollerman", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Maayane T.", +"family": "Soumagnac", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Daniel", +"family": "Stern", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Kirsty", +"family": "Taggart", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Jakob", +"family": "van Santen", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Charlotte", +"family": "Ward", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"given": "Patrick", +"family": "Woudt", +"sequence": "additional", +"affiliation": [ + +] +}, +{ +"ORCID": "http://orcid.org/0000-0001-6747-8509", +"authenticated-orcid": false, +"given": "Yuhan", +"family": "Yao", +"sequence": "additional", +"affiliation": [ + +] +} +], +"member": "297", +"published-online": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"reference": [ +{ +"key": "1295_CR1", +"doi-asserted-by": "crossref", +"first-page": "P03012", +"DOI": "10.1088/1748-0221/12/03/P03012", +"volume": "12", +"author": "MG Aartsen", +"year": "2017", +"unstructured": "Aartsen, M. G. et al. The IceCube Neutrino Observatory: instrumentation and online systems. J. Instrum. 12, P03012 (2017).", +"journal-title": "J. Instrum." +}, +{ +"key": "1295_CR2", +"unstructured": "Stein, R. IceCube-191001A—IceCube observation of a high-energy neutrino candidate event. GCN Circ. 25913 (2019)." +}, +{ +"key": "1295_CR3", +"doi-asserted-by": "crossref", +"first-page": "018002", +"DOI": "10.1088/1538-3873/aaecbe", +"volume": "131", +"author": "EC Bellm", +"year": "2019", +"unstructured": "Bellm, E. C. et al. The Zwicky Transient Facility: system overview, performance, and first results. Publ. Astron. Soc. Pac. 131, 018002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR4", +"doi-asserted-by": "crossref", +"first-page": "533", +"DOI": "10.1016/j.astropartphys.2007.03.005", +"volume": "27", +"author": "M Kowalski", +"year": "2007", +"unstructured": "Kowalski, M. & Mohr, A. Detecting neutrino transients with optical follow-up observations. Astropart. Phys. 27, 533–538 (2007).", +"journal-title": "Astropart. Phys." +}, +{ +"key": "1295_CR5", +"doi-asserted-by": "crossref", +"first-page": "329", +"DOI": "10.1088/0004-637X/693/1/329", +"volume": "693", +"author": "GR Farrar", +"year": "2009", +"unstructured": "Farrar, G. R. & Gruzinov, A. Giant AGN flares and cosmic ray bursts. Astrophys. J. 693, 329–332 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR6", +"doi-asserted-by": "crossref", +"first-page": "1354", +"DOI": "10.1093/mnras/stx863", +"volume": "469", +"author": "L Dai", +"year": "2017", +"unstructured": "Dai, L. & Fang, K. Can tidal disruption events produce the IceCube neutrinos? Mon. Not. R. Astron. Soc. 469, 1354–1359 (2017).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR7", +"doi-asserted-by": "crossref", +"first-page": "114", +"DOI": "10.3847/1538-4357/ab44ca", +"volume": "886", +"author": "K Hayasaki", +"year": "2019", +"unstructured": "Hayasaki, K. & Yamazaki, R. Neutrino emissions from tidal disruption remnants. Astrophys. J. 886, 114 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR8", +"unstructured": "Farrar, G. R. & Piran, T. Tidal disruption jets as the source of Ultra-High Energy Cosmic Rays. Preprint at https://arxiv.org/abs/1411.0704 (2014)." +}, +{ +"key": "1295_CR9", +"doi-asserted-by": "crossref", +"first-page": "3", +"DOI": "10.3847/1538-4357/aa6344", +"volume": "838", +"author": "N Senno", +"year": "2017", +"unstructured": "Senno, N., Murase, K. & Mészáros, P. High-energy neutrino flares from X-ray bright and dark tidal disruption events. Astrophys. J. 838, 3 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR10", +"doi-asserted-by": "crossref", +"first-page": "083005", +"DOI": "10.1103/PhysRevD.93.083005", +"volume": "93", +"author": "XY Wang", +"year": "2016", +"unstructured": "Wang, X. Y. & Liu, R. Y. Tidal disruption jets of supermassive black holes as hidden sources of cosmic rays: explaining the IceCube TeV–PeV neutrinos. Phys. Rev. D 93, 083005 (2016).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR11", +"doi-asserted-by": "crossref", +"first-page": "123001", +"DOI": "10.1103/PhysRevD.95.123001", +"volume": "95", +"author": "C Lunardini", +"year": "2017", +"unstructured": "Lunardini, C. & Winter, W. High energy neutrinos from the tidal disruption of stars. Phys. Rev. D 95, 123001 (2017).", +"journal-title": "Phys. Rev. D" +}, +{ +"key": "1295_CR12", +"unstructured": "Stein, R., Franckowiak, A., Necker, J., Gezari, S. & Velzen, S. V. Candidate counterparts to IceCube-191001A with ZTF. Astron. Telegr. 13160 (2019)." +}, +{ +"key": "1295_CR13", +"doi-asserted-by": "crossref", +"first-page": "078001", +"DOI": "10.1088/1538-3873/ab006c", +"volume": "131", +"author": "MJ Graham", +"year": "2019", +"unstructured": "Graham, M. J. et al. The Zwicky Transient Facility: science objectives. Publ. Astron. Soc. Pac. 131, 078001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR14", +"unstructured": "Nordin, J. et al. TNS Astronomical Transient Report 33340 (2019)." +}, +{ +"key": "1295_CR15", +"unstructured": "Nicholl, M. et al. ePESSTO+ classification of optical transients. Astron. Telegr. 12752 (2019)." +}, +{ +"key": "1295_CR16", +"unstructured": "van Velzen, S. et al. Seventeen tidal disruption events from the first half of ZTF survey observations: entering a new era of population studies. Preprint at https://arxiv.org/abs/2001.01409 (2020)." +}, +{ +"key": "1295_CR17", +"doi-asserted-by": "crossref", +"first-page": "82", +"DOI": "10.3847/1538-4357/ab1844", +"volume": "878", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. Late-time UV observations of tidal disruption flares reveal unobscured, compact accretion disks. Astrophys. J. 878, 82 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR18", +"doi-asserted-by": "crossref", +"first-page": "5655", +"DOI": "10.1093/mnras/staa192", +"volume": "492", +"author": "A Mummery", +"year": "2020", +"unstructured": "Mummery, A. & Balbus, S. A. The spectral evolution of disc dominated tidal disruption events. Mon. Not. R. Astron. Soc. 492, 5655–5674 (2020).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR19", +"doi-asserted-by": "crossref", +"first-page": "184", +"DOI": "10.1088/0004-637X/764/2/184", +"volume": "764", +"author": "NJ McConnell", +"year": "2013", +"unstructured": "McConnell, N. J. & Ma, C. P. Revisiting the scaling relations of black hole masses and host galaxy properties. Astrophys. J. 764, 184 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR20", +"doi-asserted-by": "crossref", +"first-page": "149", +"DOI": "10.3847/1538-4357/aa633b", +"volume": "838", +"author": "K Auchettl", +"year": "2017", +"unstructured": "Auchettl, K., Guillochon, J. & Ramirez-Ruiz, E. New physical insights about tidal disruption events from a comprehensive observational inventory at X-ray wavelengths. Astrophys. J. 838, 149 (2017).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR21", +"doi-asserted-by": "crossref", +"first-page": "4136", +"DOI": "10.1093/mnras/stz1602", +"volume": "487", +"author": "T Wevers", +"year": "2019", +"unstructured": "Wevers, T. et al. Black hole masses of tidal disruption event host galaxies II. Mon. Not. R. Astron. Soc. 487, 4136–4152 (2019).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR22", +"doi-asserted-by": "crossref", +"first-page": "198", +"DOI": "10.3847/1538-4357/aafe0c", +"volume": "872", +"author": "S van Velzen", +"year": "2019", +"unstructured": "van Velzen, S. et al. The first tidal disruption flare in ZTF: from photometric selection to multi-wavelength characterization. Astrophys. J. 872, 198 (2019).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR23", +"doi-asserted-by": "crossref", +"first-page": "A81", +"DOI": "10.1051/0004-6361/201117855", +"volume": "538", +"author": "G Morlino", +"year": "2012", +"unstructured": "Morlino, G. & Caprioli, D. Strong evidence for hadron acceleration in Tycho’s supernova remnant. Astron. Astrophys. 538, A81 (2012).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR24", +"doi-asserted-by": "crossref", +"first-page": "86", +"DOI": "10.3847/1538-4357/aaa8e0", +"volume": "854", +"author": "T Eftekhari", +"year": "2018", +"unstructured": "Eftekhari, T., Berger, E., Zauderer, B. A., Margutti, R. & Alexander, K. D. Radio monitoring of the tidal disruption event Swift J164449.3+573451. III. Late-time jet energetics and a deviation from equipartition. Astrophys. J. 854, 86 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR25", +"doi-asserted-by": "crossref", +"first-page": "1258", +"DOI": "10.1093/mnras/stt1645", +"volume": "436", +"author": "A Horesh", +"year": "2013", +"unstructured": "Horesh, A. et al. An early and comprehensive millimetre and centimetre wave and X-ray study of SN 2011dh: a non-equipartition blast wave expanding into a massive stellar wind. Mon. Not. R. Astron. Soc. 436, 1258–1267 (2013).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR26", +"doi-asserted-by": "crossref", +"first-page": "78", +"DOI": "10.1088/0004-637X/772/1/78", +"volume": "772", +"author": "R Barniol Duran", +"year": "2013", +"unstructured": "Barniol Duran, R., Nakar, E. & Piran, T. Radius constraints and minimal equipartition energy of relativistically moving synchrotron sources. Astrophys. J. 772, 78 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR27", +"doi-asserted-by": "crossref", +"first-page": "69", +"DOI": "10.1071/AS02053", +"volume": "20", +"author": "AG Polatidis", +"year": "2003", +"unstructured": "Polatidis, A. G. & Conway, J. E. Proper motions in compact symmetric objects. Publ. Astron. Soc. Aust. 20, 69–74 (2003).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR28", +"doi-asserted-by": "crossref", +"first-page": "L25", +"DOI": "10.3847/2041-8205/819/2/L25", +"volume": "819", +"author": "KD Alexander", +"year": "2016", +"unstructured": "Alexander, K. D., Berger, E., Guillochon, J., Zauderer, B. A. & Williams, P. K. G. Discovery of an outflow from radio observations of the tidal disruption event ASASSN-14li. Astrophys. J. Lett. 819, L25 (2016).", +"journal-title": "Astrophys. J. Lett." +}, +{ +"key": "1295_CR29", +"doi-asserted-by": "crossref", +"first-page": "127", +"DOI": "10.3847/0004-637X/827/2/127", +"volume": "827", +"author": "J Krolik", +"year": "2016", +"unstructured": "Krolik, J., Piran, T., Svirski, G. & Cheng, R. M. ASASSN-14li: a model tidal disruption event. Astrophys. J. 827, 127 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR30", +"doi-asserted-by": "crossref", +"first-page": "1", +"DOI": "10.3847/1538-4357/aab361", +"volume": "856", +"author": "DR Pasham", +"year": "2018", +"unstructured": "Pasham, D. R. & van Velzen, S. Discovery of a time lag between the soft X-ray and radio emission of the tidal disruption flare ASASSN-14li: evidence for linear disk–jet coupling. Astrophys. J. 856, 1 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR31", +"doi-asserted-by": "crossref", +"first-page": "L9", +"DOI": "10.1051/0004-6361/201834750", +"volume": "622", +"author": "NL Strotjohann", +"year": "2019", +"unstructured": "Strotjohann, N. L., Kowalski, M. & Franckowiak, A. Eddington bias for cosmic neutrino sources. Astron. Astrophys. 622, L9 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR32", +"doi-asserted-by": "crossref", +"first-page": "425", +"DOI": "10.1146/annurev.aa.22.090184.002233", +"volume": "22", +"author": "AM Hillas", +"year": "1984", +"unstructured": "Hillas, A. M. The origin of ultra-high-energy cosmic rays. Annu. Rev. Astron. Astrophys. 22, 425–444 (1984).", +"journal-title": "Annu. Rev. Astron. Astrophys." +}, +{ +"key": "1295_CR33", +"doi-asserted-by": "crossref", +"first-page": "eaat1378", +"DOI": "10.1126/science.aat1378", +"volume": "361", +"author": "IceCube Collaboration", +"year": "2018", +"unstructured": "IceCube Collaboration et al. Multimessenger observations of a flaring blazar coincident with high-energy neutrino IceCube-170922A. Science 361, eaat1378 (2018).", +"journal-title": "Science" +}, +{ +"key": "1295_CR34", +"unstructured": "Blaufuss, E., Kintscher, T., Lu, L. & Tung, C. F. The next generation of IceCube real-time neutrino alerts. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1021 (PoS, 2019)." +}, +{ +"key": "1295_CR35", +"doi-asserted-by": "crossref", +"first-page": "071101", +"DOI": "10.1103/PhysRevLett.116.071101", +"volume": "116", +"author": "K Murase", +"year": "2016", +"unstructured": "Murase, K., Guetta, D. & Ahlers, M. Hidden cosmic-ray accelerators as an origin of TeV–PeV cosmic neutrinos. Phys. Rev. Lett. 116, 071101 (2016).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR36", +"unstructured": "Stein, R. Search for neutrinos from populations of optical transients. In Proc. 36th International Cosmic Ray Conference (ICRC2019) 1016 (PoS, 2019).", +"DOI": "10.22323/1.358.1016", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR37", +"doi-asserted-by": "crossref", +"first-page": "048001", +"DOI": "10.1088/1538-3873/aaff99", +"volume": "131", +"author": "MW Coughlin", +"year": "2019", +"unstructured": "Coughlin, M. W. et al. 2900 square degree search for the optical counterpart of short gamma-ray burst GRB 180523B with the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 048001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR38", +"unstructured": "Stein, R. IceCube-200107A: IceCube observation of a high-energy neutrino candidate event. GCN Circ. 26655 (2020)." +}, +{ +"key": "1295_CR39", +"doi-asserted-by": "crossref", +"first-page": "018003", +"DOI": "10.1088/1538-3873/aae8ac", +"volume": "131", +"author": "FJ Masci", +"year": "2019", +"unstructured": "Masci, F. J. et al. The Zwicky Transient Facility: data processing, products, and archive. Publ. Astron. Soc. Pac. 131, 018003 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR40", +"doi-asserted-by": "crossref", +"first-page": "018001", +"DOI": "10.1088/1538-3873/aae904", +"volume": "131", +"author": "MT Patterson", +"year": "2019", +"unstructured": "Patterson, M. T. et al. The Zwicky Transient Facility Alert Distribution System. Publ. Astron. Soc. Pac. 131, 018001 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR41", +"unstructured": "Stein, R. & Reusch, S. robertdstein/ampel_followup_pipeline: V1.1 Release (Zenodo, 2020); https://doi.org/10.5281/zenodo.4048336", +"DOI": "10.5281/zenodo.4048336", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR42", +"doi-asserted-by": "crossref", +"first-page": "A147", +"DOI": "10.1051/0004-6361/201935634", +"volume": "631", +"author": "J Nordin", +"year": "2019", +"unstructured": "Nordin, J. et al. Transient processing and analysis using AMPEL: alert management, photometry, and evaluation of light curves. Astron. Astrophys. 631, A147 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR43", +"doi-asserted-by": "crossref", +"first-page": "038002", +"DOI": "10.1088/1538-3873/aaf3fa", +"volume": "131", +"author": "A Mahabal", +"year": "2019", +"unstructured": "Mahabal, A. et al. Machine learning for the Zwicky Transient Facility. Publ. Astron. Soc. Pac. 131, 038002 (2019).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR44", +"doi-asserted-by": "crossref", +"first-page": "075002", +"DOI": "10.1088/1538-3873/aac410", +"volume": "130", +"author": "MT Soumagnac", +"year": "2018", +"unstructured": "Soumagnac, M. T. & Ofek, E. O. catsHTM: a tool for fast accessing and cross-matching large astronomical catalogs. Publ. Astron. Soc. Pac. 130, 075002 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR45", +"doi-asserted-by": "crossref", +"first-page": "A1", +"DOI": "10.1051/0004-6361/201833051", +"volume": "616", +"author": "Gaia Collaboration", +"year": "2018", +"unstructured": "Gaia Collaboration et al. Gaia Data Release 2. Summary of the contents and survey properties. Astron. Astrophys. 616, A1 (2018).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR46", +"doi-asserted-by": "crossref", +"first-page": "128001", +"DOI": "10.1088/1538-3873/aae3d9", +"volume": "130", +"author": "Y Tachibana", +"year": "2018", +"unstructured": "Tachibana, Y. & Miller, A. A. A morphological classification model to identify unresolved PanSTARRS1 sources: application in the ZTF real-time pipeline. Publ. Astron. Soc. Pac. 130, 128001 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR47", +"unstructured": "Chambers, K. C. et al. The Pan-STARRS1 Surveys. Preprint at https://arxiv.org/abs/1612.05560 (2016)." +}, +{ +"key": "1295_CR48", +"doi-asserted-by": "crossref", +"first-page": "1868", +"DOI": "10.1088/0004-6256/140/6/1868", +"volume": "140", +"author": "EL Wright", +"year": "2010", +"unstructured": "Wright, E. L. et al. The Wide-field Infrared Survey Explorer (WISE): mission description and initial on-orbit performance. Astron. J. 140, 1868–1881 (2010).", +"journal-title": "Astron. J." +}, +{ +"key": "1295_CR49", +"doi-asserted-by": "crossref", +"first-page": "051103", +"DOI": "10.1103/PhysRevLett.124.051103", +"volume": "124", +"author": "MG Aartsen", +"year": "2020", +"unstructured": "Aartsen, M. G. et al. Time-integrated neutrino source searches with 10 years of IceCube data. Phys. Rev. Lett. 124, 051103 (2020).", +"journal-title": "Phys. Rev. Lett." +}, +{ +"key": "1295_CR50", +"unstructured": "Steele, I. A. et al. The Liverpool Telescope: performance and first results. Proc. SPIE 5489, https://doi.org/10.1117/12.551456 (2004).", +"DOI": "10.1117/12.551456", +"doi-asserted-by": "publisher" +}, +{ +"key": "1295_CR51", +"doi-asserted-by": "crossref", +"first-page": "035003", +"DOI": "10.1088/1538-3873/aaa53f", +"volume": "130", +"author": "N Blagorodnova", +"year": "2018", +"unstructured": "Blagorodnova, N. et al. The SED Machine: a robotic spectrograph for fast transient classification. Publ. Astron. Soc. Pac. 130, 035003 (2018).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR52", +"doi-asserted-by": "crossref", +"first-page": "A115", +"DOI": "10.1051/0004-6361/201935344", +"volume": "627", +"author": "M Rigault", +"year": "2019", +"unstructured": "Rigault, M. et al. Fully automated integral field spectrograph pipeline for the SEDMachine: pysedm. Astron. Astrophys. 627, A115 (2019).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR53", +"doi-asserted-by": "crossref", +"first-page": "A68", +"DOI": "10.1051/0004-6361/201628275", +"volume": "593", +"author": "C Fremling", +"year": "2016", +"unstructured": "Fremling, C. et al. PTF12os and iPTF13bvn. Two stripped-envelope supernovae from low-mass progenitors in NGC 5806. Astron. Astrophys. 593, A68 (2016).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR54", +"doi-asserted-by": "crossref", +"first-page": "72", +"DOI": "10.3847/1538-4357/aa998e", +"volume": "852", +"author": "S van Velzen", +"year": "2018", +"unstructured": "van Velzen, S. On the mass and luminosity functions of tidal disruption flares: rate suppression due to black hole event horizons. Astrophys. J. 852, 72 (2018).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR55", +"doi-asserted-by": "crossref", +"first-page": "95", +"DOI": "10.1007/s11214-005-5095-4", +"volume": "120", +"author": "PWA Roming", +"year": "2005", +"unstructured": "Roming, P. W. A. et al. The Swift Ultra-Violet/Optical Telescope. Space Sci. Rev. 120, 95–142 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR56", +"doi-asserted-by": "crossref", +"first-page": "1005", +"DOI": "10.1086/422091", +"volume": "611", +"author": "N Gehrels", +"year": "2004", +"unstructured": "Gehrels, N. et al. The Swift Gamma-Ray Burst Mission. Astrophys. J. 611, 1005–1020 (2004).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR57", +"doi-asserted-by": "crossref", +"first-page": "19", +"DOI": "10.3847/0004-637X/829/1/19", +"volume": "829", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S., Mendez, A. J., Krolik, J. H. & Gorjian, V. Discovery of transient infrared emission from dust heated by stellar tidal disruption flares. Astrophys. J. 829, 19 (2016).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR58", +"doi-asserted-by": "crossref", +"first-page": "575", +"DOI": "10.1093/mnras/stw307", +"volume": "458", +"author": "W Lu", +"year": "2016", +"unstructured": "Lu, W., Kumar, P. & Evans, N. J. Infrared emission from tidal disruption events—probing the pc-scale dust content around galactic nuclei. Mon. Not. R. Astron. Soc. 458, 575–581 (2016).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR59", +"unstructured": "Miller, J. S. & Stone, R. P. S. The Kast Double Spectrograph. Technical Report No. 66 (Lick Observatory, 1993)." +}, +{ +"key": "1295_CR60", +"doi-asserted-by": "crossref", +"first-page": "375", +"DOI": "10.1086/133562", +"volume": "107", +"author": "JB Oke", +"year": "1995", +"unstructured": "Oke, J. B. et al. The Keck Low-Resolution Imaging Spectrometer. Publ. Astron. Soc. Pac. 107, 375–385 (1995).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR61", +"doi-asserted-by": "crossref", +"first-page": "765", +"DOI": "10.1111/j.1365-2966.2005.08957.x", +"volume": "359", +"author": "A Garcia-Rissmann", +"year": "2005", +"unstructured": "Garcia-Rissmann, A. et al. An atlas of calcium triplet spectra of active galaxies. Mon. Not. R. Astron. Soc. 359, 765–780 (2005).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR62", +"doi-asserted-by": "crossref", +"first-page": "165", +"DOI": "10.1007/s11214-005-5097-2", +"volume": "120", +"author": "DN Burrows", +"year": "2005", +"unstructured": "Burrows, D. N. et al. The Swift X-Ray Telescope. Space Sci. Rev. 120, 165–195 (2005).", +"journal-title": "Space Sci. Rev." +}, +{ +"key": "1295_CR63", +"doi-asserted-by": "crossref", +"first-page": "L1", +"DOI": "10.1051/0004-6361:20000036", +"volume": "365", +"author": "F Jansen", +"year": "2001", +"unstructured": "Jansen, F. et al. XMM-Newton Observatory. I. The spacecraft and operations. Astron. Astrophys. 365, L1–L6 (2001).", +"journal-title": "Astron. Astrophys." +}, +{ +"key": "1295_CR64", +"unstructured": "HI4PI Collaboration et al. HI4PI: a full-sky H i survey based on EBHIS and GASS. Astron. Astrophys. 594, A116 (2016).", +"DOI": "10.1051/0004-6361/201629178", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR65", +"unstructured": "Arnaud, K. A. in Astronomical Data Analysis Software and Systems V (eds Jacoby, G. H. & Barnes, J.) 17 (Astronomical Society of the Pacific, 1996)." +}, +{ +"key": "1295_CR66", +"doi-asserted-by": "crossref", +"first-page": "1545", +"DOI": "10.1111/j.1365-2966.2008.13953.x", +"volume": "391", +"author": "JTL Zwart", +"year": "2008", +"unstructured": "Zwart, J. T. L. et al. The Arcminute Microkelvin Imager. Mon. Not. R. Astron. Soc. 391, 1545–1558 (2008).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR67", +"doi-asserted-by": "crossref", +"first-page": "5677", +"DOI": "10.1093/mnras/sty074", +"volume": "475", +"author": "J Hickish", +"year": "2018", +"unstructured": "Hickish, J. et al. A digital correlator upgrade for the Arcminute MicroKelvin Imager. Mon. Not. R. Astron. Soc. 475, 5677–5687 (2018).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR68", +"doi-asserted-by": "crossref", +"first-page": "1396", +"DOI": "10.1093/mnras/stv1728", +"volume": "453", +"author": "YC Perrott", +"year": "2015", +"unstructured": "Perrott, Y. C. et al. AMI galactic plane survey at 16 GHz—II. Full data release with extended coverage and improved processing. Mon. Not. R. Astron. Soc. 453, 1396–1403 (2015).", +"journal-title": "Mon. Not. R. Astron. Soc." +}, +{ +"key": "1295_CR69", +"unstructured": "McMullin, J. P., Waters, B., Schiebel, D., Young, W. & Golap, K. in Astronomical Data Analysis Software and Systems XVI (eds Shaw, R. A. et al.) 127 (Astronomical Society of the Pacific, 2007)." +}, +{ +"key": "1295_CR70", +"doi-asserted-by": "crossref", +"first-page": "1071", +"DOI": "10.1088/0004-637X/697/2/1071", +"volume": "697", +"author": "WB Atwood", +"year": "2009", +"unstructured": "Atwood, W. B. et al. The Large Area Telescope on the Fermi Gamma-ray Space Telescope mission. Astrophys. J. 697, 1071–1102 (2009).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR71", +"unstructured": "Wood, M. et al. Fermipy: an open-source Python package for analysis of Fermi-LAT Data. In Proc. 35th International Cosmic Ray Conference (ICRC2017) 824 (PoS, 2017).", +"DOI": "10.22323/1.301.0824", +"doi-asserted-by": "crossref" +}, +{ +"key": "1295_CR72", +"unstructured": "Garrappa, S. & Buson, S. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR73", +"unstructured": "The Fermi-LAT collaboration. Fermi Large Area Telescope Fourth Source Catalog. Astrophys. J. Suppl. Ser. 247, 33 (2020)." +}, +{ +"key": "1295_CR74", +"doi-asserted-by": "crossref", +"first-page": "14", +"DOI": "10.1088/0004-637X/767/1/14", +"volume": "767", +"author": "T Pursimo", +"year": "2013", +"unstructured": "Pursimo, T. et al. The Micro-Arcsecond Scintillation-Induced Variability (MASIV) survey. III. Optical identifications and new redshifts. Astrophys. J. 767, 14 (2013).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR75", +"unstructured": "Garrappa, S., Buson, S. & Fermi-LAT Collaboration. Fermi-LAT gamma-ray observations of IceCube-191001A. GCN Circ. 25932 (2019)." +}, +{ +"key": "1295_CR76", +"doi-asserted-by": "crossref", +"first-page": "133", +"DOI": "10.1088/0004-637X/802/2/133", +"volume": "802", +"author": "C Diltz", +"year": "2015", +"unstructured": "Diltz, C., Böttcher, M. & Fossati, G. Time dependent hadronic modeling of flat spectrum radio quasars. Astrophys. J. 802, 133 (2015).", +"journal-title": "Astrophys. J." +}, +{ +"key": "1295_CR77", +"doi-asserted-by": "crossref", +"first-page": "88", +"DOI": "10.1038/s41550-018-0610-1", +"volume": "3", +"author": "S Gao", +"year": "2019", +"unstructured": "Gao, S., Fedynitch, A., Winter, W. & Pohl, M. Modelling the coincident observation of a high-energy neutrino and a bright blazar flare. Nat. Astron. 3, 88–92 (2019).", +"journal-title": "Nat. Astron." +}, +{ +"key": "1295_CR78", +"unstructured": "Ayala, H. IceCube-191001A: HAWC follow-up. GCN Circ. 25936 (2019)." +}, +{ +"key": "1295_CR79", +"doi-asserted-by": "crossref", +"first-page": "62", +"DOI": "10.1126/science.aad1182", +"volume": "351", +"author": "S van Velzen", +"year": "2016", +"unstructured": "van Velzen, S. et al. A radio jet from the optical and x-ray bright stellar tidal disruption flare ASASSN-14li. Science 351, 62–65 (2016).", +"journal-title": "Science" +}, +{ +"key": "1295_CR80", +"doi-asserted-by": "crossref", +"first-page": "306", +"DOI": "10.1086/670067", +"volume": "125", +"author": "D Foreman-Mackey", +"year": "2013", +"unstructured": "Foreman-Mackey, D., Hogg, D. W., Lang, D. & Goodman, J. emcee: the MCMC Hammer. Publ. Astron. Soc. Pac. 125, 306 (2013).", +"journal-title": "Publ. Astron. Soc. Pac." +}, +{ +"key": "1295_CR81", +"doi-asserted-by": "crossref", +"first-page": "6", +"DOI": "10.3847/1538-4365/aab761", +"volume": "236", +"author": "J Guillochon", +"year": "2018", +"unstructured": "Guillochon, J. et al. MOSFiT: Modular Open Source Fitter for Transients. Astrophys. J. Suppl. Ser. 236, 6 (2018).", +"journal-title": "Astrophys. J. Suppl. Ser." +}, +{ +"key": "1295_CR82", +"doi-asserted-by": "crossref", +"first-page": "e008", +"DOI": "10.1017/pasa.2013.44", +"volume": "31", +"author": "J Granot", +"year": "2014", +"unstructured": "Granot, J. & van der Horst, A. J. Gamma-ray burst jets and their radio observations. Publ. Astron. Soc. Aust. 31, e008 (2014).", +"journal-title": "Publ. Astron. Soc. Aust." +}, +{ +"key": "1295_CR83", +"doi-asserted-by": "crossref", +"first-page": "102", +"DOI": "10.1088/0004-637X/815/2/102", +"volume": "815", +"author": "W Fong", +"year": "2015", +"unstructured": "Fong, W., Berger, E., Margutti, R. & Zauderer, B. A. A decade of short-duration gamma-ray burst broadband afterglows: energetics, circumburst densities, and jet opening angles. Astrophys. J. 815, 102 (2015).", +"journal-title": "Astrophys. J." +} +], +"container-title": [ +"Nature Astronomy" +], +"original-title": [ + +], +"language": "en", +"link": [ +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8", +"content-type": "text/html", +"content-version": "vor", +"intended-application": "text-mining" +}, +{ +"URL": "http://www.nature.com/articles/s41550-020-01295-8.pdf", +"content-type": "application/pdf", +"content-version": "vor", +"intended-application": "similarity-checking" +} +], +"deposited": { +"date-parts": [ +[ +2021, +5, +17 +] +], +"date-time": "2021-05-17T15:08:12Z", +"timestamp": 1621264092000 +}, +"score": 1.0, +"subtitle": [ + +], +"short-title": [ + +], +"issued": { +"date-parts": [ +[ +2021, +2, +22 +] +] +}, +"references-count": 83, +"journal-issue": { +"published-print": { +"date-parts": [ +[ +2021, +5 +] +] +}, +"issue": "5" +}, +"alternative-id": [ +"1295" +], +"URL": "http://dx.doi.org/10.1038/s41550-020-01295-8", +"relation": { +"cites": [ + +] +}, +"ISSN": [ +"2397-3366" +], +"issn-type": [ +{ +"value": "2397-3366", +"type": "electronic" +} +], +"assertion": [ +{ +"value": "21 July 2020", +"order": 1, +"name": "received", +"label": "Received", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "16 December 2020", +"order": 2, +"name": "accepted", +"label": "Accepted", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "22 February 2021", +"order": 3, +"name": "first_online", +"label": "First Online", +"group": { +"name": "ArticleHistory", +"label": "Article History" +} +}, +{ +"value": "The authors declare no competing interests.", +"order": 1, +"name": "Ethics", +"group": { +"name": "EthicsHeading", +"label": "Competing interests" +} +} +] +} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkConvertDatasetToJsonRDD.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkConvertDatasetToJsonRDD.scala new file mode 100644 index 000000000..3ee0c7dd6 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkConvertDatasetToJsonRDD.scala @@ -0,0 +1,42 @@ +package eu.dnetlib.dhp.sx.graph + +import com.fasterxml.jackson.databind.ObjectMapper +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.{Oaf, OtherResearchProduct, Publication, Result, Software, Dataset => OafDataset} +import org.apache.commons.io.IOUtils +import org.apache.hadoop.io.compress.GzipCodec +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +object SparkConvertDatasetToJsonRDD { + + + def main(args: Array[String]): Unit = { + val log: Logger = LoggerFactory.getLogger(getClass) + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/convert_dataset_json_params.json"))) + parser.parseArgument(args) + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + val sourcePath = parser.get("sourcePath") + log.info(s"sourcePath -> $sourcePath") + val targetPath = parser.get("targetPath") + log.info(s"targetPath -> $targetPath") + + val resultObject = List("publication","dataset","software", "otherResearchProduct") + val mapper = new ObjectMapper() + implicit val oafEncoder: Encoder[Result] = Encoders.kryo(classOf[Result]) + + + resultObject.foreach{item => + spark.read.load(s"$sourcePath/$item").as[Result].map(r=> mapper.writeValueAsString(r))(Encoders.STRING).rdd.saveAsTextFile(s"$targetPath/${item.toLowerCase}", classOf[GzipCodec]) + } + } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala new file mode 100644 index 000000000..cb41d6134 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala @@ -0,0 +1,67 @@ +package eu.dnetlib.dhp.sx.graph + +import com.fasterxml.jackson.databind.ObjectMapper +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Relation, Result, Software, Dataset => OafDataset} +import org.apache.commons.io.IOUtils +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession} +import org.slf4j.{Logger, LoggerFactory} +object SparkConvertRDDtoDataset { + + def main(args: Array[String]): Unit = { + + + val log: Logger = LoggerFactory.getLogger(getClass) + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/convert_dataset_json_params.json"))) + parser.parseArgument(args) + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + val sourcePath = parser.get("sourcePath") + log.info(s"sourcePath -> $sourcePath") + val t = parser.get("targetPath") + log.info(s"targetPath -> $t") + + val entityPath = s"$t/entities" + val relPath = s"$t/relation" + val mapper = new ObjectMapper() + implicit val datasetEncoder: Encoder[OafDataset] = Encoders.kryo(classOf[OafDataset]) + implicit val publicationEncoder: Encoder[Publication] = Encoders.kryo(classOf[Publication]) + implicit val relationEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation]) + implicit val orpEncoder: Encoder[OtherResearchProduct] = Encoders.kryo(classOf[OtherResearchProduct]) + implicit val softwareEncoder: Encoder[Software] = Encoders.kryo(classOf[Software]) + + + log.info("Converting dataset") + val rddDataset =spark.sparkContext.textFile(s"$sourcePath/dataset").map(s => mapper.readValue(s, classOf[OafDataset])) + spark.createDataset(rddDataset).as[OafDataset].write.mode(SaveMode.Overwrite).save(s"$entityPath/dataset") + + + log.info("Converting publication") + val rddPublication =spark.sparkContext.textFile(s"$sourcePath/publication").map(s => mapper.readValue(s, classOf[Publication])) + spark.createDataset(rddPublication).as[Publication].write.mode(SaveMode.Overwrite).save(s"$entityPath/publication") + + log.info("Converting software") + val rddSoftware =spark.sparkContext.textFile(s"$sourcePath/software").map(s => mapper.readValue(s, classOf[Software])) + spark.createDataset(rddSoftware).as[Software].write.mode(SaveMode.Overwrite).save(s"$entityPath/software") + + log.info("Converting otherresearchproduct") + val rddOtherResearchProduct =spark.sparkContext.textFile(s"$sourcePath/otherresearchproduct").map(s => mapper.readValue(s, classOf[OtherResearchProduct])) + spark.createDataset(rddOtherResearchProduct).as[OtherResearchProduct].write.mode(SaveMode.Overwrite).save(s"$entityPath/otherresearchproduct") + + + log.info("Converting Relation") + + + val rddRelation =spark.sparkContext.textFile(s"$sourcePath/relation").map(s => mapper.readValue(s, classOf[Relation])) + spark.createDataset(rddRelation).as[Relation].write.mode(SaveMode.Overwrite).save(s"$relPath") + + + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala index a37dd2132..350b00c5e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala @@ -70,7 +70,7 @@ object SparkCreateInputGraph { resultObject.foreach { r => log.info(s"Make ${r._1} unique") - makeDatasetUnique(s"$targetPath/extracted/${r._1}",s"$targetPath/dedup/${r._1}",spark, r._2) + makeDatasetUnique(s"$targetPath/extracted/${r._1}",s"$targetPath/preprocess/${r._1}",spark, r._2) } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateScholix.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateScholix.scala index ba483bfb2..0a7fc18fb 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateScholix.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateScholix.scala @@ -42,6 +42,7 @@ object SparkCreateScholix { val relationDS: Dataset[(String, Relation)] = spark.read.load(relationPath).as[Relation] + .filter(r => (r.getDataInfo== null || r.getDataInfo.getDeletedbyinference == false) && !r.getRelClass.toLowerCase.contains("merge")) .map(r => (r.getSource, r))(Encoders.tuple(Encoders.STRING, relEncoder)) val summaryDS: Dataset[(String, ScholixSummary)] = spark.read.load(summaryPath).as[ScholixSummary] diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateSummaryObject.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateSummaryObject.scala index a66da3e6d..0970375f5 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateSummaryObject.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkCreateSummaryObject.scala @@ -1,7 +1,7 @@ package eu.dnetlib.dhp.sx.graph import eu.dnetlib.dhp.application.ArgumentApplicationParser -import eu.dnetlib.dhp.schema.oaf.Result +import eu.dnetlib.dhp.schema.oaf.{Oaf, Result} import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils import org.apache.commons.io.IOUtils @@ -29,11 +29,12 @@ object SparkCreateSummaryObject { log.info(s"targetPath -> $targetPath") implicit val resultEncoder:Encoder[Result] = Encoders.kryo[Result] + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] implicit val summaryEncoder:Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary] - val ds:Dataset[Result] = spark.read.load(s"$sourcePath/*").as[Result] + val ds:Dataset[Result] = spark.read.load(s"$sourcePath/*").as[Result].filter(r=>r.getDataInfo== null || r.getDataInfo.getDeletedbyinference== false) ds.repartition(6000).map(r => ScholixUtils.resultToSummary(r)).filter(s => s!= null).write.mode(SaveMode.Overwrite).save(targetPath) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkResolveRelation.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkResolveRelation.scala index 6ee575e2a..b2fddec20 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkResolveRelation.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkResolveRelation.scala @@ -1,10 +1,17 @@ package eu.dnetlib.dhp.sx.graph +import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.application.ArgumentApplicationParser import eu.dnetlib.dhp.schema.oaf.{Relation, Result} import org.apache.commons.io.IOUtils +import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ +import org.json4s +import org.json4s.DefaultFormats +import org.json4s.JsonAST.{JField, JObject, JString} +import org.json4s.jackson.JsonMethods.parse import org.slf4j.{Logger, LoggerFactory} import scala.collection.JavaConverters._ @@ -25,60 +32,109 @@ object SparkResolveRelation { val relationPath = parser.get("relationPath") log.info(s"sourcePath -> $relationPath") val entityPath = parser.get("entityPath") - log.info(s"targetPath -> $entityPath") + log.info(s"entityPath -> $entityPath") val workingPath = parser.get("workingPath") log.info(s"workingPath -> $workingPath") - - implicit val oafEncoder: Encoder[Result] = Encoders.kryo(classOf[Result]) implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation]) import spark.implicits._ - val entities:Dataset[Result] = spark.read.load(s"$entityPath/*").as[Result] - entities.flatMap(e => e.getPid.asScala - .map(p => - convertPidToDNETIdentifier(p.getValue, p.getQualifier.getClassid)) - .filter(s => s!= null) - .map(s => (s,e.getId)) - ).groupByKey(_._1) - .reduceGroups((x,y) => if (x._2.startsWith("50|doi") || x._2.startsWith("50|pmid")) x else y) - .map(s =>s._2) - .write - .mode(SaveMode.Overwrite) - .save(s"$workingPath/resolvedPid") - val rPid:Dataset[(String,String)] = spark.read.load(s"$workingPath/resolvedPid").as[(String,String)] + extractPidResolvedTableFromJsonRDD(spark, entityPath, workingPath) + + val mappper = new ObjectMapper() + + val rPid:Dataset[(String,String)] = spark.read.load(s"$workingPath/relationResolvedPid").as[(String,String)] val relationDs:Dataset[(String,Relation)] = spark.read.load(relationPath).as[Relation].map(r => (r.getSource.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder)) - relationDs.joinWith(rPid, relationDs("_1").equalTo(rPid("_1")), "left").map{ + relationDs.joinWith(rPid, relationDs("_1").equalTo(rPid("_2")), "left").map{ m => val sourceResolved = m._2 val currentRelation = m._1._2 - if (sourceResolved!=null && sourceResolved._2.nonEmpty) - currentRelation.setSource(sourceResolved._2) + if (sourceResolved!=null && sourceResolved._1!=null && sourceResolved._1.nonEmpty) + currentRelation.setSource(sourceResolved._1) currentRelation }.write .mode(SaveMode.Overwrite) - .save(s"$workingPath/resolvedSource") + .save(s"$workingPath/relationResolvedSource") - val relationSourceResolved:Dataset[(String,Relation)] = spark.read.load(s"$workingPath/resolvedSource").as[Relation].map(r => (r.getTarget.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder)) - relationSourceResolved.joinWith(rPid, relationSourceResolved("_1").equalTo(rPid("_1")), "left").map{ + val relationSourceResolved:Dataset[(String,Relation)] = spark.read.load(s"$workingPath/relationResolvedSource").as[Relation].map(r => (r.getTarget.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder)) + relationSourceResolved.joinWith(rPid, relationSourceResolved("_1").equalTo(rPid("_2")), "left").map{ m => val targetResolved = m._2 val currentRelation = m._1._2 - if (targetResolved!=null && targetResolved._2.nonEmpty) - currentRelation.setTarget(targetResolved._2) + if (targetResolved!=null && targetResolved._1.nonEmpty) + currentRelation.setTarget(targetResolved._1) currentRelation }.filter(r => r.getSource.startsWith("50")&& r.getTarget.startsWith("50")) .write .mode(SaveMode.Overwrite) - .save(s"$workingPath/resolvedRelation") + .save(s"$workingPath/relation_resolved") + + spark.read.load(s"$workingPath/relation_resolved").as[Relation] + .map(r => mappper.writeValueAsString(r)) + .rdd.saveAsTextFile(s"$workingPath/relation", classOf[GzipCodec]) + } + private def extractPidsFromRecord(input:String):(String,List[(String,String)]) = { + implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats + lazy val json: json4s.JValue = parse(input) + val id:String = (json \ "id").extract[String] + val result: List[(String,String)] = for { + JObject(pids) <- json \ "pid" + JField("value", JString(pidValue)) <- pids + JField("qualifier", JObject(qualifier)) <- pids + JField("classname", JString(pidType)) <- qualifier + } yield (pidValue, pidType) + (id,result) + } + private def extractPidResolvedTableFromJsonRDD(spark: SparkSession, entityPath: String, workingPath: String) = { + import spark.implicits._ + + val d: RDD[(String,String)] = spark.sparkContext.textFile(s"$entityPath/*") + .map(i => extractPidsFromRecord(i)) + .filter(s => s != null && s._1!= null && s._2!=null && s._2.nonEmpty) + .flatMap{ p => + p._2.map(pid => + (p._1, convertPidToDNETIdentifier(pid._1, pid._2)) + ) + }.filter(r =>r._1 != null || r._2 != null) + + spark.createDataset(d) + .groupByKey(_._2) + .reduceGroups((x, y) => if (x._1.startsWith("50|doi") || x._1.startsWith("50|pmid")) x else y) + .map(s => s._2) + .write + .mode(SaveMode.Overwrite) + .save(s"$workingPath/relationResolvedPid") + } + + + /* + This method should be used once we finally convert everythings in Kryo dataset + instead of using rdd of json + */ + private def extractPidResolvedTableFromKryo(spark: SparkSession, entityPath: String, workingPath: String) = { + import spark.implicits._ + implicit val oafEncoder: Encoder[Result] = Encoders.kryo(classOf[Result]) + val entities: Dataset[Result] = spark.read.load(s"$entityPath/*").as[Result] + entities.flatMap(e => e.getPid.asScala + .map(p => + convertPidToDNETIdentifier(p.getValue, p.getQualifier.getClassid)) + .filter(s => s != null) + .map(s => (s, e.getId)) + ).groupByKey(_._1) + .reduceGroups((x, y) => if (x._2.startsWith("50|doi") || x._2.startsWith("50|pmid")) x else y) + .map(s => s._2) + .write + .mode(SaveMode.Overwrite) + .save(s"$workingPath/relationResolvedPid") + } def convertPidToDNETIdentifier(pid:String, pidType: String):String = { if (pid==null || pid.isEmpty || pidType== null || pidType.isEmpty) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/BioDBToOAF.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/BioDBToOAF.scala index a19c6fb12..90b65c8f7 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/BioDBToOAF.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/BioDBToOAF.scala @@ -199,7 +199,7 @@ object BioDBToOAF { d.setDateofacceptance(OafMapperUtils.field(i_date.get.date, DATA_INFO)) } val relevant_dates: List[StructuredProperty] = dates.filter(d => !d.date_info.contains("entry version")) - .map(date => OafMapperUtils.structuredProperty(date.date, "UNKNOWN", "UNKNOWN", ModelConstants.DNET_DATACITE_DATE, ModelConstants.DNET_DATACITE_DATE, DATA_INFO)) + .map(date => OafMapperUtils.structuredProperty(date.date, ModelConstants.UNKNOWN, ModelConstants.UNKNOWN, ModelConstants.DNET_DATACITE_DATE, ModelConstants.DNET_DATACITE_DATE, DATA_INFO)) if (relevant_dates != null && relevant_dates.nonEmpty) d.setRelevantdate(relevant_dates.asJava) d.setDateofacceptance(OafMapperUtils.field(i_date.get.date, DATA_INFO)) @@ -218,12 +218,12 @@ object BioDBToOAF { if (references_pmid != null && references_pmid.nonEmpty) { - val rel = createRelation(references_pmid.head, "pmid", d.getId, collectedFromMap("uniprot"), "relationship", "isRelatedTo", if (i_date.isDefined) i_date.get.date else null) + val rel = createRelation(references_pmid.head, "pmid", d.getId, collectedFromMap("uniprot"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, if (i_date.isDefined) i_date.get.date else null) rel.getCollectedfrom List(d, rel) } else if (references_doi != null && references_doi.nonEmpty) { - val rel = createRelation(references_doi.head, "doi", d.getId, collectedFromMap("uniprot"), "relationship", "isRelatedTo", if (i_date.isDefined) i_date.get.date else null) + val rel = createRelation(references_doi.head, "doi", d.getId, collectedFromMap("uniprot"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, if (i_date.isDefined) i_date.get.date else null) List(d, rel) } else @@ -243,7 +243,7 @@ object BioDBToOAF { rel.setCollectedfrom(List(collectedFromMap("pdb")).asJava) rel.setDataInfo(DATA_INFO) - rel.setRelType("resultResult") + rel.setRelType(ModelConstants.RESULT_RESULT) rel.setSubRelType(subRelType) rel.setRelClass(relClass) @@ -263,7 +263,7 @@ object BioDBToOAF { def createSupplementaryRelation(pid: String, pidType: String, sourceId: String, collectedFrom: KeyValue, date:String): Relation = { - createRelation(pid,pidType,sourceId,collectedFrom, "supplement","IsSupplementTo", date) + createRelation(pid,pidType,sourceId,collectedFrom, ModelConstants.SUPPLEMENT, ModelConstants.IS_SUPPLEMENT_TO, date) } @@ -392,6 +392,6 @@ object BioDBToOAF { i.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(input.date), DATA_INFO)) d.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(input.date), DATA_INFO)) - List(d, createRelation(input.pmid, "pmid", d.getId, collectedFromMap("ebi"),"relationship", "isRelatedTo", GraphCleaningFunctions.cleanDate(input.date))) + List(d, createRelation(input.pmid, "pmid", d.getId, collectedFromMap("ebi"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, GraphCleaningFunctions.cleanDate(input.date))) } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/PubMedToOaf.scala index ae4a72062..9a49deebc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/bio/pubmed/PubMedToOaf.scala @@ -16,7 +16,7 @@ object PubMedToOaf { ) def createResult(cobjQualifier: Qualifier, vocabularies: VocabularyGroup): Result = { - val result_typologies = getVocabularyTerm("dnet:result_typologies", vocabularies, cobjQualifier.getClassid) + val result_typologies = getVocabularyTerm(ModelConstants.DNET_RESULT_TYPOLOGIES, vocabularies, cobjQualifier.getClassid) result_typologies.getClassid match { case "dataset" => new Dataset case "publication" => new Publication @@ -68,11 +68,11 @@ object PubMedToOaf { //else We have to find a terms that match the vocabulary otherwise we discard it val ja = article.getPublicationTypes.asScala.find(s => "Journal Article".equalsIgnoreCase(s.getValue)) if (ja.isDefined) { - val cojbCategory = getVocabularyTerm("dnet:publication_resource", vocabularies, ja.get.getValue) + val cojbCategory = getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, ja.get.getValue) i.setInstancetype(cojbCategory) } else { val i_type = article.getPublicationTypes.asScala - .map(s => getVocabularyTerm("dnet:publication_resource", vocabularies, s.getValue)) + .map(s => getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, s.getValue)) .find(q => q != null) if (i_type.isDefined) i.setInstancetype(i_type.get) @@ -112,7 +112,7 @@ object PubMedToOaf { if (article.getLanguage != null) { - val term = vocabularies.getSynonymAsQualifier("dnet:languages", article.getLanguage) + val term = vocabularies.getSynonymAsQualifier(ModelConstants.DNET_LANGUAGES, article.getLanguage) if (term != null) result.setLanguage(term) } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala index 6a7ee7803..4dafd4fa3 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala @@ -1,11 +1,10 @@ package eu.dnetlib.dhp.sx.graph.scholix -import eu.dnetlib.dhp.schema.oaf.{Dataset, Relation, Result, StructuredProperty} -import eu.dnetlib.dhp.schema.sx.scholix.{Scholix, ScholixCollectedFrom, ScholixEntityId, ScholixIdentifier, ScholixRelationship, ScholixResource} +import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Result, StructuredProperty} +import eu.dnetlib.dhp.schema.sx.scholix._ import eu.dnetlib.dhp.schema.sx.summary.{CollectedFromType, SchemeValue, ScholixSummary, Typology} import eu.dnetlib.dhp.utils.DHPUtils -import org.apache.spark.sql.Encoders.bean import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.{Encoder, Encoders} import org.json4s @@ -301,14 +300,14 @@ object ScholixUtils { if (r.getPid == null || r.getPid.isEmpty) return null - val pids:List[ScholixIdentifier] = extractTypedIdentifierFromInstance(r) - if (pids.isEmpty) + val persistentIdentifiers:List[ScholixIdentifier] = extractTypedIdentifierFromInstance(r) + if (persistentIdentifiers.isEmpty) return null - s.setLocalIdentifier(pids.asJava) - if (r.isInstanceOf[Dataset]) - s.setTypology(Typology.dataset) - else + s.setLocalIdentifier(persistentIdentifiers.asJava) + if (r.isInstanceOf[Publication] ) s.setTypology(Typology.publication) + else + s.setTypology(Typology.dataset) s.setSubType(r.getInstance().get(0).getInstancetype.getClassname) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_dataset_json_params.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_dataset_json_params.json new file mode 100644 index 000000000..8bfdde5b0 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_dataset_json_params.json @@ -0,0 +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":"t", "paramLongName":"targetPath", "paramDescription": "the path of the raw graph", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step1/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/extractEntities/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step1/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/extractEntities/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/extractEntities/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/extractEntities/oozie_app/workflow.xml new file mode 100644 index 000000000..685976ce6 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/extractEntities/oozie_app/workflow.xml @@ -0,0 +1,85 @@ + + + + sourcePath + the working dir base path + + + targetPath + the graph Raw base path + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + Extract entities in raw graph + eu.dnetlib.dhp.sx.graph.SparkCreateInputGraph + 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} + + --masteryarn + --sourcePath${sourcePath} + --targetPath${targetPath} + + + + + + + + + + + + + + + + + yarn + cluster + Generate Input Graph for deduplication + eu.dnetlib.dhp.sx.graph.SparkConvertDatasetToJsonRDD + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.shuffle.partitions=3000 + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --masteryarn + --sourcePath${targetPath}/preprocess + --targetPath${targetPath}/dedup + + + + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step2/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step2/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step1/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml similarity index 78% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step1/oozie_app/workflow.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml index 3ea4e9d30..d8eb1fc80 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step1/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + sourcePath @@ -6,48 +6,22 @@ targetPath - the graph Raw base path + the final graph path - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + yarn cluster - Extract entities in raw graph - eu.dnetlib.dhp.sx.graph.SparkCreateInputGraph - 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} - - --masteryarn - --sourcePath${sourcePath} - --targetPath${targetPath} - - - - - - - - - yarn - cluster - Resolve Relations in raw graph - eu.dnetlib.dhp.sx.graph.SparkResolveRelation + Import JSONRDD to Dataset kryo + eu.dnetlib.dhp.sx.graph.SparkConvertRDDtoDataset dhp-graph-mapper-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} @@ -60,9 +34,8 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --masteryarn - --relationPath${targetPath}/extracted/relation - --workingPath${targetPath}/resolved/ - --entityPath${targetPath}/dedup + --sourcePath${sourcePath} + --targetPath${targetPath} @@ -87,7 +60,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --masteryarn - --sourcePath${targetPath}/dedup + --sourcePath${targetPath}/entities --targetPath${targetPath}/provision/summaries @@ -114,7 +87,7 @@ --masteryarn --summaryPath${targetPath}/provision/summaries --targetPath${targetPath}/provision/scholix - --relationPath${targetPath}/resolved/resolvedRelation + --relationPath${targetPath}/relation @@ -182,9 +155,5 @@ - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step3/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/resolverelation/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step3/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/resolverelation/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/resolverelation/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/resolverelation/oozie_app/workflow.xml new file mode 100644 index 000000000..7683ff94c --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/resolverelation/oozie_app/workflow.xml @@ -0,0 +1,62 @@ + + + + entityPath + the path of deduplicate Entities + + + relationPath + the path of relation unresolved + + + targetPath + the path of relation unresolved + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + + yarn + cluster + Resolve Relations in raw graph + eu.dnetlib.dhp.sx.graph.SparkResolveRelation + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.shuffle.partitions=3000 + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --masteryarn + --relationPath${relationPath} + --workingPath${targetPath} + --entityPath${entityPath} + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step2/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step2/oozie_app/workflow.xml deleted file mode 100644 index 9d06c42d6..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step2/oozie_app/workflow.xml +++ /dev/null @@ -1,120 +0,0 @@ - - - - workingPath - the working path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - Extract DLI Entities (Publication) - eu.dnetlib.dhp.sx.graph.SparkSplitOafTODLIEntities - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory ${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=5000 - ${sparkExtraOPT} - - -mt yarn-cluster - --workingPath${workingPath} - -epublication - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - Extract DLI Entities (Dataset) - eu.dnetlib.dhp.sx.graph.SparkSplitOafTODLIEntities - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory ${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=5000 - ${sparkExtraOPT} - - -mt yarn-cluster - --workingPath${workingPath} - -edataset - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - Extract DLI Entities (Unknown) - eu.dnetlib.dhp.sx.graph.SparkSplitOafTODLIEntities - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory ${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=5000 - ${sparkExtraOPT} - - -mt yarn-cluster - --workingPath${workingPath} - -eunknown - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - Extract DLI Entities (Relation) - eu.dnetlib.dhp.sx.graph.SparkSplitOafTODLIEntities - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory ${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=5000 - ${sparkExtraOPT} - - -mt yarn-cluster - --workingPath${workingPath} - -erelation - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step3/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step3/oozie_app/workflow.xml deleted file mode 100644 index 4d54b2afb..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/step3/oozie_app/workflow.xml +++ /dev/null @@ -1,61 +0,0 @@ - - - - sourcePath - the source path - - - targetPath - the source path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - entity - the entity to be merged - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - Merge ${entity} - eu.dnetlib.dhp.sx.graph.SparkScholexplorerCreateRawGraphJob - dhp-graph-mapper-${projectVersion}.jar - --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} - -mt yarn-cluster - --sourcePath${sourcePath}/${entity} - --targetPath${targetPath}/${entity} - --entity${entity} - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index ba4211a3f..c431b4dd8 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -24,6 +24,7 @@ import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.graph.clean.GraphCleaningFunctionsTest; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.PidType; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -250,7 +251,24 @@ public class MappersTest { final Relation r1 = (Relation) list.get(1); final Relation r2 = (Relation) list.get(2); + assertEquals(d.getId(), r1.getSource()); + assertEquals("40|corda_______::e06332dee33bec6c2ba4c98601053229", r1.getTarget()); + assertEquals(ModelConstants.RESULT_PROJECT, r1.getRelType()); + assertEquals(ModelConstants.OUTCOME, r1.getSubRelType()); + assertEquals(ModelConstants.IS_PRODUCED_BY, r1.getRelClass()); + assertTrue(r1.getValidated()); + assertEquals("2020-01-01", r1.getValidationDate()); + + assertEquals(d.getId(), r2.getTarget()); + assertEquals("40|corda_______::e06332dee33bec6c2ba4c98601053229", r2.getSource()); + assertEquals(ModelConstants.RESULT_PROJECT, r2.getRelType()); + assertEquals(ModelConstants.OUTCOME, r2.getSubRelType()); + assertEquals(ModelConstants.PRODUCES, r2.getRelClass()); + assertTrue(r2.getValidated()); + assertEquals("2020-01-01", r2.getValidationDate()); + assertValidId(d.getId()); + assertEquals("50|doi_________::000374d100a9db469bd42b69dbb40b36", d.getId()); assertEquals(2, d.getOriginalId().size()); assertTrue(d.getOriginalId().stream().anyMatch(oid -> oid.equals("oai:zenodo.org:3234526"))); assertValidId(d.getCollectedfrom().get(0).getKey()); @@ -304,10 +322,12 @@ public class MappersTest { }); assertEquals("0001", d.getInstance().get(0).getRefereed().getClassid()); assertNotNull(d.getInstance().get(0).getPid()); - assertTrue(d.getInstance().get(0).getPid().isEmpty()); + assertFalse(d.getInstance().get(0).getPid().isEmpty()); - assertEquals("doi", d.getInstance().get(0).getAlternateIdentifier().get(0).getQualifier().getClassid()); - assertEquals("10.5281/zenodo.3234526", d.getInstance().get(0).getAlternateIdentifier().get(0).getValue()); + assertEquals("doi", d.getInstance().get(0).getPid().get(0).getQualifier().getClassid()); + assertEquals("10.5281/zenodo.3234526", d.getInstance().get(0).getPid().get(0).getValue()); + + assertTrue(d.getInstance().get(0).getAlternateIdentifier().isEmpty()); assertValidId(r1.getSource()); assertValidId(r1.getTarget()); @@ -561,6 +581,31 @@ public class MappersTest { assertNotNull(d.getInstance().get(0).getUrl()); } + @Test + void testEnermaps() throws IOException { + final String xml = IOUtils.toString(getClass().getResourceAsStream("enermaps.xml")); + final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); + + System.out.println("***************"); + System.out.println(new ObjectMapper().writeValueAsString(list)); + System.out.println("***************"); + + assertEquals(1, list.size()); + assertTrue(list.get(0) instanceof Dataset); + + final Dataset d = (Dataset) list.get(0); + + assertValidId(d.getId()); + assertValidId(d.getCollectedfrom().get(0).getKey()); + assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue())); + assertEquals(1, d.getAuthor().size()); + assertEquals(1, d.getInstance().size()); + assertNotNull(d.getInstance().get(0).getUrl()); + assertNotNull(d.getContext()); + assertTrue(StringUtils.isNotBlank(d.getContext().get(0).getId())); + assertEquals("enermaps::selection::tgs00004", d.getContext().get(0).getId()); + } + @Test void testClaimFromCrossref() throws IOException { final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_crossref.xml")); @@ -713,12 +758,11 @@ public class MappersTest { } private void assertValidId(final String id) { - System.out.println(id); + // System.out.println(id); assertEquals(49, id.length()); - assertEquals('|', id.charAt(2)); - assertEquals(':', id.charAt(15)); - assertEquals(':', id.charAt(16)); + assertEquals(IdentifierFactory.ID_PREFIX_SEPARATOR, id.substring(2, 3)); + assertEquals(IdentifierFactory.ID_SEPARATOR, id.substring(15, 17)); } private List vocs() throws IOException { diff --git a/dhp-workflows/dhp-graph-provision/pom.xml b/dhp-workflows/dhp-graph-provision/pom.xml index c279436d7..e402d0600 100644 --- a/dhp-workflows/dhp-graph-provision/pom.xml +++ b/dhp-workflows/dhp-graph-provision/pom.xml @@ -9,6 +9,41 @@ dhp-graph-provision + + + + net.alchim31.maven + scala-maven-plugin + 4.0.1 + + + scala-compile-first + initialize + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + -Xmax-classfile-name + 200 + + ${scala.version} + + + + + + diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/sx/provision/DropAndCreateESIndex.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/sx/provision/DropAndCreateESIndex.java index f96a64a27..ffeb0995d 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/sx/provision/DropAndCreateESIndex.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/sx/provision/DropAndCreateESIndex.java @@ -71,6 +71,9 @@ public class DropAndCreateESIndex { log.info(STATUS_CODE_TEXT, response.getStatusLine()); } + log.info("Sleeping 60 seconds to avoid to lost the creation of index request"); + Thread.sleep(60000); + try (CloseableHttpClient client = HttpClients.createDefault()) { final String summaryConf = IOUtils diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java index 897e507e1..221049f90 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; +import eu.dnetlib.dhp.oa.provision.utils.ContextDef; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; import eu.dnetlib.dhp.schema.oaf.Dataset; 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 d5aa207d1..fb944f4ff 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 @@ -13,7 +13,7 @@ echo "Getting file from " $SCRIPT_PATH hdfs dfs -copyToLocal $SCRIPT_PATH echo "Creating indicators" -impala-shell -d ${TARGET} -q "invalidate metadata" +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 - 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/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index d79396b3b..e4e81175c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -57,12 +57,14 @@ UNION ALL SELECT * FROM ${stats_db_name}.software_sources UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; --- --- ANALYZE TABLE ${stats_db_name}.publication_sources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_sources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_sources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_sources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_sources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_sources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_sources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_sources COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file + + +create table ${stats_db_name}.result_orcid as +select distinct res.id, regexp_replace(res.orcid, 'http://orcid.org/' ,'') as orcid +from ( + SELECT substr(res.id, 4) as id, auth_pid.value as orcid + FROM ${openaire_db_name}.result res + LATERAL VIEW explode(author) a as auth + LATERAL VIEW explode(auth.pid) ap as auth_pid + LATERAL VIEW explode(auth.pid.qualifier.classid) apt as author_pid_type + WHERE res.datainfo.deletedbyinference = FALSE and res.datainfo.invisible = FALSE and author_pid_type = 'orcid') as res \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 8f364d747..8e66e05c0 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -33,13 +33,4 @@ select * from ${stats_db_name}.dataset_refereed union all select * from ${stats_db_name}.software_refereed union all -select * from ${stats_db_name}.otherresearchproduct_refereed; --- --- ANALYZE TABLE ${stats_db_name}.publication_refereed COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_refereed COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_refereed COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_refereed COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_refereed COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_refereed COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_refereed COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_refereed COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file +select * from ${stats_db_name}.otherresearchproduct_refereed; \ 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_7-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_7-createIndicatorsTables.sql index 8998cb9fc..f1ebf0d87 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_7-createIndicatorsTables.sql @@ -39,4 +39,198 @@ from publication p join result_instance ri on ri.id = p.id join datasource on datasource.id = ri.hostedby where datasource.id like '%doajarticles%') tmp -on p.id= tmp.id; \ No newline at end of file +on p.id= tmp.id; + +create table indi_project_pubs_count stored as parquet as +select pr.id id, count(p.id) total_pubs from project_results pr +join publication p on p.id=pr.result +group by pr.id; + +create table indi_project_datasets_count stored as parquet as +select pr.id id, count(d.id) total_datasets from project_results pr +join dataset d on d.id=pr.result +group by pr.id; + +create table indi_project_software_count stored as parquet as +select pr.id id, count(s.id) total_software from project_results pr +join software s on s.id=pr.result +group by pr.id; + +create table indi_project_otherresearch_count stored as parquet as +select pr.id id, count(o.id) total_other from project_results pr +join otherresearchproduct o on o.id=pr.result +group by pr.id; + +create table indi_pub_avg_year_country_oa stored as parquet as +select year, country, round(OpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageOA, +round(NonOpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageNonOA + from + (SELECT year, country, SUM(CASE + WHEN bestlicence='Open Access' THEN 1 + ELSE 0 + END) AS OpenAccess, SUM(CASE + WHEN bestlicence<>'Open Access' THEN 1 + ELSE 0 + END) AS NonOpenAccess + FROM publication p + join result_organization ro on p.id=ro.id + join organization o on o.id=ro.organization + where cast(year as int)>=2003 and cast(year as int)<=2021 + group by year, country) tmp; + +create table indi_dataset_avg_year_country_oa stored as parquet as +select year, country, round(OpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageOA, +round(NonOpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageNonOA + from + (SELECT year, country, SUM(CASE + WHEN bestlicence='Open Access' THEN 1 + ELSE 0 + END) AS OpenAccess, SUM(CASE + WHEN bestlicence<>'Open Access' THEN 1 + ELSE 0 + END) AS NonOpenAccess + FROM dataset d + join result_organization ro on d.id=ro.id + join organization o on o.id=ro.organization + where cast(year as int)>=2003 and cast(year as int)<=2021 + group by year, country) tmp; + +create table indi_software_avg_year_country_oa stored as parquet as +select year, country, round(OpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageOA, +round(NonOpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageNonOA + from + (SELECT year, country, SUM(CASE + WHEN bestlicence='Open Access' THEN 1 + ELSE 0 + END) AS OpenAccess, SUM(CASE + WHEN bestlicence<>'Open Access' THEN 1 + ELSE 0 + END) AS NonOpenAccess + FROM software s + join result_organization ro on s.id=ro.id + join SOURCER.organization o on o.id=ro.organization + where cast(year as int)>=2003 and cast(year as int)<=2021 + group by year, country) tmp; + + +create table indi_other_avg_year_country_oa stored as parquet as +select year, country, round(OpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageOA, +round(NonOpenAccess/(OpenAccess+NonOpenAccess)*100,3) as averageNonOA + from + (SELECT year, country, SUM(CASE + WHEN bestlicence='Open Access' THEN 1 + ELSE 0 + END) AS OpenAccess, SUM(CASE + WHEN bestlicence<>'Open Access' THEN 1 + ELSE 0 + END) AS NonOpenAccess + FROM otherresearchproduct orp + join result_organization ro on orp.id=ro.id + join organization o on o.id=ro.organization + where cast(year as int)>=2003 and cast(year as int)<=2021 + group by year, country) tmp; + +create table indi_pub_avg_year_context_oa stored as parquet as +with total as +(select count(distinct pc.id) no_of_pubs, year, c.name name, sum(count(distinct pc.id)) over(PARTITION by year) as total from publication_concepts pc +join context c on pc.concept like concat('%',c.id,'%') +join publication p on p.id=pc.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by c.name, year ) +select year, name, round(no_of_pubs/total*100,3) averageofpubs +from total; + +create table indi_dataset_avg_year_context_oa stored as parquet as +with total as +(select count(distinct pc.id) no_of_pubs, year, c.name name, sum(count(distinct pc.id)) over(PARTITION by year) as total from dataset_concepts pc +join context c on pc.concept like concat('%',c.id,'%') +join dataset p on p.id=pc.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by c.name, year ) +select year, name, round(no_of_pubs/total*100,3) averageofdataset +from total; + +create table indi_software_avg_year_context_oa stored as parquet as +with total as +(select count(distinct pc.id) no_of_pubs, year, c.name name, sum(count(distinct pc.id)) over(PARTITION by year) as total from software_concepts pc +join context c on pc.concept like concat('%',c.id,'%') +join software p on p.id=pc.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by c.name, year ) +select year, name, round(no_of_pubs/total*100,3) averageofsoftware +from total; + +create table indi_other_avg_year_context_oa stored as parquet as +with total as +(select count(distinct pc.id) no_of_pubs, year, c.name name, sum(count(distinct pc.id)) over(PARTITION by year) as total from otherresearchproduct_concepts pc +join context c on pc.concept like concat('%',c.id,'%') +join otherresearchproduct p on p.id=pc.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by c.name, year ) +select year, name, round(no_of_pubs/total*100,3) averageofother +from total; + +create table indi_other_avg_year_content_oa stored as parquet as +with total as +(select count(distinct pd.id) no_of_pubs, year, d.type type, sum(count(distinct pd.id)) over(PARTITION by year) as total +from otherresearchproduct_datasources pd +join datasource d on datasource=d.id +join otherresearchproduct p on p.id=pd.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by d.type, year) +select year, type, round(no_of_pubs/total*100,3) averageOfOtherresearchproduct +from total; + +create table indi_software_avg_year_content_oa stored as parquet as +with total as +(select count(distinct pd.id) no_of_pubs, year, d.type type, sum(count(distinct pd.id)) over(PARTITION by year) as total +from software_datasources pd +join datasource d on datasource=d.id +join software p on p.id=pd.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by d.type, year) +select year, type, round(no_of_pubs/total*100,3) averageOfSoftware +from total; + +create table indi_dataset_avg_year_content_oa stored as parquet as +with total as +(select count(distinct pd.id) no_of_pubs, year, d.type type, sum(count(distinct pd.id)) over(PARTITION by year) as total +from dataset_datasources pd +join datasource d on datasource=d.id +join dataset p on p.id=pd.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by d.type, year) +select year, type, round(no_of_pubs/total*100,3) averageOfDatasets +from total; + +create table indi_pub_avg_year_content_oa stored as parquet as +with total as +(select count(distinct pd.id) no_of_pubs, year, d.type type, sum(count(distinct pd.id)) over(PARTITION by year) as total +from publication_datasources pd +join datasource d on datasource=d.id +join publication p on p.id=pd.id +where cast(year as int)>=2003 and cast(year as int)<=2021 +group by d.type, year) +select year, type, round(no_of_pubs/total*100,3) averageOfPubs +from total; + +create table indi_pub_has_cc_licence stored as parquet as +select distinct p.id, (case when lic='' or lic is null then 0 else 1 end) as has_cc_license +from publication p +left outer join (select p.id, license.type as lic from publication p +join publication_licenses as license on license.id = p.id +where lower(license.type) LIKE '%creativecommons.org%' OR lower(license.type) LIKE '%cc-%') tmp +on p.id= tmp.id; + +create table indi_pub_has_cc_licence_url stored as parquet as +select distinct p.id, (case when lic_host='' or lic_host is null then 0 else 1 end) as has_cc_license_url +from publication p +left outer join (select p.id, lower(parse_url(license.type, "HOST")) as lic_host +from publication p +join publication_licenses as license on license.id = p.id +WHERE lower(parse_url(license.type, 'HOST')) = 'creativecommons.org') tmp +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 diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index 75b24b189..bb0d0ac6c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -90,27 +90,8 @@ FROM ${openaire_db_name}.publication p where p.datainfo.deletedbyinference = false; CREATE TABLE ${stats_db_name}.publication_citations AS -SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result +SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" - and p.datainfo.deletedbyinference = false; - --- ANALYZE TABLE ${stats_db_name}.publication_tmp COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_tmp COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_classifications COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_classifications COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_concepts COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_concepts COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_datasources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_datasources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_languages COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_languages COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_oids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_oids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_pids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_pids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_topics COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_topics COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.publication_citations COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.publication_citations COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file + and p.datainfo.deletedbyinference = false; \ 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 7442b7c10..5da028304 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 @@ -116,6 +116,13 @@ 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_country_oa as select * from SOURCE.indi_dataset_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_country_oa as select * from SOURCE.indi_pub_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/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index 540cc03a5..953eaad6a 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -41,7 +41,7 @@ FROM ${openaire_db_name}.dataset d WHERE d.datainfo.deletedbyinference = FALSE; CREATE TABLE ${stats_db_name}.dataset_citations AS -SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result +SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.dataset d LATERAL VIEW explode(d.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" @@ -95,21 +95,4 @@ CREATE TABLE ${stats_db_name}.dataset_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false; --- --- ANALYZE TABLE ${stats_db_name}.dataset_tmp COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_tmp COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_classifications COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_classifications COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_concepts COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_concepts COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_datasources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_datasources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_languages COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_languages COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_oids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_oids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_pids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_pids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.dataset_topics COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.dataset_topics COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file +where p.datainfo.deletedbyinference = false; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index 54345e074..0210dc8cb 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -41,7 +41,7 @@ from ${openaire_db_name}.software s where s.datainfo.deletedbyinference = false; CREATE TABLE ${stats_db_name}.software_citations AS -SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT +SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.software s LATERAL VIEW explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" @@ -95,21 +95,4 @@ CREATE TABLE ${stats_db_name}.software_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false; --- --- ANALYZE TABLE ${stats_db_name}.software_tmp COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_tmp COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_classifications COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_classifications COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_concepts COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_concepts COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_datasources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_datasources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_languages COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_languages COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_oids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_oids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_pids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_pids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.software_topics COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.software_topics COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file +where p.datainfo.deletedbyinference = false; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index 36ad5d92a..f7b302186 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -41,7 +41,7 @@ WHERE o.datainfo.deletedbyinference = FALSE; -- Otherresearchproduct_citations CREATE TABLE ${stats_db_name}.otherresearchproduct_citations AS -SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT +SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" and o.datainfo.deletedbyinference = false; @@ -86,21 +86,4 @@ where p.datainfo.deletedbyinference = false; CREATE TABLE ${stats_db_name}.otherresearchproduct_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject -where p.datainfo.deletedbyinference = false; - --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_tmp COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_tmp COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_classifications COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_classifications COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_concepts COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_concepts COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_datasources COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_datasources COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_languages COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_languages COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_oids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_oids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_pids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_pids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_topics COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.otherresearchproduct_topics COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file +where p.datainfo.deletedbyinference = false; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index 5d81e97bb..378e0f17b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -13,11 +13,17 @@ WHERE r.reltype = 'projectOrganization' and r.datainfo.deletedbyinference = false; CREATE TABLE ${stats_db_name}.project_results AS -SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result +SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result, r.datainfo.provenanceaction.classname as provenance FROM ${openaire_db_name}.relation r WHERE r.reltype = 'resultProject' and r.datainfo.deletedbyinference = false; +create table ${stats_db_name}.project_classification as +select substr(p.id, 4) as id, class.h2020programme.code, class.level1, class.level2, class.level3 +from ${openaire_db_name}.project p + lateral view explode(p.h2020classification) classifs as class +where p.datainfo.deletedbyinference=false and class.h2020programme is not null; + CREATE TABLE ${stats_db_name}.project_tmp ( id STRING, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index ae540b9b2..b3cbc9b41 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -130,12 +130,7 @@ WHERE r.reltype = 'resultOrganization' and r.datainfo.deletedbyinference = false; CREATE TABLE ${stats_db_name}.result_projects AS -select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend +select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id = pr.result - JOIN ${stats_db_name}.project_tmp p ON p.id = pr.id; - --- ANALYZE TABLE ${stats_db_name}.result_organization COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.result_organization COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.result_projects COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.result_projects COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file + JOIN ${stats_db_name}.project_tmp p ON p.id = pr.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/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index de0fedd7e..76d31eb5e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -17,7 +17,9 @@ CREATE TABLE ${stats_db_name}.datasource_tmp `latitude` STRING, `longitude` STRING, `websiteurl` STRING, - `compatibility` STRING + `compatibility` STRING, + issn_printed STRING, + issn_online STRING ) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties ('transactional' = 'true'); -- Insert statement that takes into account the piwik_id of the openAIRE graph @@ -32,7 +34,9 @@ SELECT substr(d1.id, 4) AS id, d1.latitude.value AS latitude, d1.longitude.value AS longitude, d1.websiteurl.value AS websiteurl, - d1.openairecompatibility.classid AS compatibility + d1.openairecompatibility.classid AS compatibility, + d1.journal.issnprinted AS issn_printed, + d1.journal.issnonline AS issn_online FROM ${openaire_db_name}.datasource d1 LEFT OUTER JOIN (SELECT id, split(originalidd, '\\:')[1] as piwik_id @@ -51,7 +55,7 @@ CREATE TABLE ${stats_db_name}.dual INSERT INTO ${stats_db_name}.dual VALUES ('X'); INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, - `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) + `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`, `issn_printed`, `issn_online`) SELECT 'other', 'Other', 'Repository', @@ -62,7 +66,9 @@ SELECT 'other', NULL, NULL, NULL, - 'unknown' + 'unknown', + null, + null FROM ${stats_db_name}.dual WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name = 'Unknown Repository'); DROP TABLE ${stats_db_name}.dual; @@ -97,13 +103,4 @@ where d.datainfo.deletedbyinference = false; CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result -FROM ${stats_db_name}.result_datasources; - --- ANALYZE TABLE ${stats_db_name}.datasource_tmp COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.datasource_tmp COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.datasource_languages COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.datasource_languages COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.datasource_oids COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.datasource_oids COMPUTE STATISTICS FOR COLUMNS; --- ANALYZE TABLE ${stats_db_name}.datasource_organizations COMPUTE STATISTICS; --- ANALYZE TABLE ${stats_db_name}.datasource_organizations COMPUTE STATISTICS FOR COLUMNS; \ No newline at end of file +FROM ${stats_db_name}.result_datasources; \ No newline at end of file diff --git a/pom.xml b/pom.xml index 075a45c9f..433c88093 100644 --- a/pom.xml +++ b/pom.xml @@ -741,7 +741,7 @@ 3.3.3 3.4.2 [2.12,3.0) - [2.6.14] + [2.7.15] [4.0.3] [6.0.5] [3.1.6]