[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-05 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/6


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread MaxGekk
Github user MaxGekk commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214878373
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -663,12 +662,7 @@ case class StructsToJson(
 rowSchema, writer, new JSONOptions(options, timeZoneId.get))
 
   @transient
-  lazy val rowSchema = child.dataType match {
-case st: StructType => st
-case ArrayType(st: StructType, _) => st
-case mt: MapType => mt
-case ArrayType(mt: MapType, _) => mt
-  }
+  lazy val rowSchema = child.dataType
--- End diff --

I tried to remove `lazy` and got many errors on tests like:
```
Invalid call to dataType on unresolved object, tree: 'a
org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to 
dataType on unresolved object, tree: 'a
at 
org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:105)
at 
org.apache.spark.sql.catalyst.expressions.StructsToJson.(jsonExpressions.scala:665)
```
If you don't mind, I will keep it `lazy`.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214848902
  
--- Diff: R/pkg/R/functions.R ---
@@ -1697,8 +1697,8 @@ setMethod("to_date",
   })
 
 #' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array 
of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON 
string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a 
\code{mapType}
+#' or an array into a Column of JSON string.
--- End diff --

`\code{arrayType}`. It seems missed.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214844675
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -685,33 +679,29 @@ case class StructsToJson(
 (row: Any) =>
--- End diff --

`child.dataType` -> `intputSchema`


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214844726
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -663,12 +662,7 @@ case class StructsToJson(
 rowSchema, writer, new JSONOptions(options, timeZoneId.get))
 
   @transient
-  lazy val rowSchema = child.dataType match {
-case st: StructType => st
-case ArrayType(st: StructType, _) => st
-case mt: MapType => mt
-case ArrayType(mt: MapType, _) => mt
-  }
+  lazy val rowSchema = child.dataType
--- End diff --

Let's make it `val`.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214843579
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -685,33 +679,29 @@ case class StructsToJson(
 (row: Any) =>
   gen.write(row.asInstanceOf[InternalRow])
   getAndReset()
-  case ArrayType(_: StructType, _) =>
+  case _: ArrayType =>
 (arr: Any) =>
   gen.write(arr.asInstanceOf[ArrayData])
   getAndReset()
   case _: MapType =>
 (map: Any) =>
   gen.write(map.asInstanceOf[MapData])
   getAndReset()
-  case ArrayType(_: MapType, _) =>
-(arr: Any) =>
-  gen.write(arr.asInstanceOf[ArrayData])
-  getAndReset()
 }
   }
 
   override def dataType: DataType = StringType
 
   override def checkInputDataTypes(): TypeCheckResult = child.dataType 
match {
-case _: StructType | ArrayType(_: StructType, _) =>
+case _: StructType =>
--- End diff --

nit: `case _: StructType` and use it instead of 
`rowSchema.asInstanceOf[StructType]`.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214843691
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -685,33 +679,29 @@ case class StructsToJson(
 (row: Any) =>
   gen.write(row.asInstanceOf[InternalRow])
   getAndReset()
-  case ArrayType(_: StructType, _) =>
+  case _: ArrayType =>
 (arr: Any) =>
   gen.write(arr.asInstanceOf[ArrayData])
   getAndReset()
   case _: MapType =>
 (map: Any) =>
   gen.write(map.asInstanceOf[MapData])
   getAndReset()
-  case ArrayType(_: MapType, _) =>
-(arr: Any) =>
-  gen.write(arr.asInstanceOf[ArrayData])
-  getAndReset()
 }
   }
 
   override def dataType: DataType = StringType
 
   override def checkInputDataTypes(): TypeCheckResult = child.dataType 
match {
-case _: StructType | ArrayType(_: StructType, _) =>
+case _: StructType =>
   try {
 JacksonUtils.verifySchema(rowSchema.asInstanceOf[StructType])
 TypeCheckResult.TypeCheckSuccess
   } catch {
 case e: UnsupportedOperationException =>
   TypeCheckResult.TypeCheckFailure(e.getMessage)
   }
-case _: MapType | ArrayType(_: MapType, _) =>
+case _: MapType =>
--- End diff --

nit `case mapType: Maptype =>` and use it below likewise.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214843459
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -685,33 +679,29 @@ case class StructsToJson(
 (row: Any) =>
   gen.write(row.asInstanceOf[InternalRow])
   getAndReset()
-  case ArrayType(_: StructType, _) =>
+  case _: ArrayType =>
 (arr: Any) =>
   gen.write(arr.asInstanceOf[ArrayData])
   getAndReset()
   case _: MapType =>
 (map: Any) =>
   gen.write(map.asInstanceOf[MapData])
   getAndReset()
-  case ArrayType(_: MapType, _) =>
-(arr: Any) =>
-  gen.write(arr.asInstanceOf[ArrayData])
-  getAndReset()
 }
   }
 
   override def dataType: DataType = StringType
 
   override def checkInputDataTypes(): TypeCheckResult = child.dataType 
match {
--- End diff --

nit: `child.dataType` -> `inputSchema`


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-04 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214842392
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -663,12 +662,7 @@ case class StructsToJson(
 rowSchema, writer, new JSONOptions(options, timeZoneId.get))
 
   @transient
-  lazy val rowSchema = child.dataType match {
-case st: StructType => st
-case ArrayType(st: StructType, _) => st
-case mt: MapType => mt
-case ArrayType(mt: MapType, _) => mt
-  }
+  lazy val rowSchema = child.dataType
--- End diff --

nit: `rowSchema` -> `intputSchema`. I named this to `rowSchema` because it 
was always the schema for the row itself. Now, it seems can be other types as 
well.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-02 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214572336
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala
 ---
@@ -32,29 +32,29 @@ object JacksonUtils {
 }
   }
 
-  /**
-   * Verify if the schema is supported in JSON parsing.
-   */
-  def verifySchema(schema: StructType): Unit = {
-def verifyType(name: String, dataType: DataType): Unit = dataType 
match {
-  case NullType | BooleanType | ByteType | ShortType | IntegerType | 
LongType | FloatType |
-   DoubleType | StringType | TimestampType | DateType | BinaryType 
| _: DecimalType =>
+  def verifyType(name: String, dataType: DataType): Unit = dataType match {
--- End diff --

We can do:

```
def verifyType(name: String, dataType: DataType): Unit = {
  dataType match {
case ...
  }
}
```

to reduce the diff.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-02 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214572178
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ---
@@ -43,20 +42,22 @@ private[sql] class JacksonGenerator(
   // we can directly access data in `ArrayData` without the help of 
`SpecificMutableRow`.
   private type ValueWriter = (SpecializedGetters, Int) => Unit
 
-  // `JackGenerator` can only be initialized with a `StructType` or a 
`MapType`.
-  require(dataType.isInstanceOf[StructType] || 
dataType.isInstanceOf[MapType],
-s"JacksonGenerator only supports to be initialized with a 
${StructType.simpleString} " +
-  s"or ${MapType.simpleString} but got ${dataType.catalogString}")
+  // `JackGenerator` can only be initialized with a `StructType`, a 
`MapType` or a `ArrayType`.
+  require(dataType.isInstanceOf[StructType] || 
dataType.isInstanceOf[MapType]
+|| dataType.isInstanceOf[ArrayType],
+s"JacksonGenerator only supports to be initialized with a 
${StructType.simpleString}, " +
+  s"${MapType.simpleString} or ${ArrayType.simpleString} but got 
${dataType.catalogString}")
 
   // `ValueWriter`s for all fields of the schema
   private lazy val rootFieldWriters: Array[ValueWriter] = dataType match {
 case st: StructType => st.map(_.dataType).map(makeWriter).toArray
 case _ => throw new UnsupportedOperationException(
-  s"Initial type ${dataType.catalogString} must be a struct")
+  s"Initial type ${dataType.catalogString} must be a 
${StructType.simpleString}")
   }
 
   // `ValueWriter` for array data storing rows of the schema.
   private lazy val arrElementWriter: ValueWriter = dataType match {
+case at: ArrayType => makeWriter(at.elementType)
 case st: StructType =>
--- End diff --

Can we do `case _: StructType | _: MapType => makeWriter(dataType)`?


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-09-01 Thread MaxGekk
Github user MaxGekk commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214509654
  
--- Diff: R/pkg/R/functions.R ---
@@ -1697,8 +1697,8 @@ setMethod("to_date",
   })
 
 #' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array 
of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON 
string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a 
\code{mapType}
+#' or an array into a Column of JSON string.
--- End diff --

I added tests for Python and R. Please, take a look at them.


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214243817
  
--- Diff: R/pkg/R/functions.R ---
@@ -1697,8 +1697,8 @@ setMethod("to_date",
   })
 
 #' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array 
of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON 
string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a 
\code{mapType}
+#' or an array into a Column of JSON string.
--- End diff --

Let's add one simple python doctest as well


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214243115
  
--- Diff: R/pkg/R/functions.R ---
@@ -1697,8 +1697,8 @@ setMethod("to_date",
   })
 
 #' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array 
of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON 
string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a 
\code{mapType}
+#' or an array into a Column of JSON string.
--- End diff --

it should
could we add some tests for this in R?


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread MaxGekk
Github user MaxGekk commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214188879
  
--- Diff: R/pkg/R/functions.R ---
@@ -1697,8 +1697,8 @@ setMethod("to_date",
   })
 
 #' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array 
of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON 
string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a 
\code{mapType}
+#' or an array into a Column of JSON string.
--- End diff --

I am not sure


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214157587
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala ---
@@ -469,4 +469,53 @@ class JsonFunctionsSuite extends QueryTest with 
SharedSQLContext {
 
 checkAnswer(sql("""select json[0] from jsonTable"""), Seq(Row(null)))
   }
+
+  test("to_json - array of primitive type") {
--- End diff --

typo: `primitive type` -> `primitive types`


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214157102
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -3493,11 +3493,11 @@ object functions {
   def schema_of_json(e: Column): Column = withExpr(new 
SchemaOfJson(e.expr))
 
   /**
-   * (Scala-specific) Converts a column containing a `StructType`, 
`ArrayType` of `StructType`s,
-   * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the 
specified schema.
+   * (Scala-specific) Converts a column containing a `StructType`, 
`ArrayType` or
+   * a `MapType` into a JSON string with the specified schema.
* Throws an exception, in the case of an unsupported type.
*
-   * @param e a column containing a struct or array of the structs.
+   * @param e a column containing a struct, a array or a map.
--- End diff --

typo: `a array` -> `an array`


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214156762
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ---
@@ -43,20 +42,22 @@ private[sql] class JacksonGenerator(
   // we can directly access data in `ArrayData` without the help of 
`SpecificMutableRow`.
   private type ValueWriter = (SpecializedGetters, Int) => Unit
 
-  // `JackGenerator` can only be initialized with a `StructType` or a 
`MapType`.
-  require(dataType.isInstanceOf[StructType] || 
dataType.isInstanceOf[MapType],
+  // `JackGenerator` can only be initialized with a `StructType`, a 
`MapType` or a `ArrayType`.
+  require(dataType.isInstanceOf[StructType] || 
dataType.isInstanceOf[MapType]
+|| dataType.isInstanceOf[ArrayType],
 s"JacksonGenerator only supports to be initialized with a 
${StructType.simpleString} " +
--- End diff --

maybe need a `,` between `${StructType.simpleString}` and 
`${MapType.simpleString}`?


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214156048
  
--- Diff: R/pkg/R/functions.R ---
@@ -1697,8 +1697,8 @@ setMethod("to_date",
   })
 
 #' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array 
of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON 
string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a 
\code{mapType}
+#' or an array into a Column of JSON string.
--- End diff --

does `\code{arrayType}` work here? 


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-30 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r214155095
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ---
@@ -28,7 +27,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, 
DateTimeUtils, MapData}
 import org.apache.spark.sql.types._
 
 /**
- * `JackGenerator` can only be initialized with a `StructType` or a 
`MapType`.
+ * `JackGenerator` can only be initialized with a `StructType`, a 
`MapType` ot `ArrayType`.
--- End diff --

typo: `ot`


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-27 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r213177782
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -613,8 +613,7 @@ case class JsonToStructs(
 }
 
 /**
- * Converts a [[StructType]], [[ArrayType]] of [[StructType]]s, [[MapType]]
- * or [[ArrayType]] of [[MapType]]s to a json output string.
+ * Converts a [[StructType]], [[ArrayType]] or [[MapType]] to a json 
output string.
--- End diff --

not a big deal but `JSON` while we are here


---

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



[GitHub] spark pull request #22226: [SPARK-25252][SQL] Support arrays of any types by...

2018-08-27 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/6#discussion_r213177790
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2289,12 +2289,10 @@ def from_json(col, schema, options={}):
 @since(2.1)
 def to_json(col, options={}):
 """
-Converts a column containing a :class:`StructType`, :class:`ArrayType` 
of
-:class:`StructType`\\s, a :class:`MapType` or :class:`ArrayType` of 
:class:`MapType`\\s
+Converts a column containing a :class:`StructType`, :class:`ArrayType` 
or a :class:`MapType`
 into a JSON string. Throws an exception, in the case of an unsupported 
type.
 
-:param col: name of column containing the struct, array of the 
structs, the map or
-array of the maps.
+:param col: name of column containing a struct, an array or a map.
 :param options: options to control converting. accepts the same 
options as the json datasource
--- End diff --

ditto


---

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