diff --git a/python/sparknlp/annotator/matcher/date_matcher.py b/python/sparknlp/annotator/matcher/date_matcher.py index 55c38010448295..8bf9583eef6884 100755 --- a/python/sparknlp/annotator/matcher/date_matcher.py +++ b/python/sparknlp/annotator/matcher/date_matcher.py @@ -67,6 +67,11 @@ class DateMatcherUtils(Params): "source language for explicit translation", typeConverter=TypeConverters.toString) + relaxedFactoryStrategy = Param(Params._dummy(), + "relaxedFactoryStrategy", + "Matched Strategy to searches relaxed dates", + typeConverter=TypeConverters.toString) + def setInputFormats(self, value): """Sets input formats patterns to match in the documents. @@ -159,6 +164,19 @@ def setAnchorDateDay(self, value): """ return self._set(anchorDateDay=value) + def setRelaxedFactoryStrategy(self, matchStrategy=MatchStrategy.MATCH_FIRST): + """ Sets matched strategy to search relaxed dates by ordered rules by more exhaustive to less Strategy. + + Not all of the date information needs to be included. For example + ``"YYYY"`` is also a valid input. + + Parameters + ---------- + matchStrategy : MatchStrategy + Matched strategy to search relaxed dates by ordered rules by more exhaustive to less Strategy + """ + return self._set(relaxedFactoryStrategy=matchStrategy) + class DateMatcher(AnnotatorModel, DateMatcherUtils): """Matches standard date formats into a provided format diff --git a/python/sparknlp/common/__init__.py b/python/sparknlp/common/__init__.py index 8a83c082e82516..f5cdcb45079544 100644 --- a/python/sparknlp/common/__init__.py +++ b/python/sparknlp/common/__init__.py @@ -22,3 +22,4 @@ from sparknlp.common.storage import * from sparknlp.common.utils import * from sparknlp.common.annotator_type import * +from sparknlp.common.match_strategy import * diff --git a/python/sparknlp/common/match_strategy.py b/python/sparknlp/common/match_strategy.py new file mode 100644 index 00000000000000..1fa455aeaf2588 --- /dev/null +++ b/python/sparknlp/common/match_strategy.py @@ -0,0 +1,33 @@ +# Copyright 2017-2023 John Snow Labs +# +# Licensed 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. + +"""Allowed strategies for RuleFactory applications regarding replacement""" + + +class MatchStrategy(object): + """Object that contains constants for how for matched strategies used in RuleFactory. + + Possible values are: + + ================================== =============================================================================== + Value Description + ================================== =============================================================================== + ``MatchStrategy.MATCH_ALL`` This strategy matches all occurrences of all rules in the given text. + ``MatchStrategy.MATCH_FIRST`` This strategy matches only the first occurrence of each rule in the given text. + ``MatchStrategy.MATCH_COMPLETE`` This strategy matches only the first occurrence of each rule in the given text. + ================================== =============================================================================== + """ + MATCH_ALL = "MATCH_ALL" + MATCH_FIRST = "MATCH_FIRST" + MATCH_COMPLETE = "MATCH_COMPLETE" diff --git a/python/test/annotator/matcher/multi_date_matcher_test.py b/python/test/annotator/matcher/multi_date_matcher_test.py new file mode 100644 index 00000000000000..a7fe4636fa30dc --- /dev/null +++ b/python/test/annotator/matcher/multi_date_matcher_test.py @@ -0,0 +1,50 @@ +# Copyright 2017-2022 John Snow Labs +# +# Licensed 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. +import unittest + +import pytest + +from sparknlp.annotator import * +from sparknlp.base import * +from pyspark.sql.functions import size +from test.util import SparkContextForTest + + +@pytest.mark.fast +class MultiDateMatcherTestSpec(unittest.TestCase): + + def setUp(self): + text = """ + Lease Period Monthly Installment of Base Rent. + January 1, 2021 –December 31, 2021 $20,304.85 . + January 1, 2022 –December 31, 2022 $20,914.00 . + """ + self.data = SparkContextForTest.spark.createDataFrame([[text]]).toDF("text") + + def runTest(self): + document_assembler = DocumentAssembler() \ + .setInputCol("text") \ + .setOutputCol("document") + date_matcher = MultiDateMatcher() \ + .setInputCols(["document"]) \ + .setOutputCol("date") \ + .setOutputFormat("yyyy/MM/dd") \ + .setRelaxedFactoryStrategy(MatchStrategy.MATCH_ALL) + + pipeline = Pipeline(stages=[document_assembler, date_matcher]) + model = pipeline.fit(self.data) + result = model.transform(self.data) + + actual_dates = result.select(size("date.result")).collect()[0][0] + self.assertEquals(actual_dates, 4) diff --git a/src/main/scala/com/johnsnowlabs/nlp/annotators/DateMatcherUtils.scala b/src/main/scala/com/johnsnowlabs/nlp/annotators/DateMatcherUtils.scala index 3f3095f1c51031..d44a921dab4076 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/annotators/DateMatcherUtils.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/annotators/DateMatcherUtils.scala @@ -16,7 +16,8 @@ package com.johnsnowlabs.nlp.annotators -import com.johnsnowlabs.nlp.util.regex.{MatchStrategy, RuleFactory} +import com.johnsnowlabs.nlp.util.io.MatchStrategy +import com.johnsnowlabs.nlp.util.regex.RuleFactory import org.apache.spark.ml.param._ import java.util.Calendar @@ -249,6 +250,31 @@ trait DateMatcherUtils extends Params { */ def setSourceLanguage(value: String): this.type = set(sourceLanguage, value) + /** Matched strategy to search relaxed dates by ordered rules by more exhaustive to less + * Strategy + * + * @group param + */ + val relaxedFactoryStrategy: Param[String] = + new Param(this, "relaxedFactoryStrategy", "Matched Strategy to searches relaxed dates") + + /** To set matched strategy to search relaxed dates by ordered rules by more exhaustive to less + * Strategy + * + * @group param + */ + def setRelaxedFactoryStrategy( + matchStrategy: MatchStrategy.Format = MatchStrategy.MATCH_FIRST): this.type = { + set(relaxedFactoryStrategy, matchStrategy.toString) + } + + /** To get matched strategy to search relaxed dates by ordered rules by more exhaustive to less + * Strategy + * + * @group param + */ + def getRelaxedFactoryStrategy: String = $(relaxedFactoryStrategy) + setDefault( inputFormats -> Array(""), outputFormat -> "yyyy/MM/dd", @@ -257,7 +283,8 @@ trait DateMatcherUtils extends Params { anchorDateDay -> -1, readMonthFirst -> true, defaultDayWhenMissing -> 1, - sourceLanguage -> "en") + sourceLanguage -> "en", + relaxedFactoryStrategy -> MatchStrategy.MATCH_FIRST.toString) protected val formalFactoryInputFormats = new RuleFactory(MatchStrategy.MATCH_ALL) @@ -322,11 +349,10 @@ trait DateMatcherUtils extends Params { .addRule(formalDateAlt2, "formal date with year at beginning") .addRule(formalDateShort, "formal date short version") - /** Searches relaxed dates by ordered rules by more exhaustive to less Strategy used is to match - * first only. any other matches discarded Auto completes short versions of months. Any two - * digit year is considered to be XX century + /** Searches relaxed dates by ordered rules by more exhaustive to less Strategy. Auto completes + * short versions of months. Any two digit year is considered to be XX century */ - protected val relaxedFactory: RuleFactory = new RuleFactory(MatchStrategy.MATCH_FIRST) + protected lazy val relaxedFactory: RuleFactory = new RuleFactory(getRelaxedFactoryStrategy) .addRule(relaxedDayNumbered, "relaxed days") .addRule(relaxedMonths.r, "relaxed months exclusive") .addRule(relaxedYear, "relaxed year") diff --git a/src/main/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcher.scala b/src/main/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcher.scala index 53561604988481..b711a532c63619 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcher.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcher.scala @@ -248,6 +248,14 @@ class MultiDateMatcher(override val uid: String) private def extractRelaxedDate(text: String): Seq[MatchedDateTime] = { val possibleDates = relaxedFactory.findMatch(text) + val possibleDatesByIndexMatch = possibleDates.groupBy(_.indexMatch) + possibleDatesByIndexMatch.flatMap { case (_, possibleDates) => + computePossibleDates(possibleDates) + }.toSeq + } + + private def computePossibleDates( + possibleDates: Seq[RuleFactory.RuleMatch]): Seq[MatchedDateTime] = { var dayMatch = $(defaultDayWhenMissing) var monthMatch = defaultMonthWhenMissing var yearMatch = defaultYearWhenMissing diff --git a/src/main/scala/com/johnsnowlabs/nlp/annotators/RegexMatcherModel.scala b/src/main/scala/com/johnsnowlabs/nlp/annotators/RegexMatcherModel.scala index 9b34ca3f8f21b7..921aa9b6b85c65 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/annotators/RegexMatcherModel.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/annotators/RegexMatcherModel.scala @@ -19,8 +19,8 @@ package com.johnsnowlabs.nlp.annotators import com.johnsnowlabs.nlp.AnnotatorType.{CHUNK, DOCUMENT} import com.johnsnowlabs.nlp._ import com.johnsnowlabs.nlp.serialization.ArrayFeature -import com.johnsnowlabs.nlp.util.regex.MatchStrategy.MatchStrategy -import com.johnsnowlabs.nlp.util.regex.{MatchStrategy, RegexRule, RuleFactory, TransformStrategy} +import com.johnsnowlabs.nlp.util.io.MatchStrategy +import com.johnsnowlabs.nlp.util.regex.{RegexRule, RuleFactory, TransformStrategy} import org.apache.spark.ml.param.Param import org.apache.spark.ml.util.Identifiable @@ -104,7 +104,7 @@ class RegexMatcherModel(override val uid: String) def getExternalRules: Array[(String, String)] = $$(externalRules) /** MATCH_ALL|MATCH_FIRST|MATCH_COMPLETE */ - private def getFactoryStrategy: MatchStrategy = $(strategy) match { + private def getFactoryStrategy: MatchStrategy.Format = $(strategy) match { case "MATCH_ALL" => MatchStrategy.MATCH_ALL case "MATCH_FIRST" => MatchStrategy.MATCH_FIRST case "MATCH_COMPLETE" => MatchStrategy.MATCH_COMPLETE diff --git a/src/main/scala/com/johnsnowlabs/nlp/annotators/Tokenizer.scala b/src/main/scala/com/johnsnowlabs/nlp/annotators/Tokenizer.scala index 694811d52a99d2..cf4c6d132769c4 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/annotators/Tokenizer.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/annotators/Tokenizer.scala @@ -19,8 +19,8 @@ package com.johnsnowlabs.nlp.annotators import com.johnsnowlabs.nlp.AnnotatorApproach import com.johnsnowlabs.nlp.AnnotatorType.{DOCUMENT, TOKEN} import com.johnsnowlabs.nlp.annotators.param.ExternalResourceParam -import com.johnsnowlabs.nlp.util.io.{ExternalResource, ReadAs, ResourceHelper} -import com.johnsnowlabs.nlp.util.regex.{MatchStrategy, RuleFactory} +import com.johnsnowlabs.nlp.util.io.{ExternalResource, MatchStrategy, ReadAs, ResourceHelper} +import com.johnsnowlabs.nlp.util.regex.RuleFactory import org.apache.spark.ml.PipelineModel import org.apache.spark.ml.param.{BooleanParam, IntParam, Param, StringArrayParam} import org.apache.spark.ml.util.{DefaultParamsReadable, Identifiable} diff --git a/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticContentFormatter.scala b/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticContentFormatter.scala index 9b8240fd56edc4..447e0673502ea9 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticContentFormatter.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticContentFormatter.scala @@ -18,7 +18,7 @@ package com.johnsnowlabs.nlp.annotators.sbd.pragmatic import com.johnsnowlabs.nlp.annotators.sbd.pragmatic.PragmaticDictionaries._ import com.johnsnowlabs.nlp.annotators.sbd.pragmatic.PragmaticSymbols._ -import com.johnsnowlabs.nlp.util.regex.MatchStrategy._ +import com.johnsnowlabs.nlp.util.io.MatchStrategy.MATCH_ALL import com.johnsnowlabs.nlp.util.regex.TransformStrategy._ import com.johnsnowlabs.nlp.util.regex.{RegexRule, RuleFactory} diff --git a/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticMethod.scala b/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticMethod.scala index c2900ac8766df8..caec7375b45da3 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticMethod.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/annotators/sbd/pragmatic/PragmaticMethod.scala @@ -17,7 +17,7 @@ package com.johnsnowlabs.nlp.annotators.sbd.pragmatic import com.johnsnowlabs.nlp.annotators.common.Sentence -import com.johnsnowlabs.nlp.util.regex.MatchStrategy.MATCH_ALL +import com.johnsnowlabs.nlp.util.io.MatchStrategy.MATCH_ALL import com.johnsnowlabs.nlp.util.regex.RuleFactory import com.johnsnowlabs.nlp.util.regex.TransformStrategy.{ REPLACE_ALL_WITH_SYMBOL, diff --git a/src/main/scala/com/johnsnowlabs/nlp/util/io/MatchStrategy.scala b/src/main/scala/com/johnsnowlabs/nlp/util/io/MatchStrategy.scala new file mode 100644 index 00000000000000..1aaa3d5006006f --- /dev/null +++ b/src/main/scala/com/johnsnowlabs/nlp/util/io/MatchStrategy.scala @@ -0,0 +1,37 @@ +/* + * Copyright 2017-2023 John Snow Labs + * + * Licensed 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. + */ + +package com.johnsnowlabs.nlp.util.io + +import com.johnsnowlabs.nlp.util.regex.RuleFactory + +/** Allowed strategies for [[RuleFactory]] applications regarding replacement */ +object MatchStrategy extends Enumeration { + + implicit def str2frmt(v: String): Format = { + v.toUpperCase match { + case "MATCH_ALL" => MATCH_ALL + case "MATCH_FIRST" => MATCH_FIRST + case "MATCH_COMPLETE" => MATCH_COMPLETE + case _ => + throw new MatchError( + s"Invalid MatchStrategy. Must be either of ${this.values.mkString("|")}") + } + } + + type Format = Value + val MATCH_ALL, MATCH_FIRST, MATCH_COMPLETE = Value +} diff --git a/src/main/scala/com/johnsnowlabs/nlp/util/regex/RuleFactory.scala b/src/main/scala/com/johnsnowlabs/nlp/util/regex/RuleFactory.scala index 177f436149065c..e7da0c8bd347e6 100644 --- a/src/main/scala/com/johnsnowlabs/nlp/util/regex/RuleFactory.scala +++ b/src/main/scala/com/johnsnowlabs/nlp/util/regex/RuleFactory.scala @@ -17,6 +17,7 @@ package com.johnsnowlabs.nlp.util.regex import com.johnsnowlabs.nlp.annotators.sbd.pragmatic.RuleSymbols +import com.johnsnowlabs.nlp.util.io.MatchStrategy import scala.util.matching.Regex @@ -27,7 +28,7 @@ import scala.util.matching.Regex * How to decide when replacing or transforming content with Regex */ class RuleFactory( - matchStrategy: MatchStrategy.MatchStrategy, + matchStrategy: MatchStrategy.Format, transformStrategy: TransformStrategy.TransformStrategy = TransformStrategy.NO_TRANSFORM) extends RuleSymbols with Serializable { @@ -69,7 +70,9 @@ class RuleFactory( matchStrategy match { case MATCH_ALL => rules.flatMap(rule => - rule.regex.findAllMatchIn(text).map(m => RuleMatch(m, rule.identifier))) + rule.regex.findAllMatchIn(text).zipWithIndex.map { case (currentMatch, index) => + RuleMatch(currentMatch, rule.identifier, index) + }) case MATCH_FIRST => rules.flatMap(rule => rule.regex.findFirstMatchIn(text).map(m => RuleMatch(m, rule.identifier))) @@ -224,7 +227,7 @@ object RuleFactory { /** Specific partial constructor for [[RuleFactory]] where MatchStrategy might change on runtime */ def lateMatching(transformStrategy: TransformStrategy.TransformStrategy)( - matchStrategy: MatchStrategy.MatchStrategy): RuleFactory = + matchStrategy: MatchStrategy.Format): RuleFactory = new RuleFactory(matchStrategy, transformStrategy) /** Internal representation of a regex match @@ -234,7 +237,7 @@ object RuleFactory { * @param identifier * user provided identification of a rule */ - case class RuleMatch(content: Regex.Match, identifier: String) + case class RuleMatch(content: Regex.Match, identifier: String, indexMatch: Int = -1) } /** Allowed strategies for [[RuleFactory]] applications regarding replacement */ @@ -244,9 +247,3 @@ object TransformStrategy extends Enumeration { REPLACE_WITH_SYMBOL_AND_BREAK, PROTECT_FROM_BREAK, BREAK_AND_PROTECT_FROM_BREAK, REPLACE_EACH_WITH_SYMBOL, REPLACE_EACH_WITH_SYMBOL_AND_BREAK = Value } - -/** Allowed strategies for [[RuleFactory]] applications regarding matching */ -object MatchStrategy extends Enumeration { - type MatchStrategy = Value - val MATCH_ALL, MATCH_FIRST, MATCH_COMPLETE = Value -} diff --git a/src/test/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcherTestSpec.scala b/src/test/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcherTestSpec.scala index a470090af5da0c..2768c0ebab76a6 100644 --- a/src/test/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcherTestSpec.scala +++ b/src/test/scala/com/johnsnowlabs/nlp/annotators/MultiDateMatcherTestSpec.scala @@ -17,6 +17,7 @@ package com.johnsnowlabs.nlp.annotators import com.johnsnowlabs.nlp.AnnotatorType.DATE +import com.johnsnowlabs.nlp.util.io.MatchStrategy import com.johnsnowlabs.nlp.{Annotation, AnnotatorType, DataBuilder} import com.johnsnowlabs.tags.FastTest import org.apache.spark.sql.{Dataset, Row} @@ -307,4 +308,31 @@ class MultiDateMatcherTestSpec extends AnyFlatSpec with DateMatcherBehaviors { assert(results == expectedDates) } + + "a MultiDateMatcher" should "correctly find all possible dates in a text" taggedAs FastTest in { + + val data: Dataset[Row] = DataBuilder.multipleDataBuild(Array(""" + Lease Period Monthly Installment of Base Rent + January 1, 2021 –December 31, 2021 $20,304.85* + January 1, 2022 –December 31, 2022 $20,914.00 + """)) + + val dateMatcher = new MultiDateMatcher() + .setInputCols(Array("document")) + .setOutputCol("date") + .setOutputFormat("yyyy/MM/dd") + .setRelaxedFactoryStrategy(MatchStrategy.MATCH_ALL) + .transform(data) + + val results = Annotation.collect(dateMatcher, "date").flatten.toSeq.sortBy(_.end) + + val expectedDates = Seq( + Annotation(DATE, 67, 81, "2021/01/01", Map("sentence" -> "0")), + Annotation(DATE, 84, 100, "2021/12/31", Map("sentence" -> "0")), + Annotation(DATE, 103, 138, "2022/01/20", Map("sentence" -> "0")), + Annotation(DATE, 132, 157, "2022/12/01", Map("sentence" -> "0"))) + + assert(results == expectedDates) + } + }