Skip to content

Commit 0b859c8

Browse files
committed
First draft of java API.
1 parent ada310a commit 0b859c8

File tree

6 files changed

+262
-37
lines changed

6 files changed

+262
-37
lines changed
Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,44 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.examples.sql;
19+
20+
import org.apache.spark.api.java.JavaSparkContext;
21+
import org.apache.spark.api.java.function.VoidFunction;
22+
23+
import org.apache.spark.sql.api.java.JavaSQLContext;
24+
import org.apache.spark.sql.api.java.JavaSchemaRDD;
25+
import org.apache.spark.sql.api.java.JavaRow;
26+
27+
public final class JavaSparkSQL {
28+
public static void main(String[] args) throws Exception {
29+
JavaSparkContext ctx = new JavaSparkContext("local", "JavaSparkSQL",
30+
System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkSQL.class));
31+
JavaSQLContext sqlCtx = new JavaSQLContext(ctx);
32+
33+
JavaSchemaRDD parquetFile = sqlCtx.parquetFile("pair.parquet");
34+
parquetFile.registerAsTable("parquet");
35+
36+
JavaSchemaRDD queryResult = sqlCtx.sql("SELECT * FROM parquet");
37+
queryResult.foreach(new VoidFunction<JavaRow>() {
38+
@Override
39+
public void call(JavaRow row) throws Exception {
40+
System.out.println(row.get(0) + " " + row.get(1));
41+
}
42+
});
43+
}
44+
}

sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala

Lines changed: 5 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,13 @@
1717

1818
package org.apache.spark.sql
1919

20+
import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
2021
import org.apache.spark.rdd.RDD
2122
import org.apache.spark.sql.catalyst.analysis._
2223
import org.apache.spark.sql.catalyst.expressions._
2324
import org.apache.spark.sql.catalyst.plans.logical._
2425
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
2526
import org.apache.spark.sql.catalyst.types.BooleanType
26-
import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
2727

2828
/**
2929
* <span class="badge" style="float: right; background-color: darkblue;">ALPHA COMPONENT</span>
@@ -92,23 +92,10 @@ import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
9292
*/
9393
class SchemaRDD(
9494
@transient val sqlContext: SQLContext,
95-
@transient val logicalPlan: LogicalPlan)
96-
extends RDD[Row](sqlContext.sparkContext, Nil) {
95+
@transient protected[spark] val logicalPlan: LogicalPlan)
96+
extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike {
9797

98-
/**
99-
* A lazily computed query execution workflow. All other RDD operations are passed
100-
* through to the RDD that is produced by this workflow.
101-
*
102-
* We want this to be lazy because invoking the whole query optimization pipeline can be
103-
* expensive.
104-
*/
105-
@transient
106-
protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan)
107-
108-
override def toString =
109-
s"""${super.toString}
110-
|== Query Plan ==
111-
|${queryExecution.executedPlan}""".stripMargin.trim
98+
def baseSchemaRDD = this
11299

113100
// =========================================================================================
114101
// RDD functions: Copy the interal row representation so we present immutable data to users.
@@ -312,31 +299,12 @@ class SchemaRDD(
312299
sqlContext,
313300
InsertIntoTable(UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite))
314301

315-
/**
316-
* Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that
317-
* are written out using this method can be read back in as a SchemaRDD using the ``function
318-
*
319-
* @group schema
320-
*/
321-
def saveAsParquetFile(path: String): Unit = {
322-
sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
323-
}
324-
325-
/**
326-
* Registers this RDD as a temporary table using the given name. The lifetime of this temporary
327-
* table is tied to the [[SQLContext]] that was used to create this SchemaRDD.
328-
*
329-
* @group schema
330-
*/
331-
def registerAsTable(tableName: String): Unit = {
332-
sqlContext.registerRDDAsTable(this, tableName)
333-
}
334-
335302
/**
336303
* Returns this RDD as a SchemaRDD.
337304
* @group schema
338305
*/
339306
def toSchemaRDD = this
340307

308+
/** FOR INTERNAL USE ONLY */
341309
def analyze = sqlContext.analyzer(logicalPlan)
342310
}
Lines changed: 66 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,66 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql
19+
20+
import org.apache.spark.sql.catalyst.plans.logical._
21+
22+
/**
23+
* Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java)
24+
*/
25+
trait SchemaRDDLike {
26+
@transient val sqlContext: SQLContext
27+
@transient protected[spark] val logicalPlan: LogicalPlan
28+
29+
private[sql] def baseSchemaRDD: SchemaRDD
30+
31+
/**
32+
* A lazily computed query execution workflow. All other RDD operations are passed
33+
* through to the RDD that is produced by this workflow.
34+
*
35+
* We want this to be lazy because invoking the whole query optimization pipeline can be
36+
* expensive.
37+
*/
38+
@transient
39+
protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan)
40+
41+
override def toString =
42+
s"""${super.toString}
43+
|== Query Plan ==
44+
|${queryExecution.executedPlan}""".stripMargin.trim
45+
46+
47+
/**
48+
* Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that
49+
* are written out using this method can be read back in as a SchemaRDD using the ``function
50+
*
51+
* @group schema
52+
*/
53+
def saveAsParquetFile(path: String): Unit = {
54+
sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
55+
}
56+
57+
/**
58+
* Registers this RDD as a temporary table using the given name. The lifetime of this temporary
59+
* table is tied to the [[SQLContext]] that was used to create this SchemaRDD.
60+
*
61+
* @group schema
62+
*/
63+
def registerAsTable(tableName: String): Unit = {
64+
sqlContext.registerRDDAsTable(baseSchemaRDD, tableName)
65+
}
66+
}
Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,51 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.api.java
19+
20+
import org.apache.spark.api.java.JavaSparkContext
21+
22+
import org.apache.spark.sql._
23+
24+
class JavaSQLContext(sparkContext: JavaSparkContext) {
25+
26+
val sqlContext = new SQLContext(sparkContext)
27+
28+
def sql(sqlQuery: String): JavaSchemaRDD = {
29+
val result = new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery))
30+
// We force query optimization to happen right away instead of letting it happen lazily like
31+
// when using the query DSL. This is so DDL commands behave as expected. This is only
32+
// generates the RDD lineage for DML queries, but do not perform any execution.
33+
result.queryExecution.toRdd
34+
result
35+
}
36+
37+
/**
38+
* Loads a parquet file, returning the result as a [[SchemaRDD]].
39+
*/
40+
def parquetFile(path: String): JavaSchemaRDD =
41+
new JavaSchemaRDD(sqlContext, parquet.ParquetRelation("ParquetFile", path))
42+
43+
44+
/**
45+
* Registers the given RDD as a temporary table in the catalog. Temporary tables exist only
46+
* during the lifetime of this instance of SQLContext.
47+
*/
48+
def registerRDDAsTable(rdd: JavaSchemaRDD, tableName: String): Unit = {
49+
sqlContext.registerRDDAsTable(rdd.baseSchemaRDD, tableName)
50+
}
51+
}
Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.api.java
19+
20+
import org.apache.spark.api.java.{JavaRDDLike, JavaRDD}
21+
import org.apache.spark.sql._
22+
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
23+
import org.apache.spark.rdd.RDD
24+
25+
class JavaSchemaRDD(
26+
@transient val sqlContext: SQLContext,
27+
@transient protected[spark] val logicalPlan: LogicalPlan)
28+
extends JavaRDDLike[JavaRow, JavaRDD[JavaRow]]
29+
with SchemaRDDLike {
30+
31+
private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan)
32+
33+
override val classTag = scala.reflect.classTag[JavaRow]
34+
35+
override def wrapRDD(rdd: RDD[JavaRow]): JavaRDD[JavaRow] = JavaRDD.fromRDD(rdd)
36+
37+
val rdd = baseSchemaRDD.map(new JavaRow(_))
38+
}
Lines changed: 58 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,58 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.api.java
19+
20+
import org.apache.spark.sql.catalyst.expressions.Row
21+
22+
/**
23+
* A result row from a SparkSQL query.
24+
*/
25+
class JavaRow(row: Row) {
26+
27+
def length: Int = row.length
28+
29+
def get(i: Int): Any =
30+
row(i)
31+
32+
def isNullAt(i: Int) = get(i) == null
33+
34+
def getInt(i: Int): Int =
35+
row.getInt(i)
36+
37+
def getLong(i: Int): Long =
38+
row.getLong(i)
39+
40+
def getDouble(i: Int): Double =
41+
row.getDouble(i)
42+
43+
def getBoolean(i: Int): Boolean =
44+
row.getBoolean(i)
45+
46+
def getShort(i: Int): Short =
47+
row.getShort(i)
48+
49+
def getByte(i: Int): Byte =
50+
row.getByte(i)
51+
52+
def getFloat(i: Int): Float =
53+
row.getFloat(i)
54+
55+
def getString(i: Int): String =
56+
row.getString(i)
57+
}
58+

0 commit comments

Comments
 (0)