Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Olivier Girardot
Yes thanks !

Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

 The image didn't go through.

 I think you were referring to:
   override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)

 Cheers

 On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

  Hi everyone,
  I had an issue trying to use Spark SQL from Java (8 or 7), I tried to
  reproduce it in a small test case close to the actual documentation
  
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
 ,
  so sorry for the long mail, but this is Java :
 
  import org.apache.spark.api.java.JavaRDD;
  import org.apache.spark.api.java.JavaSparkContext;
  import org.apache.spark.sql.DataFrame;
  import org.apache.spark.sql.SQLContext;
 
  import java.io.Serializable;
  import java.util.ArrayList;
  import java.util.Arrays;
  import java.util.List;
 
  class Movie implements Serializable {
  private int id;
  private String name;
 
  public Movie(int id, String name) {
  this.id = id;
  this.name = name;
  }
 
  public int getId() {
  return id;
  }
 
  public void setId(int id) {
  this.id = id;
  }
 
  public String getName() {
  return name;
  }
 
  public void setName(String name) {
  this.name = name;
  }
  }
 
  public class SparkSQLTest {
  public static void main(String[] args) {
  SparkConf conf = new SparkConf();
  conf.setAppName(My Application);
  conf.setMaster(local);
  JavaSparkContext sc = new JavaSparkContext(conf);
 
  ArrayListMovie movieArrayList = new ArrayListMovie();
  movieArrayList.add(new Movie(1, Indiana Jones));
 
  JavaRDDMovie movies = sc.parallelize(movieArrayList);
 
  SQLContext sqlContext = new SQLContext(sc);
  DataFrame frame = sqlContext.applySchema(movies, Movie.class);
  frame.registerTempTable(movies);
 
  sqlContext.sql(select name from movies)
 
  *.map(row - row.getString(0)) // this is what i would
 expect to work *.collect();
  }
  }
 
 
  But this does not compile, here's the compilation error :
 
  [ERROR]
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
  method map in class org.apache.spark.sql.DataFrame cannot be applied to
  given types;
  [ERROR] *required:
  scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
  [ERROR]* found: (row)-Na[...]ng(0) *
  [ERROR] *reason: cannot infer type-variable(s) R *
  [ERROR] *(actual and formal argument lists differ in length) *
  [ERROR]
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
  method map in class org.apache.spark.sql.DataFrame cannot be applied to
  given types;
  [ERROR] required:
  scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
  [ERROR] found: (row)-row[...]ng(0)
  [ERROR] reason: cannot infer type-variable(s) R
  [ERROR] (actual and formal argument lists differ in length)
  [ERROR] - [Help 1]
 
  Because in the DataFrame the *map *method is defined as :
 
  [image: Images intégrées 1]
 
  And once this is translated to bytecode the actual Java signature uses a
  Function1 and adds a ClassTag parameter.
  I can try to go around this and use the scala.reflect.ClassTag$ like
 that :
 
  ClassTag$.MODULE$.apply(String.class)
 
  To get the second ClassTag parameter right, but then instantiating a
 java.util.Function or using the Java 8 lambdas fail to work, and if I try
 to instantiate a proper scala Function1... well this is a world of pain.
 
  This is a regression introduced by the 1.3.x DataFrame because
 JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and are not
 callable with JFunctions), I can open a Jira if you want ?
 
  Regards,
 
  --
  *Olivier Girardot* | Associé
  o.girar...@lateral-thoughts.com
  +33 6 24 09 17 94
 



Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Ted Yu
The image didn't go through.

I think you were referring to:
  override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)

Cheers

On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
o.girar...@lateral-thoughts.com wrote:

 Hi everyone,
 I had an issue trying to use Spark SQL from Java (8 or 7), I tried to
 reproduce it in a small test case close to the actual documentation
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection,
 so sorry for the long mail, but this is Java :

 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.SQLContext;

 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;

 class Movie implements Serializable {
 private int id;
 private String name;

 public Movie(int id, String name) {
 this.id = id;
 this.name = name;
 }

 public int getId() {
 return id;
 }

 public void setId(int id) {
 this.id = id;
 }

 public String getName() {
 return name;
 }

 public void setName(String name) {
 this.name = name;
 }
 }

 public class SparkSQLTest {
 public static void main(String[] args) {
 SparkConf conf = new SparkConf();
 conf.setAppName(My Application);
 conf.setMaster(local);
 JavaSparkContext sc = new JavaSparkContext(conf);

 ArrayListMovie movieArrayList = new ArrayListMovie();
 movieArrayList.add(new Movie(1, Indiana Jones));

 JavaRDDMovie movies = sc.parallelize(movieArrayList);

 SQLContext sqlContext = new SQLContext(sc);
 DataFrame frame = sqlContext.applySchema(movies, Movie.class);
 frame.registerTempTable(movies);

 sqlContext.sql(select name from movies)

 *.map(row - row.getString(0)) // this is what i would expect 
 to work *.collect();
 }
 }


 But this does not compile, here's the compilation error :

 [ERROR]
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
 method map in class org.apache.spark.sql.DataFrame cannot be applied to
 given types;
 [ERROR] *required:
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
 [ERROR]* found: (row)-Na[...]ng(0) *
 [ERROR] *reason: cannot infer type-variable(s) R *
 [ERROR] *(actual and formal argument lists differ in length) *
 [ERROR]
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
 method map in class org.apache.spark.sql.DataFrame cannot be applied to
 given types;
 [ERROR] required:
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
 [ERROR] found: (row)-row[...]ng(0)
 [ERROR] reason: cannot infer type-variable(s) R
 [ERROR] (actual and formal argument lists differ in length)
 [ERROR] - [Help 1]

 Because in the DataFrame the *map *method is defined as :

 [image: Images intégrées 1]

 And once this is translated to bytecode the actual Java signature uses a
 Function1 and adds a ClassTag parameter.
 I can try to go around this and use the scala.reflect.ClassTag$ like that :

 ClassTag$.MODULE$.apply(String.class)

 To get the second ClassTag parameter right, but then instantiating a 
 java.util.Function or using the Java 8 lambdas fail to work, and if I try to 
 instantiate a proper scala Function1... well this is a world of pain.

 This is a regression introduced by the 1.3.x DataFrame because JavaSchemaRDD 
 used to be JavaRDDLike but DataFrame's are not (and are not callable with 
 JFunctions), I can open a Jira if you want ?

 Regards,

 --
 *Olivier Girardot* | Associé
 o.girar...@lateral-thoughts.com
 +33 6 24 09 17 94



Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Reynold Xin
I think in 1.3 and above, you'd need to do

.sql(...).javaRDD().map(..)

On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I tried to
   reproduce it in a small test case close to the actual documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies, Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be applied to
   given types;
   [ERROR] *required:
   scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be applied to
   given types;
   [ERROR] required:
   scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature uses
 a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$ like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then instantiating a
  java.util.Function or using the Java 8 lambdas fail to work, and if I try
  to instantiate a proper scala Function1... well this is a world of pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and are not
  callable with JFunctions), I can open a Jira if you want ?
  
   Regards,
  
   --
   *Olivier Girardot* | Associé
   o.girar...@lateral-thoughts.com
   +33 6 24 09 17 94
  
 



Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Olivier Girardot
Ok, do you want me to open a pull request to fix the dedicated
documentation ?

Le ven. 17 avr. 2015 à 18:14, Reynold Xin r...@databricks.com a écrit :

 I think in 1.3 and above, you'd need to do

 .sql(...).javaRDD().map(..)

 On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I tried to
   reproduce it in a small test case close to the actual documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies, Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be applied
 to
   given types;
   [ERROR] *required:
   scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
 *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be applied
 to
   given types;
   [ERROR] required:
   scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature
 uses a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$ like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then instantiating a
  java.util.Function or using the Java 8 lambdas fail to work, and if I
 try
  to instantiate a proper scala Function1... well this is a world of pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and are
 not
  callable with JFunctions), I can open a Jira if you want ?
  
   Regards,
  
   --
   *Olivier Girardot* | Associé
   o.girar...@lateral-thoughts.com
   +33 6 24 09 17 94
  
 





Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Reynold Xin
Please do! Thanks.


On Fri, Apr 17, 2015 at 2:36 PM, Olivier Girardot 
o.girar...@lateral-thoughts.com wrote:

 Ok, do you want me to open a pull request to fix the dedicated
 documentation ?

 Le ven. 17 avr. 2015 à 18:14, Reynold Xin r...@databricks.com a écrit :

 I think in 1.3 and above, you'd need to do

 .sql(...).javaRDD().map(..)

 On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I tried to
   reproduce it in a small test case close to the actual documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies,
 Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i
 would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be applied
 to
   given types;
   [ERROR] *required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be applied
 to
   given types;
   [ERROR] required:
   scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature
 uses a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$ like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then instantiating a
  java.util.Function or using the Java 8 lambdas fail to work, and if I
 try
  to instantiate a proper scala Function1... well this is a world of
 pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and are
 not
  callable with JFunctions), I can open a Jira if you want ?
  
   Regards,
  
   --
   *Olivier Girardot* | Associé
   o.girar...@lateral-thoughts.com
   +33 6 24 09 17 94
  
 





Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Reynold Xin
No there isn't a convention. Although if you want to show java 8, you
should also show java 6/7 syntax since there are still more 7 users than 8.


On Fri, Apr 17, 2015 at 3:36 PM, Olivier Girardot 
o.girar...@lateral-thoughts.com wrote:

 Is there any convention *not* to show java 8 versions in the documentation
 ?

 Le ven. 17 avr. 2015 à 21:39, Reynold Xin r...@databricks.com a écrit :

 Please do! Thanks.


 On Fri, Apr 17, 2015 at 2:36 PM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Ok, do you want me to open a pull request to fix the dedicated
 documentation ?

 Le ven. 17 avr. 2015 à 18:14, Reynold Xin r...@databricks.com a
 écrit :

 I think in 1.3 and above, you'd need to do

 .sql(...).javaRDD().map(..)

 On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I tried
 to
   reproduce it in a small test case close to the actual documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies,
 Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i
 would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] *required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature
 uses a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$
 like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then instantiating
 a
  java.util.Function or using the Java 8 lambdas fail to work, and if
 I try
  to instantiate a proper scala Function1... well this is a world of
 pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and
 are not
  callable with JFunctions), I can open a Jira if you want ?
  
   Regards,
  
   --
   *Olivier Girardot* | Associé
   o.girar...@lateral-thoughts.com
   

Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Olivier Girardot
another PR I guess :) here's the associated Jira
https://issues.apache.org/jira/browse/SPARK-6988

Le ven. 17 avr. 2015 à 23:00, Reynold Xin r...@databricks.com a écrit :

 No there isn't a convention. Although if you want to show java 8, you
 should also show java 6/7 syntax since there are still more 7 users than 8.


 On Fri, Apr 17, 2015 at 3:36 PM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Is there any convention *not* to show java 8 versions in the
 documentation ?

 Le ven. 17 avr. 2015 à 21:39, Reynold Xin r...@databricks.com a écrit :

 Please do! Thanks.


 On Fri, Apr 17, 2015 at 2:36 PM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Ok, do you want me to open a pull request to fix the dedicated
 documentation ?

 Le ven. 17 avr. 2015 à 18:14, Reynold Xin r...@databricks.com a
 écrit :

 I think in 1.3 and above, you'd need to do

 .sql(...).javaRDD().map(..)

 On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I
 tried to
   reproduce it in a small test case close to the actual
 documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies,
 Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i
 would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] *required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature
 uses a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$
 like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then
 instantiating a
  java.util.Function or using the Java 8 lambdas fail to work, and if
 I try
  to instantiate a proper scala Function1... well this is a world of
 pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's 

Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Olivier Girardot
and the PR: https://github.com/apache/spark/pull/5564

Thank you !

Olivier.

Le ven. 17 avr. 2015 à 23:00, Reynold Xin r...@databricks.com a écrit :

 No there isn't a convention. Although if you want to show java 8, you
 should also show java 6/7 syntax since there are still more 7 users than 8.


 On Fri, Apr 17, 2015 at 3:36 PM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Is there any convention *not* to show java 8 versions in the
 documentation ?

 Le ven. 17 avr. 2015 à 21:39, Reynold Xin r...@databricks.com a écrit :

 Please do! Thanks.


 On Fri, Apr 17, 2015 at 2:36 PM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Ok, do you want me to open a pull request to fix the dedicated
 documentation ?

 Le ven. 17 avr. 2015 à 18:14, Reynold Xin r...@databricks.com a
 écrit :

 I think in 1.3 and above, you'd need to do

 .sql(...).javaRDD().map(..)

 On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I
 tried to
   reproduce it in a small test case close to the actual
 documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies,
 Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i
 would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] *required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature
 uses a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$
 like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then
 instantiating a
  java.util.Function or using the Java 8 lambdas fail to work, and if
 I try
  to instantiate a proper scala Function1... well this is a world of
 pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and
 are 

Re: [Spark SQL] Java map/flatMap api broken with DataFrame in 1.3.{0,1}

2015-04-17 Thread Olivier Girardot
Is there any convention *not* to show java 8 versions in the documentation ?

Le ven. 17 avr. 2015 à 21:39, Reynold Xin r...@databricks.com a écrit :

 Please do! Thanks.


 On Fri, Apr 17, 2015 at 2:36 PM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Ok, do you want me to open a pull request to fix the dedicated
 documentation ?

 Le ven. 17 avr. 2015 à 18:14, Reynold Xin r...@databricks.com a écrit :

 I think in 1.3 and above, you'd need to do

 .sql(...).javaRDD().map(..)

 On Fri, Apr 17, 2015 at 9:22 AM, Olivier Girardot 
 o.girar...@lateral-thoughts.com wrote:

 Yes thanks !

 Le ven. 17 avr. 2015 à 16:20, Ted Yu yuzhih...@gmail.com a écrit :

  The image didn't go through.
 
  I think you were referring to:
override def map[R: ClassTag](f: Row = R): RDD[R] = rdd.map(f)
 
  Cheers
 
  On Fri, Apr 17, 2015 at 6:07 AM, Olivier Girardot 
  o.girar...@lateral-thoughts.com wrote:
 
   Hi everyone,
   I had an issue trying to use Spark SQL from Java (8 or 7), I tried
 to
   reproduce it in a small test case close to the actual documentation
   
 
 https://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection
  ,
   so sorry for the long mail, but this is Java :
  
   import org.apache.spark.api.java.JavaRDD;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.SQLContext;
  
   import java.io.Serializable;
   import java.util.ArrayList;
   import java.util.Arrays;
   import java.util.List;
  
   class Movie implements Serializable {
   private int id;
   private String name;
  
   public Movie(int id, String name) {
   this.id = id;
   this.name = name;
   }
  
   public int getId() {
   return id;
   }
  
   public void setId(int id) {
   this.id = id;
   }
  
   public String getName() {
   return name;
   }
  
   public void setName(String name) {
   this.name = name;
   }
   }
  
   public class SparkSQLTest {
   public static void main(String[] args) {
   SparkConf conf = new SparkConf();
   conf.setAppName(My Application);
   conf.setMaster(local);
   JavaSparkContext sc = new JavaSparkContext(conf);
  
   ArrayListMovie movieArrayList = new ArrayListMovie();
   movieArrayList.add(new Movie(1, Indiana Jones));
  
   JavaRDDMovie movies = sc.parallelize(movieArrayList);
  
   SQLContext sqlContext = new SQLContext(sc);
   DataFrame frame = sqlContext.applySchema(movies,
 Movie.class);
   frame.registerTempTable(movies);
  
   sqlContext.sql(select name from movies)
  
   *.map(row - row.getString(0)) // this is what i
 would
  expect to work *.collect();
   }
   }
  
  
   But this does not compile, here's the compilation error :
  
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/MainSQL.java:[37,47]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] *required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR *
   [ERROR]* found: (row)-Na[...]ng(0) *
   [ERROR] *reason: cannot infer type-variable(s) R *
   [ERROR] *(actual and formal argument lists differ in length) *
   [ERROR]
  
 
 /Users/ogirardot/Documents/spark/java-project/src/main/java/org/apache/spark/SampleSHit.java:[56,17]
   method map in class org.apache.spark.sql.DataFrame cannot be
 applied to
   given types;
   [ERROR] required:
  
 scala.Function1org.apache.spark.sql.Row,R,scala.reflect.ClassTagR
   [ERROR] found: (row)-row[...]ng(0)
   [ERROR] reason: cannot infer type-variable(s) R
   [ERROR] (actual and formal argument lists differ in length)
   [ERROR] - [Help 1]
  
   Because in the DataFrame the *map *method is defined as :
  
   [image: Images intégrées 1]
  
   And once this is translated to bytecode the actual Java signature
 uses a
   Function1 and adds a ClassTag parameter.
   I can try to go around this and use the scala.reflect.ClassTag$ like
  that :
  
   ClassTag$.MODULE$.apply(String.class)
  
   To get the second ClassTag parameter right, but then instantiating a
  java.util.Function or using the Java 8 lambdas fail to work, and if I
 try
  to instantiate a proper scala Function1... well this is a world of
 pain.
  
   This is a regression introduced by the 1.3.x DataFrame because
  JavaSchemaRDD used to be JavaRDDLike but DataFrame's are not (and are
 not
  callable with JFunctions), I can open a Jira if you want ?
  
   Regards,
  
   --
   *Olivier Girardot* | Associé
   o.girar...@lateral-thoughts.com
   +33 6 24 09 17 94