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;
|
package eu.dnetlib.dhp.schema.scholexplorer;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.HashMap;
|
import java.util.stream.Collectors;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
|
||||||
|
@ -78,6 +76,25 @@ public class DLIUnknown extends Oaf implements Serializable {
|
||||||
if ("complete".equalsIgnoreCase(p.completionStatus))
|
if ("complete".equalsIgnoreCase(p.completionStatus))
|
||||||
completionStatus = "complete";
|
completionStatus = "complete";
|
||||||
dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom());
|
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(
|
private List<ProvenaceInfo> mergeProvenance(
|
||||||
|
|
|
@ -100,7 +100,7 @@ public class DedupRecordFactory {
|
||||||
.forEach(
|
.forEach(
|
||||||
pub -> {
|
pub -> {
|
||||||
try {
|
try {
|
||||||
Publication publication = mapper.readValue(pub, Publication.class);
|
DLIPublication publication = mapper.readValue(pub, DLIPublication.class);
|
||||||
|
|
||||||
p.mergeFrom(publication);
|
p.mergeFrom(publication);
|
||||||
p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor()));
|
p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor()));
|
||||||
|
|
|
@ -55,6 +55,7 @@ public class SparkCreateSimRels {
|
||||||
.as(Encoders.kryo(Oaf.class))
|
.as(Encoders.kryo(Oaf.class))
|
||||||
.map((MapFunction<Oaf, String>) p -> new ObjectMapper().writeValueAsString(p), Encoders.STRING())
|
.map((MapFunction<Oaf, String>) p -> new ObjectMapper().writeValueAsString(p), Encoders.STRING())
|
||||||
.javaRDD()
|
.javaRDD()
|
||||||
|
.repartition(1000)
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
s -> {
|
s -> {
|
||||||
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||||
|
|
|
@ -1,21 +1,16 @@
|
||||||
|
|
||||||
package eu.dnetlib.dedup.sx;
|
package eu.dnetlib.dedup.sx;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
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.api.java.function.MapFunction;
|
||||||
import org.apache.spark.sql.*;
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
import org.apache.spark.sql.SaveMode;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
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.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation;
|
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils;
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
public class SparkPropagateRelationsJob {
|
public class SparkPropagateRelationsJob {
|
||||||
|
@ -44,37 +39,39 @@ public class SparkPropagateRelationsJob {
|
||||||
.as(Encoders.bean(Relation.class))
|
.as(Encoders.bean(Relation.class))
|
||||||
.where("relClass == 'merges'");
|
.where("relClass == 'merges'");
|
||||||
|
|
||||||
final Dataset<DLIRelation> rels = spark
|
final Dataset<Relation> rels = spark
|
||||||
.read()
|
.read()
|
||||||
.load(relationPath)
|
.load(relationPath)
|
||||||
.as(Encoders.kryo(DLIRelation.class))
|
.as(Encoders.kryo(Relation.class))
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<DLIRelation, DLIRelation>) r -> r,
|
(MapFunction<Relation, Relation>) r -> r,
|
||||||
Encoders.bean(DLIRelation.class));
|
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")
|
.joinWith(merge, merge.col("target").equalTo(rels.col("source")), "left_outer")
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<Tuple2<DLIRelation, Relation>, DLIRelation>) r -> {
|
(MapFunction<Tuple2<Relation, Relation>, Relation>) r -> {
|
||||||
final Relation mergeRelation = r._2();
|
final Relation mergeRelation = r._2();
|
||||||
final DLIRelation relation = r._1();
|
final Relation relation = r._1();
|
||||||
if (mergeRelation != null)
|
if (mergeRelation != null)
|
||||||
relation.setSource(mergeRelation.getSource());
|
relation.setSource(mergeRelation.getSource());
|
||||||
|
if (relation.getDataInfo() == null)
|
||||||
|
relation.setDataInfo(OafUtils.generateDataInfo("0.9", false));
|
||||||
return relation;
|
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")
|
.joinWith(merge, merge.col("target").equalTo(firstJoin.col("target")), "left_outer")
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<Tuple2<DLIRelation, Relation>, DLIRelation>) r -> {
|
(MapFunction<Tuple2<Relation, Relation>, Relation>) r -> {
|
||||||
final Relation mergeRelation = r._2();
|
final Relation mergeRelation = r._2();
|
||||||
final DLIRelation relation = r._1();
|
final Relation relation = r._1();
|
||||||
if (mergeRelation != null)
|
if (mergeRelation != null)
|
||||||
relation.setTarget(mergeRelation.getSource());
|
relation.setTarget(mergeRelation.getSource());
|
||||||
return relation;
|
return relation;
|
||||||
},
|
},
|
||||||
Encoders.kryo(DLIRelation.class));
|
Encoders.kryo(Relation.class));
|
||||||
|
|
||||||
secondJoin.write().mode(SaveMode.Overwrite).save(targetRelPath);
|
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.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
|
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.commons.io.IOUtils
|
||||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||||
import org.slf4j.LoggerFactory
|
import org.slf4j.LoggerFactory
|
||||||
|
@ -11,7 +11,7 @@ import org.apache.spark.sql.functions.col
|
||||||
object SparkUpdateEntityWithDedupInfo {
|
object SparkUpdateEntityWithDedupInfo {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
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)
|
val logger = LoggerFactory.getLogger(SparkUpdateEntityWithDedupInfo.getClass)
|
||||||
parser.parseArgument(args)
|
parser.parseArgument(args)
|
||||||
|
|
||||||
|
@ -24,7 +24,7 @@ object SparkUpdateEntityWithDedupInfo {
|
||||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||||
implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
|
implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
|
||||||
implicit val dlirelEncoder: Encoder[DLIRelation] = Encoders.kryo[DLIRelation]
|
|
||||||
|
|
||||||
|
|
||||||
val spark: SparkSession = SparkSession
|
val spark: SparkSession = SparkSession
|
||||||
|
|
|
@ -53,6 +53,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory ${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
${sparkExtraOPT}
|
${sparkExtraOPT}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||||
|
@ -77,6 +78,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory ${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
${sparkExtraOPT}
|
${sparkExtraOPT}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||||
|
@ -101,6 +103,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory ${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
${sparkExtraOPT}
|
${sparkExtraOPT}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||||
|
@ -125,6 +128,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory ${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
${sparkExtraOPT}
|
${sparkExtraOPT}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||||
|
@ -149,6 +153,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory ${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
${sparkExtraOPT}
|
${sparkExtraOPT}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||||
|
|
|
@ -24,7 +24,9 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
private final boolean invisible;
|
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 = "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 = "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(
|
protected static final Qualifier ORCID_PID_TYPE = qualifier(
|
||||||
"ORCID", "Open Researcher and Contributor ID", DNET_PID_TYPES, DNET_PID_TYPES);
|
"ORCID", "Open Researcher and Contributor ID", DNET_PID_TYPES, DNET_PID_TYPES);
|
||||||
protected static final Qualifier MAG_PID_TYPE = qualifier(
|
protected static final Qualifier MAG_PID_TYPE = qualifier(
|
||||||
|
@ -55,7 +57,11 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
|
DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
|
||||||
|
|
||||||
final Document doc = DocumentHelper
|
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(
|
final KeyValue collectedFrom = getProvenanceDatasource(
|
||||||
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
||||||
|
|
|
@ -16,6 +16,8 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
import org.dom4j.Document;
|
import org.dom4j.Document;
|
||||||
import org.dom4j.Node;
|
import org.dom4j.Node;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.common.PacePerson;
|
import eu.dnetlib.dhp.common.PacePerson;
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||||
|
@ -366,7 +368,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<StructuredProperty> prepareResultPids(final Document doc, final DataInfo info) {
|
protected List<StructuredProperty> prepareResultPids(final Document doc, final DataInfo info) {
|
||||||
final List<StructuredProperty> res = new ArrayList<>();
|
final Set<StructuredProperty> res = new HashSet();
|
||||||
res
|
res
|
||||||
.addAll(
|
.addAll(
|
||||||
prepareListStructPropsWithValidQualifier(
|
prepareListStructPropsWithValidQualifier(
|
||||||
|
@ -382,7 +384,7 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
|
||||||
doc,
|
doc,
|
||||||
"//datacite:alternateIdentifier[@alternateIdentifierType != 'URL' and @alternateIdentifierType != 'landingPage']",
|
"//datacite:alternateIdentifier[@alternateIdentifierType != 'URL' and @alternateIdentifierType != 'landingPage']",
|
||||||
"@alternateIdentifierType", DNET_PID_TYPES, info));
|
"@alternateIdentifierType", DNET_PID_TYPES, info));
|
||||||
return res;
|
return Lists.newArrayList(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,6 +1,6 @@
|
||||||
package eu.dnetlib.dhp.sx.ebi
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
|
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.{Encoder, Encoders}
|
||||||
import org.apache.spark.sql.expressions.Aggregator
|
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
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
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.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.sx.ebi.model.{PMArticle, PMAuthor, PMJournal}
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils
|
import eu.dnetlib.dhp.utils.DHPUtils
|
||||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
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")}"
|
val dnetPublicationId = s"50|${DHPUtils.md5(s"$pmid::pmid")}"
|
||||||
|
|
||||||
targets.flatMap(l => {
|
targets.flatMap(l => {
|
||||||
val relation = new DLIRelation
|
val relation = new Relation
|
||||||
val inverseRelation = new DLIRelation
|
val inverseRelation = new Relation
|
||||||
val targetDnetId = s"50|${DHPUtils.md5(s"${l.tpid.toLowerCase.trim}::${l.tpidType.toLowerCase.trim}")}"
|
val targetDnetId = s"50|${DHPUtils.md5(s"${l.tpid.toLowerCase.trim}::${l.tpidType.toLowerCase.trim}")}"
|
||||||
val relInfo = relationMapper.get(l.relation.toLowerCase)
|
val relInfo = relationMapper.get(l.relation.toLowerCase)
|
||||||
val relationSemantic = relInfo.getOriginal
|
val relationSemantic = relInfo.getOriginal
|
||||||
|
@ -177,7 +177,7 @@ case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:Strin
|
||||||
val workingPath = parser.get("workingPath")
|
val workingPath = parser.get("workingPath")
|
||||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||||
implicit val oafpubEncoder: Encoder[Publication] = Encoders.kryo[Publication]
|
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 datEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
||||||
implicit val atEncoder: Encoder[Author] = Encoders.kryo(classOf[Author])
|
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]
|
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")
|
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")
|
.write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset_ebi")
|
||||||
|
|
||||||
|
|
||||||
val rel: Dataset[DLIRelation] = spark.read.load(s"$workingPath/relation").as[DLIRelation]
|
val rel: Dataset[Relation] = spark.read.load(s"$workingPath/relation").as[Relation]
|
||||||
val relupdate : Dataset[DLIRelation] = spark.read.load(s"$workingPath/ebi_garr/baseline_links_updates_relation").as[DLIRelation]
|
val relupdate : Dataset[Relation] = spark.read.load(s"$workingPath/ebi_garr/baseline_links_updates_relation").as[Relation]
|
||||||
|
|
||||||
|
|
||||||
rel.union(relupdate)
|
rel.union(relupdate)
|
||||||
.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||||
.groupByKey(_._1)(Encoders.STRING)
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
.agg(EBIAggregator.getDLIRelationAggregator().toColumn)
|
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||||
.map(p => p._2)
|
.map(p => p._2)
|
||||||
.write.mode(SaveMode.Overwrite)
|
.write.mode(SaveMode.Overwrite)
|
||||||
.save(s"$workingPath/baseline_relation_ebi")
|
.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.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
|
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.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser}
|
||||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
|
@ -38,7 +38,7 @@ object SparkCreateEBIDataFrame {
|
||||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||||
implicit val datasetEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
implicit val datasetEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
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")
|
// logger.info("Extract Publication and relation from publication_xml")
|
||||||
// val oafPubsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/publication_xml").map(s =>
|
// 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")
|
// 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 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 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))
|
publication.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, pubEncoder))
|
||||||
.groupByKey(_._1)(Encoders.STRING)
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
.agg(EBIAggregator.getDLIPublicationAggregator().toColumn)
|
.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))
|
relations.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||||
.groupByKey(_._1)(Encoders.STRING)
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
.agg(EBIAggregator.getDLIRelationAggregator().toColumn)
|
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||||
.map(p => p._2)
|
.map(p => p._2)
|
||||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/relation")
|
.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.SaveMode;
|
||||||
import org.apache.spark.sql.SparkSession;
|
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 eu.dnetlib.dhp.utils.DHPUtils;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
|
@ -55,18 +55,18 @@ public class SparkSXGeneratePidSimlarity {
|
||||||
.equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::")))
|
.equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::")))
|
||||||
.distinct();
|
.distinct();
|
||||||
|
|
||||||
JavaRDD<DLIRelation> simRel = datasetSimRel
|
JavaRDD<Relation> simRel = datasetSimRel
|
||||||
.union(publicationSimRel)
|
.union(publicationSimRel)
|
||||||
.map(
|
.map(
|
||||||
s -> {
|
s -> {
|
||||||
final DLIRelation r = new DLIRelation();
|
final Relation r = new Relation();
|
||||||
r.setSource(s._1());
|
r.setSource(s._1());
|
||||||
r.setTarget(s._2());
|
r.setTarget(s._2());
|
||||||
r.setRelType("similar");
|
r.setRelType("similar");
|
||||||
return r;
|
return r;
|
||||||
});
|
});
|
||||||
spark
|
spark
|
||||||
.createDataset(simRel.rdd(), Encoders.bean(DLIRelation.class))
|
.createDataset(simRel.rdd(), Encoders.bean(Relation.class))
|
||||||
.distinct()
|
.distinct()
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Overwrite)
|
.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.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
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.schema.scholexplorer.DLIUnknown;
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||||
import net.minidev.json.JSONArray;
|
import net.minidev.json.JSONArray;
|
||||||
|
@ -156,9 +155,9 @@ public class SparkScholexplorerCreateRawGraphJob {
|
||||||
SparkSXGeneratePidSimlarity
|
SparkSXGeneratePidSimlarity
|
||||||
.generateDataFrame(
|
.generateDataFrame(
|
||||||
spark, sc, inputPath.replace("/relation", ""), targetPath.replace("/relation", ""));
|
spark, sc, inputPath.replace("/relation", ""), targetPath.replace("/relation", ""));
|
||||||
RDD<DLIRelation> rdd = union
|
RDD<Relation> rdd = union
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
(PairFunction<String, String, DLIRelation>) f -> {
|
(PairFunction<String, String, Relation>) f -> {
|
||||||
final String source = getJPathString(SOURCEJSONPATH, f);
|
final String source = getJPathString(SOURCEJSONPATH, f);
|
||||||
final String target = getJPathString(TARGETJSONPATH, f);
|
final String target = getJPathString(TARGETJSONPATH, f);
|
||||||
final String reltype = getJPathString(RELJSONPATH, f);
|
final String reltype = getJPathString(RELJSONPATH, f);
|
||||||
|
@ -175,7 +174,7 @@ public class SparkScholexplorerCreateRawGraphJob {
|
||||||
source.toLowerCase(),
|
source.toLowerCase(),
|
||||||
reltype.toLowerCase(),
|
reltype.toLowerCase(),
|
||||||
target.toLowerCase())),
|
target.toLowerCase())),
|
||||||
mapper.readValue(f, DLIRelation.class));
|
mapper.readValue(f, Relation.class));
|
||||||
})
|
})
|
||||||
.reduceByKey(
|
.reduceByKey(
|
||||||
(a, b) -> {
|
(a, b) -> {
|
||||||
|
@ -186,7 +185,7 @@ public class SparkScholexplorerCreateRawGraphJob {
|
||||||
.rdd();
|
.rdd();
|
||||||
|
|
||||||
spark
|
spark
|
||||||
.createDataset(rdd, Encoders.bean(DLIRelation.class))
|
.createDataset(rdd, Encoders.bean(Relation.class))
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Overwrite)
|
.mode(SaveMode.Overwrite)
|
||||||
.save(targetPath);
|
.save(targetPath);
|
||||||
|
|
|
@ -1,8 +1,8 @@
|
||||||
package eu.dnetlib.dhp.sx.graph
|
package eu.dnetlib.dhp.sx.graph
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation}
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation, DLIUnknown}
|
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
|
||||||
import eu.dnetlib.dhp.sx.ebi.EBIAggregator
|
import eu.dnetlib.dhp.sx.ebi.EBIAggregator
|
||||||
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal}
|
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal}
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
|
@ -12,7 +12,7 @@ import org.slf4j.LoggerFactory
|
||||||
object SparkSplitOafTODLIEntities {
|
object SparkSplitOafTODLIEntities {
|
||||||
|
|
||||||
|
|
||||||
def getKeyRelation(rel:DLIRelation):String = {
|
def getKeyRelation(rel:Relation):String = {
|
||||||
s"${rel.getSource}::${rel.getRelType}::${rel.getTarget}"
|
s"${rel.getSource}::${rel.getRelType}::${rel.getTarget}"
|
||||||
|
|
||||||
|
|
||||||
|
@ -30,13 +30,14 @@ object SparkSplitOafTODLIEntities {
|
||||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||||
implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
|
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
|
val spark:SparkSession = SparkSession
|
||||||
.builder()
|
.builder()
|
||||||
.appName(SparkSplitOafTODLIEntities.getClass.getSimpleName)
|
.appName(SparkSplitOafTODLIEntities.getClass.getSimpleName)
|
||||||
|
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||||
.master(parser.get("master"))
|
.master(parser.get("master"))
|
||||||
.getOrCreate()
|
.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_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_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
|
OAFDataset
|
||||||
.filter(s => s != null && s.isInstanceOf[DLIRelation])
|
.filter(s => s != null && s.isInstanceOf[Relation])
|
||||||
.map(s =>s.asInstanceOf[DLIRelation])
|
.map(s =>s.asInstanceOf[Relation])
|
||||||
.union(ebi_relation)
|
.union(ebi_relation)
|
||||||
.map(d => (getKeyRelation(d), d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
.map(d => (getKeyRelation(d), d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||||
.groupByKey(_._1)(Encoders.STRING)
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
.agg(EBIAggregator.getDLIRelationAggregator().toColumn)
|
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||||
.map(p => p._2)
|
.map(p => p._2)
|
||||||
.repartition(1000)
|
.repartition(1000)
|
||||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/graph/relation")
|
.write.mode(SaveMode.Overwrite).save(s"$workingPath/graph/relation")
|
||||||
|
|
|
@ -1,8 +1,8 @@
|
||||||
package eu.dnetlib.dhp.sx.graph
|
package eu.dnetlib.dhp.sx.graph
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation}
|
||||||
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.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser}
|
||||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
|
@ -40,7 +40,7 @@ object SparkXMLToOAFDataset {
|
||||||
implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf]
|
implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf]
|
||||||
implicit val datasetEncoder:Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
implicit val datasetEncoder:Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||||
implicit val publicationEncoder:Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
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
|
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.parser.utility.VtdUtilityParser;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
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.DLIUnknown;
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo;
|
import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo;
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||||
|
@ -175,8 +174,8 @@ public abstract class AbstractScholexplorerParser {
|
||||||
.stream()
|
.stream()
|
||||||
.flatMap(
|
.flatMap(
|
||||||
n -> {
|
n -> {
|
||||||
final List<DLIRelation> rels = new ArrayList<>();
|
final List<Relation> rels = new ArrayList<>();
|
||||||
DLIRelation r = new DLIRelation();
|
Relation r = new Relation();
|
||||||
r.setSource(parsedObject.getId());
|
r.setSource(parsedObject.getId());
|
||||||
final String relatedPid = n.getTextValue();
|
final String relatedPid = n.getTextValue();
|
||||||
final String relatedPidType = n.getAttributes().get("relatedIdentifierType");
|
final String relatedPidType = n.getAttributes().get("relatedIdentifierType");
|
||||||
|
@ -184,7 +183,6 @@ public abstract class AbstractScholexplorerParser {
|
||||||
String relationSemantic = n.getAttributes().get("relationType");
|
String relationSemantic = n.getAttributes().get("relationType");
|
||||||
String inverseRelation;
|
String inverseRelation;
|
||||||
final String targetId = generateId(relatedPid, relatedPidType, relatedType);
|
final String targetId = generateId(relatedPid, relatedPidType, relatedType);
|
||||||
r.setDateOfCollection(dateOfCollection);
|
|
||||||
if (relationMapper.containsKey(relationSemantic.toLowerCase())) {
|
if (relationMapper.containsKey(relationSemantic.toLowerCase())) {
|
||||||
RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase());
|
RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase());
|
||||||
relationSemantic = relInfo.getOriginal();
|
relationSemantic = relInfo.getOriginal();
|
||||||
|
@ -199,14 +197,13 @@ public abstract class AbstractScholexplorerParser {
|
||||||
r.setCollectedfrom(parsedObject.getCollectedfrom());
|
r.setCollectedfrom(parsedObject.getCollectedfrom());
|
||||||
r.setDataInfo(di);
|
r.setDataInfo(di);
|
||||||
rels.add(r);
|
rels.add(r);
|
||||||
r = new DLIRelation();
|
r = new Relation();
|
||||||
r.setDataInfo(di);
|
r.setDataInfo(di);
|
||||||
r.setSource(targetId);
|
r.setSource(targetId);
|
||||||
r.setTarget(parsedObject.getId());
|
r.setTarget(parsedObject.getId());
|
||||||
r.setRelType(inverseRelation);
|
r.setRelType(inverseRelation);
|
||||||
r.setRelClass("datacite");
|
r.setRelClass("datacite");
|
||||||
r.setCollectedfrom(parsedObject.getCollectedfrom());
|
r.setCollectedfrom(parsedObject.getCollectedfrom());
|
||||||
r.setDateOfCollection(dateOfCollection);
|
|
||||||
rels.add(r);
|
rels.add(r);
|
||||||
if ("unknown".equalsIgnoreCase(relatedType))
|
if ("unknown".equalsIgnoreCase(relatedType))
|
||||||
result
|
result
|
||||||
|
|
|
@ -276,7 +276,6 @@ public class MappersTest {
|
||||||
System.out.println("***************");
|
System.out.println("***************");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void testClaimDedup() throws IOException {
|
void testClaimDedup() throws IOException {
|
||||||
final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_dedup.xml"));
|
final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_dedup.xml"));
|
||||||
|
@ -287,6 +286,29 @@ public class MappersTest {
|
||||||
System.out.println("***************");
|
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) {
|
private void assertValidId(final String id) {
|
||||||
assertEquals(49, id.length());
|
assertEquals(49, id.length());
|
||||||
assertEquals('|', id.charAt(2));
|
assertEquals('|', id.charAt(2));
|
||||||
|
|
|
@ -951,6 +951,7 @@ dnet:countries @=@ ZW @=@ ABW
|
||||||
dnet:protocols @=@ oai @=@ OAI-PMH
|
dnet:protocols @=@ oai @=@ OAI-PMH
|
||||||
dnet:protocols @=@ oai @=@ OAI_PMH
|
dnet:protocols @=@ oai @=@ OAI_PMH
|
||||||
dnet:pid_types @=@ orcid @=@ ORCID12
|
dnet:pid_types @=@ orcid @=@ ORCID12
|
||||||
|
dnet:pid_types @=@ handle @=@ hdl
|
||||||
dnet:review_levels @=@ 0000 @=@ UNKNOWN
|
dnet:review_levels @=@ 0000 @=@ UNKNOWN
|
||||||
dnet:review_levels @=@ 0002 @=@ 80 大阪経大学会「Working Paper」
|
dnet:review_levels @=@ 0002 @=@ 80 大阪経大学会「Working Paper」
|
||||||
dnet:review_levels @=@ 0002 @=@ AO
|
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 @=@ urn @=@ urn
|
||||||
dnet:pid_types @=@ dnet:pid_types @=@ who @=@ WHO Identifier
|
dnet:pid_types @=@ dnet:pid_types @=@ who @=@ WHO Identifier
|
||||||
dnet:pid_types @=@ dnet:pid_types @=@ drks @=@ DRKS 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/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/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
|
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"?>
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
<oai:record xmlns:oai="http://www.openarchives.org/OAI/2.0/"
|
<oai:record xmlns:oai="http://www.openarchives.org/OAI/2.0/"
|
||||||
xmlns:oaf="http://namespace.openaire.eu/oaf"
|
xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||||
xmlns:dc="http://purl.org/dc/elements/1.1/">
|
xmlns:dc="http://purl.org/dc/elements/1.1/">
|
||||||
<oai:header>
|
<oai:header>
|
||||||
<dri:repositoryId>aaadf8b3-01a8-4cc2-9964-63cfb19df3b4_UmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZXMvUmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZVR5cGU=</dri:repositoryId>
|
<dri:repositoryId>aaadf8b3-01a8-4cc2-9964-63cfb19df3b4_UmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZXMvUmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZVR5cGU=</dri:repositoryId>
|
||||||
<dri:recordIdentifier>oai:pangaea.de:doi:10.1594/PANGAEA.821876</dri:recordIdentifier>
|
<dri:recordIdentifier>oai:pangaea.de:doi:10.1594/PANGAEA.432865</dri:recordIdentifier>
|
||||||
<dri:datasourceprefix>r3d100010134</dri:datasourceprefix>
|
<dri:datasourceprefix>r3d100010134</dri:datasourceprefix>
|
||||||
<dri:objIdentifier>r3d100010134::000083be706192d2d839915694ecfd47</dri:objIdentifier>
|
<dri:objIdentifier>r3d100010134::00002f60593fd1f758fb838fafb46795</dri:objIdentifier>
|
||||||
<dri:resolvedDate>2020-01-08T04:12:12.287</dri:resolvedDate>
|
<dri:dateOfCollection>2020-02-18T03:05:02.534Z</dri:dateOfCollection>
|
||||||
<dri:dateOfCollection>2020-01-08T03:24:10.865Z</dri:dateOfCollection>
|
<oaf:datasourceprefix/>
|
||||||
<oaf:datasourceprefix/>
|
<identifier>oai:pangaea.de:doi:10.1594/PANGAEA.432865</identifier>
|
||||||
<identifier>oai:pangaea.de:doi:10.1594/PANGAEA.821876</identifier>
|
<setSpec>citable topicOceans</setSpec>
|
||||||
<setSpec>citable</setSpec>
|
</oai:header>
|
||||||
</oai:header>
|
<oai:metadata>
|
||||||
<metadata>
|
<resource xmlns="http://datacite.org/schema/kernel-3">
|
||||||
<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.432865</identifier>
|
||||||
<identifier identifierType="doi">10.1594/pangaea.821876</identifier>
|
<titles xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
<creators> <creator><creatorName>Macke, Andreas</creatorName></creator><creator><creatorName>Kalisch, John</creatorName></creator> </creators>
|
<title>Daily sea level from coastal tide gauge station Woods_Hole in 1978 (Research quality database)</title>
|
||||||
<titles> <title>Total Sky Imager observations during POLARSTERN cruise ANT-XXVI/4 on 2010-05-14 with links to images</title> </titles>
|
</titles>
|
||||||
|
<publisher xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">PANGAEA - Data Publisher for Earth & Environmental Science</publisher>
|
||||||
<publisher>PANGAEA - Data Publisher for Earth & Environmental Science</publisher>
|
<publicationYear xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">2006</publicationYear>
|
||||||
<dates>
|
<dates xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
<date dateType="Collected">2010-05-14T00:13:47/2010-05-14T23:55:47</date>
|
<date dateType="Collected">1978-01-01T12:00:00/1978-12-31T12:00:00</date>
|
||||||
</dates>
|
</dates>
|
||||||
<subjects>
|
<creators xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
|
<creator>
|
||||||
<subject subjectScheme="Parameter">DATE/TIME</subject>
|
<creatorName>WOCE Sea Level, WSL</creatorName>
|
||||||
|
</creator>
|
||||||
<subject subjectScheme="Parameter">LATITUDE</subject>
|
</creators>
|
||||||
|
<subjects xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
<subject subjectScheme="Parameter">LONGITUDE</subject>
|
<subject subjectScheme="Parameter">DATE/TIME</subject>
|
||||||
|
<subject subjectScheme="Parameter">Sea level</subject>
|
||||||
<subject subjectScheme="Parameter">Uniform resource locator/link to image</subject>
|
<subject subjectScheme="Method">Tide gauge station</subject>
|
||||||
|
<subject subjectScheme="Campaign">SeaLevel</subject>
|
||||||
<subject subjectScheme="Method">Total Sky Imager</subject>
|
<subject subjectScheme="Project">World Ocean Circulation Experiment (WOCE)</subject>
|
||||||
|
</subjects>
|
||||||
<subject subjectScheme="Campaign">ANT-XXVI/4</subject>
|
<resourceType resourceTypeGeneral="Dataset"/>
|
||||||
|
<relatedIdentifiers>
|
||||||
<subject subjectScheme="Basis">Polarstern</subject>
|
<relatedIdentifier relatedIdentifierType="URL" relationType="isDocumentedBy"
|
||||||
|
inverseRelationType="documents">http://store.pangaea.de/Projects/WOCE/SeaLevel_rqds/Woods_Hole.txt</relatedIdentifier>
|
||||||
</subjects>
|
</relatedIdentifiers>
|
||||||
<resourceType resourceTypeGeneral="dataset">dataset</resourceType>
|
</resource>
|
||||||
<relatedIdentifiers>
|
</oai:metadata>
|
||||||
|
<oaf:about>
|
||||||
<relatedIdentifier relatedIdentifierType="dnet" relationType="isPartOf" inverseRelationType="hasPart" entityType="dataset">dli_resolver::cf447a378b0b6603593f8b0e57242695</relatedIdentifier>
|
<oaf:datainfo>
|
||||||
|
<oaf:collectedFrom completionStatus="complete" id="dli_________::r3d100010134" name="Pangaea"/>
|
||||||
<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>
|
<oaf:completionStatus>complete</oaf:completionStatus>
|
||||||
|
<oaf:provisionMode>collected</oaf:provisionMode>
|
||||||
<relatedIdentifier relatedIdentifierType="dnet" relationType="references" inverseRelationType="isReferencedBy" entityType="publication">dli_resolver::f0f5975d20991cffd222c6002ddd5821</relatedIdentifier>
|
</oaf:datainfo>
|
||||||
|
</oaf:about>
|
||||||
</relatedIdentifiers>
|
</oai:record>
|
||||||
</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>
|
|
|
@ -6,7 +6,7 @@ import java.time.format.DateTimeFormatter
|
||||||
import eu.dnetlib.dhp.common.PacePerson
|
import eu.dnetlib.dhp.common.PacePerson
|
||||||
import eu.dnetlib.dhp.schema.action.AtomicAction
|
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.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 eu.dnetlib.dhp.utils.DHPUtils
|
||||||
import org.apache.commons.lang3.StringUtils
|
import org.apache.commons.lang3.StringUtils
|
||||||
import org.codehaus.jackson.map.ObjectMapper
|
import org.codehaus.jackson.map.ObjectMapper
|
||||||
|
@ -273,29 +273,29 @@ object DLIToOAF {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def convertDLIRelation(r: DLIRelation): Relation = {
|
// def convertDLIRelation(r: DLIRelation): Relation = {
|
||||||
|
//
|
||||||
val result = new Relation
|
// val result = new Relation
|
||||||
if (!relationTypeMapping.contains(r.getRelType))
|
// if (!relationTypeMapping.contains(r.getRelType))
|
||||||
return null
|
// return null
|
||||||
|
//
|
||||||
if (r.getCollectedFrom == null || r.getCollectedFrom.size() == 0 || (r.getCollectedFrom.size() == 1 && r.getCollectedFrom.get(0) == null))
|
// if (r.getProperties == null || r.getProperties.size() == 0 || (r.getProperties.size() == 1 && r.getProperties.get(0) == null))
|
||||||
return null
|
// return null
|
||||||
val t = relationTypeMapping.get(r.getRelType)
|
// val t = relationTypeMapping.get(r.getRelType)
|
||||||
|
//
|
||||||
result.setRelType("resultResult")
|
// result.setRelType("resultResult")
|
||||||
result.setRelClass(t.get._1)
|
// result.setRelClass(t.get._1)
|
||||||
result.setSubRelType(t.get._2)
|
// result.setSubRelType(t.get._2)
|
||||||
result.setCollectedfrom(r.getCollectedFrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
|
// result.setCollectedfrom(r.getProperties.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
|
||||||
result.setSource(generateId(r.getSource))
|
// result.setSource(generateId(r.getSource))
|
||||||
result.setTarget(generateId(r.getTarget))
|
// result.setTarget(generateId(r.getTarget))
|
||||||
|
//
|
||||||
if (result.getSource.equals(result.getTarget))
|
// if (result.getSource.equals(result.getTarget))
|
||||||
return null
|
// return null
|
||||||
result.setDataInfo(generateDataInfo())
|
// result.setDataInfo(generateDataInfo())
|
||||||
|
//
|
||||||
result
|
// result
|
||||||
}
|
// }
|
||||||
|
|
||||||
|
|
||||||
def convertDLIDatasetTOOAF(d: DLIDataset): Dataset = {
|
def convertDLIDatasetTOOAF(d: DLIDataset): Dataset = {
|
||||||
|
|
|
@ -2,7 +2,7 @@ package eu.dnetlib.dhp.`export`
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Instance, Publication, Relation, Dataset => OafDataset}
|
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.commons.io.IOUtils
|
||||||
import org.apache.hadoop.io.Text
|
import org.apache.hadoop.io.Text
|
||||||
import org.apache.hadoop.io.compress.GzipCodec
|
import org.apache.hadoop.io.compress.GzipCodec
|
||||||
|
@ -39,14 +39,13 @@ object SparkExportContentForOpenAire {
|
||||||
implicit val pubEncoder: Encoder[Publication] = Encoders.bean(classOf[Publication])
|
implicit val pubEncoder: Encoder[Publication] = Encoders.bean(classOf[Publication])
|
||||||
implicit val datEncoder: Encoder[OafDataset] = Encoders.bean(classOf[OafDataset])
|
implicit val datEncoder: Encoder[OafDataset] = Encoders.bean(classOf[OafDataset])
|
||||||
implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
|
implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
|
||||||
implicit val dliRelEncoder: Encoder[DLIRelation] = Encoders.bean(classOf[DLIRelation])
|
|
||||||
import spark.implicits._
|
import spark.implicits._
|
||||||
|
|
||||||
|
|
||||||
val relRDD:RDD[Relation] = sc.textFile(s"$workingPath/relation_j")
|
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)
|
.filter(p => p.getDataInfo.getDeletedbyinference == false)
|
||||||
.map(DLIToOAF.convertDLIRelation).filter(p=>p!= null)
|
|
||||||
spark.createDataset(relRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationDS")
|
spark.createDataset(relRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationDS")
|
||||||
|
|
||||||
val datRDD:RDD[OafDataset] = sc.textFile(s"$workingPath/dataset")
|
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
|
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}
|
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
|
val relatedPublication = relation
|
||||||
.where("target like '50%'")
|
.where("target like '50%'")
|
||||||
|
@ -34,7 +54,7 @@ object DatasetJoiner {
|
||||||
coalesce(col("dataset"),lit(0)).alias("relatedDataset"),
|
coalesce(col("dataset"),lit(0)).alias("relatedDataset"),
|
||||||
coalesce(col("unknown"),lit(0)).alias("relatedUnknown")
|
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.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
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.apache.spark.sql.SparkSession;
|
||||||
import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
|
import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary;
|
|
||||||
|
|
||||||
public class SparkIndexCollectionOnES {
|
public class SparkIndexCollectionOnES {
|
||||||
|
|
||||||
|
@ -39,33 +36,20 @@ public class SparkIndexCollectionOnES {
|
||||||
final String sourcePath = parser.get("sourcePath");
|
final String sourcePath = parser.get("sourcePath");
|
||||||
final String index = parser.get("index");
|
final String index = parser.get("index");
|
||||||
final String idPath = parser.get("idPath");
|
final String idPath = parser.get("idPath");
|
||||||
final String type = parser.get("type");
|
final String cluster = parser.get("cluster");
|
||||||
final String indexHost = parser.get("esHost");
|
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 SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
|
||||||
|
|
||||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
JavaRDD<String> inputRdd;
|
JavaRDD<String> inputRdd = sc.textFile(sourcePath);
|
||||||
|
|
||||||
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);
|
|
||||||
|
|
||||||
Map<String, String> esCfg = new HashMap<>();
|
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", clusterMap.get(cluster));
|
||||||
esCfg.put("es.nodes", indexHost);
|
|
||||||
esCfg.put("es.mapping.id", idPath);
|
esCfg.put("es.mapping.id", idPath);
|
||||||
esCfg.put("es.batch.write.retry.count", "8");
|
esCfg.put("es.batch.write.retry.count", "8");
|
||||||
esCfg.put("es.batch.write.retry.wait", "60s");
|
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.provision.RelatedItemInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
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.oaf.StructuredProperty;
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
||||||
|
@ -138,54 +140,20 @@ public class ScholixSummary implements Serializable {
|
||||||
this.datasources = datasources;
|
this.datasources = datasources;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ScholixSummary fromJsonOAF(final Typology oafType, final String oafJson) {
|
public static ScholixSummary fromOAF(final Oaf oaf) {
|
||||||
try {
|
try {
|
||||||
final ObjectMapper mapper = new ObjectMapper();
|
|
||||||
final RelatedItemInfo relatedItemInfo = new RelatedItemInfo();
|
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(
|
if (oaf instanceof DLIPublication)
|
||||||
final Typology oafType, final String oafJson, final String relEntityJson) {
|
return summaryFromPublication((DLIPublication) oaf, relatedItemInfo);
|
||||||
try {
|
if (oaf instanceof DLIDataset)
|
||||||
final ObjectMapper mapper = new ObjectMapper();
|
return summaryFromDataset((DLIDataset) oaf, relatedItemInfo);
|
||||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
if (oaf instanceof DLIUnknown)
|
||||||
|
return summaryFromUnknown((DLIUnknown) oaf, relatedItemInfo);
|
||||||
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));
|
|
||||||
}
|
|
||||||
|
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
return null;
|
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
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "h",
|
"paramName": "c",
|
||||||
"paramLongName": "esHost",
|
"paramLongName": "cluster",
|
||||||
"paramDescription": "the index host name",
|
"paramDescription": "the index cluster",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
|
|
||||||
|
|
||||||
{
|
|
||||||
"paramName": "t",
|
|
||||||
"paramLongName": "type",
|
|
||||||
"paramDescription": "should be scholix or summary",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "id",
|
"paramName": "id",
|
||||||
"paramLongName": "idPath",
|
"paramLongName": "idPath",
|
||||||
|
|
|
@ -7,4 +7,8 @@
|
||||||
<name>oozie.action.sharelib.for.spark</name>
|
<name>oozie.action.sharelib.for.spark</name>
|
||||||
<value>spark2</value>
|
<value>spark2</value>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||||
|
<value>true</value>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
|
@ -8,6 +8,14 @@
|
||||||
<name>graphPath</name>
|
<name>graphPath</name>
|
||||||
<description>the graph path</description>
|
<description>the graph path</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>index</name>
|
||||||
|
<description>the index name</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>esCluster</name>
|
||||||
|
<description>the Index cluster</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
|
@ -16,18 +24,6 @@
|
||||||
<name>sparkExecutorMemory</name>
|
<name>sparkExecutorMemory</name>
|
||||||
<description>memory for individual executor</description>
|
<description>memory for individual executor</description>
|
||||||
</property>
|
</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>
|
</parameters>
|
||||||
|
|
||||||
<start to="DeleteTargetPath"/>
|
<start to="DeleteTargetPath"/>
|
||||||
|
@ -53,7 +49,7 @@
|
||||||
<name>calculate for each ID the number of related Dataset, publication and Unknown</name>
|
<name>calculate for each ID the number of related Dataset, publication and Unknown</name>
|
||||||
<class>eu.dnetlib.dhp.provision.SparkExtractRelationCount</class>
|
<class>eu.dnetlib.dhp.provision.SparkExtractRelationCount</class>
|
||||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
<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>-mt</arg> <arg>yarn-cluster</arg>
|
||||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||||
<arg>--relationPath</arg><arg>${graphPath}/relation</arg>
|
<arg>--relationPath</arg><arg>${graphPath}/relation</arg>
|
||||||
|
@ -69,9 +65,9 @@
|
||||||
<master>yarn-cluster</master>
|
<master>yarn-cluster</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>generate Summary</name>
|
<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>
|
<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>-mt</arg> <arg>yarn-cluster</arg>
|
||||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||||
<arg>--graphPath</arg><arg>${graphPath}</arg>
|
<arg>--graphPath</arg><arg>${graphPath}</arg>
|
||||||
|
@ -87,56 +83,88 @@
|
||||||
<master>yarn-cluster</master>
|
<master>yarn-cluster</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>generate Scholix</name>
|
<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>
|
<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>-mt</arg> <arg>yarn-cluster</arg>
|
||||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||||
<arg>--graphPath</arg><arg>${graphPath}</arg>
|
<arg>--graphPath</arg><arg>${graphPath}</arg>
|
||||||
</spark>
|
</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"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</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"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
|
@ -4,7 +4,7 @@ import java.time.LocalDateTime
|
||||||
import java.time.format.DateTimeFormatter
|
import java.time.format.DateTimeFormatter
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
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.SparkConf
|
||||||
import org.apache.spark.rdd.RDD
|
import org.apache.spark.rdd.RDD
|
||||||
import org.apache.spark.sql.SparkSession
|
import org.apache.spark.sql.SparkSession
|
||||||
|
@ -65,7 +65,7 @@ class ExportDLITOOAFTest {
|
||||||
val json = Source.fromInputStream(getClass.getResourceAsStream("relation.json")).mkString
|
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))
|
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,
|
"lastupdatetimestamp": null,
|
||||||
"relType": "IsReferencedBy",
|
"relType": "IsReferencedBy",
|
||||||
"source": "60|4ee78ab329b49416b45c3774c132f244",
|
"source": "60|4ee78ab329b49416b45c3774c132f244",
|
||||||
"collectedFrom": [
|
"collectedfrom": [
|
||||||
{
|
{
|
||||||
"dataInfo": null,
|
"dataInfo": null,
|
||||||
"value": "Europe PMC",
|
"value": "Europe PMC",
|
||||||
|
|
|
@ -48,6 +48,80 @@
|
||||||
<artifactId>spark-solr</artifactId>
|
<artifactId>spark-solr</artifactId>
|
||||||
</dependency>
|
</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>
|
<dependency>
|
||||||
<groupId>org.apache.httpcomponents</groupId>
|
<groupId>org.apache.httpcomponents</groupId>
|
||||||
<artifactId>httpclient</artifactId>
|
<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.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
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 Logger log = LoggerFactory.getLogger(XmlIndexingJob.class);
|
||||||
|
|
||||||
private static final Integer DEFAULT_BATCH_SIZE = 1000;
|
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 {
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
||||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -208,23 +203,4 @@ public class XmlIndexingJob {
|
||||||
format));
|
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>
|
<name>batchSize</name>
|
||||||
<description>number of records to be included in each indexing request</description>
|
<description>number of records to be included in each indexing request</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>solrDeletionQuery</name>
|
||||||
|
<value>*:*</value>
|
||||||
|
<description>query used in the deleted by query operation</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemoryForJoining</name>
|
<name>sparkDriverMemoryForJoining</name>
|
||||||
<description>memory for driver process</description>
|
<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_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="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="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>
|
<case to="to_solr_index">${wf:conf('resumeFrom') eq 'to_solr_index'}</case>
|
||||||
<default to="prepare_relations"/>
|
<default to="prepare_relations"/>
|
||||||
</switch>
|
</switch>
|
||||||
|
@ -584,12 +589,25 @@
|
||||||
|
|
||||||
<decision name="should_index">
|
<decision name="should_index">
|
||||||
<switch>
|
<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>
|
<case to="End">${wf:conf('shouldIndex') eq 'false'}</case>
|
||||||
<default to="to_solr_index"/>
|
<default to="drop_solr_collection"/>
|
||||||
</switch>
|
</switch>
|
||||||
</decision>
|
</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">
|
<action name="to_solr_index">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
|
@ -615,6 +633,17 @@
|
||||||
<arg>--format</arg><arg>${format}</arg>
|
<arg>--format</arg><arg>${format}</arg>
|
||||||
<arg>--batchSize</arg><arg>${batchSize}</arg>
|
<arg>--batchSize</arg><arg>${batchSize}</arg>
|
||||||
</spark>
|
</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"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
|
@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.provision;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
|
@ -20,8 +21,6 @@ import org.junit.jupiter.api.io.TempDir;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
|
|
||||||
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
|
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
|
||||||
|
@ -31,7 +30,8 @@ public class PrepareRelationsJobTest {
|
||||||
|
|
||||||
public static final String SUBRELTYPE = "subRelType";
|
public static final String SUBRELTYPE = "subRelType";
|
||||||
public static final String OUTCOME = "outcome";
|
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;
|
private static SparkSession spark;
|
||||||
|
|
||||||
|
@ -64,7 +64,7 @@ public class PrepareRelationsJobTest {
|
||||||
@Test
|
@Test
|
||||||
public void testRunPrepareRelationsJob(@TempDir Path testPath) throws Exception {
|
public void testRunPrepareRelationsJob(@TempDir Path testPath) throws Exception {
|
||||||
|
|
||||||
final int maxRelations = 10;
|
final int maxRelations = 20;
|
||||||
PrepareRelationsJob
|
PrepareRelationsJob
|
||||||
.main(
|
.main(
|
||||||
new String[] {
|
new String[] {
|
||||||
|
@ -73,7 +73,8 @@ public class PrepareRelationsJobTest {
|
||||||
"-outputPath", testPath.toString(),
|
"-outputPath", testPath.toString(),
|
||||||
"-relPartitions", "10",
|
"-relPartitions", "10",
|
||||||
"-relationFilter", "asd",
|
"-relationFilter", "asd",
|
||||||
"-maxRelations", String.valueOf(maxRelations)
|
"-sourceMaxRelations", String.valueOf(maxRelations),
|
||||||
|
"-targetMaxRelations", String.valueOf(maxRelations * 100)
|
||||||
});
|
});
|
||||||
|
|
||||||
Dataset<Relation> out = spark
|
Dataset<Relation> out = spark
|
||||||
|
@ -82,19 +83,31 @@ public class PrepareRelationsJobTest {
|
||||||
.as(Encoders.bean(Relation.class))
|
.as(Encoders.bean(Relation.class))
|
||||||
.cache();
|
.cache();
|
||||||
|
|
||||||
Assertions.assertEquals(10, out.count());
|
Assertions.assertEquals(maxRelations, out.count());
|
||||||
|
|
||||||
Dataset<Row> freq = out
|
Dataset<Row> freq = out
|
||||||
.toDF()
|
.toDF()
|
||||||
.cube(SUBRELTYPE)
|
.cube(SUBRELTYPE)
|
||||||
.count()
|
.count()
|
||||||
.filter((FilterFunction<Row>) value -> !value.isNullAt(0));
|
.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(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>
|
<dependency>
|
||||||
<groupId>org.apache.solr</groupId>
|
<groupId>org.apache.solr</groupId>
|
||||||
<artifactId>solr-solrj</artifactId>
|
<artifactId>solr-solrj</artifactId>
|
||||||
<version>7.5.0</version>
|
<version>${solr.version}</version>
|
||||||
<exclusions>
|
<exclusions>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
|
@ -241,6 +241,19 @@
|
||||||
</exclusion>
|
</exclusion>
|
||||||
</exclusions>
|
</exclusions>
|
||||||
</dependency>
|
</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>
|
<dependency>
|
||||||
<groupId>org.apache.httpcomponents</groupId>
|
<groupId>org.apache.httpcomponents</groupId>
|
||||||
|
@ -625,5 +638,6 @@
|
||||||
<mongodb.driver.version>3.4.2</mongodb.driver.version>
|
<mongodb.driver.version>3.4.2</mongodb.driver.version>
|
||||||
<vtd.version>[2.12,3.0)</vtd.version>
|
<vtd.version>[2.12,3.0)</vtd.version>
|
||||||
<dnet.openaire.broker.common>3.1.0</dnet.openaire.broker.common>
|
<dnet.openaire.broker.common>3.1.0</dnet.openaire.broker.common>
|
||||||
|
<solr.version>7.5.0</solr.version>
|
||||||
</properties>
|
</properties>
|
||||||
</project>
|
</project>
|
||||||
|
|
Loading…
Reference in New Issue