+1 to using a UDF. E.g., TransmogrifAI uses <https://github.com/salesforce/TransmogrifAI/blob/ef6d3267cf4379a0805d6add400d7b0e328e4aa1/core/src/main/scala/com/salesforce/op/stages/impl/feature/PhoneNumberParser.scala#L274> libphonenumber https://github.com/google/libphonenumber that normalizes phone numbers to a tuple (country code, national number), so you can find more sophisticated matches for phones written in different notations.
If you simplify it for DataFrame/SQL-only use, you can create a Scala UDF: $SPARK_HOME/bin/spark-shell --packages com.googlecode.libphonenumber:libphonenumber:8.13.9 scala> :paste // Entering paste mode (ctrl-D to finish) import com.google.i18n.phonenumbers._ import scala.collection.JavaConverters._ val extractPhonesUDF = udf((x: String) => PhoneNumberUtil.getInstance() .findNumbers(x, "US").asScala.toSeq .map(x => (x.number.getCountryCode, x.number.getNationalNumber))) spark.udf.register("EXTRACT_PHONES", extractPhonesUDF) sql(""" SELECT EXTRACT_PHONES('+496811234567,+1(415)7654321') AS needles, EXTRACT_PHONES('Call our HQ in Germany at (+49) 0681/1234567, in Paris at : +33 01 12 34 56 78, or the SF office at 415-765-4321') AS haystack, ARRAY_INTERSECT(needles, haystack) AS needles_in_haystack """).show(truncate=false) // Exiting paste mode, now interpreting. +-----------------------------------+----------------------------------------------------+-----------------------------------+ |needles |haystack |needles_in_haystack | +-----------------------------------+----------------------------------------------------+-----------------------------------+ |[{49, 6811234567}, {1, 4157654321}]|[{49, 6811234567}, {33, 112345678}, {1, 4157654321}]|[{49, 6811234567}, {1, 4157654321}]| +-----------------------------------+----------------------------------------------------+-----------------------------------+ On Sun, Apr 2, 2023 at 7:18 AM Sean Owen <sro...@gmail.com> wrote: > That won't work, you can't use Spark within Spark like that. > If it were exact matches, the best solution would be to load both datasets > and join on telephone number. > For this case, I think your best bet is a UDF that contains the telephone > numbers as a list and decides whether a given number matches something in > the set. Then use that to filter, then work with the data set. > There are probably clever fast ways of efficiently determining if a string > is a prefix of a group of strings in Python you could use too. > > On Sun, Apr 2, 2023 at 3:17 AM Philippe de Rochambeau <phi...@free.fr> > wrote: > >> Many thanks, Mich. >> Is « foreach » the best construct to lookup items is a dataset such as >> the below « telephonedirectory » data set? >> >> val telrdd = spark.sparkContext.parallelize(Seq(« tel1 » , « tel2 » , « >> tel3 » …)) // the telephone sequence >> >> // was read for a CSV file >> >> val ds = spark.read.parquet(« /path/to/telephonedirectory » ) >> >> rdd .foreach(tel => { >> longAcc.select(« * » ).rlike(« + » + tel) >> }) >> >> >> >> >> Le 1 avr. 2023 à 22:36, Mich Talebzadeh <mich.talebza...@gmail.com> a >> écrit : >> >> This may help >> >> Spark rlike() Working with Regex Matching Example >> <https://sparkbyexamples.com/spark/spark-rlike-regex-matching-examples/>s >> Mich Talebzadeh, >> Lead Solutions Architect/Engineering Lead >> Palantir Technologies Limited >> >> view my Linkedin profile >> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/> >> >> >> https://en.everybodywiki.com/Mich_Talebzadeh >> >> >> *Disclaimer:* Use it at your own risk. Any and all responsibility for >> any loss, damage or destruction of data or any other property which may >> arise from relying on this email's technical content is explicitly >> disclaimed. The author will in no case be liable for any monetary damages >> arising from such loss, damage or destruction. >> >> >> >> >> On Sat, 1 Apr 2023 at 19:32, Philippe de Rochambeau <phi...@free.fr> >> wrote: >> >>> Hello, >>> I’m looking for an efficient way in Spark to search for a series of >>> telephone numbers, contained in a CSV file, in a data set column. >>> >>> In pseudo code, >>> >>> for tel in [tel1, tel2, …. tel40,000] >>> search for tel in dataset using .like(« %tel% ») >>> end for >>> >>> I’m using the like function because the telephone numbers in the data >>> set main contain prefixes, such as « + « ; e.g., « +3312224444 ». >>> >>> Any suggestions would be welcome. >>> >>> Many thanks. >>> >>> Philippe >>> >>> >>> >>> >>> >>> --------------------------------------------------------------------- >>> To unsubscribe e-mail: user-unsubscr...@spark.apache.org >>> >>> >>