diff options
author | Cheng Lian <lian.cs.zju@gmail.com> | 2014-06-16 21:30:29 +0200 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2014-06-16 21:32:51 +0200 |
commit | 273afcb254fb5384204c56bdcb3b9b760bcfab3f (patch) | |
tree | 5ac5382b2df7d612d0c4f93a2799facda54b6273 /sql/core/src | |
parent | cdf2b04570871848442ca9f9e2316a37e4aaaae0 (diff) | |
download | spark-273afcb254fb5384204c56bdcb3b9b760bcfab3f.tar.gz spark-273afcb254fb5384204c56bdcb3b9b760bcfab3f.tar.bz2 spark-273afcb254fb5384204c56bdcb3b9b760bcfab3f.zip |
[SQL][SPARK-2094] Follow up of PR #1071 for Java API
Updated `JavaSQLContext` and `JavaHiveContext` similar to what we've done to `SQLContext` and `HiveContext` in PR #1071. Added corresponding test case for Spark SQL Java API.
Author: Cheng Lian <lian.cs.zju@gmail.com>
Closes #1085 from liancheng/spark-2094-java and squashes the following commits:
29b8a51 [Cheng Lian] Avoided instantiating JavaSparkContext & JavaHiveContext to workaround test failure
92bb4fb [Cheng Lian] Marked test cases in JavaHiveQLSuite with "ignore"
22aec97 [Cheng Lian] Follow up of PR #1071 for Java API
Diffstat (limited to 'sql/core/src')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala | 16 |
1 files changed, 4 insertions, 12 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 6f7d431b9a..352260fa15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -40,19 +40,13 @@ class JavaSQLContext(val sqlContext: SQLContext) { /** * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD */ - def sql(sqlQuery: String): JavaSchemaRDD = { - val result = new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery)) - // We force query optimization to happen right away instead of letting it happen lazily like - // when using the query DSL. This is so DDL commands behave as expected. This is only - // generates the RDD lineage for DML queries, but do not perform any execution. - result.queryExecution.toRdd - result - } + def sql(sqlQuery: String): JavaSchemaRDD = + new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery)) /** * :: Experimental :: * Creates an empty parquet file with the schema of class `beanClass`, which can be registered as - * a table. This registered table can be used as the target of future insertInto` operations. + * a table. This registered table can be used as the target of future `insertInto` operations. * * {{{ * JavaSQLContext sqlCtx = new JavaSQLContext(...) @@ -62,7 +56,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { * }}} * * @param beanClass A java bean class object that will be used to determine the schema of the - * parquet file. s + * parquet file. * @param path The path where the directory containing parquet metadata should be created. * Data inserted into this table will also be stored at this location. * @param allowExisting When false, an exception will be thrown if this directory already exists. @@ -100,14 +94,12 @@ class JavaSQLContext(val sqlContext: SQLContext) { new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) } - /** * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. */ def parquetFile(path: String): JavaSchemaRDD = new JavaSchemaRDD(sqlContext, ParquetRelation(path)) - /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only * during the lifetime of this instance of SQLContext. |