Skip to content

Commit be5d5c1

Browse files
cafreemanDavies Liu
authored andcommitted
refactor schema functions
Refactored `structType` and `structField` so that they can be used to create schemas from R for use with `createDataFrame`. Moved everything to `schema.R` Added new methods to `SQLUtils.scala` for handling `StructType` and `StructField` on the JVM side
1 parent 40338a4 commit be5d5c1

File tree

3 files changed

+197
-100
lines changed

3 files changed

+197
-100
lines changed

R/pkg/R/SQLContext.R

Lines changed: 0 additions & 97 deletions
Original file line numberDiff line numberDiff line change
@@ -65,30 +65,6 @@ infer_type <- function(x) {
6565
}
6666
}
6767

68-
#' dump the schema into JSON string
69-
tojson <- function(x) {
70-
if (inherits(x, "struct")) {
71-
# schema object
72-
l <- paste(lapply(x, tojson), collapse = ", ")
73-
paste('{\"type\":\"struct\", \"fields\":','[', l, ']}', sep = '')
74-
} else if (inherits(x, "field")) {
75-
# field object
76-
names <- names(x)
77-
items <- lapply(names, function(n) {
78-
safe_n <- gsub('"', '\\"', n)
79-
paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '')
80-
})
81-
d <- paste(items, collapse = ", ")
82-
paste('{', d, '}', sep = '')
83-
} else if (is.character(x)) {
84-
paste('"', x, '"', sep = '')
85-
} else if (is.logical(x)) {
86-
if (x) "true" else "false"
87-
} else {
88-
stop(paste("unexpected type:", class(x)))
89-
}
90-
}
91-
9268
#' Create a DataFrame from an RDD
9369
#'
9470
#' Converts an RDD to a DataFrame by infer the types.
@@ -516,76 +492,3 @@ createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, .
516492
sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options)
517493
dataFrame(sdf)
518494
}
519-
520-
#' Create a Schema object
521-
#'
522-
#' Create an object of type "struct" that contains the metadata for a DataFrame. Intended for
523-
#' use with createDataFrame and toDF.
524-
#'
525-
#' @param field a Field object (created with the field() function)
526-
#' @param ... additional Field objects
527-
#' @return a Schema object
528-
#' @export
529-
#' @examples
530-
#'\dontrun{
531-
#' sc <- sparkR.init()
532-
#' sqlCtx <- sparkRSQL.init(sc)
533-
#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) })
534-
#' schema <- buildSchema(field("a", "integer"), field("b", "string"))
535-
#' df <- createDataFrame(sqlCtx, rdd, schema)
536-
#' }
537-
buildSchema <- function(field, ...) {
538-
fields <- list(field, ...)
539-
if (!all(sapply(fields, inherits, "field"))) {
540-
stop("All arguments must be Field objects.")
541-
}
542-
543-
structure(fields, class = "struct")
544-
}
545-
546-
# print method for "struct" object
547-
print.struct <- function(x, ...) {
548-
cat(sapply(x, function(field) { paste("|-", "name = \"", field$name,
549-
"\", type = \"", field$type,
550-
"\", nullable = ", field$nullable, "\n",
551-
sep = "") })
552-
, sep = "")
553-
}
554-
555-
#' Create a Field object
556-
#'
557-
#' Create a Field object that contains the metadata for a single field in a schema.
558-
#'
559-
#' @param name The name of the field
560-
#' @param type The data type of the field
561-
#' @param nullable A logical vector indicating whether or not the field is nullable
562-
#' @return a Field object
563-
#' @export
564-
#' @examples
565-
#'\dontrun{
566-
#' sc <- sparkR.init()
567-
#' sqlCtx <- sparkRSQL.init(sc)
568-
#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) })
569-
#' field1 <- field("a", "integer", TRUE)
570-
#' field2 <- field("b", "string", TRUE)
571-
#' schema <- buildSchema(field1, field2)
572-
#' df <- createDataFrame(sqlCtx, rdd, schema)
573-
#' }
574-
field <- function(name, type, nullable = TRUE) {
575-
if (class(name) != "character") {
576-
stop("Field name must be a string.")
577-
}
578-
if (class(type) != "character") {
579-
stop("Field type must be a string.")
580-
}
581-
if (class(nullable) != "logical") {
582-
stop("nullable must be either TRUE or FALSE")
583-
}
584-
structure(list("name" = name, "type" = type, "nullable" = nullable), class = "field")
585-
}
586-
587-
# print method for Field objects
588-
print.field <- function(x, ...) {
589-
cat("name = \"", x$name, "\", type = \"", x$type, "\", nullable = ", x$nullable, sep = "")
590-
}
591-

pkg/R/schema.R

Lines changed: 169 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,169 @@
1+
#' structType
2+
#'
3+
#' Create a structType object that contains the metadata for a DataFrame. Intended for
4+
#' use with createDataFrame and toDF.
5+
#'
6+
#' @param x a Field object (created with the field() function)
7+
#' @param ... additional Field objects
8+
#' @return a structType object
9+
#' @export
10+
#' @examples
11+
#'\dontrun{
12+
#' sc <- sparkR.init()
13+
#' sqlCtx <- sparkRSQL.init(sc)
14+
#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) })
15+
#' schema <- buildSchema(field("a", "integer"), field("b", "string"))
16+
#' df <- createDataFrame(sqlCtx, rdd, schema)
17+
#' }
18+
structType <- function(x, ...) {
19+
UseMethod("structType", x)
20+
}
21+
22+
structType.jobj <- function(x) {
23+
obj <- structure(list(), class = "structType")
24+
obj$jobj <- x
25+
obj$fields <- function() { lapply(callJMethod(x, "fields"), structField) }
26+
obj
27+
}
28+
29+
structType.structField <- function(x, ...) {
30+
fields <- list(x, ...)
31+
if (!all(sapply(fields, inherits, "structField"))) {
32+
stop("All arguments must be structField objects.")
33+
}
34+
sfObjList <- lapply(fields, function(field) {
35+
field$jobj
36+
})
37+
stObj <- callJStatic("edu.berkeley.cs.amplab.sparkr.SQLUtils",
38+
"createStructType",
39+
listToSeq(sfObjList))
40+
structType(stObj)
41+
}
42+
43+
#' Print a Spark StructType.
44+
#'
45+
#' This function prints the contents of a StructType returned from the
46+
#' SparkR JVM backend.
47+
#'
48+
#' @param x A StructType object
49+
#' @param ... further arguments passed to or from other methods
50+
print.structType <- function(x, ...) {
51+
cat("StructType\n",
52+
sapply(x$fields(), function(field) { paste("|-", "name = \"", field$name(),
53+
"\", type = \"", field$dataType.toString(),
54+
"\", nullable = ", field$nullable(), "\n",
55+
sep = "") })
56+
, sep = "")
57+
}
58+
59+
#' structField
60+
#'
61+
#' Create a structField object that contains the metadata for a single field in a schema.
62+
#'
63+
#' @param x The name of the field
64+
#' @param type The data type of the field
65+
#' @param nullable A logical vector indicating whether or not the field is nullable
66+
#' @return a Field object
67+
#' @export
68+
#' @examples
69+
#'\dontrun{
70+
#' sc <- sparkR.init()
71+
#' sqlCtx <- sparkRSQL.init(sc)
72+
#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) })
73+
#' field1 <- field("a", "integer", TRUE)
74+
#' field2 <- field("b", "string", TRUE)
75+
#' schema <- buildSchema(field1, field2)
76+
#' df <- createDataFrame(sqlCtx, rdd, schema)
77+
#' }
78+
79+
structField <- function(x, ...) {
80+
UseMethod("structField", x)
81+
}
82+
83+
structField.jobj <- function(x) {
84+
obj <- structure(list(), class = "structField")
85+
obj$jobj <- x
86+
obj$name <- function() { callJMethod(x, "name") }
87+
obj$dataType <- function() { callJMethod(x, "dataType") }
88+
obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") }
89+
obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") }
90+
obj$nullable <- function() { callJMethod(x, "nullable") }
91+
obj
92+
}
93+
94+
structField.character <- function(x, type, nullable = TRUE) {
95+
if (class(x) != "character") {
96+
stop("Field name must be a string.")
97+
}
98+
if (class(type) != "character") {
99+
stop("Field type must be a string.")
100+
}
101+
if (class(nullable) != "logical") {
102+
stop("nullable must be either TRUE or FALSE")
103+
}
104+
options <- c("byte",
105+
"integer",
106+
"double",
107+
"numeric",
108+
"character",
109+
"string",
110+
"binary",
111+
"raw",
112+
"logical",
113+
"boolean",
114+
"timestamp",
115+
"date")
116+
dataType <- if (type %in% options) {
117+
type
118+
} else {
119+
stop(paste("Unsupported type for Dataframe:", type))
120+
}
121+
sfObj <- callJStatic("edu.berkeley.cs.amplab.sparkr.SQLUtils",
122+
"createStructField",
123+
x,
124+
dataType,
125+
nullable)
126+
structField(sfObj)
127+
}
128+
129+
#' Print a Spark StructField.
130+
#'
131+
#' This function prints the contents of a StructField returned from the
132+
#' SparkR JVM backend.
133+
#'
134+
#' @param x A StructField object
135+
#' @param ... further arguments passed to or from other methods
136+
print.structField <- function(x, ...) {
137+
cat("StructField(name = \"", x$name(),
138+
"\", type = \"", x$dataType.toString(),
139+
"\", nullable = ", x$nullable(),
140+
")",
141+
sep = "")
142+
}
143+
144+
# cfreeman: Don't think we need this function since we can create
145+
# structType in R and pass to createDataFrame
146+
#
147+
# #' dump the schema into JSON string
148+
# tojson <- function(x) {
149+
# if (inherits(x, "struct")) {
150+
# # schema object
151+
# l <- paste(lapply(x, tojson), collapse = ", ")
152+
# paste('{\"type\":\"struct\", \"fields\":','[', l, ']}', sep = '')
153+
# } else if (inherits(x, "field")) {
154+
# # field object
155+
# names <- names(x)
156+
# items <- lapply(names, function(n) {
157+
# safe_n <- gsub('"', '\\"', n)
158+
# paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '')
159+
# })
160+
# d <- paste(items, collapse = ", ")
161+
# paste('{', d, '}', sep = '')
162+
# } else if (is.character(x)) {
163+
# paste('"', x, '"', sep = '')
164+
# } else if (is.logical(x)) {
165+
# if (x) "true" else "false"
166+
# } else {
167+
# stop(paste("unexpected type:", class(x)))
168+
# }
169+
# }

sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala

Lines changed: 28 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
2323
import org.apache.spark.api.r.SerDe
2424
import org.apache.spark.rdd.RDD
2525
import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
26-
import org.apache.spark.sql.types.{DataType, StructType}
26+
import org.apache.spark.sql.types._
2727
import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode}
2828

2929
private[r] object SQLUtils {
@@ -39,8 +39,33 @@ private[r] object SQLUtils {
3939
arr.toSeq
4040
}
4141

42-
def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = {
43-
val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
42+
def createStructType(fields : Seq[StructField]) : StructType = {
43+
StructType(fields)
44+
}
45+
46+
def DataTypeObject(dataType: String): DataType = {
47+
dataType match {
48+
case "byte" => org.apache.spark.sql.types.ByteType
49+
case "integer" => org.apache.spark.sql.types.IntegerType
50+
case "double" => org.apache.spark.sql.types.DoubleType
51+
case "numeric" => org.apache.spark.sql.types.DoubleType
52+
case "character" => org.apache.spark.sql.types.StringType
53+
case "string" => org.apache.spark.sql.types.StringType
54+
case "binary" => org.apache.spark.sql.types.BinaryType
55+
case "raw" => org.apache.spark.sql.types.BinaryType
56+
case "logical" => org.apache.spark.sql.types.BooleanType
57+
case "boolean" => org.apache.spark.sql.types.BooleanType
58+
case "timestamp" => org.apache.spark.sql.types.TimestampType
59+
case "date" => org.apache.spark.sql.types.DateType
60+
case _ => throw new IllegalArgumentException(s"Invaid type $dataType")
61+
}
62+
}
63+
64+
def createStructField(name: String, dataType: String, nullable: Boolean): StructField = {
65+
val dtObj = DataTypeObject(dataType)
66+
StructField(name, dtObj, nullable)
67+
}
68+
4469
val num = schema.fields.size
4570
val rowRDD = rdd.map(bytesToRow)
4671
sqlContext.createDataFrame(rowRDD, schema)

0 commit comments

Comments
 (0)