aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorNong Li <nong@databricks.com>2016-01-26 17:34:01 -0800
committerReynold Xin <rxin@databricks.com>2016-01-26 17:34:01 -0800
commit555127387accdd7c1cf236912941822ba8af0a52 (patch)
tree1452dd0aedfcffe948bd2454d66fc032e824e62c /sql/hive
parent1dac964c1b996d38c65818414fc8401961a1de8a (diff)
downloadspark-555127387accdd7c1cf236912941822ba8af0a52.tar.gz
spark-555127387accdd7c1cf236912941822ba8af0a52.tar.bz2
spark-555127387accdd7c1cf236912941822ba8af0a52.zip
[SPARK-12854][SQL] Implement complex types support in ColumnarBatch
This patch adds support for complex types for ColumnarBatch. ColumnarBatch supports structs and arrays. There is a simple mapping between the richer catalyst types to these two. Strings are treated as an array of bytes. ColumnarBatch will contain a column for each node of the schema. Non-complex schemas consists of just leaf nodes. Structs represent an internal node with one child for each field. Arrays are internal nodes with one child. Structs just contain nullability. Arrays contain offsets and lengths into the child array. This structure is able to handle arbitrary nesting. It has the key property that we maintain columnar throughout and that primitive types are only stored in the leaf nodes and contiguous across rows. For example, if the schema is ``` array<array<int>> ``` There are three columns in the schema. The internal nodes each have one children. The leaf node contains all the int data stored consecutively. As part of this, this patch adds append APIs in addition to the Put APIs (e.g. putLong(rowid, v) vs appendLong(v)). These APIs are necessary when the batch contains variable length elements. The vectors are not fixed length and will grow as necessary. This should make the usage a lot simpler for the writer. Author: Nong Li <nong@databricks.com> Closes #10820 from nongli/spark-12854.
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala3
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala3
2 files changed, 4 insertions, 2 deletions
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index 76b36aa891..3e4cf3f79e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.hive.execution
import scala.collection.JavaConverters._
+import scala.util.Random
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
@@ -879,7 +880,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
RandomDataGenerator.forType(
dataType = schemaForGenerator,
nullable = true,
- seed = Some(System.nanoTime()))
+ new Random(System.nanoTime()))
val dataGenerator =
maybeDataGenerator
.getOrElse(fail(s"Failed to create data generator for schema $schemaForGenerator"))
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
index 3f9ecf6965..1a4b3ece72 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.sources
import scala.collection.JavaConverters._
+import scala.util.Random
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -122,7 +123,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes
val dataGenerator = RandomDataGenerator.forType(
dataType = dataType,
nullable = true,
- seed = Some(System.nanoTime())
+ new Random(System.nanoTime())
).getOrElse {
fail(s"Failed to create data generator for schema $dataType")
}