Re: How to sort rdd filled with existing data structures?

2014-09-24 Thread Liquan Pei
You only need to define an ordering of student, no need to modify the class
definition of student. It's like a Comparator class in java.

Currently, you have to map the rdd to sort by value.

Liquan

On Wed, Sep 24, 2014 at 9:52 AM, Sean Owen  wrote:

> See the scaladoc for how to define an implicit ordering to use with
> sortByKey:
>
>
> http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.OrderedRDDFunctions
>
> Off the top of my head, I think this is 90% correct to order by age for
> example:
>
> implicit val studentOrdering: Ordering[Student] = Ordering.by(_.age)
>
> On Wed, Sep 24, 2014 at 3:07 PM, Tao Xiao 
> wrote:
> > Hi ,
> >
> > I have the following rdd :
> >
> >   val conf = new SparkConf()
> >   .setAppName("<< Testing Sorting >>")
> >   val sc = new SparkContext(conf)
> >
> >   val L = List(
> >   (new Student("XiaoTao", 80, 29), "I'm Xiaotao"),
> >   (new Student("CCC", 100, 24), "I'm CCC"),
> >   (new Student("Jack", 90, 25), "I'm Jack"),
> >   (new Student("Tom", 60, 35), "I'm Tom"),
> >   (new Student("Lucy", 78, 22), "I'm Lucy"))
> >
> >   val rdd = sc.parallelize(L, 3)
> >
> >
> > where Student is a class defined as follows:
> >
> > class Student(val name:String, val score:Int, val age:Int)  {
> >
> >  override def toString =
> >  "name:" + name + ", score:" + score + ", age:" + age
> >
> > }
> >
> >
> >
> > I want to sort the rdd by key, but when I wrote rdd.sortByKey it
> complained
> > that "No implicit Ordering defined", which means I must extend the class
> > with Ordered and provide a method named  compare.  The problem is that
> the
> > class Student is from a third-party library so I cannot change its
> > definition. I'd like to know if there is a sorting method that I can
> provide
> > it a customized compare function so that it can sort the rdd according to
> > the sorting function I provide.
> >
> > One more question, if I want to sort RDD[(k, v)] by value , do I have to
> map
> > that rdd so that its key and value exchange their positions in the tuple?
> > Are there any functions that allow us to sort rdd by things other than
> key ?
> >
> > Thanks
> >
> >
> >
>
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
>
>


-- 
Liquan Pei
Department of Physics
University of Massachusetts Amherst


Re: How to sort rdd filled with existing data structures?

2014-09-24 Thread Sean Owen
See the scaladoc for how to define an implicit ordering to use with sortByKey:

http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.OrderedRDDFunctions

Off the top of my head, I think this is 90% correct to order by age for example:

implicit val studentOrdering: Ordering[Student] = Ordering.by(_.age)

On Wed, Sep 24, 2014 at 3:07 PM, Tao Xiao  wrote:
> Hi ,
>
> I have the following rdd :
>
>   val conf = new SparkConf()
>   .setAppName("<< Testing Sorting >>")
>   val sc = new SparkContext(conf)
>
>   val L = List(
>   (new Student("XiaoTao", 80, 29), "I'm Xiaotao"),
>   (new Student("CCC", 100, 24), "I'm CCC"),
>   (new Student("Jack", 90, 25), "I'm Jack"),
>   (new Student("Tom", 60, 35), "I'm Tom"),
>   (new Student("Lucy", 78, 22), "I'm Lucy"))
>
>   val rdd = sc.parallelize(L, 3)
>
>
> where Student is a class defined as follows:
>
> class Student(val name:String, val score:Int, val age:Int)  {
>
>  override def toString =
>  "name:" + name + ", score:" + score + ", age:" + age
>
> }
>
>
>
> I want to sort the rdd by key, but when I wrote rdd.sortByKey it complained
> that "No implicit Ordering defined", which means I must extend the class
> with Ordered and provide a method named  compare.  The problem is that the
> class Student is from a third-party library so I cannot change its
> definition. I'd like to know if there is a sorting method that I can provide
> it a customized compare function so that it can sort the rdd according to
> the sorting function I provide.
>
> One more question, if I want to sort RDD[(k, v)] by value , do I have to map
> that rdd so that its key and value exchange their positions in the tuple?
> Are there any functions that allow us to sort rdd by things other than key ?
>
> Thanks
>
>
>

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



How to sort rdd filled with existing data structures?

2014-09-24 Thread Tao Xiao
Hi ,

I have the following rdd :

  val conf = new SparkConf()
  .setAppName("<< Testing Sorting >>")
  val sc = new SparkContext(conf)

  val L = List(
  (new Student("XiaoTao", 80, 29), "I'm Xiaotao"),
  (new Student("CCC", 100, 24), "I'm CCC"),
  (new Student("Jack", 90, 25), "I'm Jack"),
  (new Student("Tom", 60, 35), "I'm Tom"),
  (new Student("Lucy", 78, 22), "I'm Lucy"))

  val rdd = sc.parallelize(L, 3)


where Student is a class defined as follows:

class Student(val name:String, val score:Int, val age:Int)  {

 override def toString =
 "name:" + name + ", score:" + score + ", age:" + age

}



I want to sort the *rdd *by key, but when I wrote rdd.sortByKey it
complained that "No implicit Ordering defined", which means I must extend
the class with *Ordered *and provide a method named  *compare*.  The
problem is that the class Student is from a third-party library so I cannot
change its definition. I'd like to know if there is a sorting method that I
can provide it a customized compare function so that it can sort the rdd
according to the sorting function I provide.

One more question, if I want to sort RDD[(k, v)] by value , do I have to
map that rdd so that its key and value exchange their positions in the
tuple? Are there any functions that allow us to sort rdd by things other
than key ?

Thanks