Repository: spark
Updated Branches:
  refs/heads/master 23d5f8864 -> 67d06880e


http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
deleted file mode 100644
index a9d63e7..0000000
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.types
-
-import java.text.SimpleDateFormat
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-
-
-private[sql] object DataTypeConversions {
-
-  def productToRow(product: Product, schema: StructType): Row = {
-    val mutableRow = new GenericMutableRow(product.productArity)
-    val schemaFields = schema.fields.toArray
-
-    var i = 0
-    while (i < mutableRow.length) {
-      mutableRow(i) =
-        ScalaReflection.convertToCatalyst(product.productElement(i), 
schemaFields(i).dataType)
-      i += 1
-    }
-
-    mutableRow
-  }
-
-  def stringToTime(s: String): java.util.Date = {
-    if (!s.contains('T')) {
-      // JDBC escape string
-      if (s.contains(' ')) {
-        java.sql.Timestamp.valueOf(s)
-      } else {
-        java.sql.Date.valueOf(s)
-      }
-    } else if (s.endsWith("Z")) {
-      // this is zero timezone of ISO8601
-      stringToTime(s.substring(0, s.length - 1) + "GMT-00:00")
-    } else if (s.indexOf("GMT") == -1) {
-      // timezone with ISO8601
-      val inset = "+00.00".length
-      val s0 = s.substring(0, s.length - inset)
-      val s1 = s.substring(s.length - inset, s.length)
-      if (s0.substring(s0.lastIndexOf(':')).contains('.')) {
-        stringToTime(s0 + "GMT" + s1)
-      } else {
-        stringToTime(s0 + ".0GMT" + s1)
-      }
-    } else {
-      // ISO8601 with GMT insert
-      val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( 
"yyyy-MM-dd'T'HH:mm:ss.SSSz" )
-      ISO8601GMT.parse(s)
-    }
-  }
-
-  /** Converts Java objects to catalyst rows / types */
-  def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) 
match {
-    case (obj, udt: UserDefinedType[_]) => 
ScalaReflection.convertToCatalyst(obj, udt) // Scala type
-    case (d: java.math.BigDecimal, _) => Decimal(d)
-    case (other, _) => other
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala
index 8a1a3b8..504fb05 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.types
 
 import java.sql.Date
+import java.text.SimpleDateFormat
 import java.util.{Calendar, TimeZone}
 
 import org.apache.spark.sql.catalyst.expressions.Cast
@@ -57,4 +58,32 @@ object DateUtils {
   }
 
   def toString(days: Int): String = 
Cast.threadLocalDateFormat.get.format(toJavaDate(days))
+
+  def stringToTime(s: String): java.util.Date = {
+    if (!s.contains('T')) {
+      // JDBC escape string
+      if (s.contains(' ')) {
+        java.sql.Timestamp.valueOf(s)
+      } else {
+        java.sql.Date.valueOf(s)
+      }
+    } else if (s.endsWith("Z")) {
+      // this is zero timezone of ISO8601
+      stringToTime(s.substring(0, s.length - 1) + "GMT-00:00")
+    } else if (s.indexOf("GMT") == -1) {
+      // timezone with ISO8601
+      val inset = "+00.00".length
+      val s0 = s.substring(0, s.length - inset)
+      val s1 = s.substring(s.length - inset, s.length)
+      if (s0.substring(s0.lastIndexOf(':')).contains('.')) {
+        stringToTime(s0 + "GMT" + s1)
+      } else {
+        stringToTime(s0 + ".0GMT" + s1)
+      }
+    } else {
+      // ISO8601 with GMT insert
+      val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( 
"yyyy-MM-dd'T'HH:mm:ss.SSSz" )
+      ISO8601GMT.parse(s)
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
index eee00e3..bbc0b66 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
@@ -260,7 +260,7 @@ class ScalaReflectionSuite extends FunSuite {
     val data = PrimitiveData(1, 1, 1, 1, 1, 1, true)
     val convertedData = Row(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 
1.toByte, true)
     val dataType = schemaFor[PrimitiveData].dataType
-    assert(convertToCatalyst(data, dataType) === convertedData)
+    assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === 
convertedData)
   }
 
   test("convert Option[Product] to catalyst") {
@@ -270,7 +270,7 @@ class ScalaReflectionSuite extends FunSuite {
     val dataType = schemaFor[OptionalData].dataType
     val convertedData = Row(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 
2.toByte, true,
       Row(1, 1, 1, 1, 1, 1, true))
-    assert(convertToCatalyst(data, dataType) === convertedData)
+    assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === 
convertedData)
   }
 
   test("infer schema from case class with multiple constructors") {

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index 5c6016a..9b9adf8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -33,7 +33,7 @@ import org.apache.spark.api.java.JavaRDD
 import org.apache.spark.api.python.SerDeUtil
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.sql.catalyst.{ScalaReflection, SqlParser}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, 
SqlParser}
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, 
ResolvedStar}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.{JoinType, Inner}
@@ -713,7 +713,7 @@ class DataFrame private[sql](
     val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
     val attributes = schema.toAttributes
     val rowFunction =
-      f.andThen(_.map(ScalaReflection.convertToCatalyst(_, 
schema).asInstanceOf[Row]))
+      f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, 
schema).asInstanceOf[Row]))
     val generator = UserDefinedGenerator(attributes, rowFunction, 
input.map(_.expr))
 
     Generate(generator, join = true, outer = false, None, logicalPlan)
@@ -734,7 +734,7 @@ class DataFrame private[sql](
     val dataType = ScalaReflection.schemaFor[B].dataType
     val attributes = AttributeReference(outputColumn, dataType)() :: Nil
     def rowFunction(row: Row): TraversableOnce[Row] = {
-      f(row(0).asInstanceOf[A]).map(o => 
Row(ScalaReflection.convertToCatalyst(o, dataType)))
+      f(row(0).asInstanceOf[A]).map(o => 
Row(CatalystTypeConverters.convertToCatalyst(o, dataType)))
     }
     val generator = UserDefinedGenerator(attributes, rowFunction, 
apply(inputColumn).expr :: Nil)
 
@@ -961,7 +961,10 @@ class DataFrame private[sql](
   lazy val rdd: RDD[Row] = {
     // use a local variable to make sure the map closure doesn't capture the 
whole DataFrame
     val schema = this.schema
-    
queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, 
schema))
+    queryExecution.executedPlan.execute().mapPartitions { rows =>
+      val converter = CatalystTypeConverters.createToScalaConverter(schema)
+      rows.map(converter(_).asInstanceOf[Row])
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 39dd14e..c25ef58 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -31,9 +31,9 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer}
-import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan, OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.catalyst.{ScalaReflection, expressions}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, 
expressions}
 import org.apache.spark.sql.execution.{Filter, _}
 import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, 
JDBCRelation}
 import org.apache.spark.sql.json._
@@ -404,7 +404,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
     // TODO: use MutableProjection when rowRDD is another DataFrame and the 
applied
     // schema differs from the existing schema on any field data type.
     val catalystRows = if (needsConversion) {
-      rowRDD.map(ScalaReflection.convertToCatalyst(_, 
schema).asInstanceOf[Row])
+      val converter = CatalystTypeConverters.createToCatalystConverter(schema)
+      rowRDD.map(converter(_).asInstanceOf[Row])
     } else {
       rowRDD
     }
@@ -459,7 +460,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
       iter.map { row =>
         new GenericRow(
           extractors.zip(attributeSeq).map { case (e, attr) =>
-            DataTypeConversions.convertJavaToCatalyst(e.invoke(row), 
attr.dataType)
+            CatalystTypeConverters.convertToCatalyst(e.invoke(row), 
attr.dataType)
           }.toArray[Any]
         ) : Row
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index d895572..656bdd7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -20,14 +20,12 @@ package org.apache.spark.sql.execution
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{Row, SQLContext}
-import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.CatalystTypeConverters
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow}
+import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, 
Attribute}
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
 import org.apache.spark.sql.types.StructType
 
-import scala.collection.immutable
-
 /**
  * :: DeveloperApi ::
  */
@@ -39,13 +37,15 @@ object RDDConversions {
         Iterator.empty
       } else {
         val bufferedIterator = iterator.buffered
-        val mutableRow = new 
GenericMutableRow(bufferedIterator.head.productArity)
+        val mutableRow = new SpecificMutableRow(schema.fields.map(_.dataType))
         val schemaFields = schema.fields.toArray
+        val converters = schemaFields.map {
+          f => CatalystTypeConverters.createToCatalystConverter(f.dataType)
+        }
         bufferedIterator.map { r =>
           var i = 0
           while (i < mutableRow.length) {
-            mutableRow(i) =
-              ScalaReflection.convertToCatalyst(r.productElement(i), 
schemaFields(i).dataType)
+            mutableRow(i) = converters(i)(r.productElement(i))
             i += 1
           }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
index 5bd699a..8a8c3a4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.CatalystTypeConverters
 import org.apache.spark.sql.catalyst.expressions.Attribute
 
 
@@ -32,9 +32,15 @@ case class LocalTableScan(output: Seq[Attribute], rows: 
Seq[Row]) extends LeafNo
 
   override def execute(): RDD[Row] = rdd
 
-  override def executeCollect(): Array[Row] =
-    rows.map(ScalaReflection.convertRowToScala(_, schema)).toArray
 
-  override def executeTake(limit: Int): Array[Row] =
-    rows.map(ScalaReflection.convertRowToScala(_, schema)).take(limit).toArray
+  override def executeCollect(): Array[Row] = {
+    val converter = CatalystTypeConverters.createToScalaConverter(schema)
+    rows.map(converter(_).asInstanceOf[Row]).toArray
+  }
+
+
+  override def executeTake(limit: Int): Array[Row] = {
+    val converter = CatalystTypeConverters.createToScalaConverter(schema)
+    rows.map(converter(_).asInstanceOf[Row]).take(limit).toArray
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index d239637..fabcf6b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.Logging
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.catalyst.{ScalaReflection, trees}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.plans.QueryPlan
@@ -80,8 +80,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with 
Logging with Serializ
   /**
    * Runs this query returning the result as an array.
    */
+
   def executeCollect(): Array[Row] = {
-    execute().map(ScalaReflection.convertRowToScala(_, schema)).collect()
+    execute().mapPartitions { iter =>
+      val converter = CatalystTypeConverters.createToScalaConverter(schema)
+      iter.map(converter(_).asInstanceOf[Row])
+    }.collect()
   }
 
   /**
@@ -125,7 +129,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with 
Logging with Serializ
       partsScanned += numPartsToTry
     }
 
-    buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema))
+    val converter = CatalystTypeConverters.createToScalaConverter(schema)
+    buf.toArray.map(converter(_).asInstanceOf[Row])
   }
 
   protected def newProjection(

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index 1f5251a..6eec520 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -21,7 +21,7 @@ import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf}
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.{RDD, ShuffledRDD}
 import org.apache.spark.shuffle.sort.SortShuffleManager
-import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.CatalystTypeConverters
 import org.apache.spark.sql.catalyst.errors._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical._
@@ -139,9 +139,10 @@ case class TakeOrdered(limit: Int, sortOrder: 
Seq[SortOrder], child: SparkPlan)
 
   private def collectData(): Array[Row] = 
child.execute().map(_.copy()).takeOrdered(limit)(ord)
 
-  // TODO: Is this copying for no reason?
-  override def executeCollect(): Array[Row] =
-    collectData().map(ScalaReflection.convertRowToScala(_, this.schema))
+  override def executeCollect(): Array[Row] = {
+    val converter = CatalystTypeConverters.createToScalaConverter(schema)
+    collectData().map(converter(_).asInstanceOf[Row])
+  }
 
   // TODO: Terminal split should be implemented differently from non-terminal 
split.
   // TODO: Pick num splits based on |limit|.

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 0b770f2..b1e8521 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -391,7 +391,7 @@ private[sql] object JsonRDD extends Logging {
     value match {
       // only support string as date
       case value: java.lang.String =>
-        DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime)
+        DateUtils.millisToDays(DateUtils.stringToTime(value).getTime)
       case value: java.sql.Date => DateUtils.fromJavaDate(value)
     }
   }
@@ -400,7 +400,7 @@ private[sql] object JsonRDD extends Logging {
     value match {
       case value: java.lang.Integer => new 
Timestamp(value.asInstanceOf[Int].toLong)
       case value: java.lang.Long => new Timestamp(value)
-      case value: java.lang.String => 
toTimestamp(DataTypeConversions.stringToTime(value).getTime)
+      case value: java.lang.String => 
toTimestamp(DateUtils.stringToTime(value).getTime)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java 
b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
index 1ff2d5a..6d0fbe8 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
@@ -20,6 +20,8 @@ package test.org.apache.spark.sql;
 import java.io.Serializable;
 import java.util.Arrays;
 
+import scala.collection.Seq;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -127,6 +129,12 @@ public class JavaDataFrameSuite {
       schema.apply("b"));
     Row first = df.select("a", "b").first();
     Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0);
-    Assert.assertArrayEquals(bean.getB(), first.<Integer[]>getAs(1));
+    // Now Java lists and maps are converetd to Scala Seq's and Map's. Once we 
get a Seq below,
+    // verify that it has the expected length, and contains expected elements.
+    Seq<Integer> result = first.getAs(1);
+    Assert.assertEquals(bean.getB().length, result.length());
+    for (int i = 0; i < result.length(); i++) {
+      Assert.assertEquals(bean.getB()[i], result.apply(i));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/67d06880/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index 1fe0b76..fd0e274 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -895,8 +895,7 @@ class JsonSuite extends QueryTest {
     )
   }
 
-  test("SPARK-4228 DataFrame to JSON")
-  {
+  test("SPARK-4228 DataFrame to JSON") {
     val schema1 = StructType(
       StructField("f1", IntegerType, false) ::
       StructField("f2", StringType, false) ::


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

Reply via email to