forked from D-Net/dnet-hadoop
merge branch with master
This commit is contained in:
commit
2d67476417
|
@ -1,30 +0,0 @@
|
|||
|
||||
package eu.dnetlib.dhp.schema.scholexplorer;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
|
||||
public class DLIRelation extends Relation {
|
||||
|
||||
private String dateOfCollection;
|
||||
|
||||
private List<KeyValue> collectedFrom;
|
||||
|
||||
public List<KeyValue> getCollectedFrom() {
|
||||
return collectedFrom;
|
||||
}
|
||||
|
||||
public void setCollectedFrom(List<KeyValue> collectedFrom) {
|
||||
this.collectedFrom = collectedFrom;
|
||||
}
|
||||
|
||||
public String getDateOfCollection() {
|
||||
return dateOfCollection;
|
||||
}
|
||||
|
||||
public void setDateOfCollection(String dateOfCollection) {
|
||||
this.dateOfCollection = dateOfCollection;
|
||||
}
|
||||
}
|
|
@ -2,10 +2,8 @@
|
|||
package eu.dnetlib.dhp.schema.scholexplorer;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
|
@ -78,6 +76,25 @@ public class DLIUnknown extends Oaf implements Serializable {
|
|||
if ("complete".equalsIgnoreCase(p.completionStatus))
|
||||
completionStatus = "complete";
|
||||
dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom());
|
||||
if (StringUtils.isEmpty(id) && StringUtils.isNoneEmpty(p.getId()))
|
||||
id = p.getId();
|
||||
if (StringUtils.isEmpty(dateofcollection) && StringUtils.isNoneEmpty(p.getDateofcollection()))
|
||||
dateofcollection = p.getDateofcollection();
|
||||
|
||||
if (StringUtils.isEmpty(dateoftransformation) && StringUtils.isNoneEmpty(p.getDateoftransformation()))
|
||||
dateofcollection = p.getDateoftransformation();
|
||||
pid = mergeLists(pid, p.getPid());
|
||||
}
|
||||
|
||||
protected <T> List<T> mergeLists(final List<T>... lists) {
|
||||
|
||||
return Arrays
|
||||
.stream(lists)
|
||||
.filter(Objects::nonNull)
|
||||
.flatMap(List::stream)
|
||||
.filter(Objects::nonNull)
|
||||
.distinct()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private List<ProvenaceInfo> mergeProvenance(
|
||||
|
|
|
@ -100,7 +100,7 @@ public class DedupRecordFactory {
|
|||
.forEach(
|
||||
pub -> {
|
||||
try {
|
||||
Publication publication = mapper.readValue(pub, Publication.class);
|
||||
DLIPublication publication = mapper.readValue(pub, DLIPublication.class);
|
||||
|
||||
p.mergeFrom(publication);
|
||||
p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor()));
|
||||
|
|
|
@ -55,6 +55,7 @@ public class SparkCreateSimRels {
|
|||
.as(Encoders.kryo(Oaf.class))
|
||||
.map((MapFunction<Oaf, String>) p -> new ObjectMapper().writeValueAsString(p), Encoders.STRING())
|
||||
.javaRDD()
|
||||
.repartition(1000)
|
||||
.mapToPair(
|
||||
s -> {
|
||||
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||
|
|
|
@ -1,21 +1,16 @@
|
|||
|
||||
package eu.dnetlib.dedup.sx;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkPropagateRelationsJob {
|
||||
|
@ -44,37 +39,39 @@ public class SparkPropagateRelationsJob {
|
|||
.as(Encoders.bean(Relation.class))
|
||||
.where("relClass == 'merges'");
|
||||
|
||||
final Dataset<DLIRelation> rels = spark
|
||||
final Dataset<Relation> rels = spark
|
||||
.read()
|
||||
.load(relationPath)
|
||||
.as(Encoders.kryo(DLIRelation.class))
|
||||
.as(Encoders.kryo(Relation.class))
|
||||
.map(
|
||||
(MapFunction<DLIRelation, DLIRelation>) r -> r,
|
||||
Encoders.bean(DLIRelation.class));
|
||||
(MapFunction<Relation, Relation>) r -> r,
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
final Dataset<DLIRelation> firstJoin = rels
|
||||
final Dataset<Relation> firstJoin = rels
|
||||
.joinWith(merge, merge.col("target").equalTo(rels.col("source")), "left_outer")
|
||||
.map(
|
||||
(MapFunction<Tuple2<DLIRelation, Relation>, DLIRelation>) r -> {
|
||||
(MapFunction<Tuple2<Relation, Relation>, Relation>) r -> {
|
||||
final Relation mergeRelation = r._2();
|
||||
final DLIRelation relation = r._1();
|
||||
final Relation relation = r._1();
|
||||
if (mergeRelation != null)
|
||||
relation.setSource(mergeRelation.getSource());
|
||||
if (relation.getDataInfo() == null)
|
||||
relation.setDataInfo(OafUtils.generateDataInfo("0.9", false));
|
||||
return relation;
|
||||
},
|
||||
Encoders.bean(DLIRelation.class));
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
final Dataset<DLIRelation> secondJoin = firstJoin
|
||||
final Dataset<Relation> secondJoin = firstJoin
|
||||
.joinWith(merge, merge.col("target").equalTo(firstJoin.col("target")), "left_outer")
|
||||
.map(
|
||||
(MapFunction<Tuple2<DLIRelation, Relation>, DLIRelation>) r -> {
|
||||
(MapFunction<Tuple2<Relation, Relation>, Relation>) r -> {
|
||||
final Relation mergeRelation = r._2();
|
||||
final DLIRelation relation = r._1();
|
||||
final Relation relation = r._1();
|
||||
if (mergeRelation != null)
|
||||
relation.setTarget(mergeRelation.getSource());
|
||||
return relation;
|
||||
},
|
||||
Encoders.kryo(DLIRelation.class));
|
||||
Encoders.kryo(Relation.class));
|
||||
|
||||
secondJoin.write().mode(SaveMode.Overwrite).save(targetRelPath);
|
||||
}
|
||||
|
|
|
@ -2,7 +2,7 @@ package eu.dnetlib.dedup.sx
|
|||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation, DLIUnknown, OafUtils}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown, OafUtils}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.LoggerFactory
|
||||
|
@ -11,7 +11,7 @@ import org.apache.spark.sql.functions.col
|
|||
object SparkUpdateEntityWithDedupInfo {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkUpdateEntityWithDedupInfo.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/argumentparser/input_extract_entities_parameters.json")))
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkUpdateEntityWithDedupInfo.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json")))
|
||||
val logger = LoggerFactory.getLogger(SparkUpdateEntityWithDedupInfo.getClass)
|
||||
parser.parseArgument(args)
|
||||
|
||||
|
@ -24,7 +24,7 @@ object SparkUpdateEntityWithDedupInfo {
|
|||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||
implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
|
||||
implicit val dlirelEncoder: Encoder[DLIRelation] = Encoders.kryo[DLIRelation]
|
||||
|
||||
|
||||
|
||||
val spark: SparkSession = SparkSession
|
||||
|
|
|
@ -53,6 +53,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
|
@ -77,6 +78,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
|
@ -101,6 +103,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
|
@ -125,6 +128,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
|
@ -149,6 +153,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
|
|
|
@ -24,7 +24,9 @@ public abstract class AbstractMdRecordToOafMapper {
|
|||
private final boolean invisible;
|
||||
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_4 = "http://datacite.org/schema/kernel-4";
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_4_SLASH = "http://datacite.org/schema/kernel-4/";
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3";
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_3_SLASH = "http://datacite.org/schema/kernel-3/";
|
||||
protected static final Qualifier ORCID_PID_TYPE = qualifier(
|
||||
"ORCID", "Open Researcher and Contributor ID", DNET_PID_TYPES, DNET_PID_TYPES);
|
||||
protected static final Qualifier MAG_PID_TYPE = qualifier(
|
||||
|
@ -55,7 +57,11 @@ public abstract class AbstractMdRecordToOafMapper {
|
|||
DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
|
||||
|
||||
final Document doc = DocumentHelper
|
||||
.parseText(xml.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3));
|
||||
.parseText(
|
||||
xml
|
||||
.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3)
|
||||
.replaceAll(DATACITE_SCHEMA_KERNEL_4_SLASH, DATACITE_SCHEMA_KERNEL_3)
|
||||
.replaceAll(DATACITE_SCHEMA_KERNEL_3_SLASH, DATACITE_SCHEMA_KERNEL_3));
|
||||
|
||||
final KeyValue collectedFrom = getProvenanceDatasource(
|
||||
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
||||
|
|
|
@ -16,6 +16,8 @@ import org.apache.commons.lang3.StringUtils;
|
|||
import org.dom4j.Document;
|
||||
import org.dom4j.Node;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import eu.dnetlib.dhp.common.PacePerson;
|
||||
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
|
@ -366,7 +368,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
|
|||
|
||||
@Override
|
||||
protected List<StructuredProperty> prepareResultPids(final Document doc, final DataInfo info) {
|
||||
final List<StructuredProperty> res = new ArrayList<>();
|
||||
final Set<StructuredProperty> res = new HashSet();
|
||||
res
|
||||
.addAll(
|
||||
prepareListStructPropsWithValidQualifier(
|
||||
|
@ -382,7 +384,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
|
|||
doc,
|
||||
"//datacite:alternateIdentifier[@alternateIdentifierType != 'URL' and @alternateIdentifierType != 'landingPage']",
|
||||
"@alternateIdentifierType", DNET_PID_TYPES, info));
|
||||
return res;
|
||||
return Lists.newArrayList(res);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
package eu.dnetlib.dhp.sx.ebi
|
||||
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation, DLIUnknown}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
|
||||
import org.apache.spark.sql.{Encoder, Encoders}
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
|
||||
|
@ -168,26 +168,7 @@ object EBIAggregator {
|
|||
}
|
||||
|
||||
|
||||
def getDLIRelationAggregator(): Aggregator[(String, DLIRelation), DLIRelation, DLIRelation] = new Aggregator[(String, DLIRelation), DLIRelation, DLIRelation]{
|
||||
|
||||
override def zero: DLIRelation = new DLIRelation()
|
||||
|
||||
override def reduce(b: DLIRelation, a: (String, DLIRelation)): DLIRelation = {
|
||||
a._2
|
||||
}
|
||||
|
||||
|
||||
override def merge(a: DLIRelation, b: DLIRelation): DLIRelation = {
|
||||
if(b!= null) b else a
|
||||
}
|
||||
override def finish(reduction: DLIRelation): DLIRelation = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[DLIRelation] =
|
||||
Encoders.kryo(classOf[DLIRelation])
|
||||
|
||||
override def outputEncoder: Encoder[DLIRelation] =
|
||||
Encoders.kryo(classOf[DLIRelation])
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
package eu.dnetlib.dhp.sx.ebi
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, Instance, Journal, KeyValue, Oaf, Publication, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, Instance, Journal, KeyValue, Oaf, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils.createQualifier
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation, OafUtils, ProvenaceInfo}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, OafUtils, ProvenaceInfo}
|
||||
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal}
|
||||
import eu.dnetlib.dhp.utils.DHPUtils
|
||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||
|
@ -115,8 +115,8 @@ case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:Strin
|
|||
val dnetPublicationId = s"50|${DHPUtils.md5(s"$pmid::pmid")}"
|
||||
|
||||
targets.flatMap(l => {
|
||||
val relation = new DLIRelation
|
||||
val inverseRelation = new DLIRelation
|
||||
val relation = new Relation
|
||||
val inverseRelation = new Relation
|
||||
val targetDnetId = s"50|${DHPUtils.md5(s"${l.tpid.toLowerCase.trim}::${l.tpidType.toLowerCase.trim}")}"
|
||||
val relInfo = relationMapper.get(l.relation.toLowerCase)
|
||||
val relationSemantic = relInfo.getOriginal
|
||||
|
@ -177,7 +177,7 @@ case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:Strin
|
|||
val workingPath = parser.get("workingPath")
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
implicit val oafpubEncoder: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
implicit val relEncoder: Encoder[DLIRelation] = Encoders.kryo(classOf[DLIRelation])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
||||
implicit val atEncoder: Encoder[Author] = Encoders.kryo(classOf[Author])
|
||||
|
@ -197,7 +197,7 @@ case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:Strin
|
|||
|
||||
val oDataset:Dataset[Oaf] = spark.read.load(s"$workingPath/baseline_links_updates_oaf").as[Oaf]
|
||||
|
||||
oDataset.filter(p =>p.isInstanceOf[DLIRelation]).map(p => p.asInstanceOf[DLIRelation]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_relation")
|
||||
oDataset.filter(p =>p.isInstanceOf[Relation]).map(p => p.asInstanceOf[Relation]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_relation")
|
||||
oDataset.filter(p =>p.isInstanceOf[DLIDataset]).map(p => p.asInstanceOf[DLIDataset]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_dataset")
|
||||
|
||||
|
||||
|
@ -230,14 +230,14 @@ case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:Strin
|
|||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset_ebi")
|
||||
|
||||
|
||||
val rel: Dataset[DLIRelation] = spark.read.load(s"$workingPath/relation").as[DLIRelation]
|
||||
val relupdate : Dataset[DLIRelation] = spark.read.load(s"$workingPath/ebi_garr/baseline_links_updates_relation").as[DLIRelation]
|
||||
val rel: Dataset[Relation] = spark.read.load(s"$workingPath/relation").as[Relation]
|
||||
val relupdate : Dataset[Relation] = spark.read.load(s"$workingPath/ebi_garr/baseline_links_updates_relation").as[Relation]
|
||||
|
||||
|
||||
rel.union(relupdate)
|
||||
.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIRelationAggregator().toColumn)
|
||||
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite)
|
||||
.save(s"$workingPath/baseline_relation_ebi")
|
||||
|
|
|
@ -2,7 +2,7 @@ package eu.dnetlib.dhp.sx.ebi
|
|||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
|
||||
import eu.dnetlib.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser}
|
||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||
import org.apache.commons.io.IOUtils
|
||||
|
@ -38,7 +38,7 @@ object SparkCreateEBIDataFrame {
|
|||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
implicit val datasetEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
||||
implicit val relEncoder: Encoder[DLIRelation] = Encoders.kryo(classOf[DLIRelation])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||
|
||||
// logger.info("Extract Publication and relation from publication_xml")
|
||||
// val oafPubsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/publication_xml").map(s =>
|
||||
|
@ -63,7 +63,7 @@ object SparkCreateEBIDataFrame {
|
|||
// spark.createDataset(oafDatsRDD).write.mode(SaveMode.Append).save(s"$workingPath/oaf")
|
||||
val dataset: Dataset[DLIDataset] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[DLIDataset]).map(d => d.asInstanceOf[DLIDataset])
|
||||
val publication: Dataset[DLIPublication] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[DLIPublication]).map(d => d.asInstanceOf[DLIPublication])
|
||||
val relations: Dataset[DLIRelation] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[DLIRelation]).map(d => d.asInstanceOf[DLIRelation])
|
||||
val relations: Dataset[Relation] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[Relation]).map(d => d.asInstanceOf[Relation])
|
||||
publication.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, pubEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIPublicationAggregator().toColumn)
|
||||
|
@ -78,7 +78,7 @@ object SparkCreateEBIDataFrame {
|
|||
|
||||
relations.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIRelationAggregator().toColumn)
|
||||
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/relation")
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ import org.apache.spark.sql.Encoders;
|
|||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
|
@ -55,18 +55,18 @@ public class SparkSXGeneratePidSimlarity {
|
|||
.equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::")))
|
||||
.distinct();
|
||||
|
||||
JavaRDD<DLIRelation> simRel = datasetSimRel
|
||||
JavaRDD<Relation> simRel = datasetSimRel
|
||||
.union(publicationSimRel)
|
||||
.map(
|
||||
s -> {
|
||||
final DLIRelation r = new DLIRelation();
|
||||
final Relation r = new Relation();
|
||||
r.setSource(s._1());
|
||||
r.setTarget(s._2());
|
||||
r.setRelType("similar");
|
||||
return r;
|
||||
});
|
||||
spark
|
||||
.createDataset(simRel.rdd(), Encoders.bean(DLIRelation.class))
|
||||
.createDataset(simRel.rdd(), Encoders.bean(Relation.class))
|
||||
.distinct()
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
|
|
|
@ -31,7 +31,6 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import net.minidev.json.JSONArray;
|
||||
|
@ -156,9 +155,9 @@ public class SparkScholexplorerCreateRawGraphJob {
|
|||
SparkSXGeneratePidSimlarity
|
||||
.generateDataFrame(
|
||||
spark, sc, inputPath.replace("/relation", ""), targetPath.replace("/relation", ""));
|
||||
RDD<DLIRelation> rdd = union
|
||||
RDD<Relation> rdd = union
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, DLIRelation>) f -> {
|
||||
(PairFunction<String, String, Relation>) f -> {
|
||||
final String source = getJPathString(SOURCEJSONPATH, f);
|
||||
final String target = getJPathString(TARGETJSONPATH, f);
|
||||
final String reltype = getJPathString(RELJSONPATH, f);
|
||||
|
@ -175,7 +174,7 @@ public class SparkScholexplorerCreateRawGraphJob {
|
|||
source.toLowerCase(),
|
||||
reltype.toLowerCase(),
|
||||
target.toLowerCase())),
|
||||
mapper.readValue(f, DLIRelation.class));
|
||||
mapper.readValue(f, Relation.class));
|
||||
})
|
||||
.reduceByKey(
|
||||
(a, b) -> {
|
||||
|
@ -186,7 +185,7 @@ public class SparkScholexplorerCreateRawGraphJob {
|
|||
.rdd();
|
||||
|
||||
spark
|
||||
.createDataset(rdd, Encoders.bean(DLIRelation.class))
|
||||
.createDataset(rdd, Encoders.bean(Relation.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(targetPath);
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation, DLIUnknown}
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
|
||||
import eu.dnetlib.dhp.sx.ebi.EBIAggregator
|
||||
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal}
|
||||
import org.apache.commons.io.IOUtils
|
||||
|
@ -12,7 +12,7 @@ import org.slf4j.LoggerFactory
|
|||
object SparkSplitOafTODLIEntities {
|
||||
|
||||
|
||||
def getKeyRelation(rel:DLIRelation):String = {
|
||||
def getKeyRelation(rel:Relation):String = {
|
||||
s"${rel.getSource}::${rel.getRelType}::${rel.getTarget}"
|
||||
|
||||
|
||||
|
@ -30,13 +30,14 @@ object SparkSplitOafTODLIEntities {
|
|||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||
implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
|
||||
implicit val relEncoder: Encoder[DLIRelation] = Encoders.kryo[DLIRelation]
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo[Relation]
|
||||
|
||||
|
||||
|
||||
val spark:SparkSession = SparkSession
|
||||
.builder()
|
||||
.appName(SparkSplitOafTODLIEntities.getClass.getSimpleName)
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate()
|
||||
|
||||
|
@ -47,7 +48,7 @@ object SparkSplitOafTODLIEntities {
|
|||
|
||||
val ebi_dataset:Dataset[DLIDataset] = spark.read.load(s"$workingPath/ebi/baseline_dataset_ebi").as[DLIDataset]
|
||||
val ebi_publication:Dataset[DLIPublication] = spark.read.load(s"$workingPath/ebi/baseline_publication_ebi").as[DLIPublication]
|
||||
val ebi_relation:Dataset[DLIRelation] = spark.read.load(s"$workingPath/ebi/baseline_relation_ebi").as[DLIRelation]
|
||||
val ebi_relation:Dataset[Relation] = spark.read.load(s"$workingPath/ebi/baseline_relation_ebi").as[Relation]
|
||||
|
||||
|
||||
|
||||
|
@ -86,12 +87,12 @@ object SparkSplitOafTODLIEntities {
|
|||
|
||||
|
||||
OAFDataset
|
||||
.filter(s => s != null && s.isInstanceOf[DLIRelation])
|
||||
.map(s =>s.asInstanceOf[DLIRelation])
|
||||
.filter(s => s != null && s.isInstanceOf[Relation])
|
||||
.map(s =>s.asInstanceOf[Relation])
|
||||
.union(ebi_relation)
|
||||
.map(d => (getKeyRelation(d), d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIRelationAggregator().toColumn)
|
||||
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.repartition(1000)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/graph/relation")
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
|
||||
import eu.dnetlib.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser}
|
||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||
import org.apache.commons.io.IOUtils
|
||||
|
@ -40,7 +40,7 @@ object SparkXMLToOAFDataset {
|
|||
implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf]
|
||||
implicit val datasetEncoder:Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||
implicit val publicationEncoder:Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||
implicit val relationEncoder:Encoder[DLIRelation] = Encoders.kryo[DLIRelation]
|
||||
implicit val relationEncoder:Encoder[Relation] = Encoders.kryo[Relation]
|
||||
|
||||
val relationMapper = RelationMapper.load
|
||||
|
||||
|
|
|
@ -14,7 +14,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
|
||||
import eu.dnetlib.dhp.parser.utility.VtdUtilityParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
|
@ -175,8 +174,8 @@ public abstract class AbstractScholexplorerParser {
|
|||
.stream()
|
||||
.flatMap(
|
||||
n -> {
|
||||
final List<DLIRelation> rels = new ArrayList<>();
|
||||
DLIRelation r = new DLIRelation();
|
||||
final List<Relation> rels = new ArrayList<>();
|
||||
Relation r = new Relation();
|
||||
r.setSource(parsedObject.getId());
|
||||
final String relatedPid = n.getTextValue();
|
||||
final String relatedPidType = n.getAttributes().get("relatedIdentifierType");
|
||||
|
@ -184,7 +183,6 @@ public abstract class AbstractScholexplorerParser {
|
|||
String relationSemantic = n.getAttributes().get("relationType");
|
||||
String inverseRelation;
|
||||
final String targetId = generateId(relatedPid, relatedPidType, relatedType);
|
||||
r.setDateOfCollection(dateOfCollection);
|
||||
if (relationMapper.containsKey(relationSemantic.toLowerCase())) {
|
||||
RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase());
|
||||
relationSemantic = relInfo.getOriginal();
|
||||
|
@ -199,14 +197,13 @@ public abstract class AbstractScholexplorerParser {
|
|||
r.setCollectedfrom(parsedObject.getCollectedfrom());
|
||||
r.setDataInfo(di);
|
||||
rels.add(r);
|
||||
r = new DLIRelation();
|
||||
r = new Relation();
|
||||
r.setDataInfo(di);
|
||||
r.setSource(targetId);
|
||||
r.setTarget(parsedObject.getId());
|
||||
r.setRelType(inverseRelation);
|
||||
r.setRelClass("datacite");
|
||||
r.setCollectedfrom(parsedObject.getCollectedfrom());
|
||||
r.setDateOfCollection(dateOfCollection);
|
||||
rels.add(r);
|
||||
if ("unknown".equalsIgnoreCase(relatedType))
|
||||
result
|
||||
|
|
|
@ -276,7 +276,6 @@ public class MappersTest {
|
|||
System.out.println("***************");
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
void testClaimDedup() throws IOException {
|
||||
final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_dedup.xml"));
|
||||
|
@ -287,6 +286,29 @@ public class MappersTest {
|
|||
System.out.println("***************");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNakala() throws IOException {
|
||||
final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_nakala.xml"));
|
||||
final List<Oaf> list = new OdfToOafMapper(vocs, false).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(0, d.getSubject().size());
|
||||
assertEquals(1, d.getInstance().size());
|
||||
assertEquals(1, d.getPid().size());
|
||||
}
|
||||
|
||||
private void assertValidId(final String id) {
|
||||
assertEquals(49, id.length());
|
||||
assertEquals('|', id.charAt(2));
|
||||
|
|
|
@ -951,6 +951,7 @@ dnet:countries @=@ ZW @=@ ABW
|
|||
dnet:protocols @=@ oai @=@ OAI-PMH
|
||||
dnet:protocols @=@ oai @=@ OAI_PMH
|
||||
dnet:pid_types @=@ orcid @=@ ORCID12
|
||||
dnet:pid_types @=@ handle @=@ hdl
|
||||
dnet:review_levels @=@ 0000 @=@ UNKNOWN
|
||||
dnet:review_levels @=@ 0002 @=@ 80 大阪経大学会「Working Paper」
|
||||
dnet:review_levels @=@ 0002 @=@ AO
|
||||
|
|
|
@ -1045,6 +1045,7 @@ dnet:pid_types @=@ dnet:pid_types @=@ pmid @=@ pmid
|
|||
dnet:pid_types @=@ dnet:pid_types @=@ urn @=@ urn
|
||||
dnet:pid_types @=@ dnet:pid_types @=@ who @=@ WHO Identifier
|
||||
dnet:pid_types @=@ dnet:pid_types @=@ drks @=@ DRKS Identifier
|
||||
dnet:pid_types @=@ dnet:pid_types @=@ handle @=@ Handle
|
||||
dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/ACM @=@ An ACM classification term that can be associated to your publications
|
||||
dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/ARXIV @=@ An ARXIV classification term that can be associated to your publications
|
||||
dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/DDC @=@ A Dewey Decimal classification term (DDC) that can be associated to your publications
|
||||
|
|
|
@ -0,0 +1,88 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
|
||||
xmlns:oaf="http://namespace.openaire.eu/oaf" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<oai:header xmlns="http://namespace.openaire.eu/"
|
||||
xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
|
||||
<dri:objIdentifier>r3f5b9831893::cca7367159bc3ff90cd2f75bf9dc21c4</dri:objIdentifier>
|
||||
<dri:recordIdentifier>oai:nakala.fr:hdl_11280_847e01df</dri:recordIdentifier>
|
||||
<dri:dateOfCollection>2020-08-01T00:16:24.742Z</dri:dateOfCollection>
|
||||
<oaf:datasourceprefix>r3f5b9831893</oaf:datasourceprefix>
|
||||
<identifier xmlns="http://www.openarchives.org/OAI/2.0/">oai:nakala.fr:hdl_11280_847e01df</identifier>
|
||||
<datestamp xmlns="http://www.openarchives.org/OAI/2.0/">2020-06-08T01:01:38Z</datestamp>
|
||||
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_2b09fc10</setSpec>
|
||||
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_c1bc48d0</setSpec>
|
||||
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_57c8db3a</setSpec>
|
||||
<dr:dateOfTransformation>2020-08-01T00:31:35.625Z</dr:dateOfTransformation>
|
||||
</oai:header>
|
||||
<metadata>
|
||||
<datacite:resource xmlns="http://www.openarchives.org/OAI/2.0/"
|
||||
xmlns:datacite="http://datacite.org/schema/kernel-4"
|
||||
xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:oai="http://www.openarchives.org/OAI/2.0/"
|
||||
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://datacite.org/schema/kernel-4 https://schema.datacite.org/meta/kernel-4/metadata.xsd">
|
||||
<datacite:alternateIdentifier identifierType="URL" xmlns:datacite="http://datacite.org/schema/kernel-4/">277</datacite:alternateIdentifier>
|
||||
<datacite:identifier identifierType="Handle" xmlns:datacite="http://datacite.org/schema/kernel-4/">http://hdl.handle.net/11280/847e01df</datacite:identifier>
|
||||
<alternateIdentifiers>
|
||||
<alternateIdentifier alternateIdentifierType="URL">http://hdl.handle.net/http://hdl.handle.net/11280/847e01df</alternateIdentifier>
|
||||
</alternateIdentifiers>
|
||||
<datacite:alternateIdentifier identifierType="URL" xmlns:datacite="http://datacite.org/schema/kernel-4/">http://nakala.fr/data/11280/847e01df</datacite:alternateIdentifier>
|
||||
<datacite:creators xmlns:datacite="http://datacite.org/schema/kernel-4/">
|
||||
<datacite:creator>
|
||||
<datacite:creatorName>DHAAP</datacite:creatorName>
|
||||
</datacite:creator>
|
||||
</datacite:creators>
|
||||
<datacite:titles xmlns:datacite="http://datacite.org/schema/kernel-4/">
|
||||
<datacite:title>CVP_Notice277-1 place du Docteur Antoine Béclère _PHO02.jpg</datacite:title>
|
||||
</datacite:titles>
|
||||
<datacite:descriptions xmlns:datacite="http://datacite.org/schema/kernel-4/">
|
||||
<datacite:description descriptionType="Abstract">Hôpital Saint-Antoine. Fragment de dalle funéraire trouvée en décembre 1932. Paris (XIIème arr.). Photographie d'Albert Citerne (1876-1970). Plaque de verre, 1932. Département Histoire de l'Architecture et Archéologie de Paris.</datacite:description>
|
||||
<datacite:description descriptionType="Abstract">Nfa_1146</datacite:description>
|
||||
<datacite:description descriptionType="Abstract">Hôpital Saint-Antoine. Fragment de dalle funéraire trouvée en décembre 1932. Paris (XIIème arr.). Photographie d'Albert Citerne (1876-1970). Plaque de verre, 1932. Département Histoire de l'Architecture et Archéologie de Paris.</datacite:description>
|
||||
</datacite:descriptions>
|
||||
<datacite:publisher xmlns:datacite="http://datacite.org/schema/kernel-4/">Nakala by Huma-Num</datacite:publisher>
|
||||
<datacite:contributors xmlns:datacite="http://datacite.org/schema/kernel-4/">
|
||||
<datacite:contributor contributorType="Other">
|
||||
<datacite:contributorName>DHAAP, Pôle Archéologique</datacite:contributorName>
|
||||
</datacite:contributor>
|
||||
</datacite:contributors>
|
||||
<datacite:dates xmlns:datacite="http://datacite.org/schema/kernel-4/">
|
||||
<datacite:date dateType="Created">1932</datacite:date>
|
||||
</datacite:dates>
|
||||
<datacite:resourceType resourceTypeGeneral="Image" xmlns:datacite="http://datacite.org/schema/kernel-4/">StillImage</datacite:resourceType>
|
||||
<datacite:rightsList xmlns:datacite="http://datacite.org/schema/kernel-4/">
|
||||
<datacite:rights rightsURI="info:eu-repo/semantics/openAccess"/>
|
||||
</datacite:rightsList>
|
||||
</datacite:resource>
|
||||
<oaf:identifier identifierType="handle">http://hdl.handle.net/11280/847e01df</oaf:identifier>
|
||||
<dr:CobjCategory type="dataset">0025</dr:CobjCategory>
|
||||
<oaf:dateAccepted/>
|
||||
<oaf:accessrights>OPEN</oaf:accessrights>
|
||||
<oaf:language>und</oaf:language>
|
||||
<oaf:hostedBy id="re3data_____::r3d100012102" name="NAKALA"/>
|
||||
<oaf:collectedFrom id="re3data_____::r3d100012102" name="NAKALA"/>
|
||||
</metadata>
|
||||
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
|
||||
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
|
||||
<originDescription altered="true" harvestDate="2020-08-01T00:16:24.742Z">
|
||||
<baseURL>https%3A%2F%2Fwww.nakala.fr%2Foai_oa%2F11280%2F8892ab4b</baseURL>
|
||||
<identifier>oai:nakala.fr:hdl_11280_847e01df</identifier>
|
||||
<datestamp>2020-06-08T01:01:38Z</datestamp>
|
||||
<metadataNamespace/>
|
||||
</originDescription>
|
||||
</provenance>
|
||||
<oaf:datainfo>
|
||||
<oaf:inferred>false</oaf:inferred>
|
||||
<oaf:deletedbyinference>false</oaf:deletedbyinference>
|
||||
<oaf:trust>0.9</oaf:trust>
|
||||
<oaf:inferenceprovenance/>
|
||||
<oaf:provenanceaction classid="sysimport:crosswalk:datasetarchive"
|
||||
classname="sysimport:crosswalk:datasetarchive"
|
||||
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
|
||||
</oaf:datainfo>
|
||||
</about>
|
||||
</record>
|
|
@ -1,66 +1,53 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<oai:record xmlns:oai="http://www.openarchives.org/OAI/2.0/"
|
||||
xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:dc="http://purl.org/dc/elements/1.1/">
|
||||
<oai:header>
|
||||
<dri:repositoryId>aaadf8b3-01a8-4cc2-9964-63cfb19df3b4_UmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZXMvUmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZVR5cGU=</dri:repositoryId>
|
||||
<dri:recordIdentifier>oai:pangaea.de:doi:10.1594/PANGAEA.821876</dri:recordIdentifier>
|
||||
<dri:datasourceprefix>r3d100010134</dri:datasourceprefix>
|
||||
<dri:objIdentifier>r3d100010134::000083be706192d2d839915694ecfd47</dri:objIdentifier>
|
||||
<dri:resolvedDate>2020-01-08T04:12:12.287</dri:resolvedDate>
|
||||
<dri:dateOfCollection>2020-01-08T03:24:10.865Z</dri:dateOfCollection>
|
||||
<oaf:datasourceprefix/>
|
||||
<identifier>oai:pangaea.de:doi:10.1594/PANGAEA.821876</identifier>
|
||||
<setSpec>citable</setSpec>
|
||||
</oai:header>
|
||||
<metadata>
|
||||
<resource xmlns="http://datacite.org/schema/kernel-3" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://datacite.org/schema/kernel-3 http://schema.datacite.org/meta/kernel-3/metadata.xsd">
|
||||
<identifier identifierType="doi">10.1594/pangaea.821876</identifier>
|
||||
<creators> <creator><creatorName>Macke, Andreas</creatorName></creator><creator><creatorName>Kalisch, John</creatorName></creator> </creators>
|
||||
<titles> <title>Total Sky Imager observations during POLARSTERN cruise ANT-XXVI/4 on 2010-05-14 with links to images</title> </titles>
|
||||
|
||||
<publisher>PANGAEA - Data Publisher for Earth & Environmental Science</publisher>
|
||||
<dates>
|
||||
<date dateType="Collected">2010-05-14T00:13:47/2010-05-14T23:55:47</date>
|
||||
</dates>
|
||||
<subjects>
|
||||
|
||||
<subject subjectScheme="Parameter">DATE/TIME</subject>
|
||||
|
||||
<subject subjectScheme="Parameter">LATITUDE</subject>
|
||||
|
||||
<subject subjectScheme="Parameter">LONGITUDE</subject>
|
||||
|
||||
<subject subjectScheme="Parameter">Uniform resource locator/link to image</subject>
|
||||
|
||||
<subject subjectScheme="Method">Total Sky Imager</subject>
|
||||
|
||||
<subject subjectScheme="Campaign">ANT-XXVI/4</subject>
|
||||
|
||||
<subject subjectScheme="Basis">Polarstern</subject>
|
||||
|
||||
</subjects>
|
||||
<resourceType resourceTypeGeneral="dataset">dataset</resourceType>
|
||||
<relatedIdentifiers>
|
||||
|
||||
<relatedIdentifier relatedIdentifierType="dnet" relationType="isPartOf" inverseRelationType="hasPart" entityType="dataset">dli_resolver::cf447a378b0b6603593f8b0e57242695</relatedIdentifier>
|
||||
|
||||
<relatedIdentifier relatedIdentifierType="URL" relationType="references" inverseRelationType="isReferencedBy" entityType="unknown">http://hs.pangaea.de/images/airphoto/ps/ps75/2010-05-14/ant-xxvi_4_2010-05-14_tsi-images-links.zip</relatedIdentifier>
|
||||
|
||||
<relatedIdentifier relatedIdentifierType="dnet" relationType="references" inverseRelationType="isReferencedBy" entityType="publication">dli_resolver::f0f5975d20991cffd222c6002ddd5821</relatedIdentifier>
|
||||
|
||||
</relatedIdentifiers>
|
||||
</resource>
|
||||
</metadata>
|
||||
<oaf:about xmlns:oaf="http://namespace.dnet.eu/oaf">
|
||||
<oaf:datainfo >
|
||||
<oaf:completionStatus>complete</oaf:completionStatus>
|
||||
|
||||
<oaf:collectedFrom id="dli_________::r3d100010134" name="Pangaea" completionStatus="complete"/>
|
||||
|
||||
</oaf:datainfo>
|
||||
</oaf:about>
|
||||
|
||||
|
||||
</oai:record>
|
||||
xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:dc="http://purl.org/dc/elements/1.1/">
|
||||
<oai:header>
|
||||
<dri:repositoryId>aaadf8b3-01a8-4cc2-9964-63cfb19df3b4_UmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZXMvUmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZVR5cGU=</dri:repositoryId>
|
||||
<dri:recordIdentifier>oai:pangaea.de:doi:10.1594/PANGAEA.432865</dri:recordIdentifier>
|
||||
<dri:datasourceprefix>r3d100010134</dri:datasourceprefix>
|
||||
<dri:objIdentifier>r3d100010134::00002f60593fd1f758fb838fafb46795</dri:objIdentifier>
|
||||
<dri:dateOfCollection>2020-02-18T03:05:02.534Z</dri:dateOfCollection>
|
||||
<oaf:datasourceprefix/>
|
||||
<identifier>oai:pangaea.de:doi:10.1594/PANGAEA.432865</identifier>
|
||||
<setSpec>citable topicOceans</setSpec>
|
||||
</oai:header>
|
||||
<oai:metadata>
|
||||
<resource xmlns="http://datacite.org/schema/kernel-3">
|
||||
<identifier identifierType="doi">10.1594/pangaea.432865</identifier>
|
||||
<titles xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<title>Daily sea level from coastal tide gauge station Woods_Hole in 1978 (Research quality database)</title>
|
||||
</titles>
|
||||
<publisher xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">PANGAEA - Data Publisher for Earth & Environmental Science</publisher>
|
||||
<publicationYear xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">2006</publicationYear>
|
||||
<dates xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<date dateType="Collected">1978-01-01T12:00:00/1978-12-31T12:00:00</date>
|
||||
</dates>
|
||||
<creators xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<creator>
|
||||
<creatorName>WOCE Sea Level, WSL</creatorName>
|
||||
</creator>
|
||||
</creators>
|
||||
<subjects xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<subject subjectScheme="Parameter">DATE/TIME</subject>
|
||||
<subject subjectScheme="Parameter">Sea level</subject>
|
||||
<subject subjectScheme="Method">Tide gauge station</subject>
|
||||
<subject subjectScheme="Campaign">SeaLevel</subject>
|
||||
<subject subjectScheme="Project">World Ocean Circulation Experiment (WOCE)</subject>
|
||||
</subjects>
|
||||
<resourceType resourceTypeGeneral="Dataset"/>
|
||||
<relatedIdentifiers>
|
||||
<relatedIdentifier relatedIdentifierType="URL" relationType="isDocumentedBy"
|
||||
inverseRelationType="documents">http://store.pangaea.de/Projects/WOCE/SeaLevel_rqds/Woods_Hole.txt</relatedIdentifier>
|
||||
</relatedIdentifiers>
|
||||
</resource>
|
||||
</oai:metadata>
|
||||
<oaf:about>
|
||||
<oaf:datainfo>
|
||||
<oaf:collectedFrom completionStatus="complete" id="dli_________::r3d100010134" name="Pangaea"/>
|
||||
<oaf:completionStatus>complete</oaf:completionStatus>
|
||||
<oaf:provisionMode>collected</oaf:provisionMode>
|
||||
</oaf:datainfo>
|
||||
</oaf:about>
|
||||
</oai:record>
|
|
@ -6,7 +6,7 @@ import java.time.format.DateTimeFormatter
|
|||
import eu.dnetlib.dhp.common.PacePerson
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
|
||||
import eu.dnetlib.dhp.utils.DHPUtils
|
||||
import org.apache.commons.lang3.StringUtils
|
||||
import org.codehaus.jackson.map.ObjectMapper
|
||||
|
@ -273,29 +273,29 @@ object DLIToOAF {
|
|||
}
|
||||
|
||||
|
||||
def convertDLIRelation(r: DLIRelation): Relation = {
|
||||
|
||||
val result = new Relation
|
||||
if (!relationTypeMapping.contains(r.getRelType))
|
||||
return null
|
||||
|
||||
if (r.getCollectedFrom == null || r.getCollectedFrom.size() == 0 || (r.getCollectedFrom.size() == 1 && r.getCollectedFrom.get(0) == null))
|
||||
return null
|
||||
val t = relationTypeMapping.get(r.getRelType)
|
||||
|
||||
result.setRelType("resultResult")
|
||||
result.setRelClass(t.get._1)
|
||||
result.setSubRelType(t.get._2)
|
||||
result.setCollectedfrom(r.getCollectedFrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
|
||||
result.setSource(generateId(r.getSource))
|
||||
result.setTarget(generateId(r.getTarget))
|
||||
|
||||
if (result.getSource.equals(result.getTarget))
|
||||
return null
|
||||
result.setDataInfo(generateDataInfo())
|
||||
|
||||
result
|
||||
}
|
||||
// def convertDLIRelation(r: DLIRelation): Relation = {
|
||||
//
|
||||
// val result = new Relation
|
||||
// if (!relationTypeMapping.contains(r.getRelType))
|
||||
// return null
|
||||
//
|
||||
// if (r.getProperties == null || r.getProperties.size() == 0 || (r.getProperties.size() == 1 && r.getProperties.get(0) == null))
|
||||
// return null
|
||||
// val t = relationTypeMapping.get(r.getRelType)
|
||||
//
|
||||
// result.setRelType("resultResult")
|
||||
// result.setRelClass(t.get._1)
|
||||
// result.setSubRelType(t.get._2)
|
||||
// result.setCollectedfrom(r.getProperties.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
|
||||
// result.setSource(generateId(r.getSource))
|
||||
// result.setTarget(generateId(r.getTarget))
|
||||
//
|
||||
// if (result.getSource.equals(result.getTarget))
|
||||
// return null
|
||||
// result.setDataInfo(generateDataInfo())
|
||||
//
|
||||
// result
|
||||
// }
|
||||
|
||||
|
||||
def convertDLIDatasetTOOAF(d: DLIDataset): Dataset = {
|
||||
|
|
|
@ -2,7 +2,7 @@ package eu.dnetlib.dhp.`export`
|
|||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Instance, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.hadoop.io.Text
|
||||
import org.apache.hadoop.io.compress.GzipCodec
|
||||
|
@ -39,14 +39,13 @@ object SparkExportContentForOpenAire {
|
|||
implicit val pubEncoder: Encoder[Publication] = Encoders.bean(classOf[Publication])
|
||||
implicit val datEncoder: Encoder[OafDataset] = Encoders.bean(classOf[OafDataset])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
|
||||
implicit val dliRelEncoder: Encoder[DLIRelation] = Encoders.bean(classOf[DLIRelation])
|
||||
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
val relRDD:RDD[Relation] = sc.textFile(s"$workingPath/relation_j")
|
||||
.map(s => new ObjectMapper().readValue(s, classOf[DLIRelation]))
|
||||
.map(s => new ObjectMapper().readValue(s, classOf[Relation]))
|
||||
.filter(p => p.getDataInfo.getDeletedbyinference == false)
|
||||
.map(DLIToOAF.convertDLIRelation).filter(p=>p!= null)
|
||||
spark.createDataset(relRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationDS")
|
||||
|
||||
val datRDD:RDD[OafDataset] = sc.textFile(s"$workingPath/dataset")
|
||||
|
|
|
@ -0,0 +1,97 @@
|
|||
|
||||
package eu.dnetlib.dhp.provision;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.http.client.methods.CloseableHttpResponse;
|
||||
import org.apache.http.client.methods.HttpDelete;
|
||||
import org.apache.http.client.methods.HttpPut;
|
||||
import org.apache.http.entity.StringEntity;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.impl.client.HttpClients;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
|
||||
public class DropAndCreateESIndex {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
DropAndCreateESIndex.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/provision/dropAndCreateIndex.json")));
|
||||
parser.parseArgument(args);
|
||||
|
||||
final String index = parser.get("index");
|
||||
|
||||
final String cluster = parser.get("cluster");
|
||||
final String clusterJson = IOUtils
|
||||
.toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/cluster.json"));
|
||||
|
||||
final Map<String, String> clusterMap = new ObjectMapper().readValue(clusterJson, Map.class);
|
||||
|
||||
final String ip = clusterMap.get(cluster).split(",")[0];
|
||||
|
||||
System.out.println(ip);
|
||||
|
||||
final String url = "http://%s:9200/%s_%s";
|
||||
|
||||
CloseableHttpClient client = HttpClients.createDefault();
|
||||
|
||||
HttpDelete delete = new HttpDelete(String.format(url, ip, index, "object"));
|
||||
|
||||
CloseableHttpResponse response = client.execute(delete);
|
||||
|
||||
System.out.println("deleting Index SUMMARY");
|
||||
System.out.println(response.getStatusLine());
|
||||
client.close();
|
||||
client = HttpClients.createDefault();
|
||||
|
||||
delete = new HttpDelete(String.format(url, ip, index, "scholix"));
|
||||
|
||||
response = client.execute(delete);
|
||||
|
||||
System.out.println("deleting Index SCHOLIX");
|
||||
System.out.println(response.getStatusLine());
|
||||
client.close();
|
||||
client = HttpClients.createDefault();
|
||||
|
||||
final String summaryConf = IOUtils
|
||||
.toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/summary_index.json"));
|
||||
|
||||
final String scholixConf = IOUtils
|
||||
.toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/scholix_index.json"));
|
||||
|
||||
HttpPut put = new HttpPut(String.format(url, ip, index, "object"));
|
||||
|
||||
StringEntity entity = new StringEntity(summaryConf);
|
||||
put.setEntity(entity);
|
||||
put.setHeader("Accept", "application/json");
|
||||
put.setHeader("Content-type", "application/json");
|
||||
|
||||
System.out.println("creating First Index SUMMARY");
|
||||
response = client.execute(put);
|
||||
|
||||
client.close();
|
||||
client = HttpClients.createDefault();
|
||||
|
||||
System.out.println(response.getStatusLine());
|
||||
|
||||
System.out.println("creating Index SCHOLIX");
|
||||
put = new HttpPut(String.format(url, ip, index, "scholix"));
|
||||
|
||||
entity = new StringEntity(scholixConf);
|
||||
put.setEntity(entity);
|
||||
put.setHeader("Accept", "application/json");
|
||||
put.setHeader("Content-type", "application/json");
|
||||
|
||||
response = client.execute(put);
|
||||
System.out.println(response.getStatusLine());
|
||||
client.close();
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
package eu.dnetlib.dhp.provision
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.provision.scholix.Scholix
|
||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
|
||||
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.codehaus.jackson.map.ObjectMapper
|
||||
|
||||
object SparkConvertDatasetToJson {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertDatasetToJson.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/dataset2Json.json")))
|
||||
parser.parseArgument(args)
|
||||
val conf = new SparkConf
|
||||
val spark = SparkSession.builder.config(conf).appName(SparkConvertDatasetToJson.getClass.getSimpleName).master(parser.get("master")).getOrCreate
|
||||
|
||||
implicit val summaryEncoder: Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
|
||||
implicit val scholixEncoder: Encoder[Scholix] = Encoders.kryo[Scholix]
|
||||
|
||||
|
||||
val workingPath = parser.get("workingPath")
|
||||
|
||||
|
||||
|
||||
spark.read.load(s"$workingPath/summary").as[ScholixSummary]
|
||||
.map(s => new ObjectMapper().writeValueAsString(s))(Encoders.STRING)
|
||||
.rdd.repartition(500).saveAsTextFile(s"$workingPath/summary_json", classOf[GzipCodec])
|
||||
|
||||
spark.read.load(s"$workingPath/scholix").as[Scholix]
|
||||
.map(s => new ObjectMapper().writeValueAsString(s))(Encoders.STRING)
|
||||
.rdd.repartition(2000).saveAsTextFile(s"$workingPath/scholix_json", classOf[GzipCodec])
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,34 +0,0 @@
|
|||
|
||||
package eu.dnetlib.dhp.provision;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.sql.*;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
|
||||
/**
|
||||
* SparkExtractRelationCount is a spark job that takes in input relation RDD and retrieve for each item in relation
|
||||
* which are the number of - Related Dataset - Related Publication - Related Unknown
|
||||
*/
|
||||
public class SparkExtractRelationCount {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkExtractRelationCount.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/provision/input_related_entities_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkExtractRelationCount.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final String workingDirPath = parser.get("workingDirPath");
|
||||
|
||||
final String relationPath = parser.get("relationPath");
|
||||
DatasetJoiner.startJoin(spark, relationPath, workingDirPath + "/relatedItemCount");
|
||||
}
|
||||
}
|
|
@ -1,12 +1,32 @@
|
|||
package eu.dnetlib.dhp.provision
|
||||
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.functions.{coalesce, col, count, lit}
|
||||
|
||||
object DatasetJoiner {
|
||||
|
||||
def startJoin(spark: SparkSession, relPath:String, targetPath:String) {
|
||||
val relation = spark.read.load(relPath)
|
||||
/**
|
||||
* SparkExtractRelationCount is a spark job that takes in input relation RDD and retrieve for each item in relation
|
||||
* which are the number of - Related Dataset - Related Publication - Related Unknown
|
||||
*/
|
||||
object SparkExtractRelationCount {
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkExtractRelationCount.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/input_related_entities_parameters.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark = SparkSession.builder.appName(SparkExtractRelationCount.getClass.getSimpleName).master(parser.get("master")).getOrCreate
|
||||
|
||||
val workingDirPath = parser.get("workingDirPath")
|
||||
|
||||
val relationPath = parser.get("relationPath")
|
||||
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo[Relation]
|
||||
|
||||
val relation = spark.read.load(relationPath).as[Relation].map(r =>r)(Encoders.bean(classOf[Relation]))
|
||||
|
||||
val relatedPublication = relation
|
||||
.where("target like '50%'")
|
||||
|
@ -34,7 +54,7 @@ object DatasetJoiner {
|
|||
coalesce(col("dataset"),lit(0)).alias("relatedDataset"),
|
||||
coalesce(col("unknown"),lit(0)).alias("relatedUnknown")
|
||||
)
|
||||
firstJoin.write.mode("overwrite").save(targetPath)
|
||||
firstJoin.write.mode(SaveMode.Overwrite).save(s"$workingDirPath/relatedItemCount")
|
||||
}
|
||||
|
||||
}
|
|
@ -1,109 +0,0 @@
|
|||
|
||||
package eu.dnetlib.dhp.provision;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.provision.scholix.*;
|
||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkGenerateScholix {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkGenerateScholix.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("spark.sql.shuffle.partitions", "4000");
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(SparkExtractRelationCount.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
conf
|
||||
.registerKryoClasses(
|
||||
new Class[] {
|
||||
Scholix.class, ScholixCollectedFrom.class, ScholixEntityId.class,
|
||||
ScholixIdentifier.class, ScholixRelationship.class, ScholixResource.class
|
||||
});
|
||||
|
||||
final String graphPath = parser.get("graphPath");
|
||||
final String workingDirPath = parser.get("workingDirPath");
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
final Dataset<ScholixSummary> scholixSummary = spark
|
||||
.read()
|
||||
.load(workingDirPath + "/summary")
|
||||
.as(Encoders.bean(ScholixSummary.class));
|
||||
final Dataset<Relation> rels = spark.read().load(graphPath + "/relation").as(Encoders.bean(Relation.class));
|
||||
|
||||
Dataset<Scholix> firstJoin = scholixSummary
|
||||
.joinWith(rels, scholixSummary.col("id").equalTo(rels.col("source")))
|
||||
.map(
|
||||
(MapFunction<Tuple2<ScholixSummary, Relation>, Scholix>) f -> Scholix
|
||||
.generateScholixWithSource(f._1(), f._2()),
|
||||
Encoders.bean(Scholix.class));
|
||||
|
||||
firstJoin.write().mode(SaveMode.Overwrite).save(workingDirPath + "/scholix_1");
|
||||
|
||||
Dataset<Scholix> scholix_final = spark
|
||||
.read()
|
||||
.load(workingDirPath + "/scholix_1")
|
||||
.as(Encoders.bean(Scholix.class));
|
||||
|
||||
scholixSummary
|
||||
.map(
|
||||
(MapFunction<ScholixSummary, ScholixResource>) ScholixResource::fromSummary,
|
||||
Encoders.bean(ScholixResource.class))
|
||||
.repartition(1000)
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(workingDirPath + "/scholix_target");
|
||||
|
||||
Dataset<ScholixResource> target = spark
|
||||
.read()
|
||||
.load(workingDirPath + "/scholix_target")
|
||||
.as(Encoders.bean(ScholixResource.class));
|
||||
|
||||
scholix_final
|
||||
.joinWith(
|
||||
target, scholix_final.col("identifier").equalTo(target.col("dnetIdentifier")), "inner")
|
||||
.map(
|
||||
(MapFunction<Tuple2<Scholix, ScholixResource>, Scholix>) f -> {
|
||||
final Scholix scholix = f._1();
|
||||
final ScholixResource scholixTarget = f._2();
|
||||
scholix.setTarget(scholixTarget);
|
||||
scholix.generateIdentifier();
|
||||
scholix.generatelinkPublisher();
|
||||
return scholix;
|
||||
},
|
||||
Encoders.kryo(Scholix.class))
|
||||
.javaRDD()
|
||||
.map(
|
||||
s -> {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
return mapper.writeValueAsString(s);
|
||||
})
|
||||
.saveAsTextFile(workingDirPath + "/scholix_json", GzipCodec.class);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
package eu.dnetlib.dhp.provision
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.provision.scholix.{Scholix, ScholixResource}
|
||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
|
||||
object SparkGenerateScholixIndex {
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkGenerateScholixIndex.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json")))
|
||||
parser.parseArgument(args)
|
||||
val conf = new SparkConf
|
||||
conf.set("spark.sql.shuffle.partitions", "4000")
|
||||
val spark = SparkSession.builder.config(conf).appName(SparkGenerateScholixIndex.getClass.getSimpleName).master(parser.get("master")).getOrCreate
|
||||
|
||||
val graphPath = parser.get("graphPath")
|
||||
val workingDirPath = parser.get("workingDirPath")
|
||||
|
||||
|
||||
implicit val summaryEncoder:Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
|
||||
implicit val relEncoder:Encoder[Relation] = Encoders.kryo[Relation]
|
||||
implicit val scholixEncoder:Encoder[Scholix] = Encoders.kryo[Scholix]
|
||||
implicit val tupleScholix:Encoder[(String,Scholix)]=Encoders.tuple(Encoders.STRING, scholixEncoder)
|
||||
|
||||
|
||||
val scholixSummary:Dataset[(String,ScholixSummary)] = spark.read.load(s"$workingDirPath/summary").as[ScholixSummary]
|
||||
.map(s => (s.getId, s))(Encoders.tuple(Encoders.STRING, summaryEncoder))
|
||||
val sourceRelations:Dataset[(String,Relation)]= spark.read.load(s"$graphPath/relation").as[Relation]
|
||||
.map(r => (r.getSource,r))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
|
||||
scholixSummary.joinWith(sourceRelations, scholixSummary("_1").equalTo(sourceRelations("_1")), "inner")
|
||||
.map(r=> {
|
||||
val summary = r._1._2
|
||||
val relation = r._2._2
|
||||
|
||||
(relation.getTarget, Scholix.generateScholixWithSource(summary,relation))
|
||||
|
||||
}).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/scholix_source")
|
||||
|
||||
val sTarget:Dataset[(String,Scholix)] = spark.read.load(s"$workingDirPath/scholix_source").as[(String, Scholix)]
|
||||
|
||||
sTarget.joinWith(scholixSummary, sTarget("_1").equalTo(scholixSummary("_1")), "inner").map(i => {
|
||||
val summary = i._2._2
|
||||
val scholix = i._1._2
|
||||
|
||||
val scholixResource = ScholixResource.fromSummary(summary)
|
||||
scholix.setTarget(scholixResource)
|
||||
scholix.generateIdentifier()
|
||||
scholix.generatelinkPublisher()
|
||||
scholix
|
||||
}).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/scholix")
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,106 +0,0 @@
|
|||
|
||||
package eu.dnetlib.dhp.provision;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkGenerateSummary {
|
||||
|
||||
private static final String jsonIDPath = "$.id";
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkGenerateSummary.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkExtractRelationCount.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final String graphPath = parser.get("graphPath");
|
||||
final String workingDirPath = parser.get("workingDirPath");
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
Dataset<RelatedItemInfo> rInfo = spark
|
||||
.read()
|
||||
.load(workingDirPath + "/relatedItemCount")
|
||||
.as(Encoders.bean(RelatedItemInfo.class));
|
||||
|
||||
Dataset<ScholixSummary> entity = spark
|
||||
.createDataset(
|
||||
sc
|
||||
.textFile(
|
||||
graphPath + "/publication," + graphPath + "/dataset," + graphPath + "/unknown")
|
||||
.map(
|
||||
s -> ScholixSummary
|
||||
.fromJsonOAF(
|
||||
ProvisionUtil.getItemTypeFromId(DHPUtils.getJPathString(jsonIDPath, s)),
|
||||
s))
|
||||
.rdd(),
|
||||
Encoders.bean(ScholixSummary.class));
|
||||
|
||||
Dataset<ScholixSummary> summaryComplete = rInfo
|
||||
.joinWith(entity, rInfo.col("source").equalTo(entity.col("id")))
|
||||
.map(
|
||||
(MapFunction<Tuple2<RelatedItemInfo, ScholixSummary>, ScholixSummary>) t -> {
|
||||
ScholixSummary scholixSummary = t._2();
|
||||
RelatedItemInfo relatedItemInfo = t._1();
|
||||
scholixSummary.setRelatedDatasets(relatedItemInfo.getRelatedDataset());
|
||||
scholixSummary
|
||||
.setRelatedPublications(
|
||||
relatedItemInfo.getRelatedPublication());
|
||||
scholixSummary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown());
|
||||
return scholixSummary;
|
||||
},
|
||||
Encoders.bean(ScholixSummary.class));
|
||||
|
||||
summaryComplete.write().save(workingDirPath + "/summary");
|
||||
|
||||
// JavaPairRDD<String, String> relationCount =
|
||||
// sc.textFile(workingDirPath+"/relatedItemCount").mapToPair((PairFunction<String, String,
|
||||
// String>) i -> new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i));
|
||||
//
|
||||
// JavaPairRDD<String, String> entities =
|
||||
// sc.textFile(graphPath + "/publication")
|
||||
// .filter(ProvisionUtil::isNotDeleted)
|
||||
// .mapToPair((PairFunction<String, String, String>) i -> new
|
||||
// Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i))
|
||||
// .union(
|
||||
// sc.textFile(graphPath + "/dataset")
|
||||
// .filter(ProvisionUtil::isNotDeleted)
|
||||
// .mapToPair((PairFunction<String, String, String>)
|
||||
// i ->
|
||||
// new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i))
|
||||
// )
|
||||
// .union(
|
||||
// sc.textFile(graphPath + "/unknown")
|
||||
// .filter(ProvisionUtil::isNotDeleted)
|
||||
// .mapToPair((PairFunction<String, String, String>)
|
||||
// i ->
|
||||
// new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i))
|
||||
// );
|
||||
// entities.join(relationCount).map((Function<Tuple2<String, Tuple2<String, String>>,
|
||||
// String>) k ->
|
||||
// ScholixSummary.fromJsonOAF(ProvisionUtil.getItemTypeFromId(k._1()),
|
||||
// k._2()._1(), k._2()._2())).saveAsTextFile(workingDirPath+"/summary", GzipCodec.class);
|
||||
//
|
||||
//
|
||||
// ;
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
package eu.dnetlib.dhp.provision
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
|
||||
object SparkGenerateSummaryIndex {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkGenerateSummaryIndex.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark = SparkSession.builder.appName(SparkGenerateSummaryIndex.getClass.getSimpleName).master(parser.get("master")).getOrCreate
|
||||
|
||||
val graphPath = parser.get("graphPath")
|
||||
val workingDirPath = parser.get("workingDirPath")
|
||||
|
||||
implicit val relatedItemInfoEncoders: Encoder[RelatedItemInfo] = Encoders.bean(classOf[RelatedItemInfo])
|
||||
implicit val datasetEncoder:Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||
implicit val publicationEncoder:Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||
implicit val relationEncoder:Encoder[Relation] = Encoders.kryo[Relation]
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
|
||||
implicit val oafWithIdEncoder: Encoder[(String, Oaf)] = Encoders.tuple(Encoders.STRING, oafEncoder)
|
||||
implicit val scholixSummaryEncoder: Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
|
||||
implicit val scholixSummaryEncoderTuple: Encoder[(String,ScholixSummary)] = Encoders.tuple(Encoders.STRING,scholixSummaryEncoder)
|
||||
|
||||
|
||||
val pubs = spark.read.load(s"$graphPath/publication").as[Oaf].map(o => (o.asInstanceOf[DLIPublication].getId, o))
|
||||
val dats = spark.read.load(s"$graphPath/dataset").as[Oaf].map(o => (o.asInstanceOf[DLIDataset].getId, o))
|
||||
val ukn = spark.read.load(s"$graphPath/unknown").as[Oaf].map(o => (o.asInstanceOf[DLIUnknown].getId, o))
|
||||
|
||||
|
||||
val summary:Dataset[(String,ScholixSummary)] = pubs.union(dats).union(ukn).map(o =>{
|
||||
val s = ScholixSummary.fromOAF(o._2)
|
||||
(s.getId,s)
|
||||
})
|
||||
|
||||
|
||||
val relatedItemInfoDs:Dataset[RelatedItemInfo] = spark.read.load(s"$workingDirPath/relatedItemCount").as[RelatedItemInfo]
|
||||
|
||||
|
||||
summary.joinWith(relatedItemInfoDs, summary("_1").equalTo(relatedItemInfoDs("source")), "inner")
|
||||
.map(i => {
|
||||
val summary = i._1._2
|
||||
val relatedItemInfo = i._2
|
||||
summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset)
|
||||
summary.setRelatedPublications(relatedItemInfo.getRelatedPublication)
|
||||
summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown)
|
||||
summary
|
||||
}).filter(s => s.getLocalIdentifier != null).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/summary")
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -8,15 +8,12 @@ import org.apache.commons.io.IOUtils;
|
|||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary;
|
||||
|
||||
public class SparkIndexCollectionOnES {
|
||||
|
||||
|
@ -39,33 +36,20 @@ public class SparkIndexCollectionOnES {
|
|||
final String sourcePath = parser.get("sourcePath");
|
||||
final String index = parser.get("index");
|
||||
final String idPath = parser.get("idPath");
|
||||
final String type = parser.get("type");
|
||||
final String indexHost = parser.get("esHost");
|
||||
final String cluster = parser.get("cluster");
|
||||
final String clusterJson = IOUtils
|
||||
.toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/cluster.json"));
|
||||
|
||||
final Map<String, String> clusterMap = new ObjectMapper().readValue(clusterJson, Map.class);
|
||||
|
||||
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<String> inputRdd;
|
||||
|
||||
if ("summary".equalsIgnoreCase(type))
|
||||
inputRdd = spark
|
||||
.read()
|
||||
.load(sourcePath)
|
||||
.as(Encoders.bean(ScholixSummary.class))
|
||||
.map(
|
||||
(MapFunction<ScholixSummary, String>) f -> {
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
return mapper.writeValueAsString(f);
|
||||
},
|
||||
Encoders.STRING())
|
||||
.javaRDD();
|
||||
else
|
||||
inputRdd = sc.textFile(sourcePath);
|
||||
JavaRDD<String> inputRdd = sc.textFile(sourcePath);
|
||||
|
||||
Map<String, String> esCfg = new HashMap<>();
|
||||
// esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54");
|
||||
esCfg.put("es.nodes", indexHost);
|
||||
esCfg.put("es.nodes", clusterMap.get(cluster));
|
||||
esCfg.put("es.mapping.id", idPath);
|
||||
esCfg.put("es.batch.write.retry.count", "8");
|
||||
esCfg.put("es.batch.write.retry.wait", "60s");
|
||||
|
|
|
@ -12,6 +12,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
|
||||
import eu.dnetlib.dhp.provision.RelatedItemInfo;
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
||||
|
@ -138,54 +140,20 @@ public class ScholixSummary implements Serializable {
|
|||
this.datasources = datasources;
|
||||
}
|
||||
|
||||
public static ScholixSummary fromJsonOAF(final Typology oafType, final String oafJson) {
|
||||
public static ScholixSummary fromOAF(final Oaf oaf) {
|
||||
try {
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final RelatedItemInfo relatedItemInfo = new RelatedItemInfo();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
switch (oafType) {
|
||||
case dataset:
|
||||
return summaryFromDataset(mapper.readValue(oafJson, DLIDataset.class), relatedItemInfo);
|
||||
case publication:
|
||||
return summaryFromPublication(
|
||||
mapper.readValue(oafJson, DLIPublication.class), relatedItemInfo);
|
||||
case unknown:
|
||||
return summaryFromUnknown(mapper.readValue(oafJson, DLIUnknown.class), relatedItemInfo);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static String fromJsonOAF(
|
||||
final Typology oafType, final String oafJson, final String relEntityJson) {
|
||||
try {
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
||||
RelatedItemInfo relatedItemInfo = mapper.readValue(relEntityJson, RelatedItemInfo.class);
|
||||
|
||||
switch (oafType) {
|
||||
case dataset:
|
||||
return mapper
|
||||
.writeValueAsString(
|
||||
summaryFromDataset(mapper.readValue(oafJson, DLIDataset.class), relatedItemInfo));
|
||||
case publication:
|
||||
return mapper
|
||||
.writeValueAsString(
|
||||
summaryFromPublication(
|
||||
mapper.readValue(oafJson, DLIPublication.class), relatedItemInfo));
|
||||
case unknown:
|
||||
return mapper
|
||||
.writeValueAsString(
|
||||
summaryFromUnknown(mapper.readValue(oafJson, DLIUnknown.class), relatedItemInfo));
|
||||
}
|
||||
if (oaf instanceof DLIPublication)
|
||||
return summaryFromPublication((DLIPublication) oaf, relatedItemInfo);
|
||||
if (oaf instanceof DLIDataset)
|
||||
return summaryFromDataset((DLIDataset) oaf, relatedItemInfo);
|
||||
if (oaf instanceof DLIUnknown)
|
||||
return summaryFromUnknown((DLIUnknown) oaf, relatedItemInfo);
|
||||
|
||||
} catch (Throwable e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,4 @@
|
|||
{
|
||||
"cluster1": "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54",
|
||||
"cluster2": "10.19.65.55, 10.19.65.56, 10.19.65.57, 10.19.65.58"
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
[
|
||||
{
|
||||
"paramName": "m",
|
||||
"paramLongName": "master",
|
||||
"paramDescription": "master should be local or yarn",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "w",
|
||||
"paramLongName": "workingPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,14 @@
|
|||
[
|
||||
{
|
||||
"paramName": "c",
|
||||
"paramLongName": "cluster",
|
||||
"paramDescription": "should be cluster1 or cluster2",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "i",
|
||||
"paramLongName": "index",
|
||||
"paramDescription": "index name",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -18,19 +18,12 @@
|
|||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "h",
|
||||
"paramLongName": "esHost",
|
||||
"paramDescription": "the index host name",
|
||||
"paramName": "c",
|
||||
"paramLongName": "cluster",
|
||||
"paramDescription": "the index cluster",
|
||||
"paramRequired": true
|
||||
},
|
||||
|
||||
|
||||
{
|
||||
"paramName": "t",
|
||||
"paramLongName": "type",
|
||||
"paramDescription": "should be scholix or summary",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "id",
|
||||
"paramLongName": "idPath",
|
||||
|
|
|
@ -7,4 +7,8 @@
|
|||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -8,6 +8,14 @@
|
|||
<name>graphPath</name>
|
||||
<description>the graph path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>index</name>
|
||||
<description>the index name</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>esCluster</name>
|
||||
<description>the Index cluster</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
|
@ -16,18 +24,6 @@
|
|||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<!-- <property>-->
|
||||
<!-- <name>index</name>-->
|
||||
<!-- <description>index name</description>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>idScholix</name>-->
|
||||
<!-- <description>the identifier name of the scholix </description>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>idSummary</name>-->
|
||||
<!-- <description>the identifier name of the summary</description>-->
|
||||
<!-- </property>-->
|
||||
</parameters>
|
||||
|
||||
<start to="DeleteTargetPath"/>
|
||||
|
@ -53,7 +49,7 @@
|
|||
<name>calculate for each ID the number of related Dataset, publication and Unknown</name>
|
||||
<class>eu.dnetlib.dhp.provision.SparkExtractRelationCount</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT}</spark-opts>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT}</spark-opts>
|
||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||
<arg>--relationPath</arg><arg>${graphPath}/relation</arg>
|
||||
|
@ -69,9 +65,9 @@
|
|||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>generate Summary</name>
|
||||
<class>eu.dnetlib.dhp.provision.SparkGenerateSummary</class>
|
||||
<class>eu.dnetlib.dhp.provision.SparkGenerateSummaryIndex</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT}</spark-opts>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.sql.shuffle.partitions=4000 ${sparkExtraOPT}</spark-opts>
|
||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||
<arg>--graphPath</arg><arg>${graphPath}</arg>
|
||||
|
@ -87,56 +83,88 @@
|
|||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>generate Scholix</name>
|
||||
<class>eu.dnetlib.dhp.provision.SparkGenerateScholix</class>
|
||||
<class>eu.dnetlib.dhp.provision.SparkGenerateScholixIndex</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>--executor-memory 6G --driver-memory=${sparkDriverMemory} ${sparkExtraOPT}</spark-opts>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.sql.shuffle.partitions=4000 ${sparkExtraOPT}</spark-opts>
|
||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||
<arg>--graphPath</arg><arg>${graphPath}</arg>
|
||||
</spark>
|
||||
<ok to="datasetToJson"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="datasetToJson">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>generate Scholix</name>
|
||||
<class>eu.dnetlib.dhp.provision.SparkConvertDatasetToJson</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.sql.shuffle.partitions=4000 ${sparkExtraOPT}</spark-opts>
|
||||
<arg>-m</arg> <arg>yarn-cluster</arg>
|
||||
<arg>--workingPath</arg><arg>${workingDirPath}</arg>
|
||||
</spark>
|
||||
<ok to="DropAndCreateIndex"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="DropAndCreateIndex">
|
||||
<java>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<main-class>eu.dnetlib.dhp.provision.DropAndCreateESIndex</main-class>
|
||||
<arg>-i</arg><arg>${index}</arg>
|
||||
<arg>-c</arg><arg>${esCluster}</arg>
|
||||
</java>
|
||||
<ok to="indexSummary"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="indexSummary">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>index summary</name>
|
||||
<class>eu.dnetlib.dhp.provision.SparkIndexCollectionOnES</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="8" </spark-opts>
|
||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${workingDirPath}/summary_json</arg>
|
||||
<arg>--index</arg><arg>${index}_object</arg>
|
||||
<arg>--idPath</arg><arg>id</arg>
|
||||
<arg>--cluster</arg><arg>${esCluster}</arg>
|
||||
</spark>
|
||||
<ok to="indexScholix"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="indexScholix">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>index scholix</name>
|
||||
<class>eu.dnetlib.dhp.provision.SparkIndexCollectionOnES</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="8" </spark-opts>
|
||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${workingDirPath}/scholix_json</arg>
|
||||
<arg>--index</arg><arg>${index}_scholix</arg>
|
||||
<arg>--idPath</arg><arg>identifier</arg>
|
||||
<arg>--cluster</arg><arg>${esCluster}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<!-- <action name="indexSummary">-->
|
||||
<!-- <spark xmlns="uri:oozie:spark-action:0.2">-->
|
||||
<!-- <job-tracker>${jobTracker}</job-tracker>-->
|
||||
<!-- <name-node>${nameNode}</name-node>-->
|
||||
<!-- <master>yarn-cluster</master>-->
|
||||
<!-- <mode>cluster</mode>-->
|
||||
<!-- <name>index Summary</name>-->
|
||||
<!-- <class>eu.dnetlib.dhp.provision.SparkIndexCollectionOnES</class>-->
|
||||
<!-- <jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>-->
|
||||
<!-- <spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="32" </spark-opts>-->
|
||||
<!-- <arg>-mt</arg> <arg>yarn-cluster</arg>-->
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDirPath}/summary</arg>-->
|
||||
<!-- <arg>--index</arg><arg>${index}_object</arg>-->
|
||||
<!-- <arg>--idPath</arg><arg>id</arg>-->
|
||||
<!-- <arg>--type</arg><arg>summary</arg>-->
|
||||
<!-- </spark>-->
|
||||
<!-- <ok to="indexScholix"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
<!-- <action name="indexScholix">-->
|
||||
<!-- <spark xmlns="uri:oozie:spark-action:0.2">-->
|
||||
<!-- <job-tracker>${jobTracker}</job-tracker>-->
|
||||
<!-- <name-node>${nameNode}</name-node>-->
|
||||
<!-- <master>yarn-cluster</master>-->
|
||||
<!-- <mode>cluster</mode>-->
|
||||
<!-- <name>index scholix</name>-->
|
||||
<!-- <class>eu.dnetlib.dhp.provision.SparkIndexCollectionOnES</class>-->
|
||||
<!-- <jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>-->
|
||||
<!-- <spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="8" </spark-opts>-->
|
||||
<!-- <arg>-mt</arg> <arg>yarn-cluster</arg>-->
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDirPath}/scholix_json</arg>-->
|
||||
<!-- <arg>--index</arg><arg>${index}_scholix</arg>-->
|
||||
<!-- <arg>--idPath</arg><arg>identifier</arg>-->
|
||||
<!-- <arg>--type</arg><arg>scholix</arg>-->
|
||||
<!-- </spark>-->
|
||||
<!-- <ok to="End"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -4,7 +4,7 @@ import java.time.LocalDateTime
|
|||
import java.time.format.DateTimeFormatter
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.SparkSession
|
||||
|
@ -65,7 +65,7 @@ class ExportDLITOOAFTest {
|
|||
val json = Source.fromInputStream(getClass.getResourceAsStream("relation.json")).mkString
|
||||
|
||||
|
||||
val oaf =DLIToOAF.convertDLIRelation(mapper.readValue(json, classOf[DLIRelation]))
|
||||
val oaf =mapper.readValue(json, classOf[Relation])
|
||||
|
||||
println(mapper.writeValueAsString(oaf))
|
||||
|
||||
|
|
|
@ -0,0 +1,13 @@
|
|||
|
||||
package eu.dnetlib.dhp.provision;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class DropAndCreateESIndexTest {
|
||||
|
||||
public void testDropAndCreate() throws Exception {
|
||||
DropAndCreateESIndex.main("-c localhost -i dli_shadow".split(" "));
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -13,7 +13,7 @@
|
|||
"lastupdatetimestamp": null,
|
||||
"relType": "IsReferencedBy",
|
||||
"source": "60|4ee78ab329b49416b45c3774c132f244",
|
||||
"collectedFrom": [
|
||||
"collectedfrom": [
|
||||
{
|
||||
"dataInfo": null,
|
||||
"value": "Europe PMC",
|
||||
|
|
|
@ -48,6 +48,80 @@
|
|||
<artifactId>spark-solr</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.solr</groupId>
|
||||
<artifactId>solr-test-framework</artifactId>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.carrotsearch</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.carrotsearch.randomizedtesting</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
|
||||
<exclusion>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.fasterxml.jackson.dataformat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.janino</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.woodstox</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
|
||||
|
||||
|
||||
<exclusion>
|
||||
<groupId>com.github.ben-manes.caffeine</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.google.protobuf</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.lmax</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.tdunning</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-core</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
<artifactId>httpclient</artifactId>
|
||||
|
|
|
@ -0,0 +1,101 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.solr.client.solrj.SolrResponse;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.client.solrj.response.UpdateResponse;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ZkServers;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
|
||||
public class SolrAdminApplication extends SolrApplication implements Closeable {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SolrAdminApplication.class);
|
||||
|
||||
enum Action {
|
||||
DELETE_BY_QUERY, COMMIT
|
||||
}
|
||||
|
||||
private CloudSolrClient solrClient;
|
||||
|
||||
public static void main(final String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SolrAdminApplication.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/oa/provision/input_solradmin_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
|
||||
final String isLookupUrl = parser.get("isLookupUrl");
|
||||
log.info("isLookupUrl: {}", isLookupUrl);
|
||||
|
||||
final String format = parser.get("format");
|
||||
log.info("format: {}", format);
|
||||
|
||||
final Action action = Action.valueOf(parser.get("action"));
|
||||
log.info("action: {}", action);
|
||||
|
||||
final String query = parser.get("query");
|
||||
log.info("query: {}", query);
|
||||
|
||||
final boolean commit = Optional
|
||||
.ofNullable(parser.get("commit"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(false);
|
||||
log.info("commit: {}", commit);
|
||||
|
||||
final ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
||||
|
||||
final String zkHost = getZkHost(isLookup);
|
||||
log.info("zkHost: {}", zkHost);
|
||||
|
||||
final String collection = format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION;
|
||||
log.info("collection: {}", collection);
|
||||
|
||||
try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) {
|
||||
app.execute(action, collection, query, commit);
|
||||
}
|
||||
}
|
||||
|
||||
public SolrAdminApplication(String zkHost) {
|
||||
final ZkServers zk = ZkServers.newInstance(zkHost);
|
||||
this.solrClient = new CloudSolrClient.Builder(zk.getHosts(), zk.getChroot()).build();
|
||||
}
|
||||
|
||||
public SolrResponse commit(String collection) throws IOException, SolrServerException {
|
||||
return execute(Action.COMMIT, collection, null, true);
|
||||
}
|
||||
|
||||
public SolrResponse execute(Action action, String collection, String query, boolean commit)
|
||||
throws IOException, SolrServerException {
|
||||
switch (action) {
|
||||
|
||||
case DELETE_BY_QUERY:
|
||||
UpdateResponse rsp = solrClient.deleteByQuery(collection, query);
|
||||
if (commit) {
|
||||
solrClient.commit(collection);
|
||||
}
|
||||
return rsp;
|
||||
case COMMIT:
|
||||
return solrClient.commit(collection);
|
||||
default:
|
||||
throw new IllegalArgumentException("action not managed: " + action.toString());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
solrClient.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
|
||||
public abstract class SolrApplication {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SolrApplication.class);
|
||||
|
||||
protected static final String LAYOUT = "index";
|
||||
protected static final String INTERPRETATION = "openaire";
|
||||
protected static final String SEPARATOR = "-";
|
||||
protected static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'";
|
||||
|
||||
/**
|
||||
* Method retrieves from the information system the zookeeper quorum of the Solr server
|
||||
*
|
||||
* @param isLookup
|
||||
* @return the zookeeper quorum of the Solr server
|
||||
* @throws ISLookUpException
|
||||
*/
|
||||
protected static String getZkHost(ISLookUpService isLookup) throws ISLookUpException {
|
||||
return doLookup(
|
||||
isLookup,
|
||||
"for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='IndexServiceResourceType'] return $x//PROTOCOL[./@name='solr']/@address/string()");
|
||||
}
|
||||
|
||||
protected static String doLookup(ISLookUpService isLookup, String xquery) throws ISLookUpException {
|
||||
log.info(String.format("running xquery: %s", xquery));
|
||||
final String res = isLookup.getResourceProfileByQuery(xquery);
|
||||
log.info(String.format("got response (100 chars): %s", StringUtils.left(res, 100) + " ..."));
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
|
@ -35,17 +35,12 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpDocumentNotFoundException;
|
|||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
|
||||
public class XmlIndexingJob {
|
||||
public class XmlIndexingJob extends SolrApplication {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(XmlIndexingJob.class);
|
||||
|
||||
private static final Integer DEFAULT_BATCH_SIZE = 1000;
|
||||
|
||||
private static final String LAYOUT = "index";
|
||||
private static final String INTERPRETATION = "openaire";
|
||||
private static final String SEPARATOR = "-";
|
||||
public static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'";
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -208,23 +203,4 @@ public class XmlIndexingJob {
|
|||
format));
|
||||
}
|
||||
|
||||
/**
|
||||
* Method retrieves from the information system the zookeeper quorum of the Solr server
|
||||
*
|
||||
* @param isLookup
|
||||
* @return the zookeeper quorum of the Solr server
|
||||
* @throws ISLookUpException
|
||||
*/
|
||||
private static String getZkHost(ISLookUpService isLookup) throws ISLookUpException {
|
||||
return doLookup(
|
||||
isLookup,
|
||||
"for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='IndexServiceResourceType'] return $x//PROTOCOL[./@name='solr']/@address/string()");
|
||||
}
|
||||
|
||||
private static String doLookup(ISLookUpService isLookup, String xquery) throws ISLookUpException {
|
||||
log.info(String.format("running xquery: %s", xquery));
|
||||
final String res = isLookup.getResourceProfileByQuery(xquery);
|
||||
log.info(String.format("got response (100 chars): %s", StringUtils.left(res, 100) + " ..."));
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,59 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision.utils;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
public class ZkServers {
|
||||
|
||||
private static final Log log = LogFactory.getLog(ZkServers.class);
|
||||
public static final String SEPARATOR = "/";
|
||||
|
||||
private List<String> hosts;
|
||||
|
||||
private Optional<String> chroot;
|
||||
|
||||
public static ZkServers newInstance(final String zkUrl) {
|
||||
|
||||
// quorum0:2182,quorum1:2182,quorum2:2182,quorum3:2182,quorum4:2182/solr-dev-openaire
|
||||
String urls = zkUrl;
|
||||
final Optional<String> chRoot = Optional.of(SEPARATOR + StringUtils.substringAfterLast(zkUrl, SEPARATOR));
|
||||
if (chRoot.isPresent() && StringUtils.isNotBlank(chRoot.get())) {
|
||||
log.debug(String.format("found zk chroot %s", chRoot));
|
||||
urls = zkUrl.replace(chRoot.get(), "");
|
||||
}
|
||||
|
||||
final List<String> urlList = Lists.newArrayList(Splitter.on(",").omitEmptyStrings().split(urls));
|
||||
log.debug(String.format("zk urls %s", zkUrl));
|
||||
|
||||
return new ZkServers(urlList, chRoot);
|
||||
}
|
||||
|
||||
public ZkServers(List<String> hosts, Optional<String> chroot) {
|
||||
this.hosts = hosts;
|
||||
this.chroot = chroot;
|
||||
}
|
||||
|
||||
public List<String> getHosts() {
|
||||
return hosts;
|
||||
}
|
||||
|
||||
public void setHosts(List<String> hosts) {
|
||||
this.hosts = hosts;
|
||||
}
|
||||
|
||||
public Optional<String> getChroot() {
|
||||
return chroot;
|
||||
}
|
||||
|
||||
public void setChroot(Optional<String> chroot) {
|
||||
this.chroot = chroot;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
[
|
||||
{
|
||||
"paramName": "isu",
|
||||
"paramLongName": "isLookupUrl",
|
||||
"paramDescription": "the URL to the ISLookUp Service",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "f",
|
||||
"paramLongName": "format",
|
||||
"paramDescription": "metadata format profile name",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "a",
|
||||
"paramLongName": "action",
|
||||
"paramDescription": "the action to be performed by the application",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "q",
|
||||
"paramLongName": "query",
|
||||
"paramDescription": "the query",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "c",
|
||||
"paramLongName": "commit",
|
||||
"paramDescription": "should the action be followed by a commit?",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -37,7 +37,11 @@
|
|||
<name>batchSize</name>
|
||||
<description>number of records to be included in each indexing request</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>solrDeletionQuery</name>
|
||||
<value>*:*</value>
|
||||
<description>query used in the deleted by query operation</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemoryForJoining</name>
|
||||
<description>memory for driver process</description>
|
||||
|
@ -109,6 +113,7 @@
|
|||
<case to="fork_join_related_entities">${wf:conf('resumeFrom') eq 'fork_join_related_entities'}</case>
|
||||
<case to="fork_join_all_entities">${wf:conf('resumeFrom') eq 'fork_join_all_entities'}</case>
|
||||
<case to="convert_to_xml">${wf:conf('resumeFrom') eq 'convert_to_xml'}</case>
|
||||
<case to="drop_solr_collection">${wf:conf('resumeFrom') eq 'drop_solr_collection'}</case>
|
||||
<case to="to_solr_index">${wf:conf('resumeFrom') eq 'to_solr_index'}</case>
|
||||
<default to="prepare_relations"/>
|
||||
</switch>
|
||||
|
@ -584,12 +589,25 @@
|
|||
|
||||
<decision name="should_index">
|
||||
<switch>
|
||||
<case to="to_solr_index">${wf:conf('shouldIndex') eq 'true'}</case>
|
||||
<case to="drop_solr_collection">${wf:conf('shouldIndex') eq 'true'}</case>
|
||||
<case to="End">${wf:conf('shouldIndex') eq 'false'}</case>
|
||||
<default to="to_solr_index"/>
|
||||
<default to="drop_solr_collection"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
<action name="drop_solr_collection">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.oa.provision.SolrAdminApplication</main-class>
|
||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
<arg>--format</arg><arg>${format}</arg>
|
||||
<arg>--action</arg><arg>DELETE_BY_QUERY</arg>
|
||||
<arg>--query</arg><arg>${solrDeletionQuery}</arg>
|
||||
<arg>--commit</arg><arg>true</arg>
|
||||
</java>
|
||||
<ok to="to_solr_index"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="to_solr_index">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
|
@ -615,6 +633,17 @@
|
|||
<arg>--format</arg><arg>${format}</arg>
|
||||
<arg>--batchSize</arg><arg>${batchSize}</arg>
|
||||
</spark>
|
||||
<ok to="commit_solr_collection"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="commit_solr_collection">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.oa.provision.SolrAdminApplication</main-class>
|
||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
<arg>--format</arg><arg>${format}</arg>
|
||||
<arg>--action</arg><arg>COMMIT</arg>
|
||||
</java>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
|
|
@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.provision;
|
|||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
|
@ -20,8 +21,6 @@ import org.junit.jupiter.api.io.TempDir;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
|
||||
|
@ -31,7 +30,8 @@ public class PrepareRelationsJobTest {
|
|||
|
||||
public static final String SUBRELTYPE = "subRelType";
|
||||
public static final String OUTCOME = "outcome";
|
||||
public static final String SUPPLEMENT = "supplement";
|
||||
public static final String PARTICIPATION = "participation";
|
||||
public static final String AFFILIATION = "affiliation";
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
|
@ -64,7 +64,7 @@ public class PrepareRelationsJobTest {
|
|||
@Test
|
||||
public void testRunPrepareRelationsJob(@TempDir Path testPath) throws Exception {
|
||||
|
||||
final int maxRelations = 10;
|
||||
final int maxRelations = 20;
|
||||
PrepareRelationsJob
|
||||
.main(
|
||||
new String[] {
|
||||
|
@ -73,7 +73,8 @@ public class PrepareRelationsJobTest {
|
|||
"-outputPath", testPath.toString(),
|
||||
"-relPartitions", "10",
|
||||
"-relationFilter", "asd",
|
||||
"-maxRelations", String.valueOf(maxRelations)
|
||||
"-sourceMaxRelations", String.valueOf(maxRelations),
|
||||
"-targetMaxRelations", String.valueOf(maxRelations * 100)
|
||||
});
|
||||
|
||||
Dataset<Relation> out = spark
|
||||
|
@ -82,19 +83,31 @@ public class PrepareRelationsJobTest {
|
|||
.as(Encoders.bean(Relation.class))
|
||||
.cache();
|
||||
|
||||
Assertions.assertEquals(10, out.count());
|
||||
Assertions.assertEquals(maxRelations, out.count());
|
||||
|
||||
Dataset<Row> freq = out
|
||||
.toDF()
|
||||
.cube(SUBRELTYPE)
|
||||
.count()
|
||||
.filter((FilterFunction<Row>) value -> !value.isNullAt(0));
|
||||
long outcome = freq.filter(freq.col(SUBRELTYPE).equalTo(OUTCOME)).collectAsList().get(0).getAs("count");
|
||||
long supplement = freq.filter(freq.col(SUBRELTYPE).equalTo(SUPPLEMENT)).collectAsList().get(0).getAs("count");
|
||||
|
||||
Assertions.assertTrue(outcome > supplement);
|
||||
log.info(freq.collectAsList().toString());
|
||||
|
||||
long outcome = getRows(freq, OUTCOME).get(0).getAs("count");
|
||||
long participation = getRows(freq, PARTICIPATION).get(0).getAs("count");
|
||||
long affiliation = getRows(freq, AFFILIATION).get(0).getAs("count");
|
||||
|
||||
Assertions.assertTrue(participation == outcome);
|
||||
Assertions.assertTrue(outcome > affiliation);
|
||||
Assertions.assertTrue(participation > affiliation);
|
||||
|
||||
Assertions.assertEquals(7, outcome);
|
||||
Assertions.assertEquals(3, supplement);
|
||||
Assertions.assertEquals(7, participation);
|
||||
Assertions.assertEquals(6, affiliation);
|
||||
}
|
||||
|
||||
protected List<Row> getRows(Dataset<Row> freq, String col) {
|
||||
return freq.filter(freq.col(SUBRELTYPE).equalTo(col)).collectAsList();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,130 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.solr.client.solrj.SolrResponse;
|
||||
import org.apache.solr.client.solrj.embedded.JettyConfig;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.client.solrj.impl.XMLResponseParser;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
|
||||
import org.apache.solr.client.solrj.request.QueryRequest;
|
||||
import org.apache.solr.client.solrj.request.RequestWriter;
|
||||
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
|
||||
import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
|
||||
import org.apache.solr.client.solrj.response.SolrPingResponse;
|
||||
import org.apache.solr.client.solrj.response.UpdateResponse;
|
||||
import org.apache.solr.cloud.MiniSolrCloudCluster;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
import org.apache.solr.common.params.CoreAdminParams;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
public class SolrAdminApplicationTest {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SolrAdminApplicationTest.class);
|
||||
public static final String DEFAULT_COLLECTION = "testCollection";
|
||||
public static final String CONFIG_NAME = "testConfig";
|
||||
|
||||
private static MiniSolrCloudCluster miniCluster;
|
||||
private static CloudSolrClient cloudSolrClient;
|
||||
|
||||
@TempDir
|
||||
public static Path tempDir;
|
||||
|
||||
@BeforeAll
|
||||
public static void setup() throws Exception {
|
||||
|
||||
// random unassigned HTTP port
|
||||
final int jettyPort = 0;
|
||||
|
||||
final JettyConfig jettyConfig = JettyConfig.builder().setPort(jettyPort).build();
|
||||
|
||||
// create a MiniSolrCloudCluster instance
|
||||
miniCluster = new MiniSolrCloudCluster(2, tempDir, jettyConfig);
|
||||
|
||||
// Upload Solr configuration directory to ZooKeeper
|
||||
String solrZKConfigDir = "src/test/resources/eu/dnetlib/dhp/oa/provision/solr/conf/testConfig";
|
||||
File configDir = new File(solrZKConfigDir);
|
||||
|
||||
miniCluster.uploadConfigSet(configDir.toPath(), CONFIG_NAME);
|
||||
|
||||
// override settings in the solrconfig include
|
||||
System.setProperty("solr.tests.maxBufferedDocs", "100000");
|
||||
System.setProperty("solr.tests.maxIndexingThreads", "-1");
|
||||
System.setProperty("solr.tests.ramBufferSizeMB", "100");
|
||||
|
||||
// use non-test classes so RandomizedRunner isn't necessary
|
||||
System.setProperty("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
|
||||
System.setProperty("solr.directoryFactory", "solr.RAMDirectoryFactory");
|
||||
|
||||
cloudSolrClient = miniCluster.getSolrClient();
|
||||
cloudSolrClient.setRequestWriter(new RequestWriter());
|
||||
cloudSolrClient.setParser(new XMLResponseParser());
|
||||
cloudSolrClient.setDefaultCollection(DEFAULT_COLLECTION);
|
||||
cloudSolrClient.connect();
|
||||
|
||||
log.info(new ConfigSetAdminRequest.List().process(cloudSolrClient).toString());
|
||||
log.info(CollectionAdminRequest.ClusterStatus.getClusterStatus().process(cloudSolrClient).toString());
|
||||
|
||||
createCollection(cloudSolrClient, DEFAULT_COLLECTION, 2, 1, CONFIG_NAME);
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void shutDown() throws Exception {
|
||||
miniCluster.shutdown();
|
||||
}
|
||||
|
||||
protected static NamedList<Object> createCollection(CloudSolrClient client, String name, int numShards,
|
||||
int replicationFactor, String configName) throws Exception {
|
||||
ModifiableSolrParams modParams = new ModifiableSolrParams();
|
||||
modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name());
|
||||
modParams.set("name", name);
|
||||
modParams.set("numShards", numShards);
|
||||
modParams.set("replicationFactor", replicationFactor);
|
||||
modParams.set("collection.configName", configName);
|
||||
QueryRequest request = new QueryRequest(modParams);
|
||||
request.setPath("/admin/collections");
|
||||
return client.request(request);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPing() throws Exception {
|
||||
SolrPingResponse pingResponse = cloudSolrClient.ping();
|
||||
log.info("pingResponse: '{}'", pingResponse.getStatus());
|
||||
Assert.assertTrue(pingResponse.getStatus() == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAdminApplication_DELETE() throws Exception {
|
||||
|
||||
SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost());
|
||||
|
||||
UpdateResponse rsp = (UpdateResponse) admin
|
||||
.execute(SolrAdminApplication.Action.DELETE_BY_QUERY, DEFAULT_COLLECTION, "*:*", false);
|
||||
|
||||
Assertions.assertTrue(rsp.getStatus() == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAdminApplication_COMMIT() throws Exception {
|
||||
|
||||
SolrAdminApplication admin = new SolrAdminApplication(miniCluster.getSolrClient().getZkHost());
|
||||
|
||||
UpdateResponse rsp = (UpdateResponse) admin.commit(DEFAULT_COLLECTION);
|
||||
|
||||
Assertions.assertTrue(rsp.getStatus() == 0);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,8 @@
|
|||
# Set of Catalan contractions for ElisionFilter
|
||||
# TODO: load this as a resource from the analyzer and sync it in build.xml
|
||||
d
|
||||
l
|
||||
m
|
||||
n
|
||||
s
|
||||
t
|
|
@ -0,0 +1,15 @@
|
|||
# Set of French contractions for ElisionFilter
|
||||
# TODO: load this as a resource from the analyzer and sync it in build.xml
|
||||
l
|
||||
m
|
||||
t
|
||||
qu
|
||||
n
|
||||
s
|
||||
j
|
||||
d
|
||||
c
|
||||
jusqu
|
||||
quoiqu
|
||||
lorsqu
|
||||
puisqu
|
|
@ -0,0 +1,5 @@
|
|||
# Set of Irish contractions for ElisionFilter
|
||||
# TODO: load this as a resource from the analyzer and sync it in build.xml
|
||||
d
|
||||
m
|
||||
b
|
|
@ -0,0 +1,23 @@
|
|||
# Set of Italian contractions for ElisionFilter
|
||||
# TODO: load this as a resource from the analyzer and sync it in build.xml
|
||||
c
|
||||
l
|
||||
all
|
||||
dall
|
||||
dell
|
||||
nell
|
||||
sull
|
||||
coll
|
||||
pell
|
||||
gl
|
||||
agl
|
||||
dagl
|
||||
degl
|
||||
negl
|
||||
sugl
|
||||
un
|
||||
m
|
||||
t
|
||||
s
|
||||
v
|
||||
d
|
|
@ -0,0 +1,5 @@
|
|||
# Set of Irish hyphenations for StopFilter
|
||||
# TODO: load this as a resource from the analyzer and sync it in build.xml
|
||||
h
|
||||
n
|
||||
t
|
|
@ -0,0 +1,6 @@
|
|||
# Set of overrides for the dutch stemmer
|
||||
# TODO: load this as a resource from the analyzer and sync it in build.xml
|
||||
fiets fiets
|
||||
bromfiets bromfiets
|
||||
ei eier
|
||||
kind kinder
|
|
@ -0,0 +1,420 @@
|
|||
#
|
||||
# This file defines a Japanese stoptag set for JapanesePartOfSpeechStopFilter.
|
||||
#
|
||||
# Any token with a part-of-speech tag that exactly matches those defined in this
|
||||
# file are removed from the token stream.
|
||||
#
|
||||
# Set your own stoptags by uncommenting the lines below. Note that comments are
|
||||
# not allowed on the same line as a stoptag. See LUCENE-3745 for frequency lists,
|
||||
# etc. that can be useful for building you own stoptag set.
|
||||
#
|
||||
# The entire possible tagset is provided below for convenience.
|
||||
#
|
||||
#####
|
||||
# noun: unclassified nouns
|
||||
#名詞
|
||||
#
|
||||
# noun-common: Common nouns or nouns where the sub-classification is undefined
|
||||
#名詞-一般
|
||||
#
|
||||
# noun-proper: Proper nouns where the sub-classification is undefined
|
||||
#名詞-固有名詞
|
||||
#
|
||||
# noun-proper-misc: miscellaneous proper nouns
|
||||
#名詞-固有名詞-一般
|
||||
#
|
||||
# noun-proper-person: Personal names where the sub-classification is undefined
|
||||
#名詞-固有名詞-人名
|
||||
#
|
||||
# noun-proper-person-misc: names that cannot be divided into surname and
|
||||
# given name; foreign names; names where the surname or given name is unknown.
|
||||
# e.g. お市の方
|
||||
#名詞-固有名詞-人名-一般
|
||||
#
|
||||
# noun-proper-person-surname: Mainly Japanese surnames.
|
||||
# e.g. 山田
|
||||
#名詞-固有名詞-人名-姓
|
||||
#
|
||||
# noun-proper-person-given_name: Mainly Japanese given names.
|
||||
# e.g. 太郎
|
||||
#名詞-固有名詞-人名-名
|
||||
#
|
||||
# noun-proper-organization: Names representing organizations.
|
||||
# e.g. 通産省, NHK
|
||||
#名詞-固有名詞-組織
|
||||
#
|
||||
# noun-proper-place: Place names where the sub-classification is undefined
|
||||
#名詞-固有名詞-地域
|
||||
#
|
||||
# noun-proper-place-misc: Place names excluding countries.
|
||||
# e.g. アジア, バルセロナ, 京都
|
||||
#名詞-固有名詞-地域-一般
|
||||
#
|
||||
# noun-proper-place-country: Country names.
|
||||
# e.g. 日本, オーストラリア
|
||||
#名詞-固有名詞-地域-国
|
||||
#
|
||||
# noun-pronoun: Pronouns where the sub-classification is undefined
|
||||
#名詞-代名詞
|
||||
#
|
||||
# noun-pronoun-misc: miscellaneous pronouns:
|
||||
# e.g. それ, ここ, あいつ, あなた, あちこち, いくつ, どこか, なに, みなさん, みんな, わたくし, われわれ
|
||||
#名詞-代名詞-一般
|
||||
#
|
||||
# noun-pronoun-contraction: Spoken language contraction made by combining a
|
||||
# pronoun and the particle 'wa'.
|
||||
# e.g. ありゃ, こりゃ, こりゃあ, そりゃ, そりゃあ
|
||||
#名詞-代名詞-縮約
|
||||
#
|
||||
# noun-adverbial: Temporal nouns such as names of days or months that behave
|
||||
# like adverbs. Nouns that represent amount or ratios and can be used adverbially,
|
||||
# e.g. 金曜, 一月, 午後, 少量
|
||||
#名詞-副詞可能
|
||||
#
|
||||
# noun-verbal: Nouns that take arguments with case and can appear followed by
|
||||
# 'suru' and related verbs (する, できる, なさる, くださる)
|
||||
# e.g. インプット, 愛着, 悪化, 悪戦苦闘, 一安心, 下取り
|
||||
#名詞-サ変接続
|
||||
#
|
||||
# noun-adjective-base: The base form of adjectives, words that appear before な ("na")
|
||||
# e.g. 健康, 安易, 駄目, だめ
|
||||
#名詞-形容動詞語幹
|
||||
#
|
||||
# noun-numeric: Arabic numbers, Chinese numerals, and counters like 何 (回), 数.
|
||||
# e.g. 0, 1, 2, 何, 数, 幾
|
||||
#名詞-数
|
||||
#
|
||||
# noun-affix: noun affixes where the sub-classification is undefined
|
||||
#名詞-非自立
|
||||
#
|
||||
# noun-affix-misc: Of adnominalizers, the case-marker の ("no"), and words that
|
||||
# attach to the base form of inflectional words, words that cannot be classified
|
||||
# into any of the other categories below. This category includes indefinite nouns.
|
||||
# e.g. あかつき, 暁, かい, 甲斐, 気, きらい, 嫌い, くせ, 癖, こと, 事, ごと, 毎, しだい, 次第,
|
||||
# 順, せい, 所為, ついで, 序で, つもり, 積もり, 点, どころ, の, はず, 筈, はずみ, 弾み,
|
||||
# 拍子, ふう, ふり, 振り, ほう, 方, 旨, もの, 物, 者, ゆえ, 故, ゆえん, 所以, わけ, 訳,
|
||||
# わり, 割り, 割, ん-口語/, もん-口語/
|
||||
#名詞-非自立-一般
|
||||
#
|
||||
# noun-affix-adverbial: noun affixes that that can behave as adverbs.
|
||||
# e.g. あいだ, 間, あげく, 挙げ句, あと, 後, 余り, 以外, 以降, 以後, 以上, 以前, 一方, うえ,
|
||||
# 上, うち, 内, おり, 折り, かぎり, 限り, きり, っきり, 結果, ころ, 頃, さい, 際, 最中, さなか,
|
||||
# 最中, じたい, 自体, たび, 度, ため, 為, つど, 都度, とおり, 通り, とき, 時, ところ, 所,
|
||||
# とたん, 途端, なか, 中, のち, 後, ばあい, 場合, 日, ぶん, 分, ほか, 他, まえ, 前, まま,
|
||||
# 儘, 侭, みぎり, 矢先
|
||||
#名詞-非自立-副詞可能
|
||||
#
|
||||
# noun-affix-aux: noun affixes treated as 助動詞 ("auxiliary verb") in school grammars
|
||||
# with the stem よう(だ) ("you(da)").
|
||||
# e.g. よう, やう, 様 (よう)
|
||||
#名詞-非自立-助動詞語幹
|
||||
#
|
||||
# noun-affix-adjective-base: noun affixes that can connect to the indeclinable
|
||||
# connection form な (aux "da").
|
||||
# e.g. みたい, ふう
|
||||
#名詞-非自立-形容動詞語幹
|
||||
#
|
||||
# noun-special: special nouns where the sub-classification is undefined.
|
||||
#名詞-特殊
|
||||
#
|
||||
# noun-special-aux: The そうだ ("souda") stem form that is used for reporting news, is
|
||||
# treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the base
|
||||
# form of inflectional words.
|
||||
# e.g. そう
|
||||
#名詞-特殊-助動詞語幹
|
||||
#
|
||||
# noun-suffix: noun suffixes where the sub-classification is undefined.
|
||||
#名詞-接尾
|
||||
#
|
||||
# noun-suffix-misc: Of the nouns or stem forms of other parts of speech that connect
|
||||
# to ガル or タイ and can combine into compound nouns, words that cannot be classified into
|
||||
# any of the other categories below. In general, this category is more inclusive than
|
||||
# 接尾語 ("suffix") and is usually the last element in a compound noun.
|
||||
# e.g. おき, かた, 方, 甲斐 (がい), がかり, ぎみ, 気味, ぐるみ, (~した) さ, 次第, 済 (ず) み,
|
||||
# よう, (でき)っこ, 感, 観, 性, 学, 類, 面, 用
|
||||
#名詞-接尾-一般
|
||||
#
|
||||
# noun-suffix-person: Suffixes that form nouns and attach to person names more often
|
||||
# than other nouns.
|
||||
# e.g. 君, 様, 著
|
||||
#名詞-接尾-人名
|
||||
#
|
||||
# noun-suffix-place: Suffixes that form nouns and attach to place names more often
|
||||
# than other nouns.
|
||||
# e.g. 町, 市, 県
|
||||
#名詞-接尾-地域
|
||||
#
|
||||
# noun-suffix-verbal: Of the suffixes that attach to nouns and form nouns, those that
|
||||
# can appear before スル ("suru").
|
||||
# e.g. 化, 視, 分け, 入り, 落ち, 買い
|
||||
#名詞-接尾-サ変接続
|
||||
#
|
||||
# noun-suffix-aux: The stem form of そうだ (様態) that is used to indicate conditions,
|
||||
# is treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the
|
||||
# conjunctive form of inflectional words.
|
||||
# e.g. そう
|
||||
#名詞-接尾-助動詞語幹
|
||||
#
|
||||
# noun-suffix-adjective-base: Suffixes that attach to other nouns or the conjunctive
|
||||
# form of inflectional words and appear before the copula だ ("da").
|
||||
# e.g. 的, げ, がち
|
||||
#名詞-接尾-形容動詞語幹
|
||||
#
|
||||
# noun-suffix-adverbial: Suffixes that attach to other nouns and can behave as adverbs.
|
||||
# e.g. 後 (ご), 以後, 以降, 以前, 前後, 中, 末, 上, 時 (じ)
|
||||
#名詞-接尾-副詞可能
|
||||
#
|
||||
# noun-suffix-classifier: Suffixes that attach to numbers and form nouns. This category
|
||||
# is more inclusive than 助数詞 ("classifier") and includes common nouns that attach
|
||||
# to numbers.
|
||||
# e.g. 個, つ, 本, 冊, パーセント, cm, kg, カ月, か国, 区画, 時間, 時半
|
||||
#名詞-接尾-助数詞
|
||||
#
|
||||
# noun-suffix-special: Special suffixes that mainly attach to inflecting words.
|
||||
# e.g. (楽し) さ, (考え) 方
|
||||
#名詞-接尾-特殊
|
||||
#
|
||||
# noun-suffix-conjunctive: Nouns that behave like conjunctions and join two words
|
||||
# together.
|
||||
# e.g. (日本) 対 (アメリカ), 対 (アメリカ), (3) 対 (5), (女優) 兼 (主婦)
|
||||
#名詞-接続詞的
|
||||
#
|
||||
# noun-verbal_aux: Nouns that attach to the conjunctive particle て ("te") and are
|
||||
# semantically verb-like.
|
||||
# e.g. ごらん, ご覧, 御覧, 頂戴
|
||||
#名詞-動詞非自立的
|
||||
#
|
||||
# noun-quotation: text that cannot be segmented into words, proverbs, Chinese poetry,
|
||||
# dialects, English, etc. Currently, the only entry for 名詞 引用文字列 ("noun quotation")
|
||||
# is いわく ("iwaku").
|
||||
#名詞-引用文字列
|
||||
#
|
||||
# noun-nai_adjective: Words that appear before the auxiliary verb ない ("nai") and
|
||||
# behave like an adjective.
|
||||
# e.g. 申し訳, 仕方, とんでも, 違い
|
||||
#名詞-ナイ形容詞語幹
|
||||
#
|
||||
#####
|
||||
# prefix: unclassified prefixes
|
||||
#接頭詞
|
||||
#
|
||||
# prefix-nominal: Prefixes that attach to nouns (including adjective stem forms)
|
||||
# excluding numerical expressions.
|
||||
# e.g. お (水), 某 (氏), 同 (社), 故 (~氏), 高 (品質), お (見事), ご (立派)
|
||||
#接頭詞-名詞接続
|
||||
#
|
||||
# prefix-verbal: Prefixes that attach to the imperative form of a verb or a verb
|
||||
# in conjunctive form followed by なる/なさる/くださる.
|
||||
# e.g. お (読みなさい), お (座り)
|
||||
#接頭詞-動詞接続
|
||||
#
|
||||
# prefix-adjectival: Prefixes that attach to adjectives.
|
||||
# e.g. お (寒いですねえ), バカ (でかい)
|
||||
#接頭詞-形容詞接続
|
||||
#
|
||||
# prefix-numerical: Prefixes that attach to numerical expressions.
|
||||
# e.g. 約, およそ, 毎時
|
||||
#接頭詞-数接続
|
||||
#
|
||||
#####
|
||||
# verb: unclassified verbs
|
||||
#動詞
|
||||
#
|
||||
# verb-main:
|
||||
#動詞-自立
|
||||
#
|
||||
# verb-auxiliary:
|
||||
#動詞-非自立
|
||||
#
|
||||
# verb-suffix:
|
||||
#動詞-接尾
|
||||
#
|
||||
#####
|
||||
# adjective: unclassified adjectives
|
||||
#形容詞
|
||||
#
|
||||
# adjective-main:
|
||||
#形容詞-自立
|
||||
#
|
||||
# adjective-auxiliary:
|
||||
#形容詞-非自立
|
||||
#
|
||||
# adjective-suffix:
|
||||
#形容詞-接尾
|
||||
#
|
||||
#####
|
||||
# adverb: unclassified adverbs
|
||||
#副詞
|
||||
#
|
||||
# adverb-misc: Words that can be segmented into one unit and where adnominal
|
||||
# modification is not possible.
|
||||
# e.g. あいかわらず, 多分
|
||||
#副詞-一般
|
||||
#
|
||||
# adverb-particle_conjunction: Adverbs that can be followed by の, は, に,
|
||||
# な, する, だ, etc.
|
||||
# e.g. こんなに, そんなに, あんなに, なにか, なんでも
|
||||
#副詞-助詞類接続
|
||||
#
|
||||
#####
|
||||
# adnominal: Words that only have noun-modifying forms.
|
||||
# e.g. この, その, あの, どの, いわゆる, なんらかの, 何らかの, いろんな, こういう, そういう, ああいう,
|
||||
# どういう, こんな, そんな, あんな, どんな, 大きな, 小さな, おかしな, ほんの, たいした,
|
||||
# 「(, も) さる (ことながら)」, 微々たる, 堂々たる, 単なる, いかなる, 我が」「同じ, 亡き
|
||||
#連体詞
|
||||
#
|
||||
#####
|
||||
# conjunction: Conjunctions that can occur independently.
|
||||
# e.g. が, けれども, そして, じゃあ, それどころか
|
||||
接続詞
|
||||
#
|
||||
#####
|
||||
# particle: unclassified particles.
|
||||
助詞
|
||||
#
|
||||
# particle-case: case particles where the subclassification is undefined.
|
||||
助詞-格助詞
|
||||
#
|
||||
# particle-case-misc: Case particles.
|
||||
# e.g. から, が, で, と, に, へ, より, を, の, にて
|
||||
助詞-格助詞-一般
|
||||
#
|
||||
# particle-case-quote: the "to" that appears after nouns, a person’s speech,
|
||||
# quotation marks, expressions of decisions from a meeting, reasons, judgements,
|
||||
# conjectures, etc.
|
||||
# e.g. ( だ) と (述べた.), ( である) と (して執行猶予...)
|
||||
助詞-格助詞-引用
|
||||
#
|
||||
# particle-case-compound: Compounds of particles and verbs that mainly behave
|
||||
# like case particles.
|
||||
# e.g. という, といった, とかいう, として, とともに, と共に, でもって, にあたって, に当たって, に当って,
|
||||
# にあたり, に当たり, に当り, に当たる, にあたる, において, に於いて,に於て, における, に於ける,
|
||||
# にかけ, にかけて, にかんし, に関し, にかんして, に関して, にかんする, に関する, に際し,
|
||||
# に際して, にしたがい, に従い, に従う, にしたがって, に従って, にたいし, に対し, にたいして,
|
||||
# に対して, にたいする, に対する, について, につき, につけ, につけて, につれ, につれて, にとって,
|
||||
# にとり, にまつわる, によって, に依って, に因って, により, に依り, に因り, による, に依る, に因る,
|
||||
# にわたって, にわたる, をもって, を以って, を通じ, を通じて, を通して, をめぐって, をめぐり, をめぐる,
|
||||
# って-口語/, ちゅう-関西弁「という」/, (何) ていう (人)-口語/, っていう-口語/, といふ, とかいふ
|
||||
助詞-格助詞-連語
|
||||
#
|
||||
# particle-conjunctive:
|
||||
# e.g. から, からには, が, けれど, けれども, けど, し, つつ, て, で, と, ところが, どころか, とも, ども,
|
||||
# ながら, なり, ので, のに, ば, ものの, や ( した), やいなや, (ころん) じゃ(いけない)-口語/,
|
||||
# (行っ) ちゃ(いけない)-口語/, (言っ) たって (しかたがない)-口語/, (それがなく)ったって (平気)-口語/
|
||||
助詞-接続助詞
|
||||
#
|
||||
# particle-dependency:
|
||||
# e.g. こそ, さえ, しか, すら, は, も, ぞ
|
||||
助詞-係助詞
|
||||
#
|
||||
# particle-adverbial:
|
||||
# e.g. がてら, かも, くらい, 位, ぐらい, しも, (学校) じゃ(これが流行っている)-口語/,
|
||||
# (それ)じゃあ (よくない)-口語/, ずつ, (私) なぞ, など, (私) なり (に), (先生) なんか (大嫌い)-口語/,
|
||||
# (私) なんぞ, (先生) なんて (大嫌い)-口語/, のみ, だけ, (私) だって-口語/, だに,
|
||||
# (彼)ったら-口語/, (お茶) でも (いかが), 等 (とう), (今後) とも, ばかり, ばっか-口語/, ばっかり-口語/,
|
||||
# ほど, 程, まで, 迄, (誰) も (が)([助詞-格助詞] および [助詞-係助詞] の前に位置する「も」)
|
||||
助詞-副助詞
|
||||
#
|
||||
# particle-interjective: particles with interjective grammatical roles.
|
||||
# e.g. (松島) や
|
||||
助詞-間投助詞
|
||||
#
|
||||
# particle-coordinate:
|
||||
# e.g. と, たり, だの, だり, とか, なり, や, やら
|
||||
助詞-並立助詞
|
||||
#
|
||||
# particle-final:
|
||||
# e.g. かい, かしら, さ, ぜ, (だ)っけ-口語/, (とまってる) で-方言/, な, ナ, なあ-口語/, ぞ, ね, ネ,
|
||||
# ねぇ-口語/, ねえ-口語/, ねん-方言/, の, のう-口語/, や, よ, ヨ, よぉ-口語/, わ, わい-口語/
|
||||
助詞-終助詞
|
||||
#
|
||||
# particle-adverbial/conjunctive/final: The particle "ka" when unknown whether it is
|
||||
# adverbial, conjunctive, or sentence final. For example:
|
||||
# (a) 「A か B か」. Ex:「(国内で運用する) か,(海外で運用する) か (.)」
|
||||
# (b) Inside an adverb phrase. Ex:「(幸いという) か (, 死者はいなかった.)」
|
||||
# 「(祈りが届いたせい) か (, 試験に合格した.)」
|
||||
# (c) 「かのように」. Ex:「(何もなかった) か (のように振る舞った.)」
|
||||
# e.g. か
|
||||
助詞-副助詞/並立助詞/終助詞
|
||||
#
|
||||
# particle-adnominalizer: The "no" that attaches to nouns and modifies
|
||||
# non-inflectional words.
|
||||
助詞-連体化
|
||||
#
|
||||
# particle-adnominalizer: The "ni" and "to" that appear following nouns and adverbs
|
||||
# that are giongo, giseigo, or gitaigo.
|
||||
# e.g. に, と
|
||||
助詞-副詞化
|
||||
#
|
||||
# particle-special: A particle that does not fit into one of the above classifications.
|
||||
# This includes particles that are used in Tanka, Haiku, and other poetry.
|
||||
# e.g. かな, けむ, ( しただろう) に, (あんた) にゃ(わからん), (俺) ん (家)
|
||||
助詞-特殊
|
||||
#
|
||||
#####
|
||||
# auxiliary-verb:
|
||||
助動詞
|
||||
#
|
||||
#####
|
||||
# interjection: Greetings and other exclamations.
|
||||
# e.g. おはよう, おはようございます, こんにちは, こんばんは, ありがとう, どうもありがとう, ありがとうございます,
|
||||
# いただきます, ごちそうさま, さよなら, さようなら, はい, いいえ, ごめん, ごめんなさい
|
||||
#感動詞
|
||||
#
|
||||
#####
|
||||
# symbol: unclassified Symbols.
|
||||
記号
|
||||
#
|
||||
# symbol-misc: A general symbol not in one of the categories below.
|
||||
# e.g. [○◎@$〒→+]
|
||||
記号-一般
|
||||
#
|
||||
# symbol-comma: Commas
|
||||
# e.g. [,、]
|
||||
記号-読点
|
||||
#
|
||||
# symbol-period: Periods and full stops.
|
||||
# e.g. [..。]
|
||||
記号-句点
|
||||
#
|
||||
# symbol-space: Full-width whitespace.
|
||||
記号-空白
|
||||
#
|
||||
# symbol-open_bracket:
|
||||
# e.g. [({‘“『【]
|
||||
記号-括弧開
|
||||
#
|
||||
# symbol-close_bracket:
|
||||
# e.g. [)}’”』」】]
|
||||
記号-括弧閉
|
||||
#
|
||||
# symbol-alphabetic:
|
||||
#記号-アルファベット
|
||||
#
|
||||
#####
|
||||
# other: unclassified other
|
||||
#その他
|
||||
#
|
||||
# other-interjection: Words that are hard to classify as noun-suffixes or
|
||||
# sentence-final particles.
|
||||
# e.g. (だ)ァ
|
||||
その他-間投
|
||||
#
|
||||
#####
|
||||
# filler: Aizuchi that occurs during a conversation or sounds inserted as filler.
|
||||
# e.g. あの, うんと, えと
|
||||
フィラー
|
||||
#
|
||||
#####
|
||||
# non-verbal: non-verbal sound.
|
||||
非言語音
|
||||
#
|
||||
#####
|
||||
# fragment:
|
||||
#語断片
|
||||
#
|
||||
#####
|
||||
# unknown: unknown part of speech.
|
||||
#未知語
|
||||
#
|
||||
##### End of file
|
|
@ -0,0 +1,125 @@
|
|||
# This file was created by Jacques Savoy and is distributed under the BSD license.
|
||||
# See http://members.unine.ch/jacques.savoy/clef/index.html.
|
||||
# Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
# Cleaned on October 11, 2009 (not normalized, so use before normalization)
|
||||
# This means that when modifying this list, you might need to add some
|
||||
# redundant entries, for example containing forms with both أ and ا
|
||||
من
|
||||
ومن
|
||||
منها
|
||||
منه
|
||||
في
|
||||
وفي
|
||||
فيها
|
||||
فيه
|
||||
و
|
||||
ف
|
||||
ثم
|
||||
او
|
||||
أو
|
||||
ب
|
||||
بها
|
||||
به
|
||||
ا
|
||||
أ
|
||||
اى
|
||||
اي
|
||||
أي
|
||||
أى
|
||||
لا
|
||||
ولا
|
||||
الا
|
||||
ألا
|
||||
إلا
|
||||
لكن
|
||||
ما
|
||||
وما
|
||||
كما
|
||||
فما
|
||||
عن
|
||||
مع
|
||||
اذا
|
||||
إذا
|
||||
ان
|
||||
أن
|
||||
إن
|
||||
انها
|
||||
أنها
|
||||
إنها
|
||||
انه
|
||||
أنه
|
||||
إنه
|
||||
بان
|
||||
بأن
|
||||
فان
|
||||
فأن
|
||||
وان
|
||||
وأن
|
||||
وإن
|
||||
التى
|
||||
التي
|
||||
الذى
|
||||
الذي
|
||||
الذين
|
||||
الى
|
||||
الي
|
||||
إلى
|
||||
إلي
|
||||
على
|
||||
عليها
|
||||
عليه
|
||||
اما
|
||||
أما
|
||||
إما
|
||||
ايضا
|
||||
أيضا
|
||||
كل
|
||||
وكل
|
||||
لم
|
||||
ولم
|
||||
لن
|
||||
ولن
|
||||
هى
|
||||
هي
|
||||
هو
|
||||
وهى
|
||||
وهي
|
||||
وهو
|
||||
فهى
|
||||
فهي
|
||||
فهو
|
||||
انت
|
||||
أنت
|
||||
لك
|
||||
لها
|
||||
له
|
||||
هذه
|
||||
هذا
|
||||
تلك
|
||||
ذلك
|
||||
هناك
|
||||
كانت
|
||||
كان
|
||||
يكون
|
||||
تكون
|
||||
وكانت
|
||||
وكان
|
||||
غير
|
||||
بعض
|
||||
قد
|
||||
نحو
|
||||
بين
|
||||
بينما
|
||||
منذ
|
||||
ضمن
|
||||
حيث
|
||||
الان
|
||||
الآن
|
||||
خلال
|
||||
بعد
|
||||
قبل
|
||||
حتى
|
||||
عند
|
||||
عندما
|
||||
لدى
|
||||
جميع
|
|
@ -0,0 +1,193 @@
|
|||
# This file was created by Jacques Savoy and is distributed under the BSD license.
|
||||
# See http://members.unine.ch/jacques.savoy/clef/index.html.
|
||||
# Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
а
|
||||
аз
|
||||
ако
|
||||
ала
|
||||
бе
|
||||
без
|
||||
беше
|
||||
би
|
||||
бил
|
||||
била
|
||||
били
|
||||
било
|
||||
близо
|
||||
бъдат
|
||||
бъде
|
||||
бяха
|
||||
в
|
||||
вас
|
||||
ваш
|
||||
ваша
|
||||
вероятно
|
||||
вече
|
||||
взема
|
||||
ви
|
||||
вие
|
||||
винаги
|
||||
все
|
||||
всеки
|
||||
всички
|
||||
всичко
|
||||
всяка
|
||||
във
|
||||
въпреки
|
||||
върху
|
||||
г
|
||||
ги
|
||||
главно
|
||||
го
|
||||
д
|
||||
да
|
||||
дали
|
||||
до
|
||||
докато
|
||||
докога
|
||||
дори
|
||||
досега
|
||||
доста
|
||||
е
|
||||
едва
|
||||
един
|
||||
ето
|
||||
за
|
||||
зад
|
||||
заедно
|
||||
заради
|
||||
засега
|
||||
затова
|
||||
защо
|
||||
защото
|
||||
и
|
||||
из
|
||||
или
|
||||
им
|
||||
има
|
||||
имат
|
||||
иска
|
||||
й
|
||||
каза
|
||||
как
|
||||
каква
|
||||
какво
|
||||
както
|
||||
какъв
|
||||
като
|
||||
кога
|
||||
когато
|
||||
което
|
||||
които
|
||||
кой
|
||||
който
|
||||
колко
|
||||
която
|
||||
къде
|
||||
където
|
||||
към
|
||||
ли
|
||||
м
|
||||
ме
|
||||
между
|
||||
мен
|
||||
ми
|
||||
мнозина
|
||||
мога
|
||||
могат
|
||||
може
|
||||
моля
|
||||
момента
|
||||
му
|
||||
н
|
||||
на
|
||||
над
|
||||
назад
|
||||
най
|
||||
направи
|
||||
напред
|
||||
например
|
||||
нас
|
||||
не
|
||||
него
|
||||
нея
|
||||
ни
|
||||
ние
|
||||
никой
|
||||
нито
|
||||
но
|
||||
някои
|
||||
някой
|
||||
няма
|
||||
обаче
|
||||
около
|
||||
освен
|
||||
особено
|
||||
от
|
||||
отгоре
|
||||
отново
|
||||
още
|
||||
пак
|
||||
по
|
||||
повече
|
||||
повечето
|
||||
под
|
||||
поне
|
||||
поради
|
||||
после
|
||||
почти
|
||||
прави
|
||||
пред
|
||||
преди
|
||||
през
|
||||
при
|
||||
пък
|
||||
първо
|
||||
с
|
||||
са
|
||||
само
|
||||
се
|
||||
сега
|
||||
си
|
||||
скоро
|
||||
след
|
||||
сме
|
||||
според
|
||||
сред
|
||||
срещу
|
||||
сте
|
||||
съм
|
||||
със
|
||||
също
|
||||
т
|
||||
тази
|
||||
така
|
||||
такива
|
||||
такъв
|
||||
там
|
||||
твой
|
||||
те
|
||||
тези
|
||||
ти
|
||||
тн
|
||||
то
|
||||
това
|
||||
тогава
|
||||
този
|
||||
той
|
||||
толкова
|
||||
точно
|
||||
трябва
|
||||
тук
|
||||
тъй
|
||||
тя
|
||||
тях
|
||||
у
|
||||
харесва
|
||||
ч
|
||||
че
|
||||
често
|
||||
чрез
|
||||
ще
|
||||
щом
|
||||
я
|
|
@ -0,0 +1,220 @@
|
|||
# Catalan stopwords from http://github.com/vcl/cue.language (Apache 2 Licensed)
|
||||
a
|
||||
abans
|
||||
ací
|
||||
ah
|
||||
així
|
||||
això
|
||||
al
|
||||
als
|
||||
aleshores
|
||||
algun
|
||||
alguna
|
||||
algunes
|
||||
alguns
|
||||
alhora
|
||||
allà
|
||||
allí
|
||||
allò
|
||||
altra
|
||||
altre
|
||||
altres
|
||||
amb
|
||||
ambdós
|
||||
ambdues
|
||||
apa
|
||||
aquell
|
||||
aquella
|
||||
aquelles
|
||||
aquells
|
||||
aquest
|
||||
aquesta
|
||||
aquestes
|
||||
aquests
|
||||
aquí
|
||||
baix
|
||||
cada
|
||||
cadascú
|
||||
cadascuna
|
||||
cadascunes
|
||||
cadascuns
|
||||
com
|
||||
contra
|
||||
d'un
|
||||
d'una
|
||||
d'unes
|
||||
d'uns
|
||||
dalt
|
||||
de
|
||||
del
|
||||
dels
|
||||
des
|
||||
després
|
||||
dins
|
||||
dintre
|
||||
donat
|
||||
doncs
|
||||
durant
|
||||
e
|
||||
eh
|
||||
el
|
||||
els
|
||||
em
|
||||
en
|
||||
encara
|
||||
ens
|
||||
entre
|
||||
érem
|
||||
eren
|
||||
éreu
|
||||
es
|
||||
és
|
||||
esta
|
||||
està
|
||||
estàvem
|
||||
estaven
|
||||
estàveu
|
||||
esteu
|
||||
et
|
||||
etc
|
||||
ets
|
||||
fins
|
||||
fora
|
||||
gairebé
|
||||
ha
|
||||
han
|
||||
has
|
||||
havia
|
||||
he
|
||||
hem
|
||||
heu
|
||||
hi
|
||||
ho
|
||||
i
|
||||
igual
|
||||
iguals
|
||||
ja
|
||||
l'hi
|
||||
la
|
||||
les
|
||||
li
|
||||
li'n
|
||||
llavors
|
||||
m'he
|
||||
ma
|
||||
mal
|
||||
malgrat
|
||||
mateix
|
||||
mateixa
|
||||
mateixes
|
||||
mateixos
|
||||
me
|
||||
mentre
|
||||
més
|
||||
meu
|
||||
meus
|
||||
meva
|
||||
meves
|
||||
molt
|
||||
molta
|
||||
moltes
|
||||
molts
|
||||
mon
|
||||
mons
|
||||
n'he
|
||||
n'hi
|
||||
ne
|
||||
ni
|
||||
no
|
||||
nogensmenys
|
||||
només
|
||||
nosaltres
|
||||
nostra
|
||||
nostre
|
||||
nostres
|
||||
o
|
||||
oh
|
||||
oi
|
||||
on
|
||||
pas
|
||||
pel
|
||||
pels
|
||||
per
|
||||
però
|
||||
perquè
|
||||
poc
|
||||
poca
|
||||
pocs
|
||||
poques
|
||||
potser
|
||||
propi
|
||||
qual
|
||||
quals
|
||||
quan
|
||||
quant
|
||||
que
|
||||
què
|
||||
quelcom
|
||||
qui
|
||||
quin
|
||||
quina
|
||||
quines
|
||||
quins
|
||||
s'ha
|
||||
s'han
|
||||
sa
|
||||
semblant
|
||||
semblants
|
||||
ses
|
||||
seu
|
||||
seus
|
||||
seva
|
||||
seva
|
||||
seves
|
||||
si
|
||||
sobre
|
||||
sobretot
|
||||
sóc
|
||||
solament
|
||||
sols
|
||||
son
|
||||
són
|
||||
sons
|
||||
sota
|
||||
sou
|
||||
t'ha
|
||||
t'han
|
||||
t'he
|
||||
ta
|
||||
tal
|
||||
també
|
||||
tampoc
|
||||
tan
|
||||
tant
|
||||
tanta
|
||||
tantes
|
||||
teu
|
||||
teus
|
||||
teva
|
||||
teves
|
||||
ton
|
||||
tons
|
||||
tot
|
||||
tota
|
||||
totes
|
||||
tots
|
||||
un
|
||||
una
|
||||
unes
|
||||
uns
|
||||
us
|
||||
va
|
||||
vaig
|
||||
vam
|
||||
van
|
||||
vas
|
||||
veu
|
||||
vosaltres
|
||||
vostra
|
||||
vostre
|
||||
vostres
|
|
@ -0,0 +1,172 @@
|
|||
a
|
||||
s
|
||||
k
|
||||
o
|
||||
i
|
||||
u
|
||||
v
|
||||
z
|
||||
dnes
|
||||
cz
|
||||
tímto
|
||||
budeš
|
||||
budem
|
||||
byli
|
||||
jseš
|
||||
můj
|
||||
svým
|
||||
ta
|
||||
tomto
|
||||
tohle
|
||||
tuto
|
||||
tyto
|
||||
jej
|
||||
zda
|
||||
proč
|
||||
máte
|
||||
tato
|
||||
kam
|
||||
tohoto
|
||||
kdo
|
||||
kteří
|
||||
mi
|
||||
nám
|
||||
tom
|
||||
tomuto
|
||||
mít
|
||||
nic
|
||||
proto
|
||||
kterou
|
||||
byla
|
||||
toho
|
||||
protože
|
||||
asi
|
||||
ho
|
||||
naši
|
||||
napište
|
||||
re
|
||||
což
|
||||
tím
|
||||
takže
|
||||
svých
|
||||
její
|
||||
svými
|
||||
jste
|
||||
aj
|
||||
tu
|
||||
tedy
|
||||
teto
|
||||
bylo
|
||||
kde
|
||||
ke
|
||||
pravé
|
||||
ji
|
||||
nad
|
||||
nejsou
|
||||
či
|
||||
pod
|
||||
téma
|
||||
mezi
|
||||
přes
|
||||
ty
|
||||
pak
|
||||
vám
|
||||
ani
|
||||
když
|
||||
však
|
||||
neg
|
||||
jsem
|
||||
tento
|
||||
článku
|
||||
články
|
||||
aby
|
||||
jsme
|
||||
před
|
||||
pta
|
||||
jejich
|
||||
byl
|
||||
ještě
|
||||
až
|
||||
bez
|
||||
také
|
||||
pouze
|
||||
první
|
||||
vaše
|
||||
která
|
||||
nás
|
||||
nový
|
||||
tipy
|
||||
pokud
|
||||
může
|
||||
strana
|
||||
jeho
|
||||
své
|
||||
jiné
|
||||
zprávy
|
||||
nové
|
||||
není
|
||||
vás
|
||||
jen
|
||||
podle
|
||||
zde
|
||||
už
|
||||
být
|
||||
více
|
||||
bude
|
||||
již
|
||||
než
|
||||
který
|
||||
by
|
||||
které
|
||||
co
|
||||
nebo
|
||||
ten
|
||||
tak
|
||||
má
|
||||
při
|
||||
od
|
||||
po
|
||||
jsou
|
||||
jak
|
||||
další
|
||||
ale
|
||||
si
|
||||
se
|
||||
ve
|
||||
to
|
||||
jako
|
||||
za
|
||||
zpět
|
||||
ze
|
||||
do
|
||||
pro
|
||||
je
|
||||
na
|
||||
atd
|
||||
atp
|
||||
jakmile
|
||||
přičemž
|
||||
já
|
||||
on
|
||||
ona
|
||||
ono
|
||||
oni
|
||||
ony
|
||||
my
|
||||
vy
|
||||
jí
|
||||
ji
|
||||
mě
|
||||
mne
|
||||
jemu
|
||||
tomu
|
||||
těm
|
||||
těmu
|
||||
němu
|
||||
němuž
|
||||
jehož
|
||||
jíž
|
||||
jelikož
|
||||
jež
|
||||
jakož
|
||||
načež
|
|
@ -0,0 +1,110 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/danish/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A Danish stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
| This is a ranked list (commonest to rarest) of stopwords derived from
|
||||
| a large text sample.
|
||||
|
||||
|
||||
og | and
|
||||
i | in
|
||||
jeg | I
|
||||
det | that (dem. pronoun)/it (pers. pronoun)
|
||||
at | that (in front of a sentence)/to (with infinitive)
|
||||
en | a/an
|
||||
den | it (pers. pronoun)/that (dem. pronoun)
|
||||
til | to/at/for/until/against/by/of/into, more
|
||||
er | present tense of "to be"
|
||||
som | who, as
|
||||
på | on/upon/in/on/at/to/after/of/with/for, on
|
||||
de | they
|
||||
med | with/by/in, along
|
||||
han | he
|
||||
af | of/by/from/off/for/in/with/on, off
|
||||
for | at/for/to/from/by/of/ago, in front/before, because
|
||||
ikke | not
|
||||
der | who/which, there/those
|
||||
var | past tense of "to be"
|
||||
mig | me/myself
|
||||
sig | oneself/himself/herself/itself/themselves
|
||||
men | but
|
||||
et | a/an/one, one (number), someone/somebody/one
|
||||
har | present tense of "to have"
|
||||
om | round/about/for/in/a, about/around/down, if
|
||||
vi | we
|
||||
min | my
|
||||
havde | past tense of "to have"
|
||||
ham | him
|
||||
hun | she
|
||||
nu | now
|
||||
over | over/above/across/by/beyond/past/on/about, over/past
|
||||
da | then, when/as/since
|
||||
fra | from/off/since, off, since
|
||||
du | you
|
||||
ud | out
|
||||
sin | his/her/its/one's
|
||||
dem | them
|
||||
os | us/ourselves
|
||||
op | up
|
||||
man | you/one
|
||||
hans | his
|
||||
hvor | where
|
||||
eller | or
|
||||
hvad | what
|
||||
skal | must/shall etc.
|
||||
selv | myself/youself/herself/ourselves etc., even
|
||||
her | here
|
||||
alle | all/everyone/everybody etc.
|
||||
vil | will (verb)
|
||||
blev | past tense of "to stay/to remain/to get/to become"
|
||||
kunne | could
|
||||
ind | in
|
||||
når | when
|
||||
være | present tense of "to be"
|
||||
dog | however/yet/after all
|
||||
noget | something
|
||||
ville | would
|
||||
jo | you know/you see (adv), yes
|
||||
deres | their/theirs
|
||||
efter | after/behind/according to/for/by/from, later/afterwards
|
||||
ned | down
|
||||
skulle | should
|
||||
denne | this
|
||||
end | than
|
||||
dette | this
|
||||
mit | my/mine
|
||||
også | also
|
||||
under | under/beneath/below/during, below/underneath
|
||||
have | have
|
||||
dig | you
|
||||
anden | other
|
||||
hende | her
|
||||
mine | my
|
||||
alt | everything
|
||||
meget | much/very, plenty of
|
||||
sit | his, her, its, one's
|
||||
sine | his, her, its, one's
|
||||
vor | our
|
||||
mod | against
|
||||
disse | these
|
||||
hvis | if
|
||||
din | your/yours
|
||||
nogle | some
|
||||
hos | by/at
|
||||
blive | be/become
|
||||
mange | many
|
||||
ad | by/through
|
||||
bliver | present tense of "to be/to become"
|
||||
hendes | her/hers
|
||||
været | be
|
||||
thi | for (conj)
|
||||
jer | you
|
||||
sådan | such, like this/like that
|
|
@ -0,0 +1,294 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/german/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A German stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
| The number of forms in this list is reduced significantly by passing it
|
||||
| through the German stemmer.
|
||||
|
||||
|
||||
aber | but
|
||||
|
||||
alle | all
|
||||
allem
|
||||
allen
|
||||
aller
|
||||
alles
|
||||
|
||||
als | than, as
|
||||
also | so
|
||||
am | an + dem
|
||||
an | at
|
||||
|
||||
ander | other
|
||||
andere
|
||||
anderem
|
||||
anderen
|
||||
anderer
|
||||
anderes
|
||||
anderm
|
||||
andern
|
||||
anderr
|
||||
anders
|
||||
|
||||
auch | also
|
||||
auf | on
|
||||
aus | out of
|
||||
bei | by
|
||||
bin | am
|
||||
bis | until
|
||||
bist | art
|
||||
da | there
|
||||
damit | with it
|
||||
dann | then
|
||||
|
||||
der | the
|
||||
den
|
||||
des
|
||||
dem
|
||||
die
|
||||
das
|
||||
|
||||
daß | that
|
||||
|
||||
derselbe | the same
|
||||
derselben
|
||||
denselben
|
||||
desselben
|
||||
demselben
|
||||
dieselbe
|
||||
dieselben
|
||||
dasselbe
|
||||
|
||||
dazu | to that
|
||||
|
||||
dein | thy
|
||||
deine
|
||||
deinem
|
||||
deinen
|
||||
deiner
|
||||
deines
|
||||
|
||||
denn | because
|
||||
|
||||
derer | of those
|
||||
dessen | of him
|
||||
|
||||
dich | thee
|
||||
dir | to thee
|
||||
du | thou
|
||||
|
||||
dies | this
|
||||
diese
|
||||
diesem
|
||||
diesen
|
||||
dieser
|
||||
dieses
|
||||
|
||||
|
||||
doch | (several meanings)
|
||||
dort | (over) there
|
||||
|
||||
|
||||
durch | through
|
||||
|
||||
ein | a
|
||||
eine
|
||||
einem
|
||||
einen
|
||||
einer
|
||||
eines
|
||||
|
||||
einig | some
|
||||
einige
|
||||
einigem
|
||||
einigen
|
||||
einiger
|
||||
einiges
|
||||
|
||||
einmal | once
|
||||
|
||||
er | he
|
||||
ihn | him
|
||||
ihm | to him
|
||||
|
||||
es | it
|
||||
etwas | something
|
||||
|
||||
euer | your
|
||||
eure
|
||||
eurem
|
||||
euren
|
||||
eurer
|
||||
eures
|
||||
|
||||
für | for
|
||||
gegen | towards
|
||||
gewesen | p.p. of sein
|
||||
hab | have
|
||||
habe | have
|
||||
haben | have
|
||||
hat | has
|
||||
hatte | had
|
||||
hatten | had
|
||||
hier | here
|
||||
hin | there
|
||||
hinter | behind
|
||||
|
||||
ich | I
|
||||
mich | me
|
||||
mir | to me
|
||||
|
||||
|
||||
ihr | you, to her
|
||||
ihre
|
||||
ihrem
|
||||
ihren
|
||||
ihrer
|
||||
ihres
|
||||
euch | to you
|
||||
|
||||
im | in + dem
|
||||
in | in
|
||||
indem | while
|
||||
ins | in + das
|
||||
ist | is
|
||||
|
||||
jede | each, every
|
||||
jedem
|
||||
jeden
|
||||
jeder
|
||||
jedes
|
||||
|
||||
jene | that
|
||||
jenem
|
||||
jenen
|
||||
jener
|
||||
jenes
|
||||
|
||||
jetzt | now
|
||||
kann | can
|
||||
|
||||
kein | no
|
||||
keine
|
||||
keinem
|
||||
keinen
|
||||
keiner
|
||||
keines
|
||||
|
||||
können | can
|
||||
könnte | could
|
||||
machen | do
|
||||
man | one
|
||||
|
||||
manche | some, many a
|
||||
manchem
|
||||
manchen
|
||||
mancher
|
||||
manches
|
||||
|
||||
mein | my
|
||||
meine
|
||||
meinem
|
||||
meinen
|
||||
meiner
|
||||
meines
|
||||
|
||||
mit | with
|
||||
muss | must
|
||||
musste | had to
|
||||
nach | to(wards)
|
||||
nicht | not
|
||||
nichts | nothing
|
||||
noch | still, yet
|
||||
nun | now
|
||||
nur | only
|
||||
ob | whether
|
||||
oder | or
|
||||
ohne | without
|
||||
sehr | very
|
||||
|
||||
sein | his
|
||||
seine
|
||||
seinem
|
||||
seinen
|
||||
seiner
|
||||
seines
|
||||
|
||||
selbst | self
|
||||
sich | herself
|
||||
|
||||
sie | they, she
|
||||
ihnen | to them
|
||||
|
||||
sind | are
|
||||
so | so
|
||||
|
||||
solche | such
|
||||
solchem
|
||||
solchen
|
||||
solcher
|
||||
solches
|
||||
|
||||
soll | shall
|
||||
sollte | should
|
||||
sondern | but
|
||||
sonst | else
|
||||
über | over
|
||||
um | about, around
|
||||
und | and
|
||||
|
||||
uns | us
|
||||
unse
|
||||
unsem
|
||||
unsen
|
||||
unser
|
||||
unses
|
||||
|
||||
unter | under
|
||||
viel | much
|
||||
vom | von + dem
|
||||
von | from
|
||||
vor | before
|
||||
während | while
|
||||
war | was
|
||||
waren | were
|
||||
warst | wast
|
||||
was | what
|
||||
weg | away, off
|
||||
weil | because
|
||||
weiter | further
|
||||
|
||||
welche | which
|
||||
welchem
|
||||
welchen
|
||||
welcher
|
||||
welches
|
||||
|
||||
wenn | when
|
||||
werde | will
|
||||
werden | will
|
||||
wie | how
|
||||
wieder | again
|
||||
will | want
|
||||
wir | we
|
||||
wird | will
|
||||
wirst | willst
|
||||
wo | where
|
||||
wollen | want
|
||||
wollte | wanted
|
||||
würde | would
|
||||
würden | would
|
||||
zu | to
|
||||
zum | zu + dem
|
||||
zur | zu + der
|
||||
zwar | indeed
|
||||
zwischen | between
|
||||
|
|
@ -0,0 +1,78 @@
|
|||
# Lucene Greek Stopwords list
|
||||
# Note: by default this file is used after GreekLowerCaseFilter,
|
||||
# so when modifying this file use 'σ' instead of 'ς'
|
||||
ο
|
||||
η
|
||||
το
|
||||
οι
|
||||
τα
|
||||
του
|
||||
τησ
|
||||
των
|
||||
τον
|
||||
την
|
||||
και
|
||||
κι
|
||||
κ
|
||||
ειμαι
|
||||
εισαι
|
||||
ειναι
|
||||
ειμαστε
|
||||
ειστε
|
||||
στο
|
||||
στον
|
||||
στη
|
||||
στην
|
||||
μα
|
||||
αλλα
|
||||
απο
|
||||
για
|
||||
προσ
|
||||
με
|
||||
σε
|
||||
ωσ
|
||||
παρα
|
||||
αντι
|
||||
κατα
|
||||
μετα
|
||||
θα
|
||||
να
|
||||
δε
|
||||
δεν
|
||||
μη
|
||||
μην
|
||||
επι
|
||||
ενω
|
||||
εαν
|
||||
αν
|
||||
τοτε
|
||||
που
|
||||
πωσ
|
||||
ποιοσ
|
||||
ποια
|
||||
ποιο
|
||||
ποιοι
|
||||
ποιεσ
|
||||
ποιων
|
||||
ποιουσ
|
||||
αυτοσ
|
||||
αυτη
|
||||
αυτο
|
||||
αυτοι
|
||||
αυτων
|
||||
αυτουσ
|
||||
αυτεσ
|
||||
αυτα
|
||||
εκεινοσ
|
||||
εκεινη
|
||||
εκεινο
|
||||
εκεινοι
|
||||
εκεινεσ
|
||||
εκεινα
|
||||
εκεινων
|
||||
εκεινουσ
|
||||
οπωσ
|
||||
ομωσ
|
||||
ισωσ
|
||||
οσο
|
||||
οτι
|
|
@ -0,0 +1,54 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
# a couple of test stopwords to test that the words are really being
|
||||
# configured from this file:
|
||||
stopworda
|
||||
stopwordb
|
||||
|
||||
# Standard english stop words taken from Lucene's StopAnalyzer
|
||||
a
|
||||
an
|
||||
and
|
||||
are
|
||||
as
|
||||
at
|
||||
be
|
||||
but
|
||||
by
|
||||
for
|
||||
if
|
||||
in
|
||||
into
|
||||
is
|
||||
it
|
||||
no
|
||||
not
|
||||
of
|
||||
on
|
||||
or
|
||||
such
|
||||
that
|
||||
the
|
||||
their
|
||||
then
|
||||
there
|
||||
these
|
||||
they
|
||||
this
|
||||
to
|
||||
was
|
||||
will
|
||||
with
|
|
@ -0,0 +1,356 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/spanish/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A Spanish stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
|
||||
| The following is a ranked list (commonest to rarest) of stopwords
|
||||
| deriving from a large sample of text.
|
||||
|
||||
| Extra words have been added at the end.
|
||||
|
||||
de | from, of
|
||||
la | the, her
|
||||
que | who, that
|
||||
el | the
|
||||
en | in
|
||||
y | and
|
||||
a | to
|
||||
los | the, them
|
||||
del | de + el
|
||||
se | himself, from him etc
|
||||
las | the, them
|
||||
por | for, by, etc
|
||||
un | a
|
||||
para | for
|
||||
con | with
|
||||
no | no
|
||||
una | a
|
||||
su | his, her
|
||||
al | a + el
|
||||
| es from SER
|
||||
lo | him
|
||||
como | how
|
||||
más | more
|
||||
pero | pero
|
||||
sus | su plural
|
||||
le | to him, her
|
||||
ya | already
|
||||
o | or
|
||||
| fue from SER
|
||||
este | this
|
||||
| ha from HABER
|
||||
sí | himself etc
|
||||
porque | because
|
||||
esta | this
|
||||
| son from SER
|
||||
entre | between
|
||||
| está from ESTAR
|
||||
cuando | when
|
||||
muy | very
|
||||
sin | without
|
||||
sobre | on
|
||||
| ser from SER
|
||||
| tiene from TENER
|
||||
también | also
|
||||
me | me
|
||||
hasta | until
|
||||
hay | there is/are
|
||||
donde | where
|
||||
| han from HABER
|
||||
quien | whom, that
|
||||
| están from ESTAR
|
||||
| estado from ESTAR
|
||||
desde | from
|
||||
todo | all
|
||||
nos | us
|
||||
durante | during
|
||||
| estados from ESTAR
|
||||
todos | all
|
||||
uno | a
|
||||
les | to them
|
||||
ni | nor
|
||||
contra | against
|
||||
otros | other
|
||||
| fueron from SER
|
||||
ese | that
|
||||
eso | that
|
||||
| había from HABER
|
||||
ante | before
|
||||
ellos | they
|
||||
e | and (variant of y)
|
||||
esto | this
|
||||
mí | me
|
||||
antes | before
|
||||
algunos | some
|
||||
qué | what?
|
||||
unos | a
|
||||
yo | I
|
||||
otro | other
|
||||
otras | other
|
||||
otra | other
|
||||
él | he
|
||||
tanto | so much, many
|
||||
esa | that
|
||||
estos | these
|
||||
mucho | much, many
|
||||
quienes | who
|
||||
nada | nothing
|
||||
muchos | many
|
||||
cual | who
|
||||
| sea from SER
|
||||
poco | few
|
||||
ella | she
|
||||
estar | to be
|
||||
| haber from HABER
|
||||
estas | these
|
||||
| estaba from ESTAR
|
||||
| estamos from ESTAR
|
||||
algunas | some
|
||||
algo | something
|
||||
nosotros | we
|
||||
|
||||
| other forms
|
||||
|
||||
mi | me
|
||||
mis | mi plural
|
||||
tú | thou
|
||||
te | thee
|
||||
ti | thee
|
||||
tu | thy
|
||||
tus | tu plural
|
||||
ellas | they
|
||||
nosotras | we
|
||||
vosotros | you
|
||||
vosotras | you
|
||||
os | you
|
||||
mío | mine
|
||||
mía |
|
||||
míos |
|
||||
mías |
|
||||
tuyo | thine
|
||||
tuya |
|
||||
tuyos |
|
||||
tuyas |
|
||||
suyo | his, hers, theirs
|
||||
suya |
|
||||
suyos |
|
||||
suyas |
|
||||
nuestro | ours
|
||||
nuestra |
|
||||
nuestros |
|
||||
nuestras |
|
||||
vuestro | yours
|
||||
vuestra |
|
||||
vuestros |
|
||||
vuestras |
|
||||
esos | those
|
||||
esas | those
|
||||
|
||||
| forms of estar, to be (not including the infinitive):
|
||||
estoy
|
||||
estás
|
||||
está
|
||||
estamos
|
||||
estáis
|
||||
están
|
||||
esté
|
||||
estés
|
||||
estemos
|
||||
estéis
|
||||
estén
|
||||
estaré
|
||||
estarás
|
||||
estará
|
||||
estaremos
|
||||
estaréis
|
||||
estarán
|
||||
estaría
|
||||
estarías
|
||||
estaríamos
|
||||
estaríais
|
||||
estarían
|
||||
estaba
|
||||
estabas
|
||||
estábamos
|
||||
estabais
|
||||
estaban
|
||||
estuve
|
||||
estuviste
|
||||
estuvo
|
||||
estuvimos
|
||||
estuvisteis
|
||||
estuvieron
|
||||
estuviera
|
||||
estuvieras
|
||||
estuviéramos
|
||||
estuvierais
|
||||
estuvieran
|
||||
estuviese
|
||||
estuvieses
|
||||
estuviésemos
|
||||
estuvieseis
|
||||
estuviesen
|
||||
estando
|
||||
estado
|
||||
estada
|
||||
estados
|
||||
estadas
|
||||
estad
|
||||
|
||||
| forms of haber, to have (not including the infinitive):
|
||||
he
|
||||
has
|
||||
ha
|
||||
hemos
|
||||
habéis
|
||||
han
|
||||
haya
|
||||
hayas
|
||||
hayamos
|
||||
hayáis
|
||||
hayan
|
||||
habré
|
||||
habrás
|
||||
habrá
|
||||
habremos
|
||||
habréis
|
||||
habrán
|
||||
habría
|
||||
habrías
|
||||
habríamos
|
||||
habríais
|
||||
habrían
|
||||
había
|
||||
habías
|
||||
habíamos
|
||||
habíais
|
||||
habían
|
||||
hube
|
||||
hubiste
|
||||
hubo
|
||||
hubimos
|
||||
hubisteis
|
||||
hubieron
|
||||
hubiera
|
||||
hubieras
|
||||
hubiéramos
|
||||
hubierais
|
||||
hubieran
|
||||
hubiese
|
||||
hubieses
|
||||
hubiésemos
|
||||
hubieseis
|
||||
hubiesen
|
||||
habiendo
|
||||
habido
|
||||
habida
|
||||
habidos
|
||||
habidas
|
||||
|
||||
| forms of ser, to be (not including the infinitive):
|
||||
soy
|
||||
eres
|
||||
es
|
||||
somos
|
||||
sois
|
||||
son
|
||||
sea
|
||||
seas
|
||||
seamos
|
||||
seáis
|
||||
sean
|
||||
seré
|
||||
serás
|
||||
será
|
||||
seremos
|
||||
seréis
|
||||
serán
|
||||
sería
|
||||
serías
|
||||
seríamos
|
||||
seríais
|
||||
serían
|
||||
era
|
||||
eras
|
||||
éramos
|
||||
erais
|
||||
eran
|
||||
fui
|
||||
fuiste
|
||||
fue
|
||||
fuimos
|
||||
fuisteis
|
||||
fueron
|
||||
fuera
|
||||
fueras
|
||||
fuéramos
|
||||
fuerais
|
||||
fueran
|
||||
fuese
|
||||
fueses
|
||||
fuésemos
|
||||
fueseis
|
||||
fuesen
|
||||
siendo
|
||||
sido
|
||||
| sed also means 'thirst'
|
||||
|
||||
| forms of tener, to have (not including the infinitive):
|
||||
tengo
|
||||
tienes
|
||||
tiene
|
||||
tenemos
|
||||
tenéis
|
||||
tienen
|
||||
tenga
|
||||
tengas
|
||||
tengamos
|
||||
tengáis
|
||||
tengan
|
||||
tendré
|
||||
tendrás
|
||||
tendrá
|
||||
tendremos
|
||||
tendréis
|
||||
tendrán
|
||||
tendría
|
||||
tendrías
|
||||
tendríamos
|
||||
tendríais
|
||||
tendrían
|
||||
tenía
|
||||
tenías
|
||||
teníamos
|
||||
teníais
|
||||
tenían
|
||||
tuve
|
||||
tuviste
|
||||
tuvo
|
||||
tuvimos
|
||||
tuvisteis
|
||||
tuvieron
|
||||
tuviera
|
||||
tuvieras
|
||||
tuviéramos
|
||||
tuvierais
|
||||
tuvieran
|
||||
tuviese
|
||||
tuvieses
|
||||
tuviésemos
|
||||
tuvieseis
|
||||
tuviesen
|
||||
teniendo
|
||||
tenido
|
||||
tenida
|
||||
tenidos
|
||||
tenidas
|
||||
tened
|
||||
|
|
@ -0,0 +1,99 @@
|
|||
# example set of basque stopwords
|
||||
al
|
||||
anitz
|
||||
arabera
|
||||
asko
|
||||
baina
|
||||
bat
|
||||
batean
|
||||
batek
|
||||
bati
|
||||
batzuei
|
||||
batzuek
|
||||
batzuetan
|
||||
batzuk
|
||||
bera
|
||||
beraiek
|
||||
berau
|
||||
berauek
|
||||
bere
|
||||
berori
|
||||
beroriek
|
||||
beste
|
||||
bezala
|
||||
da
|
||||
dago
|
||||
dira
|
||||
ditu
|
||||
du
|
||||
dute
|
||||
edo
|
||||
egin
|
||||
ere
|
||||
eta
|
||||
eurak
|
||||
ez
|
||||
gainera
|
||||
gu
|
||||
gutxi
|
||||
guzti
|
||||
haiei
|
||||
haiek
|
||||
haietan
|
||||
hainbeste
|
||||
hala
|
||||
han
|
||||
handik
|
||||
hango
|
||||
hara
|
||||
hari
|
||||
hark
|
||||
hartan
|
||||
hau
|
||||
hauei
|
||||
hauek
|
||||
hauetan
|
||||
hemen
|
||||
hemendik
|
||||
hemengo
|
||||
hi
|
||||
hona
|
||||
honek
|
||||
honela
|
||||
honetan
|
||||
honi
|
||||
hor
|
||||
hori
|
||||
horiei
|
||||
horiek
|
||||
horietan
|
||||
horko
|
||||
horra
|
||||
horrek
|
||||
horrela
|
||||
horretan
|
||||
horri
|
||||
hortik
|
||||
hura
|
||||
izan
|
||||
ni
|
||||
noiz
|
||||
nola
|
||||
non
|
||||
nondik
|
||||
nongo
|
||||
nor
|
||||
nora
|
||||
ze
|
||||
zein
|
||||
zen
|
||||
zenbait
|
||||
zenbat
|
||||
zer
|
||||
zergatik
|
||||
ziren
|
||||
zituen
|
||||
zu
|
||||
zuek
|
||||
zuen
|
||||
zuten
|
|
@ -0,0 +1,313 @@
|
|||
# This file was created by Jacques Savoy and is distributed under the BSD license.
|
||||
# See http://members.unine.ch/jacques.savoy/clef/index.html.
|
||||
# Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
# Note: by default this file is used after normalization, so when adding entries
|
||||
# to this file, use the arabic 'ي' instead of 'ی'
|
||||
انان
|
||||
نداشته
|
||||
سراسر
|
||||
خياه
|
||||
ايشان
|
||||
وي
|
||||
تاكنون
|
||||
بيشتري
|
||||
دوم
|
||||
پس
|
||||
ناشي
|
||||
وگو
|
||||
يا
|
||||
داشتند
|
||||
سپس
|
||||
هنگام
|
||||
هرگز
|
||||
پنج
|
||||
نشان
|
||||
امسال
|
||||
ديگر
|
||||
گروهي
|
||||
شدند
|
||||
چطور
|
||||
ده
|
||||
و
|
||||
دو
|
||||
نخستين
|
||||
ولي
|
||||
چرا
|
||||
چه
|
||||
وسط
|
||||
ه
|
||||
كدام
|
||||
قابل
|
||||
يك
|
||||
رفت
|
||||
هفت
|
||||
همچنين
|
||||
در
|
||||
هزار
|
||||
بله
|
||||
بلي
|
||||
شايد
|
||||
اما
|
||||
شناسي
|
||||
گرفته
|
||||
دهد
|
||||
داشته
|
||||
دانست
|
||||
داشتن
|
||||
خواهيم
|
||||
ميليارد
|
||||
وقتيكه
|
||||
امد
|
||||
خواهد
|
||||
جز
|
||||
اورده
|
||||
شده
|
||||
بلكه
|
||||
خدمات
|
||||
شدن
|
||||
برخي
|
||||
نبود
|
||||
بسياري
|
||||
جلوگيري
|
||||
حق
|
||||
كردند
|
||||
نوعي
|
||||
بعري
|
||||
نكرده
|
||||
نظير
|
||||
نبايد
|
||||
بوده
|
||||
بودن
|
||||
داد
|
||||
اورد
|
||||
هست
|
||||
جايي
|
||||
شود
|
||||
دنبال
|
||||
داده
|
||||
بايد
|
||||
سابق
|
||||
هيچ
|
||||
همان
|
||||
انجا
|
||||
كمتر
|
||||
كجاست
|
||||
گردد
|
||||
كسي
|
||||
تر
|
||||
مردم
|
||||
تان
|
||||
دادن
|
||||
بودند
|
||||
سري
|
||||
جدا
|
||||
ندارند
|
||||
مگر
|
||||
يكديگر
|
||||
دارد
|
||||
دهند
|
||||
بنابراين
|
||||
هنگامي
|
||||
سمت
|
||||
جا
|
||||
انچه
|
||||
خود
|
||||
دادند
|
||||
زياد
|
||||
دارند
|
||||
اثر
|
||||
بدون
|
||||
بهترين
|
||||
بيشتر
|
||||
البته
|
||||
به
|
||||
براساس
|
||||
بيرون
|
||||
كرد
|
||||
بعضي
|
||||
گرفت
|
||||
توي
|
||||
اي
|
||||
ميليون
|
||||
او
|
||||
جريان
|
||||
تول
|
||||
بر
|
||||
مانند
|
||||
برابر
|
||||
باشيم
|
||||
مدتي
|
||||
گويند
|
||||
اكنون
|
||||
تا
|
||||
تنها
|
||||
جديد
|
||||
چند
|
||||
بي
|
||||
نشده
|
||||
كردن
|
||||
كردم
|
||||
گويد
|
||||
كرده
|
||||
كنيم
|
||||
نمي
|
||||
نزد
|
||||
روي
|
||||
قصد
|
||||
فقط
|
||||
بالاي
|
||||
ديگران
|
||||
اين
|
||||
ديروز
|
||||
توسط
|
||||
سوم
|
||||
ايم
|
||||
دانند
|
||||
سوي
|
||||
استفاده
|
||||
شما
|
||||
كنار
|
||||
داريم
|
||||
ساخته
|
||||
طور
|
||||
امده
|
||||
رفته
|
||||
نخست
|
||||
بيست
|
||||
نزديك
|
||||
طي
|
||||
كنيد
|
||||
از
|
||||
انها
|
||||
تمامي
|
||||
داشت
|
||||
يكي
|
||||
طريق
|
||||
اش
|
||||
چيست
|
||||
روب
|
||||
نمايد
|
||||
گفت
|
||||
چندين
|
||||
چيزي
|
||||
تواند
|
||||
ام
|
||||
ايا
|
||||
با
|
||||
ان
|
||||
ايد
|
||||
ترين
|
||||
اينكه
|
||||
ديگري
|
||||
راه
|
||||
هايي
|
||||
بروز
|
||||
همچنان
|
||||
پاعين
|
||||
كس
|
||||
حدود
|
||||
مختلف
|
||||
مقابل
|
||||
چيز
|
||||
گيرد
|
||||
ندارد
|
||||
ضد
|
||||
همچون
|
||||
سازي
|
||||
شان
|
||||
مورد
|
||||
باره
|
||||
مرسي
|
||||
خويش
|
||||
برخوردار
|
||||
چون
|
||||
خارج
|
||||
شش
|
||||
هنوز
|
||||
تحت
|
||||
ضمن
|
||||
هستيم
|
||||
گفته
|
||||
فكر
|
||||
بسيار
|
||||
پيش
|
||||
براي
|
||||
روزهاي
|
||||
انكه
|
||||
نخواهد
|
||||
بالا
|
||||
كل
|
||||
وقتي
|
||||
كي
|
||||
چنين
|
||||
كه
|
||||
گيري
|
||||
نيست
|
||||
است
|
||||
كجا
|
||||
كند
|
||||
نيز
|
||||
يابد
|
||||
بندي
|
||||
حتي
|
||||
توانند
|
||||
عقب
|
||||
خواست
|
||||
كنند
|
||||
بين
|
||||
تمام
|
||||
همه
|
||||
ما
|
||||
باشند
|
||||
مثل
|
||||
شد
|
||||
اري
|
||||
باشد
|
||||
اره
|
||||
طبق
|
||||
بعد
|
||||
اگر
|
||||
صورت
|
||||
غير
|
||||
جاي
|
||||
بيش
|
||||
ريزي
|
||||
اند
|
||||
زيرا
|
||||
چگونه
|
||||
بار
|
||||
لطفا
|
||||
مي
|
||||
درباره
|
||||
من
|
||||
ديده
|
||||
همين
|
||||
گذاري
|
||||
برداري
|
||||
علت
|
||||
گذاشته
|
||||
هم
|
||||
فوق
|
||||
نه
|
||||
ها
|
||||
شوند
|
||||
اباد
|
||||
همواره
|
||||
هر
|
||||
اول
|
||||
خواهند
|
||||
چهار
|
||||
نام
|
||||
امروز
|
||||
مان
|
||||
هاي
|
||||
قبل
|
||||
كنم
|
||||
سعي
|
||||
تازه
|
||||
را
|
||||
هستند
|
||||
زير
|
||||
جلوي
|
||||
عنوان
|
||||
بود
|
|
@ -0,0 +1,97 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/finnish/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| forms of BE
|
||||
|
||||
olla
|
||||
olen
|
||||
olet
|
||||
on
|
||||
olemme
|
||||
olette
|
||||
ovat
|
||||
ole | negative form
|
||||
|
||||
oli
|
||||
olisi
|
||||
olisit
|
||||
olisin
|
||||
olisimme
|
||||
olisitte
|
||||
olisivat
|
||||
olit
|
||||
olin
|
||||
olimme
|
||||
olitte
|
||||
olivat
|
||||
ollut
|
||||
olleet
|
||||
|
||||
en | negation
|
||||
et
|
||||
ei
|
||||
emme
|
||||
ette
|
||||
eivät
|
||||
|
||||
|Nom Gen Acc Part Iness Elat Illat Adess Ablat Allat Ess Trans
|
||||
minä minun minut minua minussa minusta minuun minulla minulta minulle | I
|
||||
sinä sinun sinut sinua sinussa sinusta sinuun sinulla sinulta sinulle | you
|
||||
hän hänen hänet häntä hänessä hänestä häneen hänellä häneltä hänelle | he she
|
||||
me meidän meidät meitä meissä meistä meihin meillä meiltä meille | we
|
||||
te teidän teidät teitä teissä teistä teihin teillä teiltä teille | you
|
||||
he heidän heidät heitä heissä heistä heihin heillä heiltä heille | they
|
||||
|
||||
tämä tämän tätä tässä tästä tähän tallä tältä tälle tänä täksi | this
|
||||
tuo tuon tuotä tuossa tuosta tuohon tuolla tuolta tuolle tuona tuoksi | that
|
||||
se sen sitä siinä siitä siihen sillä siltä sille sinä siksi | it
|
||||
nämä näiden näitä näissä näistä näihin näillä näiltä näille näinä näiksi | these
|
||||
nuo noiden noita noissa noista noihin noilla noilta noille noina noiksi | those
|
||||
ne niiden niitä niissä niistä niihin niillä niiltä niille niinä niiksi | they
|
||||
|
||||
kuka kenen kenet ketä kenessä kenestä keneen kenellä keneltä kenelle kenenä keneksi| who
|
||||
ketkä keiden ketkä keitä keissä keistä keihin keillä keiltä keille keinä keiksi | (pl)
|
||||
mikä minkä minkä mitä missä mistä mihin millä miltä mille minä miksi | which what
|
||||
mitkä | (pl)
|
||||
|
||||
joka jonka jota jossa josta johon jolla jolta jolle jona joksi | who which
|
||||
jotka joiden joita joissa joista joihin joilla joilta joille joina joiksi | (pl)
|
||||
|
||||
| conjunctions
|
||||
|
||||
että | that
|
||||
ja | and
|
||||
jos | if
|
||||
koska | because
|
||||
kuin | than
|
||||
mutta | but
|
||||
niin | so
|
||||
sekä | and
|
||||
sillä | for
|
||||
tai | or
|
||||
vaan | but
|
||||
vai | or
|
||||
vaikka | although
|
||||
|
||||
|
||||
| prepositions
|
||||
|
||||
kanssa | with
|
||||
mukaan | according to
|
||||
noin | about
|
||||
poikki | across
|
||||
yli | over, across
|
||||
|
||||
| other
|
||||
|
||||
kun | when
|
||||
niin | so
|
||||
nyt | now
|
||||
itse | self
|
||||
|
|
@ -0,0 +1,186 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/french/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A French stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
au | a + le
|
||||
aux | a + les
|
||||
avec | with
|
||||
ce | this
|
||||
ces | these
|
||||
dans | with
|
||||
de | of
|
||||
des | de + les
|
||||
du | de + le
|
||||
elle | she
|
||||
en | `of them' etc
|
||||
et | and
|
||||
eux | them
|
||||
il | he
|
||||
je | I
|
||||
la | the
|
||||
le | the
|
||||
leur | their
|
||||
lui | him
|
||||
ma | my (fem)
|
||||
mais | but
|
||||
me | me
|
||||
même | same; as in moi-même (myself) etc
|
||||
mes | me (pl)
|
||||
moi | me
|
||||
mon | my (masc)
|
||||
ne | not
|
||||
nos | our (pl)
|
||||
notre | our
|
||||
nous | we
|
||||
on | one
|
||||
ou | where
|
||||
par | by
|
||||
pas | not
|
||||
pour | for
|
||||
qu | que before vowel
|
||||
que | that
|
||||
qui | who
|
||||
sa | his, her (fem)
|
||||
se | oneself
|
||||
ses | his (pl)
|
||||
son | his, her (masc)
|
||||
sur | on
|
||||
ta | thy (fem)
|
||||
te | thee
|
||||
tes | thy (pl)
|
||||
toi | thee
|
||||
ton | thy (masc)
|
||||
tu | thou
|
||||
un | a
|
||||
une | a
|
||||
vos | your (pl)
|
||||
votre | your
|
||||
vous | you
|
||||
|
||||
| single letter forms
|
||||
|
||||
c | c'
|
||||
d | d'
|
||||
j | j'
|
||||
l | l'
|
||||
à | to, at
|
||||
m | m'
|
||||
n | n'
|
||||
s | s'
|
||||
t | t'
|
||||
y | there
|
||||
|
||||
| forms of être (not including the infinitive):
|
||||
été
|
||||
étée
|
||||
étées
|
||||
étés
|
||||
étant
|
||||
suis
|
||||
es
|
||||
est
|
||||
sommes
|
||||
êtes
|
||||
sont
|
||||
serai
|
||||
seras
|
||||
sera
|
||||
serons
|
||||
serez
|
||||
seront
|
||||
serais
|
||||
serait
|
||||
serions
|
||||
seriez
|
||||
seraient
|
||||
étais
|
||||
était
|
||||
étions
|
||||
étiez
|
||||
étaient
|
||||
fus
|
||||
fut
|
||||
fûmes
|
||||
fûtes
|
||||
furent
|
||||
sois
|
||||
soit
|
||||
soyons
|
||||
soyez
|
||||
soient
|
||||
fusse
|
||||
fusses
|
||||
fût
|
||||
fussions
|
||||
fussiez
|
||||
fussent
|
||||
|
||||
| forms of avoir (not including the infinitive):
|
||||
ayant
|
||||
eu
|
||||
eue
|
||||
eues
|
||||
eus
|
||||
ai
|
||||
as
|
||||
avons
|
||||
avez
|
||||
ont
|
||||
aurai
|
||||
auras
|
||||
aura
|
||||
aurons
|
||||
aurez
|
||||
auront
|
||||
aurais
|
||||
aurait
|
||||
aurions
|
||||
auriez
|
||||
auraient
|
||||
avais
|
||||
avait
|
||||
avions
|
||||
aviez
|
||||
avaient
|
||||
eut
|
||||
eûmes
|
||||
eûtes
|
||||
eurent
|
||||
aie
|
||||
aies
|
||||
ait
|
||||
ayons
|
||||
ayez
|
||||
aient
|
||||
eusse
|
||||
eusses
|
||||
eût
|
||||
eussions
|
||||
eussiez
|
||||
eussent
|
||||
|
||||
| Later additions (from Jean-Christophe Deschamps)
|
||||
ceci | this
|
||||
cela | that
|
||||
celà | that
|
||||
cet | this
|
||||
cette | this
|
||||
ici | here
|
||||
ils | they
|
||||
les | the (pl)
|
||||
leurs | their (pl)
|
||||
quel | which
|
||||
quels | which
|
||||
quelle | which
|
||||
quelles | which
|
||||
sans | without
|
||||
soi | oneself
|
||||
|
|
@ -0,0 +1,110 @@
|
|||
|
||||
a
|
||||
ach
|
||||
ag
|
||||
agus
|
||||
an
|
||||
aon
|
||||
ar
|
||||
arna
|
||||
as
|
||||
b'
|
||||
ba
|
||||
beirt
|
||||
bhúr
|
||||
caoga
|
||||
ceathair
|
||||
ceathrar
|
||||
chomh
|
||||
chtó
|
||||
chuig
|
||||
chun
|
||||
cois
|
||||
céad
|
||||
cúig
|
||||
cúigear
|
||||
d'
|
||||
daichead
|
||||
dar
|
||||
de
|
||||
deich
|
||||
deichniúr
|
||||
den
|
||||
dhá
|
||||
do
|
||||
don
|
||||
dtí
|
||||
dá
|
||||
dár
|
||||
dó
|
||||
faoi
|
||||
faoin
|
||||
faoina
|
||||
faoinár
|
||||
fara
|
||||
fiche
|
||||
gach
|
||||
gan
|
||||
go
|
||||
gur
|
||||
haon
|
||||
hocht
|
||||
i
|
||||
iad
|
||||
idir
|
||||
in
|
||||
ina
|
||||
ins
|
||||
inár
|
||||
is
|
||||
le
|
||||
leis
|
||||
lena
|
||||
lenár
|
||||
m'
|
||||
mar
|
||||
mo
|
||||
mé
|
||||
na
|
||||
nach
|
||||
naoi
|
||||
naonúr
|
||||
ná
|
||||
ní
|
||||
níor
|
||||
nó
|
||||
nócha
|
||||
ocht
|
||||
ochtar
|
||||
os
|
||||
roimh
|
||||
sa
|
||||
seacht
|
||||
seachtar
|
||||
seachtó
|
||||
seasca
|
||||
seisear
|
||||
siad
|
||||
sibh
|
||||
sinn
|
||||
sna
|
||||
sé
|
||||
sí
|
||||
tar
|
||||
thar
|
||||
thú
|
||||
triúr
|
||||
trí
|
||||
trína
|
||||
trínár
|
||||
tríocha
|
||||
tú
|
||||
um
|
||||
ár
|
||||
é
|
||||
éis
|
||||
í
|
||||
ó
|
||||
ón
|
||||
óna
|
||||
ónár
|
|
@ -0,0 +1,161 @@
|
|||
# galican stopwords
|
||||
a
|
||||
aínda
|
||||
alí
|
||||
aquel
|
||||
aquela
|
||||
aquelas
|
||||
aqueles
|
||||
aquilo
|
||||
aquí
|
||||
ao
|
||||
aos
|
||||
as
|
||||
así
|
||||
á
|
||||
ben
|
||||
cando
|
||||
che
|
||||
co
|
||||
coa
|
||||
comigo
|
||||
con
|
||||
connosco
|
||||
contigo
|
||||
convosco
|
||||
coas
|
||||
cos
|
||||
cun
|
||||
cuns
|
||||
cunha
|
||||
cunhas
|
||||
da
|
||||
dalgunha
|
||||
dalgunhas
|
||||
dalgún
|
||||
dalgúns
|
||||
das
|
||||
de
|
||||
del
|
||||
dela
|
||||
delas
|
||||
deles
|
||||
desde
|
||||
deste
|
||||
do
|
||||
dos
|
||||
dun
|
||||
duns
|
||||
dunha
|
||||
dunhas
|
||||
e
|
||||
el
|
||||
ela
|
||||
elas
|
||||
eles
|
||||
en
|
||||
era
|
||||
eran
|
||||
esa
|
||||
esas
|
||||
ese
|
||||
eses
|
||||
esta
|
||||
estar
|
||||
estaba
|
||||
está
|
||||
están
|
||||
este
|
||||
estes
|
||||
estiven
|
||||
estou
|
||||
eu
|
||||
é
|
||||
facer
|
||||
foi
|
||||
foron
|
||||
fun
|
||||
había
|
||||
hai
|
||||
iso
|
||||
isto
|
||||
la
|
||||
las
|
||||
lle
|
||||
lles
|
||||
lo
|
||||
los
|
||||
mais
|
||||
me
|
||||
meu
|
||||
meus
|
||||
min
|
||||
miña
|
||||
miñas
|
||||
moi
|
||||
na
|
||||
nas
|
||||
neste
|
||||
nin
|
||||
no
|
||||
non
|
||||
nos
|
||||
nosa
|
||||
nosas
|
||||
noso
|
||||
nosos
|
||||
nós
|
||||
nun
|
||||
nunha
|
||||
nuns
|
||||
nunhas
|
||||
o
|
||||
os
|
||||
ou
|
||||
ó
|
||||
ós
|
||||
para
|
||||
pero
|
||||
pode
|
||||
pois
|
||||
pola
|
||||
polas
|
||||
polo
|
||||
polos
|
||||
por
|
||||
que
|
||||
se
|
||||
senón
|
||||
ser
|
||||
seu
|
||||
seus
|
||||
sexa
|
||||
sido
|
||||
sobre
|
||||
súa
|
||||
súas
|
||||
tamén
|
||||
tan
|
||||
te
|
||||
ten
|
||||
teñen
|
||||
teño
|
||||
ter
|
||||
teu
|
||||
teus
|
||||
ti
|
||||
tido
|
||||
tiña
|
||||
tiven
|
||||
túa
|
||||
túas
|
||||
un
|
||||
unha
|
||||
unhas
|
||||
uns
|
||||
vos
|
||||
vosa
|
||||
vosas
|
||||
voso
|
||||
vosos
|
||||
vós
|
|
@ -0,0 +1,235 @@
|
|||
# Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
# See http://members.unine.ch/jacques.savoy/clef/index.html.
|
||||
# This file was created by Jacques Savoy and is distributed under the BSD license.
|
||||
# Note: by default this file also contains forms normalized by HindiNormalizer
|
||||
# for spelling variation (see section below), such that it can be used whether or
|
||||
# not you enable that feature. When adding additional entries to this list,
|
||||
# please add the normalized form as well.
|
||||
अंदर
|
||||
अत
|
||||
अपना
|
||||
अपनी
|
||||
अपने
|
||||
अभी
|
||||
आदि
|
||||
आप
|
||||
इत्यादि
|
||||
इन
|
||||
इनका
|
||||
इन्हीं
|
||||
इन्हें
|
||||
इन्हों
|
||||
इस
|
||||
इसका
|
||||
इसकी
|
||||
इसके
|
||||
इसमें
|
||||
इसी
|
||||
इसे
|
||||
उन
|
||||
उनका
|
||||
उनकी
|
||||
उनके
|
||||
उनको
|
||||
उन्हीं
|
||||
उन्हें
|
||||
उन्हों
|
||||
उस
|
||||
उसके
|
||||
उसी
|
||||
उसे
|
||||
एक
|
||||
एवं
|
||||
एस
|
||||
ऐसे
|
||||
और
|
||||
कई
|
||||
कर
|
||||
करता
|
||||
करते
|
||||
करना
|
||||
करने
|
||||
करें
|
||||
कहते
|
||||
कहा
|
||||
का
|
||||
काफ़ी
|
||||
कि
|
||||
कितना
|
||||
किन्हें
|
||||
किन्हों
|
||||
किया
|
||||
किर
|
||||
किस
|
||||
किसी
|
||||
किसे
|
||||
की
|
||||
कुछ
|
||||
कुल
|
||||
के
|
||||
को
|
||||
कोई
|
||||
कौन
|
||||
कौनसा
|
||||
गया
|
||||
घर
|
||||
जब
|
||||
जहाँ
|
||||
जा
|
||||
जितना
|
||||
जिन
|
||||
जिन्हें
|
||||
जिन्हों
|
||||
जिस
|
||||
जिसे
|
||||
जीधर
|
||||
जैसा
|
||||
जैसे
|
||||
जो
|
||||
तक
|
||||
तब
|
||||
तरह
|
||||
तिन
|
||||
तिन्हें
|
||||
तिन्हों
|
||||
तिस
|
||||
तिसे
|
||||
तो
|
||||
था
|
||||
थी
|
||||
थे
|
||||
दबारा
|
||||
दिया
|
||||
दुसरा
|
||||
दूसरे
|
||||
दो
|
||||
द्वारा
|
||||
न
|
||||
नहीं
|
||||
ना
|
||||
निहायत
|
||||
नीचे
|
||||
ने
|
||||
पर
|
||||
पर
|
||||
पहले
|
||||
पूरा
|
||||
पे
|
||||
फिर
|
||||
बनी
|
||||
बही
|
||||
बहुत
|
||||
बाद
|
||||
बाला
|
||||
बिलकुल
|
||||
भी
|
||||
भीतर
|
||||
मगर
|
||||
मानो
|
||||
मे
|
||||
में
|
||||
यदि
|
||||
यह
|
||||
यहाँ
|
||||
यही
|
||||
या
|
||||
यिह
|
||||
ये
|
||||
रखें
|
||||
रहा
|
||||
रहे
|
||||
ऱ्वासा
|
||||
लिए
|
||||
लिये
|
||||
लेकिन
|
||||
व
|
||||
वर्ग
|
||||
वह
|
||||
वह
|
||||
वहाँ
|
||||
वहीं
|
||||
वाले
|
||||
वुह
|
||||
वे
|
||||
वग़ैरह
|
||||
संग
|
||||
सकता
|
||||
सकते
|
||||
सबसे
|
||||
सभी
|
||||
साथ
|
||||
साबुत
|
||||
साभ
|
||||
सारा
|
||||
से
|
||||
सो
|
||||
ही
|
||||
हुआ
|
||||
हुई
|
||||
हुए
|
||||
है
|
||||
हैं
|
||||
हो
|
||||
होता
|
||||
होती
|
||||
होते
|
||||
होना
|
||||
होने
|
||||
# additional normalized forms of the above
|
||||
अपनि
|
||||
जेसे
|
||||
होति
|
||||
सभि
|
||||
तिंहों
|
||||
इंहों
|
||||
दवारा
|
||||
इसि
|
||||
किंहें
|
||||
थि
|
||||
उंहों
|
||||
ओर
|
||||
जिंहें
|
||||
वहिं
|
||||
अभि
|
||||
बनि
|
||||
हि
|
||||
उंहिं
|
||||
उंहें
|
||||
हें
|
||||
वगेरह
|
||||
एसे
|
||||
रवासा
|
||||
कोन
|
||||
निचे
|
||||
काफि
|
||||
उसि
|
||||
पुरा
|
||||
भितर
|
||||
हे
|
||||
बहि
|
||||
वहां
|
||||
कोइ
|
||||
यहां
|
||||
जिंहों
|
||||
तिंहें
|
||||
किसि
|
||||
कइ
|
||||
यहि
|
||||
इंहिं
|
||||
जिधर
|
||||
इंहें
|
||||
अदि
|
||||
इतयादि
|
||||
हुइ
|
||||
कोनसा
|
||||
इसकि
|
||||
दुसरे
|
||||
जहां
|
||||
अप
|
||||
किंहों
|
||||
उनकि
|
||||
भि
|
||||
वरग
|
||||
हुअ
|
||||
जेसा
|
||||
नहिं
|
|
@ -0,0 +1,211 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/hungarian/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| Hungarian stop word list
|
||||
| prepared by Anna Tordai
|
||||
|
||||
a
|
||||
ahogy
|
||||
ahol
|
||||
aki
|
||||
akik
|
||||
akkor
|
||||
alatt
|
||||
által
|
||||
általában
|
||||
amely
|
||||
amelyek
|
||||
amelyekben
|
||||
amelyeket
|
||||
amelyet
|
||||
amelynek
|
||||
ami
|
||||
amit
|
||||
amolyan
|
||||
amíg
|
||||
amikor
|
||||
át
|
||||
abban
|
||||
ahhoz
|
||||
annak
|
||||
arra
|
||||
arról
|
||||
az
|
||||
azok
|
||||
azon
|
||||
azt
|
||||
azzal
|
||||
azért
|
||||
aztán
|
||||
azután
|
||||
azonban
|
||||
bár
|
||||
be
|
||||
belül
|
||||
benne
|
||||
cikk
|
||||
cikkek
|
||||
cikkeket
|
||||
csak
|
||||
de
|
||||
e
|
||||
eddig
|
||||
egész
|
||||
egy
|
||||
egyes
|
||||
egyetlen
|
||||
egyéb
|
||||
egyik
|
||||
egyre
|
||||
ekkor
|
||||
el
|
||||
elég
|
||||
ellen
|
||||
elő
|
||||
először
|
||||
előtt
|
||||
első
|
||||
én
|
||||
éppen
|
||||
ebben
|
||||
ehhez
|
||||
emilyen
|
||||
ennek
|
||||
erre
|
||||
ez
|
||||
ezt
|
||||
ezek
|
||||
ezen
|
||||
ezzel
|
||||
ezért
|
||||
és
|
||||
fel
|
||||
felé
|
||||
hanem
|
||||
hiszen
|
||||
hogy
|
||||
hogyan
|
||||
igen
|
||||
így
|
||||
illetve
|
||||
ill.
|
||||
ill
|
||||
ilyen
|
||||
ilyenkor
|
||||
ison
|
||||
ismét
|
||||
itt
|
||||
jó
|
||||
jól
|
||||
jobban
|
||||
kell
|
||||
kellett
|
||||
keresztül
|
||||
keressünk
|
||||
ki
|
||||
kívül
|
||||
között
|
||||
közül
|
||||
legalább
|
||||
lehet
|
||||
lehetett
|
||||
legyen
|
||||
lenne
|
||||
lenni
|
||||
lesz
|
||||
lett
|
||||
maga
|
||||
magát
|
||||
majd
|
||||
majd
|
||||
már
|
||||
más
|
||||
másik
|
||||
meg
|
||||
még
|
||||
mellett
|
||||
mert
|
||||
mely
|
||||
melyek
|
||||
mi
|
||||
mit
|
||||
míg
|
||||
miért
|
||||
milyen
|
||||
mikor
|
||||
minden
|
||||
mindent
|
||||
mindenki
|
||||
mindig
|
||||
mint
|
||||
mintha
|
||||
mivel
|
||||
most
|
||||
nagy
|
||||
nagyobb
|
||||
nagyon
|
||||
ne
|
||||
néha
|
||||
nekem
|
||||
neki
|
||||
nem
|
||||
néhány
|
||||
nélkül
|
||||
nincs
|
||||
olyan
|
||||
ott
|
||||
össze
|
||||
ő
|
||||
ők
|
||||
őket
|
||||
pedig
|
||||
persze
|
||||
rá
|
||||
s
|
||||
saját
|
||||
sem
|
||||
semmi
|
||||
sok
|
||||
sokat
|
||||
sokkal
|
||||
számára
|
||||
szemben
|
||||
szerint
|
||||
szinte
|
||||
talán
|
||||
tehát
|
||||
teljes
|
||||
tovább
|
||||
továbbá
|
||||
több
|
||||
úgy
|
||||
ugyanis
|
||||
új
|
||||
újabb
|
||||
újra
|
||||
után
|
||||
utána
|
||||
utolsó
|
||||
vagy
|
||||
vagyis
|
||||
valaki
|
||||
valami
|
||||
valamint
|
||||
való
|
||||
vagyok
|
||||
van
|
||||
vannak
|
||||
volt
|
||||
voltam
|
||||
voltak
|
||||
voltunk
|
||||
vissza
|
||||
vele
|
||||
viszont
|
||||
volna
|
|
@ -0,0 +1,46 @@
|
|||
# example set of Armenian stopwords.
|
||||
այդ
|
||||
այլ
|
||||
այն
|
||||
այս
|
||||
դու
|
||||
դուք
|
||||
եմ
|
||||
են
|
||||
ենք
|
||||
ես
|
||||
եք
|
||||
է
|
||||
էի
|
||||
էին
|
||||
էինք
|
||||
էիր
|
||||
էիք
|
||||
էր
|
||||
ըստ
|
||||
թ
|
||||
ի
|
||||
ին
|
||||
իսկ
|
||||
իր
|
||||
կամ
|
||||
համար
|
||||
հետ
|
||||
հետո
|
||||
մենք
|
||||
մեջ
|
||||
մի
|
||||
ն
|
||||
նա
|
||||
նաև
|
||||
նրա
|
||||
նրանք
|
||||
որ
|
||||
որը
|
||||
որոնք
|
||||
որպես
|
||||
ու
|
||||
ում
|
||||
պիտի
|
||||
վրա
|
||||
և
|
|
@ -0,0 +1,359 @@
|
|||
# from appendix D of: A Study of Stemming Effects on Information
|
||||
# Retrieval in Bahasa Indonesia
|
||||
ada
|
||||
adanya
|
||||
adalah
|
||||
adapun
|
||||
agak
|
||||
agaknya
|
||||
agar
|
||||
akan
|
||||
akankah
|
||||
akhirnya
|
||||
aku
|
||||
akulah
|
||||
amat
|
||||
amatlah
|
||||
anda
|
||||
andalah
|
||||
antar
|
||||
diantaranya
|
||||
antara
|
||||
antaranya
|
||||
diantara
|
||||
apa
|
||||
apaan
|
||||
mengapa
|
||||
apabila
|
||||
apakah
|
||||
apalagi
|
||||
apatah
|
||||
atau
|
||||
ataukah
|
||||
ataupun
|
||||
bagai
|
||||
bagaikan
|
||||
sebagai
|
||||
sebagainya
|
||||
bagaimana
|
||||
bagaimanapun
|
||||
sebagaimana
|
||||
bagaimanakah
|
||||
bagi
|
||||
bahkan
|
||||
bahwa
|
||||
bahwasanya
|
||||
sebaliknya
|
||||
banyak
|
||||
sebanyak
|
||||
beberapa
|
||||
seberapa
|
||||
begini
|
||||
beginian
|
||||
beginikah
|
||||
beginilah
|
||||
sebegini
|
||||
begitu
|
||||
begitukah
|
||||
begitulah
|
||||
begitupun
|
||||
sebegitu
|
||||
belum
|
||||
belumlah
|
||||
sebelum
|
||||
sebelumnya
|
||||
sebenarnya
|
||||
berapa
|
||||
berapakah
|
||||
berapalah
|
||||
berapapun
|
||||
betulkah
|
||||
sebetulnya
|
||||
biasa
|
||||
biasanya
|
||||
bila
|
||||
bilakah
|
||||
bisa
|
||||
bisakah
|
||||
sebisanya
|
||||
boleh
|
||||
bolehkah
|
||||
bolehlah
|
||||
buat
|
||||
bukan
|
||||
bukankah
|
||||
bukanlah
|
||||
bukannya
|
||||
cuma
|
||||
percuma
|
||||
dahulu
|
||||
dalam
|
||||
dan
|
||||
dapat
|
||||
dari
|
||||
daripada
|
||||
dekat
|
||||
demi
|
||||
demikian
|
||||
demikianlah
|
||||
sedemikian
|
||||
dengan
|
||||
depan
|
||||
di
|
||||
dia
|
||||
dialah
|
||||
dini
|
||||
diri
|
||||
dirinya
|
||||
terdiri
|
||||
dong
|
||||
dulu
|
||||
enggak
|
||||
enggaknya
|
||||
entah
|
||||
entahlah
|
||||
terhadap
|
||||
terhadapnya
|
||||
hal
|
||||
hampir
|
||||
hanya
|
||||
hanyalah
|
||||
harus
|
||||
haruslah
|
||||
harusnya
|
||||
seharusnya
|
||||
hendak
|
||||
hendaklah
|
||||
hendaknya
|
||||
hingga
|
||||
sehingga
|
||||
ia
|
||||
ialah
|
||||
ibarat
|
||||
ingin
|
||||
inginkah
|
||||
inginkan
|
||||
ini
|
||||
inikah
|
||||
inilah
|
||||
itu
|
||||
itukah
|
||||
itulah
|
||||
jangan
|
||||
jangankan
|
||||
janganlah
|
||||
jika
|
||||
jikalau
|
||||
juga
|
||||
justru
|
||||
kala
|
||||
kalau
|
||||
kalaulah
|
||||
kalaupun
|
||||
kalian
|
||||
kami
|
||||
kamilah
|
||||
kamu
|
||||
kamulah
|
||||
kan
|
||||
kapan
|
||||
kapankah
|
||||
kapanpun
|
||||
dikarenakan
|
||||
karena
|
||||
karenanya
|
||||
ke
|
||||
kecil
|
||||
kemudian
|
||||
kenapa
|
||||
kepada
|
||||
kepadanya
|
||||
ketika
|
||||
seketika
|
||||
khususnya
|
||||
kini
|
||||
kinilah
|
||||
kiranya
|
||||
sekiranya
|
||||
kita
|
||||
kitalah
|
||||
kok
|
||||
lagi
|
||||
lagian
|
||||
selagi
|
||||
lah
|
||||
lain
|
||||
lainnya
|
||||
melainkan
|
||||
selaku
|
||||
lalu
|
||||
melalui
|
||||
terlalu
|
||||
lama
|
||||
lamanya
|
||||
selama
|
||||
selama
|
||||
selamanya
|
||||
lebih
|
||||
terlebih
|
||||
bermacam
|
||||
macam
|
||||
semacam
|
||||
maka
|
||||
makanya
|
||||
makin
|
||||
malah
|
||||
malahan
|
||||
mampu
|
||||
mampukah
|
||||
mana
|
||||
manakala
|
||||
manalagi
|
||||
masih
|
||||
masihkah
|
||||
semasih
|
||||
masing
|
||||
mau
|
||||
maupun
|
||||
semaunya
|
||||
memang
|
||||
mereka
|
||||
merekalah
|
||||
meski
|
||||
meskipun
|
||||
semula
|
||||
mungkin
|
||||
mungkinkah
|
||||
nah
|
||||
namun
|
||||
nanti
|
||||
nantinya
|
||||
nyaris
|
||||
oleh
|
||||
olehnya
|
||||
seorang
|
||||
seseorang
|
||||
pada
|
||||
padanya
|
||||
padahal
|
||||
paling
|
||||
sepanjang
|
||||
pantas
|
||||
sepantasnya
|
||||
sepantasnyalah
|
||||
para
|
||||
pasti
|
||||
pastilah
|
||||
per
|
||||
pernah
|
||||
pula
|
||||
pun
|
||||
merupakan
|
||||
rupanya
|
||||
serupa
|
||||
saat
|
||||
saatnya
|
||||
sesaat
|
||||
saja
|
||||
sajalah
|
||||
saling
|
||||
bersama
|
||||
sama
|
||||
sesama
|
||||
sambil
|
||||
sampai
|
||||
sana
|
||||
sangat
|
||||
sangatlah
|
||||
saya
|
||||
sayalah
|
||||
se
|
||||
sebab
|
||||
sebabnya
|
||||
sebuah
|
||||
tersebut
|
||||
tersebutlah
|
||||
sedang
|
||||
sedangkan
|
||||
sedikit
|
||||
sedikitnya
|
||||
segala
|
||||
segalanya
|
||||
segera
|
||||
sesegera
|
||||
sejak
|
||||
sejenak
|
||||
sekali
|
||||
sekalian
|
||||
sekalipun
|
||||
sesekali
|
||||
sekaligus
|
||||
sekarang
|
||||
sekarang
|
||||
sekitar
|
||||
sekitarnya
|
||||
sela
|
||||
selain
|
||||
selalu
|
||||
seluruh
|
||||
seluruhnya
|
||||
semakin
|
||||
sementara
|
||||
sempat
|
||||
semua
|
||||
semuanya
|
||||
sendiri
|
||||
sendirinya
|
||||
seolah
|
||||
seperti
|
||||
sepertinya
|
||||
sering
|
||||
seringnya
|
||||
serta
|
||||
siapa
|
||||
siapakah
|
||||
siapapun
|
||||
disini
|
||||
disinilah
|
||||
sini
|
||||
sinilah
|
||||
sesuatu
|
||||
sesuatunya
|
||||
suatu
|
||||
sesudah
|
||||
sesudahnya
|
||||
sudah
|
||||
sudahkah
|
||||
sudahlah
|
||||
supaya
|
||||
tadi
|
||||
tadinya
|
||||
tak
|
||||
tanpa
|
||||
setelah
|
||||
telah
|
||||
tentang
|
||||
tentu
|
||||
tentulah
|
||||
tentunya
|
||||
tertentu
|
||||
seterusnya
|
||||
tapi
|
||||
tetapi
|
||||
setiap
|
||||
tiap
|
||||
setidaknya
|
||||
tidak
|
||||
tidakkah
|
||||
tidaklah
|
||||
toh
|
||||
waduh
|
||||
wah
|
||||
wahai
|
||||
sewaktu
|
||||
walau
|
||||
walaupun
|
||||
wong
|
||||
yaitu
|
||||
yakni
|
||||
yang
|
|
@ -0,0 +1,303 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/italian/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| An Italian stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
ad | a (to) before vowel
|
||||
al | a + il
|
||||
allo | a + lo
|
||||
ai | a + i
|
||||
agli | a + gli
|
||||
all | a + l'
|
||||
agl | a + gl'
|
||||
alla | a + la
|
||||
alle | a + le
|
||||
con | with
|
||||
col | con + il
|
||||
coi | con + i (forms collo, cogli etc are now very rare)
|
||||
da | from
|
||||
dal | da + il
|
||||
dallo | da + lo
|
||||
dai | da + i
|
||||
dagli | da + gli
|
||||
dall | da + l'
|
||||
dagl | da + gll'
|
||||
dalla | da + la
|
||||
dalle | da + le
|
||||
di | of
|
||||
del | di + il
|
||||
dello | di + lo
|
||||
dei | di + i
|
||||
degli | di + gli
|
||||
dell | di + l'
|
||||
degl | di + gl'
|
||||
della | di + la
|
||||
delle | di + le
|
||||
in | in
|
||||
nel | in + el
|
||||
nello | in + lo
|
||||
nei | in + i
|
||||
negli | in + gli
|
||||
nell | in + l'
|
||||
negl | in + gl'
|
||||
nella | in + la
|
||||
nelle | in + le
|
||||
su | on
|
||||
sul | su + il
|
||||
sullo | su + lo
|
||||
sui | su + i
|
||||
sugli | su + gli
|
||||
sull | su + l'
|
||||
sugl | su + gl'
|
||||
sulla | su + la
|
||||
sulle | su + le
|
||||
per | through, by
|
||||
tra | among
|
||||
contro | against
|
||||
io | I
|
||||
tu | thou
|
||||
lui | he
|
||||
lei | she
|
||||
noi | we
|
||||
voi | you
|
||||
loro | they
|
||||
mio | my
|
||||
mia |
|
||||
miei |
|
||||
mie |
|
||||
tuo |
|
||||
tua |
|
||||
tuoi | thy
|
||||
tue |
|
||||
suo |
|
||||
sua |
|
||||
suoi | his, her
|
||||
sue |
|
||||
nostro | our
|
||||
nostra |
|
||||
nostri |
|
||||
nostre |
|
||||
vostro | your
|
||||
vostra |
|
||||
vostri |
|
||||
vostre |
|
||||
mi | me
|
||||
ti | thee
|
||||
ci | us, there
|
||||
vi | you, there
|
||||
lo | him, the
|
||||
la | her, the
|
||||
li | them
|
||||
le | them, the
|
||||
gli | to him, the
|
||||
ne | from there etc
|
||||
il | the
|
||||
un | a
|
||||
uno | a
|
||||
una | a
|
||||
ma | but
|
||||
ed | and
|
||||
se | if
|
||||
perché | why, because
|
||||
anche | also
|
||||
come | how
|
||||
dov | where (as dov')
|
||||
dove | where
|
||||
che | who, that
|
||||
chi | who
|
||||
cui | whom
|
||||
non | not
|
||||
più | more
|
||||
quale | who, that
|
||||
quanto | how much
|
||||
quanti |
|
||||
quanta |
|
||||
quante |
|
||||
quello | that
|
||||
quelli |
|
||||
quella |
|
||||
quelle |
|
||||
questo | this
|
||||
questi |
|
||||
questa |
|
||||
queste |
|
||||
si | yes
|
||||
tutto | all
|
||||
tutti | all
|
||||
|
||||
| single letter forms:
|
||||
|
||||
a | at
|
||||
c | as c' for ce or ci
|
||||
e | and
|
||||
i | the
|
||||
l | as l'
|
||||
o | or
|
||||
|
||||
| forms of avere, to have (not including the infinitive):
|
||||
|
||||
ho
|
||||
hai
|
||||
ha
|
||||
abbiamo
|
||||
avete
|
||||
hanno
|
||||
abbia
|
||||
abbiate
|
||||
abbiano
|
||||
avrò
|
||||
avrai
|
||||
avrà
|
||||
avremo
|
||||
avrete
|
||||
avranno
|
||||
avrei
|
||||
avresti
|
||||
avrebbe
|
||||
avremmo
|
||||
avreste
|
||||
avrebbero
|
||||
avevo
|
||||
avevi
|
||||
aveva
|
||||
avevamo
|
||||
avevate
|
||||
avevano
|
||||
ebbi
|
||||
avesti
|
||||
ebbe
|
||||
avemmo
|
||||
aveste
|
||||
ebbero
|
||||
avessi
|
||||
avesse
|
||||
avessimo
|
||||
avessero
|
||||
avendo
|
||||
avuto
|
||||
avuta
|
||||
avuti
|
||||
avute
|
||||
|
||||
| forms of essere, to be (not including the infinitive):
|
||||
sono
|
||||
sei
|
||||
è
|
||||
siamo
|
||||
siete
|
||||
sia
|
||||
siate
|
||||
siano
|
||||
sarò
|
||||
sarai
|
||||
sarà
|
||||
saremo
|
||||
sarete
|
||||
saranno
|
||||
sarei
|
||||
saresti
|
||||
sarebbe
|
||||
saremmo
|
||||
sareste
|
||||
sarebbero
|
||||
ero
|
||||
eri
|
||||
era
|
||||
eravamo
|
||||
eravate
|
||||
erano
|
||||
fui
|
||||
fosti
|
||||
fu
|
||||
fummo
|
||||
foste
|
||||
furono
|
||||
fossi
|
||||
fosse
|
||||
fossimo
|
||||
fossero
|
||||
essendo
|
||||
|
||||
| forms of fare, to do (not including the infinitive, fa, fat-):
|
||||
faccio
|
||||
fai
|
||||
facciamo
|
||||
fanno
|
||||
faccia
|
||||
facciate
|
||||
facciano
|
||||
farò
|
||||
farai
|
||||
farà
|
||||
faremo
|
||||
farete
|
||||
faranno
|
||||
farei
|
||||
faresti
|
||||
farebbe
|
||||
faremmo
|
||||
fareste
|
||||
farebbero
|
||||
facevo
|
||||
facevi
|
||||
faceva
|
||||
facevamo
|
||||
facevate
|
||||
facevano
|
||||
feci
|
||||
facesti
|
||||
fece
|
||||
facemmo
|
||||
faceste
|
||||
fecero
|
||||
facessi
|
||||
facesse
|
||||
facessimo
|
||||
facessero
|
||||
facendo
|
||||
|
||||
| forms of stare, to be (not including the infinitive):
|
||||
sto
|
||||
stai
|
||||
sta
|
||||
stiamo
|
||||
stanno
|
||||
stia
|
||||
stiate
|
||||
stiano
|
||||
starò
|
||||
starai
|
||||
starà
|
||||
staremo
|
||||
starete
|
||||
staranno
|
||||
starei
|
||||
staresti
|
||||
starebbe
|
||||
staremmo
|
||||
stareste
|
||||
starebbero
|
||||
stavo
|
||||
stavi
|
||||
stava
|
||||
stavamo
|
||||
stavate
|
||||
stavano
|
||||
stetti
|
||||
stesti
|
||||
stette
|
||||
stemmo
|
||||
steste
|
||||
stettero
|
||||
stessi
|
||||
stesse
|
||||
stessimo
|
||||
stessero
|
||||
stando
|
|
@ -0,0 +1,127 @@
|
|||
#
|
||||
# This file defines a stopword set for Japanese.
|
||||
#
|
||||
# This set is made up of hand-picked frequent terms from segmented Japanese Wikipedia.
|
||||
# Punctuation characters and frequent kanji have mostly been left out. See LUCENE-3745
|
||||
# for frequency lists, etc. that can be useful for making your own set (if desired)
|
||||
#
|
||||
# Note that there is an overlap between these stopwords and the terms stopped when used
|
||||
# in combination with the JapanesePartOfSpeechStopFilter. When editing this file, note
|
||||
# that comments are not allowed on the same line as stopwords.
|
||||
#
|
||||
# Also note that stopping is done in a case-insensitive manner. Change your StopFilter
|
||||
# configuration if you need case-sensitive stopping. Lastly, note that stopping is done
|
||||
# using the same character width as the entries in this file. Since this StopFilter is
|
||||
# normally done after a CJKWidthFilter in your chain, you would usually want your romaji
|
||||
# entries to be in half-width and your kana entries to be in full-width.
|
||||
#
|
||||
の
|
||||
に
|
||||
は
|
||||
を
|
||||
た
|
||||
が
|
||||
で
|
||||
て
|
||||
と
|
||||
し
|
||||
れ
|
||||
さ
|
||||
ある
|
||||
いる
|
||||
も
|
||||
する
|
||||
から
|
||||
な
|
||||
こと
|
||||
として
|
||||
い
|
||||
や
|
||||
れる
|
||||
など
|
||||
なっ
|
||||
ない
|
||||
この
|
||||
ため
|
||||
その
|
||||
あっ
|
||||
よう
|
||||
また
|
||||
もの
|
||||
という
|
||||
あり
|
||||
まで
|
||||
られ
|
||||
なる
|
||||
へ
|
||||
か
|
||||
だ
|
||||
これ
|
||||
によって
|
||||
により
|
||||
おり
|
||||
より
|
||||
による
|
||||
ず
|
||||
なり
|
||||
られる
|
||||
において
|
||||
ば
|
||||
なかっ
|
||||
なく
|
||||
しかし
|
||||
について
|
||||
せ
|
||||
だっ
|
||||
その後
|
||||
できる
|
||||
それ
|
||||
う
|
||||
ので
|
||||
なお
|
||||
のみ
|
||||
でき
|
||||
き
|
||||
つ
|
||||
における
|
||||
および
|
||||
いう
|
||||
さらに
|
||||
でも
|
||||
ら
|
||||
たり
|
||||
その他
|
||||
に関する
|
||||
たち
|
||||
ます
|
||||
ん
|
||||
なら
|
||||
に対して
|
||||
特に
|
||||
せる
|
||||
及び
|
||||
これら
|
||||
とき
|
||||
では
|
||||
にて
|
||||
ほか
|
||||
ながら
|
||||
うち
|
||||
そして
|
||||
とともに
|
||||
ただし
|
||||
かつて
|
||||
それぞれ
|
||||
または
|
||||
お
|
||||
ほど
|
||||
ものの
|
||||
に対する
|
||||
ほとんど
|
||||
と共に
|
||||
といった
|
||||
です
|
||||
とも
|
||||
ところ
|
||||
ここ
|
||||
##### End of file
|
|
@ -0,0 +1,172 @@
|
|||
# Set of Latvian stopwords from A Stemming Algorithm for Latvian, Karlis Kreslins
|
||||
# the original list of over 800 forms was refined:
|
||||
# pronouns, adverbs, interjections were removed
|
||||
#
|
||||
# prepositions
|
||||
aiz
|
||||
ap
|
||||
ar
|
||||
apakš
|
||||
ārpus
|
||||
augšpus
|
||||
bez
|
||||
caur
|
||||
dēļ
|
||||
gar
|
||||
iekš
|
||||
iz
|
||||
kopš
|
||||
labad
|
||||
lejpus
|
||||
līdz
|
||||
no
|
||||
otrpus
|
||||
pa
|
||||
par
|
||||
pār
|
||||
pēc
|
||||
pie
|
||||
pirms
|
||||
pret
|
||||
priekš
|
||||
starp
|
||||
šaipus
|
||||
uz
|
||||
viņpus
|
||||
virs
|
||||
virspus
|
||||
zem
|
||||
apakšpus
|
||||
# Conjunctions
|
||||
un
|
||||
bet
|
||||
jo
|
||||
ja
|
||||
ka
|
||||
lai
|
||||
tomēr
|
||||
tikko
|
||||
turpretī
|
||||
arī
|
||||
kaut
|
||||
gan
|
||||
tādēļ
|
||||
tā
|
||||
ne
|
||||
tikvien
|
||||
vien
|
||||
kā
|
||||
ir
|
||||
te
|
||||
vai
|
||||
kamēr
|
||||
# Particles
|
||||
ar
|
||||
diezin
|
||||
droši
|
||||
diemžēl
|
||||
nebūt
|
||||
ik
|
||||
it
|
||||
taču
|
||||
nu
|
||||
pat
|
||||
tiklab
|
||||
iekšpus
|
||||
nedz
|
||||
tik
|
||||
nevis
|
||||
turpretim
|
||||
jeb
|
||||
iekam
|
||||
iekām
|
||||
iekāms
|
||||
kolīdz
|
||||
līdzko
|
||||
tiklīdz
|
||||
jebšu
|
||||
tālab
|
||||
tāpēc
|
||||
nekā
|
||||
itin
|
||||
jā
|
||||
jau
|
||||
jel
|
||||
nē
|
||||
nezin
|
||||
tad
|
||||
tikai
|
||||
vis
|
||||
tak
|
||||
iekams
|
||||
vien
|
||||
# modal verbs
|
||||
būt
|
||||
biju
|
||||
biji
|
||||
bija
|
||||
bijām
|
||||
bijāt
|
||||
esmu
|
||||
esi
|
||||
esam
|
||||
esat
|
||||
būšu
|
||||
būsi
|
||||
būs
|
||||
būsim
|
||||
būsiet
|
||||
tikt
|
||||
tiku
|
||||
tiki
|
||||
tika
|
||||
tikām
|
||||
tikāt
|
||||
tieku
|
||||
tiec
|
||||
tiek
|
||||
tiekam
|
||||
tiekat
|
||||
tikšu
|
||||
tiks
|
||||
tiksim
|
||||
tiksiet
|
||||
tapt
|
||||
tapi
|
||||
tapāt
|
||||
topat
|
||||
tapšu
|
||||
tapsi
|
||||
taps
|
||||
tapsim
|
||||
tapsiet
|
||||
kļūt
|
||||
kļuvu
|
||||
kļuvi
|
||||
kļuva
|
||||
kļuvām
|
||||
kļuvāt
|
||||
kļūstu
|
||||
kļūsti
|
||||
kļūst
|
||||
kļūstam
|
||||
kļūstat
|
||||
kļūšu
|
||||
kļūsi
|
||||
kļūs
|
||||
kļūsim
|
||||
kļūsiet
|
||||
# verbs
|
||||
varēt
|
||||
varēju
|
||||
varējām
|
||||
varēšu
|
||||
varēsim
|
||||
var
|
||||
varēji
|
||||
varējāt
|
||||
varēsi
|
||||
varēsiet
|
||||
varat
|
||||
varēja
|
||||
varēs
|
|
@ -0,0 +1,119 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/dutch/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A Dutch stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
| This is a ranked list (commonest to rarest) of stopwords derived from
|
||||
| a large sample of Dutch text.
|
||||
|
||||
| Dutch stop words frequently exhibit homonym clashes. These are indicated
|
||||
| clearly below.
|
||||
|
||||
de | the
|
||||
en | and
|
||||
van | of, from
|
||||
ik | I, the ego
|
||||
te | (1) chez, at etc, (2) to, (3) too
|
||||
dat | that, which
|
||||
die | that, those, who, which
|
||||
in | in, inside
|
||||
een | a, an, one
|
||||
hij | he
|
||||
het | the, it
|
||||
niet | not, nothing, naught
|
||||
zijn | (1) to be, being, (2) his, one's, its
|
||||
is | is
|
||||
was | (1) was, past tense of all persons sing. of 'zijn' (to be) (2) wax, (3) the washing, (4) rise of river
|
||||
op | on, upon, at, in, up, used up
|
||||
aan | on, upon, to (as dative)
|
||||
met | with, by
|
||||
als | like, such as, when
|
||||
voor | (1) before, in front of, (2) furrow
|
||||
had | had, past tense all persons sing. of 'hebben' (have)
|
||||
er | there
|
||||
maar | but, only
|
||||
om | round, about, for etc
|
||||
hem | him
|
||||
dan | then
|
||||
zou | should/would, past tense all persons sing. of 'zullen'
|
||||
of | or, whether, if
|
||||
wat | what, something, anything
|
||||
mijn | possessive and noun 'mine'
|
||||
men | people, 'one'
|
||||
dit | this
|
||||
zo | so, thus, in this way
|
||||
door | through by
|
||||
over | over, across
|
||||
ze | she, her, they, them
|
||||
zich | oneself
|
||||
bij | (1) a bee, (2) by, near, at
|
||||
ook | also, too
|
||||
tot | till, until
|
||||
je | you
|
||||
mij | me
|
||||
uit | out of, from
|
||||
der | Old Dutch form of 'van der' still found in surnames
|
||||
daar | (1) there, (2) because
|
||||
haar | (1) her, their, them, (2) hair
|
||||
naar | (1) unpleasant, unwell etc, (2) towards, (3) as
|
||||
heb | present first person sing. of 'to have'
|
||||
hoe | how, why
|
||||
heeft | present third person sing. of 'to have'
|
||||
hebben | 'to have' and various parts thereof
|
||||
deze | this
|
||||
u | you
|
||||
want | (1) for, (2) mitten, (3) rigging
|
||||
nog | yet, still
|
||||
zal | 'shall', first and third person sing. of verb 'zullen' (will)
|
||||
me | me
|
||||
zij | she, they
|
||||
nu | now
|
||||
ge | 'thou', still used in Belgium and south Netherlands
|
||||
geen | none
|
||||
omdat | because
|
||||
iets | something, somewhat
|
||||
worden | to become, grow, get
|
||||
toch | yet, still
|
||||
al | all, every, each
|
||||
waren | (1) 'were' (2) to wander, (3) wares, (3)
|
||||
veel | much, many
|
||||
meer | (1) more, (2) lake
|
||||
doen | to do, to make
|
||||
toen | then, when
|
||||
moet | noun 'spot/mote' and present form of 'to must'
|
||||
ben | (1) am, (2) 'are' in interrogative second person singular of 'to be'
|
||||
zonder | without
|
||||
kan | noun 'can' and present form of 'to be able'
|
||||
hun | their, them
|
||||
dus | so, consequently
|
||||
alles | all, everything, anything
|
||||
onder | under, beneath
|
||||
ja | yes, of course
|
||||
eens | once, one day
|
||||
hier | here
|
||||
wie | who
|
||||
werd | imperfect third person sing. of 'become'
|
||||
altijd | always
|
||||
doch | yet, but etc
|
||||
wordt | present third person sing. of 'become'
|
||||
wezen | (1) to be, (2) 'been' as in 'been fishing', (3) orphans
|
||||
kunnen | to be able
|
||||
ons | us/our
|
||||
zelf | self
|
||||
tegen | against, towards, at
|
||||
na | after, near
|
||||
reeds | already
|
||||
wil | (1) present tense of 'want', (2) 'will', noun, (3) fender
|
||||
kon | could; past tense of 'to be able'
|
||||
niets | nothing
|
||||
uw | your
|
||||
iemand | somebody
|
||||
geweest | been; past participle of 'be'
|
||||
andere | other
|
|
@ -0,0 +1,194 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/norwegian/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A Norwegian stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
| This stop word list is for the dominant bokmål dialect. Words unique
|
||||
| to nynorsk are marked *.
|
||||
|
||||
| Revised by Jan Bruusgaard <Jan.Bruusgaard@ssb.no>, Jan 2005
|
||||
|
||||
og | and
|
||||
i | in
|
||||
jeg | I
|
||||
det | it/this/that
|
||||
at | to (w. inf.)
|
||||
en | a/an
|
||||
et | a/an
|
||||
den | it/this/that
|
||||
til | to
|
||||
er | is/am/are
|
||||
som | who/that
|
||||
på | on
|
||||
de | they / you(formal)
|
||||
med | with
|
||||
han | he
|
||||
av | of
|
||||
ikke | not
|
||||
ikkje | not *
|
||||
der | there
|
||||
så | so
|
||||
var | was/were
|
||||
meg | me
|
||||
seg | you
|
||||
men | but
|
||||
ett | one
|
||||
har | have
|
||||
om | about
|
||||
vi | we
|
||||
min | my
|
||||
mitt | my
|
||||
ha | have
|
||||
hadde | had
|
||||
hun | she
|
||||
nå | now
|
||||
over | over
|
||||
da | when/as
|
||||
ved | by/know
|
||||
fra | from
|
||||
du | you
|
||||
ut | out
|
||||
sin | your
|
||||
dem | them
|
||||
oss | us
|
||||
opp | up
|
||||
man | you/one
|
||||
kan | can
|
||||
hans | his
|
||||
hvor | where
|
||||
eller | or
|
||||
hva | what
|
||||
skal | shall/must
|
||||
selv | self (reflective)
|
||||
sjøl | self (reflective)
|
||||
her | here
|
||||
alle | all
|
||||
vil | will
|
||||
bli | become
|
||||
ble | became
|
||||
blei | became *
|
||||
blitt | have become
|
||||
kunne | could
|
||||
inn | in
|
||||
når | when
|
||||
være | be
|
||||
kom | come
|
||||
noen | some
|
||||
noe | some
|
||||
ville | would
|
||||
dere | you
|
||||
som | who/which/that
|
||||
deres | their/theirs
|
||||
kun | only/just
|
||||
ja | yes
|
||||
etter | after
|
||||
ned | down
|
||||
skulle | should
|
||||
denne | this
|
||||
for | for/because
|
||||
deg | you
|
||||
si | hers/his
|
||||
sine | hers/his
|
||||
sitt | hers/his
|
||||
mot | against
|
||||
å | to
|
||||
meget | much
|
||||
hvorfor | why
|
||||
dette | this
|
||||
disse | these/those
|
||||
uten | without
|
||||
hvordan | how
|
||||
ingen | none
|
||||
din | your
|
||||
ditt | your
|
||||
blir | become
|
||||
samme | same
|
||||
hvilken | which
|
||||
hvilke | which (plural)
|
||||
sånn | such a
|
||||
inni | inside/within
|
||||
mellom | between
|
||||
vår | our
|
||||
hver | each
|
||||
hvem | who
|
||||
vors | us/ours
|
||||
hvis | whose
|
||||
både | both
|
||||
bare | only/just
|
||||
enn | than
|
||||
fordi | as/because
|
||||
før | before
|
||||
mange | many
|
||||
også | also
|
||||
slik | just
|
||||
vært | been
|
||||
være | to be
|
||||
båe | both *
|
||||
begge | both
|
||||
siden | since
|
||||
dykk | your *
|
||||
dykkar | yours *
|
||||
dei | they *
|
||||
deira | them *
|
||||
deires | theirs *
|
||||
deim | them *
|
||||
di | your (fem.) *
|
||||
då | as/when *
|
||||
eg | I *
|
||||
ein | a/an *
|
||||
eit | a/an *
|
||||
eitt | a/an *
|
||||
elles | or *
|
||||
honom | he *
|
||||
hjå | at *
|
||||
ho | she *
|
||||
hoe | she *
|
||||
henne | her
|
||||
hennar | her/hers
|
||||
hennes | hers
|
||||
hoss | how *
|
||||
hossen | how *
|
||||
ikkje | not *
|
||||
ingi | noone *
|
||||
inkje | noone *
|
||||
korleis | how *
|
||||
korso | how *
|
||||
kva | what/which *
|
||||
kvar | where *
|
||||
kvarhelst | where *
|
||||
kven | who/whom *
|
||||
kvi | why *
|
||||
kvifor | why *
|
||||
me | we *
|
||||
medan | while *
|
||||
mi | my *
|
||||
mine | my *
|
||||
mykje | much *
|
||||
no | now *
|
||||
nokon | some (masc./neut.) *
|
||||
noka | some (fem.) *
|
||||
nokor | some *
|
||||
noko | some *
|
||||
nokre | some *
|
||||
si | his/hers *
|
||||
sia | since *
|
||||
sidan | since *
|
||||
so | so *
|
||||
somt | some *
|
||||
somme | some *
|
||||
um | about*
|
||||
upp | up *
|
||||
vere | be *
|
||||
vore | was *
|
||||
verte | become *
|
||||
vort | become *
|
||||
varte | became *
|
||||
vart | became *
|
||||
|
|
@ -0,0 +1,253 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/portuguese/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A Portuguese stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
|
||||
| The following is a ranked list (commonest to rarest) of stopwords
|
||||
| deriving from a large sample of text.
|
||||
|
||||
| Extra words have been added at the end.
|
||||
|
||||
de | of, from
|
||||
a | the; to, at; her
|
||||
o | the; him
|
||||
que | who, that
|
||||
e | and
|
||||
do | de + o
|
||||
da | de + a
|
||||
em | in
|
||||
um | a
|
||||
para | for
|
||||
| é from SER
|
||||
com | with
|
||||
não | not, no
|
||||
uma | a
|
||||
os | the; them
|
||||
no | em + o
|
||||
se | himself etc
|
||||
na | em + a
|
||||
por | for
|
||||
mais | more
|
||||
as | the; them
|
||||
dos | de + os
|
||||
como | as, like
|
||||
mas | but
|
||||
| foi from SER
|
||||
ao | a + o
|
||||
ele | he
|
||||
das | de + as
|
||||
| tem from TER
|
||||
à | a + a
|
||||
seu | his
|
||||
sua | her
|
||||
ou | or
|
||||
| ser from SER
|
||||
quando | when
|
||||
muito | much
|
||||
| há from HAV
|
||||
nos | em + os; us
|
||||
já | already, now
|
||||
| está from EST
|
||||
eu | I
|
||||
também | also
|
||||
só | only, just
|
||||
pelo | per + o
|
||||
pela | per + a
|
||||
até | up to
|
||||
isso | that
|
||||
ela | he
|
||||
entre | between
|
||||
| era from SER
|
||||
depois | after
|
||||
sem | without
|
||||
mesmo | same
|
||||
aos | a + os
|
||||
| ter from TER
|
||||
seus | his
|
||||
quem | whom
|
||||
nas | em + as
|
||||
me | me
|
||||
esse | that
|
||||
eles | they
|
||||
| estão from EST
|
||||
você | you
|
||||
| tinha from TER
|
||||
| foram from SER
|
||||
essa | that
|
||||
num | em + um
|
||||
nem | nor
|
||||
suas | her
|
||||
meu | my
|
||||
às | a + as
|
||||
minha | my
|
||||
| têm from TER
|
||||
numa | em + uma
|
||||
pelos | per + os
|
||||
elas | they
|
||||
| havia from HAV
|
||||
| seja from SER
|
||||
qual | which
|
||||
| será from SER
|
||||
nós | we
|
||||
| tenho from TER
|
||||
lhe | to him, her
|
||||
deles | of them
|
||||
essas | those
|
||||
esses | those
|
||||
pelas | per + as
|
||||
este | this
|
||||
| fosse from SER
|
||||
dele | of him
|
||||
|
||||
| other words. There are many contractions such as naquele = em+aquele,
|
||||
| mo = me+o, but they are rare.
|
||||
| Indefinite article plural forms are also rare.
|
||||
|
||||
tu | thou
|
||||
te | thee
|
||||
vocês | you (plural)
|
||||
vos | you
|
||||
lhes | to them
|
||||
meus | my
|
||||
minhas
|
||||
teu | thy
|
||||
tua
|
||||
teus
|
||||
tuas
|
||||
nosso | our
|
||||
nossa
|
||||
nossos
|
||||
nossas
|
||||
|
||||
dela | of her
|
||||
delas | of them
|
||||
|
||||
esta | this
|
||||
estes | these
|
||||
estas | these
|
||||
aquele | that
|
||||
aquela | that
|
||||
aqueles | those
|
||||
aquelas | those
|
||||
isto | this
|
||||
aquilo | that
|
||||
|
||||
| forms of estar, to be (not including the infinitive):
|
||||
estou
|
||||
está
|
||||
estamos
|
||||
estão
|
||||
estive
|
||||
esteve
|
||||
estivemos
|
||||
estiveram
|
||||
estava
|
||||
estávamos
|
||||
estavam
|
||||
estivera
|
||||
estivéramos
|
||||
esteja
|
||||
estejamos
|
||||
estejam
|
||||
estivesse
|
||||
estivéssemos
|
||||
estivessem
|
||||
estiver
|
||||
estivermos
|
||||
estiverem
|
||||
|
||||
| forms of haver, to have (not including the infinitive):
|
||||
hei
|
||||
há
|
||||
havemos
|
||||
hão
|
||||
houve
|
||||
houvemos
|
||||
houveram
|
||||
houvera
|
||||
houvéramos
|
||||
haja
|
||||
hajamos
|
||||
hajam
|
||||
houvesse
|
||||
houvéssemos
|
||||
houvessem
|
||||
houver
|
||||
houvermos
|
||||
houverem
|
||||
houverei
|
||||
houverá
|
||||
houveremos
|
||||
houverão
|
||||
houveria
|
||||
houveríamos
|
||||
houveriam
|
||||
|
||||
| forms of ser, to be (not including the infinitive):
|
||||
sou
|
||||
somos
|
||||
são
|
||||
era
|
||||
éramos
|
||||
eram
|
||||
fui
|
||||
foi
|
||||
fomos
|
||||
foram
|
||||
fora
|
||||
fôramos
|
||||
seja
|
||||
sejamos
|
||||
sejam
|
||||
fosse
|
||||
fôssemos
|
||||
fossem
|
||||
for
|
||||
formos
|
||||
forem
|
||||
serei
|
||||
será
|
||||
seremos
|
||||
serão
|
||||
seria
|
||||
seríamos
|
||||
seriam
|
||||
|
||||
| forms of ter, to have (not including the infinitive):
|
||||
tenho
|
||||
tem
|
||||
temos
|
||||
tém
|
||||
tinha
|
||||
tínhamos
|
||||
tinham
|
||||
tive
|
||||
teve
|
||||
tivemos
|
||||
tiveram
|
||||
tivera
|
||||
tivéramos
|
||||
tenha
|
||||
tenhamos
|
||||
tenham
|
||||
tivesse
|
||||
tivéssemos
|
||||
tivessem
|
||||
tiver
|
||||
tivermos
|
||||
tiverem
|
||||
terei
|
||||
terá
|
||||
teremos
|
||||
terão
|
||||
teria
|
||||
teríamos
|
||||
teriam
|
|
@ -0,0 +1,233 @@
|
|||
# This file was created by Jacques Savoy and is distributed under the BSD license.
|
||||
# See http://members.unine.ch/jacques.savoy/clef/index.html.
|
||||
# Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
acea
|
||||
aceasta
|
||||
această
|
||||
aceea
|
||||
acei
|
||||
aceia
|
||||
acel
|
||||
acela
|
||||
acele
|
||||
acelea
|
||||
acest
|
||||
acesta
|
||||
aceste
|
||||
acestea
|
||||
aceşti
|
||||
aceştia
|
||||
acolo
|
||||
acum
|
||||
ai
|
||||
aia
|
||||
aibă
|
||||
aici
|
||||
al
|
||||
ăla
|
||||
ale
|
||||
alea
|
||||
ălea
|
||||
altceva
|
||||
altcineva
|
||||
am
|
||||
ar
|
||||
are
|
||||
aş
|
||||
aşadar
|
||||
asemenea
|
||||
asta
|
||||
ăsta
|
||||
astăzi
|
||||
astea
|
||||
ăstea
|
||||
ăştia
|
||||
asupra
|
||||
aţi
|
||||
au
|
||||
avea
|
||||
avem
|
||||
aveţi
|
||||
azi
|
||||
bine
|
||||
bucur
|
||||
bună
|
||||
ca
|
||||
că
|
||||
căci
|
||||
când
|
||||
care
|
||||
cărei
|
||||
căror
|
||||
cărui
|
||||
cât
|
||||
câte
|
||||
câţi
|
||||
către
|
||||
câtva
|
||||
ce
|
||||
cel
|
||||
ceva
|
||||
chiar
|
||||
cînd
|
||||
cine
|
||||
cineva
|
||||
cît
|
||||
cîte
|
||||
cîţi
|
||||
cîtva
|
||||
contra
|
||||
cu
|
||||
cum
|
||||
cumva
|
||||
curând
|
||||
curînd
|
||||
da
|
||||
dă
|
||||
dacă
|
||||
dar
|
||||
datorită
|
||||
de
|
||||
deci
|
||||
deja
|
||||
deoarece
|
||||
departe
|
||||
deşi
|
||||
din
|
||||
dinaintea
|
||||
dintr
|
||||
dintre
|
||||
drept
|
||||
după
|
||||
ea
|
||||
ei
|
||||
el
|
||||
ele
|
||||
eram
|
||||
este
|
||||
eşti
|
||||
eu
|
||||
face
|
||||
fără
|
||||
fi
|
||||
fie
|
||||
fiecare
|
||||
fii
|
||||
fim
|
||||
fiţi
|
||||
iar
|
||||
ieri
|
||||
îi
|
||||
îl
|
||||
îmi
|
||||
împotriva
|
||||
în
|
||||
înainte
|
||||
înaintea
|
||||
încât
|
||||
încît
|
||||
încotro
|
||||
între
|
||||
întrucât
|
||||
întrucît
|
||||
îţi
|
||||
la
|
||||
lângă
|
||||
le
|
||||
li
|
||||
lîngă
|
||||
lor
|
||||
lui
|
||||
mă
|
||||
mâine
|
||||
mea
|
||||
mei
|
||||
mele
|
||||
mereu
|
||||
meu
|
||||
mi
|
||||
mine
|
||||
mult
|
||||
multă
|
||||
mulţi
|
||||
ne
|
||||
nicăieri
|
||||
nici
|
||||
nimeni
|
||||
nişte
|
||||
noastră
|
||||
noastre
|
||||
noi
|
||||
noştri
|
||||
nostru
|
||||
nu
|
||||
ori
|
||||
oricând
|
||||
oricare
|
||||
oricât
|
||||
orice
|
||||
oricînd
|
||||
oricine
|
||||
oricît
|
||||
oricum
|
||||
oriunde
|
||||
până
|
||||
pe
|
||||
pentru
|
||||
peste
|
||||
pînă
|
||||
poate
|
||||
pot
|
||||
prea
|
||||
prima
|
||||
primul
|
||||
prin
|
||||
printr
|
||||
sa
|
||||
să
|
||||
săi
|
||||
sale
|
||||
sau
|
||||
său
|
||||
se
|
||||
şi
|
||||
sînt
|
||||
sîntem
|
||||
sînteţi
|
||||
spre
|
||||
sub
|
||||
sunt
|
||||
suntem
|
||||
sunteţi
|
||||
ta
|
||||
tăi
|
||||
tale
|
||||
tău
|
||||
te
|
||||
ţi
|
||||
ţie
|
||||
tine
|
||||
toată
|
||||
toate
|
||||
tot
|
||||
toţi
|
||||
totuşi
|
||||
tu
|
||||
un
|
||||
una
|
||||
unde
|
||||
undeva
|
||||
unei
|
||||
unele
|
||||
uneori
|
||||
unor
|
||||
vă
|
||||
vi
|
||||
voastră
|
||||
voastre
|
||||
voi
|
||||
voştri
|
||||
vostru
|
||||
vouă
|
||||
vreo
|
||||
vreun
|
|
@ -0,0 +1,243 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/russian/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| a russian stop word list. comments begin with vertical bar. each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
| this is a ranked list (commonest to rarest) of stopwords derived from
|
||||
| a large text sample.
|
||||
|
||||
| letter `ё' is translated to `е'.
|
||||
|
||||
и | and
|
||||
в | in/into
|
||||
во | alternative form
|
||||
не | not
|
||||
что | what/that
|
||||
он | he
|
||||
на | on/onto
|
||||
я | i
|
||||
с | from
|
||||
со | alternative form
|
||||
как | how
|
||||
а | milder form of `no' (but)
|
||||
то | conjunction and form of `that'
|
||||
все | all
|
||||
она | she
|
||||
так | so, thus
|
||||
его | him
|
||||
но | but
|
||||
да | yes/and
|
||||
ты | thou
|
||||
к | towards, by
|
||||
у | around, chez
|
||||
же | intensifier particle
|
||||
вы | you
|
||||
за | beyond, behind
|
||||
бы | conditional/subj. particle
|
||||
по | up to, along
|
||||
только | only
|
||||
ее | her
|
||||
мне | to me
|
||||
было | it was
|
||||
вот | here is/are, particle
|
||||
от | away from
|
||||
меня | me
|
||||
еще | still, yet, more
|
||||
нет | no, there isnt/arent
|
||||
о | about
|
||||
из | out of
|
||||
ему | to him
|
||||
теперь | now
|
||||
когда | when
|
||||
даже | even
|
||||
ну | so, well
|
||||
вдруг | suddenly
|
||||
ли | interrogative particle
|
||||
если | if
|
||||
уже | already, but homonym of `narrower'
|
||||
или | or
|
||||
ни | neither
|
||||
быть | to be
|
||||
был | he was
|
||||
него | prepositional form of его
|
||||
до | up to
|
||||
вас | you accusative
|
||||
нибудь | indef. suffix preceded by hyphen
|
||||
опять | again
|
||||
уж | already, but homonym of `adder'
|
||||
вам | to you
|
||||
сказал | he said
|
||||
ведь | particle `after all'
|
||||
там | there
|
||||
потом | then
|
||||
себя | oneself
|
||||
ничего | nothing
|
||||
ей | to her
|
||||
может | usually with `быть' as `maybe'
|
||||
они | they
|
||||
тут | here
|
||||
где | where
|
||||
есть | there is/are
|
||||
надо | got to, must
|
||||
ней | prepositional form of ей
|
||||
для | for
|
||||
мы | we
|
||||
тебя | thee
|
||||
их | them, their
|
||||
чем | than
|
||||
была | she was
|
||||
сам | self
|
||||
чтоб | in order to
|
||||
без | without
|
||||
будто | as if
|
||||
человек | man, person, one
|
||||
чего | genitive form of `what'
|
||||
раз | once
|
||||
тоже | also
|
||||
себе | to oneself
|
||||
под | beneath
|
||||
жизнь | life
|
||||
будет | will be
|
||||
ж | short form of intensifer particle `же'
|
||||
тогда | then
|
||||
кто | who
|
||||
этот | this
|
||||
говорил | was saying
|
||||
того | genitive form of `that'
|
||||
потому | for that reason
|
||||
этого | genitive form of `this'
|
||||
какой | which
|
||||
совсем | altogether
|
||||
ним | prepositional form of `его', `они'
|
||||
здесь | here
|
||||
этом | prepositional form of `этот'
|
||||
один | one
|
||||
почти | almost
|
||||
мой | my
|
||||
тем | instrumental/dative plural of `тот', `то'
|
||||
чтобы | full form of `in order that'
|
||||
нее | her (acc.)
|
||||
кажется | it seems
|
||||
сейчас | now
|
||||
были | they were
|
||||
куда | where to
|
||||
зачем | why
|
||||
сказать | to say
|
||||
всех | all (acc., gen. preposn. plural)
|
||||
никогда | never
|
||||
сегодня | today
|
||||
можно | possible, one can
|
||||
при | by
|
||||
наконец | finally
|
||||
два | two
|
||||
об | alternative form of `о', about
|
||||
другой | another
|
||||
хоть | even
|
||||
после | after
|
||||
над | above
|
||||
больше | more
|
||||
тот | that one (masc.)
|
||||
через | across, in
|
||||
эти | these
|
||||
нас | us
|
||||
про | about
|
||||
всего | in all, only, of all
|
||||
них | prepositional form of `они' (they)
|
||||
какая | which, feminine
|
||||
много | lots
|
||||
разве | interrogative particle
|
||||
сказала | she said
|
||||
три | three
|
||||
эту | this, acc. fem. sing.
|
||||
моя | my, feminine
|
||||
впрочем | moreover, besides
|
||||
хорошо | good
|
||||
свою | ones own, acc. fem. sing.
|
||||
этой | oblique form of `эта', fem. `this'
|
||||
перед | in front of
|
||||
иногда | sometimes
|
||||
лучше | better
|
||||
чуть | a little
|
||||
том | preposn. form of `that one'
|
||||
нельзя | one must not
|
||||
такой | such a one
|
||||
им | to them
|
||||
более | more
|
||||
всегда | always
|
||||
конечно | of course
|
||||
всю | acc. fem. sing of `all'
|
||||
между | between
|
||||
|
||||
|
||||
| b: some paradigms
|
||||
|
|
||||
| personal pronouns
|
||||
|
|
||||
| я меня мне мной [мною]
|
||||
| ты тебя тебе тобой [тобою]
|
||||
| он его ему им [него, нему, ним]
|
||||
| она ее эи ею [нее, нэи, нею]
|
||||
| оно его ему им [него, нему, ним]
|
||||
|
|
||||
| мы нас нам нами
|
||||
| вы вас вам вами
|
||||
| они их им ими [них, ним, ними]
|
||||
|
|
||||
| себя себе собой [собою]
|
||||
|
|
||||
| demonstrative pronouns: этот (this), тот (that)
|
||||
|
|
||||
| этот эта это эти
|
||||
| этого эты это эти
|
||||
| этого этой этого этих
|
||||
| этому этой этому этим
|
||||
| этим этой этим [этою] этими
|
||||
| этом этой этом этих
|
||||
|
|
||||
| тот та то те
|
||||
| того ту то те
|
||||
| того той того тех
|
||||
| тому той тому тем
|
||||
| тем той тем [тою] теми
|
||||
| том той том тех
|
||||
|
|
||||
| determinative pronouns
|
||||
|
|
||||
| (a) весь (all)
|
||||
|
|
||||
| весь вся все все
|
||||
| всего всю все все
|
||||
| всего всей всего всех
|
||||
| всему всей всему всем
|
||||
| всем всей всем [всею] всеми
|
||||
| всем всей всем всех
|
||||
|
|
||||
| (b) сам (himself etc)
|
||||
|
|
||||
| сам сама само сами
|
||||
| самого саму само самих
|
||||
| самого самой самого самих
|
||||
| самому самой самому самим
|
||||
| самим самой самим [самою] самими
|
||||
| самом самой самом самих
|
||||
|
|
||||
| stems of verbs `to be', `to have', `to do' and modal
|
||||
|
|
||||
| быть бы буд быв есть суть
|
||||
| име
|
||||
| дел
|
||||
| мог мож мочь
|
||||
| уме
|
||||
| хоч хот
|
||||
| долж
|
||||
| можн
|
||||
| нужн
|
||||
| нельзя
|
||||
|
|
@ -0,0 +1,133 @@
|
|||
| From svn.tartarus.org/snowball/trunk/website/algorithms/swedish/stop.txt
|
||||
| This file is distributed under the BSD License.
|
||||
| See http://snowball.tartarus.org/license.php
|
||||
| Also see http://www.opensource.org/licenses/bsd-license.html
|
||||
| - Encoding was converted to UTF-8.
|
||||
| - This notice was added.
|
||||
|
|
||||
| NOTE: To use this file with StopFilterFactory, you must specify format="snowball"
|
||||
|
||||
| A Swedish stop word list. Comments begin with vertical bar. Each stop
|
||||
| word is at the start of a line.
|
||||
|
||||
| This is a ranked list (commonest to rarest) of stopwords derived from
|
||||
| a large text sample.
|
||||
|
||||
| Swedish stop words occasionally exhibit homonym clashes. For example
|
||||
| så = so, but also seed. These are indicated clearly below.
|
||||
|
||||
och | and
|
||||
det | it, this/that
|
||||
att | to (with infinitive)
|
||||
i | in, at
|
||||
en | a
|
||||
jag | I
|
||||
hon | she
|
||||
som | who, that
|
||||
han | he
|
||||
på | on
|
||||
den | it, this/that
|
||||
med | with
|
||||
var | where, each
|
||||
sig | him(self) etc
|
||||
för | for
|
||||
så | so (also: seed)
|
||||
till | to
|
||||
är | is
|
||||
men | but
|
||||
ett | a
|
||||
om | if; around, about
|
||||
hade | had
|
||||
de | they, these/those
|
||||
av | of
|
||||
icke | not, no
|
||||
mig | me
|
||||
du | you
|
||||
henne | her
|
||||
då | then, when
|
||||
sin | his
|
||||
nu | now
|
||||
har | have
|
||||
inte | inte någon = no one
|
||||
hans | his
|
||||
honom | him
|
||||
skulle | 'sake'
|
||||
hennes | her
|
||||
där | there
|
||||
min | my
|
||||
man | one (pronoun)
|
||||
ej | nor
|
||||
vid | at, by, on (also: vast)
|
||||
kunde | could
|
||||
något | some etc
|
||||
från | from, off
|
||||
ut | out
|
||||
när | when
|
||||
efter | after, behind
|
||||
upp | up
|
||||
vi | we
|
||||
dem | them
|
||||
vara | be
|
||||
vad | what
|
||||
över | over
|
||||
än | than
|
||||
dig | you
|
||||
kan | can
|
||||
sina | his
|
||||
här | here
|
||||
ha | have
|
||||
mot | towards
|
||||
alla | all
|
||||
under | under (also: wonder)
|
||||
någon | some etc
|
||||
eller | or (else)
|
||||
allt | all
|
||||
mycket | much
|
||||
sedan | since
|
||||
ju | why
|
||||
denna | this/that
|
||||
själv | myself, yourself etc
|
||||
detta | this/that
|
||||
åt | to
|
||||
utan | without
|
||||
varit | was
|
||||
hur | how
|
||||
ingen | no
|
||||
mitt | my
|
||||
ni | you
|
||||
bli | to be, become
|
||||
blev | from bli
|
||||
oss | us
|
||||
din | thy
|
||||
dessa | these/those
|
||||
några | some etc
|
||||
deras | their
|
||||
blir | from bli
|
||||
mina | my
|
||||
samma | (the) same
|
||||
vilken | who, that
|
||||
er | you, your
|
||||
sådan | such a
|
||||
vår | our
|
||||
blivit | from bli
|
||||
dess | its
|
||||
inom | within
|
||||
mellan | between
|
||||
sådant | such a
|
||||
varför | why
|
||||
varje | each
|
||||
vilka | who, that
|
||||
ditt | thy
|
||||
vem | who
|
||||
vilket | who, that
|
||||
sitta | his
|
||||
sådana | such a
|
||||
vart | each
|
||||
dina | thy
|
||||
vars | whose
|
||||
vårt | our
|
||||
våra | our
|
||||
ert | your
|
||||
era | your
|
||||
vilkas | whose
|
||||
|
|
@ -0,0 +1,119 @@
|
|||
# Thai stopwords from:
|
||||
# "Opinion Detection in Thai Political News Columns
|
||||
# Based on Subjectivity Analysis"
|
||||
# Khampol Sukhum, Supot Nitsuwat, and Choochart Haruechaiyasak
|
||||
ไว้
|
||||
ไม่
|
||||
ไป
|
||||
ได้
|
||||
ให้
|
||||
ใน
|
||||
โดย
|
||||
แห่ง
|
||||
แล้ว
|
||||
และ
|
||||
แรก
|
||||
แบบ
|
||||
แต่
|
||||
เอง
|
||||
เห็น
|
||||
เลย
|
||||
เริ่ม
|
||||
เรา
|
||||
เมื่อ
|
||||
เพื่อ
|
||||
เพราะ
|
||||
เป็นการ
|
||||
เป็น
|
||||
เปิดเผย
|
||||
เปิด
|
||||
เนื่องจาก
|
||||
เดียวกัน
|
||||
เดียว
|
||||
เช่น
|
||||
เฉพาะ
|
||||
เคย
|
||||
เข้า
|
||||
เขา
|
||||
อีก
|
||||
อาจ
|
||||
อะไร
|
||||
ออก
|
||||
อย่าง
|
||||
อยู่
|
||||
อยาก
|
||||
หาก
|
||||
หลาย
|
||||
หลังจาก
|
||||
หลัง
|
||||
หรือ
|
||||
หนึ่ง
|
||||
ส่วน
|
||||
ส่ง
|
||||
สุด
|
||||
สําหรับ
|
||||
ว่า
|
||||
วัน
|
||||
ลง
|
||||
ร่วม
|
||||
ราย
|
||||
รับ
|
||||
ระหว่าง
|
||||
รวม
|
||||
ยัง
|
||||
มี
|
||||
มาก
|
||||
มา
|
||||
พร้อม
|
||||
พบ
|
||||
ผ่าน
|
||||
ผล
|
||||
บาง
|
||||
น่า
|
||||
นี้
|
||||
นํา
|
||||
นั้น
|
||||
นัก
|
||||
นอกจาก
|
||||
ทุก
|
||||
ที่สุด
|
||||
ที่
|
||||
ทําให้
|
||||
ทํา
|
||||
ทาง
|
||||
ทั้งนี้
|
||||
ทั้ง
|
||||
ถ้า
|
||||
ถูก
|
||||
ถึง
|
||||
ต้อง
|
||||
ต่างๆ
|
||||
ต่าง
|
||||
ต่อ
|
||||
ตาม
|
||||
ตั้งแต่
|
||||
ตั้ง
|
||||
ด้าน
|
||||
ด้วย
|
||||
ดัง
|
||||
ซึ่ง
|
||||
ช่วง
|
||||
จึง
|
||||
จาก
|
||||
จัด
|
||||
จะ
|
||||
คือ
|
||||
ความ
|
||||
ครั้ง
|
||||
คง
|
||||
ขึ้น
|
||||
ของ
|
||||
ขอ
|
||||
ขณะ
|
||||
ก่อน
|
||||
ก็
|
||||
การ
|
||||
กับ
|
||||
กัน
|
||||
กว่า
|
||||
กล่าว
|
|
@ -0,0 +1,212 @@
|
|||
# Turkish stopwords from LUCENE-559
|
||||
# merged with the list from "Information Retrieval on Turkish Texts"
|
||||
# (http://www.users.muohio.edu/canf/papers/JASIST2008offPrint.pdf)
|
||||
acaba
|
||||
altmış
|
||||
altı
|
||||
ama
|
||||
ancak
|
||||
arada
|
||||
aslında
|
||||
ayrıca
|
||||
bana
|
||||
bazı
|
||||
belki
|
||||
ben
|
||||
benden
|
||||
beni
|
||||
benim
|
||||
beri
|
||||
beş
|
||||
bile
|
||||
bin
|
||||
bir
|
||||
birçok
|
||||
biri
|
||||
birkaç
|
||||
birkez
|
||||
birşey
|
||||
birşeyi
|
||||
biz
|
||||
bize
|
||||
bizden
|
||||
bizi
|
||||
bizim
|
||||
böyle
|
||||
böylece
|
||||
bu
|
||||
buna
|
||||
bunda
|
||||
bundan
|
||||
bunlar
|
||||
bunları
|
||||
bunların
|
||||
bunu
|
||||
bunun
|
||||
burada
|
||||
çok
|
||||
çünkü
|
||||
da
|
||||
daha
|
||||
dahi
|
||||
de
|
||||
defa
|
||||
değil
|
||||
diğer
|
||||
diye
|
||||
doksan
|
||||
dokuz
|
||||
dolayı
|
||||
dolayısıyla
|
||||
dört
|
||||
edecek
|
||||
eden
|
||||
ederek
|
||||
edilecek
|
||||
ediliyor
|
||||
edilmesi
|
||||
ediyor
|
||||
eğer
|
||||
elli
|
||||
en
|
||||
etmesi
|
||||
etti
|
||||
ettiği
|
||||
ettiğini
|
||||
gibi
|
||||
göre
|
||||
halen
|
||||
hangi
|
||||
hatta
|
||||
hem
|
||||
henüz
|
||||
hep
|
||||
hepsi
|
||||
her
|
||||
herhangi
|
||||
herkesin
|
||||
hiç
|
||||
hiçbir
|
||||
için
|
||||
iki
|
||||
ile
|
||||
ilgili
|
||||
ise
|
||||
işte
|
||||
itibaren
|
||||
itibariyle
|
||||
kadar
|
||||
karşın
|
||||
katrilyon
|
||||
kendi
|
||||
kendilerine
|
||||
kendini
|
||||
kendisi
|
||||
kendisine
|
||||
kendisini
|
||||
kez
|
||||
ki
|
||||
kim
|
||||
kimden
|
||||
kime
|
||||
kimi
|
||||
kimse
|
||||
kırk
|
||||
milyar
|
||||
milyon
|
||||
mu
|
||||
mü
|
||||
mı
|
||||
nasıl
|
||||
ne
|
||||
neden
|
||||
nedenle
|
||||
nerde
|
||||
nerede
|
||||
nereye
|
||||
niye
|
||||
niçin
|
||||
o
|
||||
olan
|
||||
olarak
|
||||
oldu
|
||||
olduğu
|
||||
olduğunu
|
||||
olduklarını
|
||||
olmadı
|
||||
olmadığı
|
||||
olmak
|
||||
olması
|
||||
olmayan
|
||||
olmaz
|
||||
olsa
|
||||
olsun
|
||||
olup
|
||||
olur
|
||||
olursa
|
||||
oluyor
|
||||
on
|
||||
ona
|
||||
ondan
|
||||
onlar
|
||||
onlardan
|
||||
onları
|
||||
onların
|
||||
onu
|
||||
onun
|
||||
otuz
|
||||
oysa
|
||||
öyle
|
||||
pek
|
||||
rağmen
|
||||
sadece
|
||||
sanki
|
||||
sekiz
|
||||
seksen
|
||||
sen
|
||||
senden
|
||||
seni
|
||||
senin
|
||||
siz
|
||||
sizden
|
||||
sizi
|
||||
sizin
|
||||
şey
|
||||
şeyden
|
||||
şeyi
|
||||
şeyler
|
||||
şöyle
|
||||
şu
|
||||
şuna
|
||||
şunda
|
||||
şundan
|
||||
şunları
|
||||
şunu
|
||||
tarafından
|
||||
trilyon
|
||||
tüm
|
||||
üç
|
||||
üzere
|
||||
var
|
||||
vardı
|
||||
ve
|
||||
veya
|
||||
ya
|
||||
yani
|
||||
yapacak
|
||||
yapılan
|
||||
yapılması
|
||||
yapıyor
|
||||
yapmak
|
||||
yaptı
|
||||
yaptığı
|
||||
yaptığını
|
||||
yaptıkları
|
||||
yedi
|
||||
yerine
|
||||
yetmiş
|
||||
yine
|
||||
yirmi
|
||||
yoksa
|
||||
yüz
|
||||
zaten
|
|
@ -0,0 +1,29 @@
|
|||
#
|
||||
# This is a sample user dictionary for Kuromoji (JapaneseTokenizer)
|
||||
#
|
||||
# Add entries to this file in order to override the statistical model in terms
|
||||
# of segmentation, readings and part-of-speech tags. Notice that entries do
|
||||
# not have weights since they are always used when found. This is by-design
|
||||
# in order to maximize ease-of-use.
|
||||
#
|
||||
# Entries are defined using the following CSV format:
|
||||
# <text>,<token 1> ... <token n>,<reading 1> ... <reading n>,<part-of-speech tag>
|
||||
#
|
||||
# Notice that a single half-width space separates tokens and readings, and
|
||||
# that the number tokens and readings must match exactly.
|
||||
#
|
||||
# Also notice that multiple entries with the same <text> is undefined.
|
||||
#
|
||||
# Whitespace only lines are ignored. Comments are not allowed on entry lines.
|
||||
#
|
||||
|
||||
# Custom segmentation for kanji compounds
|
||||
日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞
|
||||
関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,カスタム名詞
|
||||
|
||||
# Custom segmentation for compound katakana
|
||||
トートバッグ,トート バッグ,トート バッグ,かずカナ名詞
|
||||
ショルダーバッグ,ショルダー バッグ,ショルダー バッグ,かずカナ名詞
|
||||
|
||||
# Custom reading for former sumo wrestler
|
||||
朝青龍,朝青龍,アサショウリュウ,カスタム人名
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,20 @@
|
|||
{"params":{
|
||||
"query":{
|
||||
"defType":"edismax",
|
||||
"q.alt":"*:*",
|
||||
"rows":"10",
|
||||
"fl":"*,score",
|
||||
"":{"v":0}
|
||||
},
|
||||
"facets":{
|
||||
"facet":"on",
|
||||
"facet.mincount": "1",
|
||||
"":{"v":0}
|
||||
},
|
||||
"velocity":{
|
||||
"wt": "velocity",
|
||||
"v.template":"browse",
|
||||
"v.layout": "layout",
|
||||
"":{"v":0}
|
||||
}
|
||||
}}
|
|
@ -0,0 +1,21 @@
|
|||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
#-----------------------------------------------------------------------
|
||||
# Use a protected word file to protect against the stemmer reducing two
|
||||
# unrelated words to the same base word.
|
||||
|
||||
# Some non-words that normally won't be encountered,
|
||||
# just to test that they won't be stemmed.
|
||||
dontstems
|
||||
zwhacky
|
||||
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,14 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
|
@ -0,0 +1,29 @@
|
|||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
#-----------------------------------------------------------------------
|
||||
#some test synonym mappings unlikely to appear in real input text
|
||||
aaafoo => aaabar
|
||||
bbbfoo => bbbfoo bbbbar
|
||||
cccfoo => cccbar cccbaz
|
||||
fooaaa,baraaa,bazaaa
|
||||
|
||||
# Some synonym groups specific to this example
|
||||
GB,gib,gigabyte,gigabytes
|
||||
MB,mib,megabyte,megabytes
|
||||
Television, Televisions, TV, TVs
|
||||
#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming
|
||||
#after us won't split it into two words.
|
||||
|
||||
# Synonym mappings can be used for spelling correction too
|
||||
pixima => pixma
|
||||
|
16
pom.xml
16
pom.xml
|
@ -222,7 +222,7 @@
|
|||
<dependency>
|
||||
<groupId>org.apache.solr</groupId>
|
||||
<artifactId>solr-solrj</artifactId>
|
||||
<version>7.5.0</version>
|
||||
<version>${solr.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>*</artifactId>
|
||||
|
@ -241,6 +241,19 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.solr</groupId>
|
||||
<artifactId>solr-test-framework</artifactId>
|
||||
<version>${solr.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-core</artifactId>
|
||||
<version>3.2.6</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
|
@ -625,5 +638,6 @@
|
|||
<mongodb.driver.version>3.4.2</mongodb.driver.version>
|
||||
<vtd.version>[2.12,3.0)</vtd.version>
|
||||
<dnet.openaire.broker.common>3.1.0</dnet.openaire.broker.common>
|
||||
<solr.version>7.5.0</solr.version>
|
||||
</properties>
|
||||
</project>
|
||||
|
|
Loading…
Reference in New Issue