At the Spark Summit, the Typesafe people had a toy implementation of a 
full-text index that you could use as a starting point. The bare code is 
available in github at 
https://github.com/deanwampler/spark-workshop/blob/eb077a734aad166235de85494def8fe3d4d2ca66/src/main/scala/spark/InvertedIndex5b.scala.
 You can get a nice presentation of it through Typesafe's "Activator" - see 
http://typesafe.com/activator/template/spark-workshop for more on that.

Please keep the list informed on how things go - I'd like this functionality as 
well.

Ron

> -----Original Message-----
> From: Philip Ogren [mailto:philip.og...@oracle.com]
> Sent: Monday, August 04, 2014 11:08 AM
> To: user@spark.apache.org
> Subject: Re: creating a distributed index
> 
> After playing around with mapPartition I think this does exactly what I want.
> I can pass in a function to mapPartition that looks like this:
> 
>          def f1(iter: Iterator[String]): Iterator[MyIndex] = {
>              val idx: MyIndex = new MyIndex()
>              while (iter.hasNext) {
>                  val text: String = iter.next;
>                  idx.add(text)
>              }
>              List(idx).iterator
>          }
> 
>          val indexRDD: RDD[MyIndex] = rdd.mapPartitions(f1, true)
> 
> Now I can perform a match operation like so:
> 
> val matchRdd: RDD[Match] = indexRDD.flatMap(index =>
> index.`match`(myquery))
> 
> I'm sure it won't take much imagination to figure out how to the the
> matching in a batch way.
> 
> If anyone has done anything along these lines I'd love to have some
> feedback.
> 
> Thanks,
> Philip
> 
> 
> 
> On 08/04/2014 09:46 AM, Philip Ogren wrote:
> > This looks like a really cool feature and it seems likely that this
> > will be extremely useful for things we are doing.  However, I'm not
> > sure it is quite what I need here.  With an inverted index you don't
> > actually look items up by their keys but instead try to match against
> > some input string.  So, if I created an inverted index on 1M
> > strings/documents in a single JVM I could subsequently submit a
> > string-valued query to retrieve the n-best matches.  I'd like to do
> > something like build ten 1M string/document indexes across ten nodes,
> > submit a string-valued query to each of the ten indexes and aggregate
> > the n-best matches from the ten sets of results.  Would this be
> > possible with IndexedRDD or some other feature of Spark?
> >
> > Thanks,
> > Philip
> >
> >
> > On 08/01/2014 04:26 PM, Ankur Dave wrote:
> >> At 2014-08-01 14:50:22 -0600, Philip Ogren <philip.og...@oracle.com>
> >> wrote:
> >>> It seems that I could do this with mapPartition so that each element
> >>> in a partition gets added to an index for that partition.
> >>> [...]
> >>> Would it then be possible to take a string and query each
> >>> partition's index with it? Or better yet, take a batch of strings
> >>> and query each string in the batch against each partition's index?
> >> I proposed a key-value store based on RDDs called IndexedRDD that
> >> does exactly what you described. It uses mapPartitions to construct
> >> an index within each partition, then exposes get and multiget methods
> >> to allow looking up values associated with given keys.
> >>
> >> It will hopefully make it into Spark 1.2.0. Until then you can try it
> >> out by merging in the pull request locally:
> >> https://github.com/apache/spark/pull/1297.
> >>
> >> See JIRA for details and slides on how it works:
> >> https://issues.apache.org/jira/browse/SPARK-2365.
> >>
> >> Ankur
> >
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: user-unsubscr...@spark.apache.org For
> > additional commands, e-mail: user-h...@spark.apache.org
> >
> 
> 
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org For additional
> commands, e-mail: user-h...@spark.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org

Reply via email to