Commit graph

14665 commits

Author SHA1 Message Date
Jakob Odersky 6883a5120c [SPARK-13171][CORE] Replace future calls with Future
Trivial search-and-replace to eliminate deprecation warnings in Scala 2.11.
Also works with 2.10

Author: Jakob Odersky <jakob@odersky.com>

Closes #11085 from jodersky/SPARK-13171.
2016-02-05 19:00:12 -08:00
Davies Liu 875f507929 [SPARK-13215] [SQL] remove fallback in codegen
Since we remove the configuration for codegen, we are heavily reply on codegen (also TungstenAggregate require the generated MutableProjection to update UnsafeRow), should remove the fallback, which could make user confusing, see the discussion in SPARK-13116.

Author: Davies Liu <davies@databricks.com>

Closes #11097 from davies/remove_fallback.
2016-02-05 15:07:43 -08:00
Luc Bourlier 0bb5b73387 [SPARK-13002][MESOS] Send initial request of executors for dyn allocation
Fix for [SPARK-13002](https://issues.apache.org/jira/browse/SPARK-13002) about the initial number of executors when running with dynamic allocation on Mesos.
Instead of fixing it just for the Mesos case, made the change in `ExecutorAllocationManager`. It is already driving the number of executors running on Mesos, only no the initial value.

The `None` and `Some(0)` are internal details on the computation of resources to reserved, in the Mesos backend scheduler. `executorLimitOption` has to be initialized correctly, otherwise the Mesos backend scheduler will, either, create to many executors at launch, or not create any executors and not be able to recover from this state.

Removed the 'special case' description in the doc. It was not totally accurate, and is not needed anymore.

This doesn't fix the same problem visible with Spark standalone. There is no straightforward way to send the initial value in standalone mode.

Somebody knowing this part of the yarn support should review this change.

Author: Luc Bourlier <luc.bourlier@typesafe.com>

Closes #11047 from skyluc/issue/initial-dyn-alloc-2.
2016-02-05 14:37:42 -08:00
Bill Chambers 66e1383de2 [SPARK-13214][DOCS] update dynamicAllocation documentation
Author: Bill Chambers <bill@databricks.com>

Closes #11094 from anabranch/dynamic-docs.
2016-02-05 14:35:39 -08:00
Wenchen Fan 1ed354a536 [SPARK-12939][SQL] migrate encoder resolution logic to Analyzer
https://issues.apache.org/jira/browse/SPARK-12939

Now we will catch `ObjectOperator` in `Analyzer` and resolve the `fromRowExpression/deserializer` inside it.  Also update the `MapGroups` and `CoGroup` to pass in `dataAttributes`, so that we can correctly resolve value deserializer(the `child.output` contains both groupking key and values, which may mess things up if they have same-name attribtues). End-to-end tests are added.

follow-ups:

* remove encoders from typed aggregate expression.
* completely remove resolve/bind in `ExpressionEncoder`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10852 from cloud-fan/bug.
2016-02-05 14:34:12 -08:00
Shixiong Zhu 7b73f1719c [SPARK-13166][SQL] Rename DataStreamReaderWriterSuite to DataFrameReaderWriterSuite
A follow up PR for #11062 because it didn't rename the test suite.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11096 from zsxwing/rename.
2016-02-05 13:44:34 -08:00
Reynold Xin 82d84ff2dd [SPARK-13187][SQL] Add boolean/long/double options in DataFrameReader/Writer
This patch adds option function for boolean, long, and double types. This makes it slightly easier for Spark users to specify options without turning them into strings. Using the JSON data source as an example.

Before this patch:
```scala
sqlContext.read.option("primitivesAsString", "true").json("/path/to/json")
```

After this patch:
Before this patch:
```scala
sqlContext.read.option("primitivesAsString", true).json("/path/to/json")
```

Author: Reynold Xin <rxin@databricks.com>

Closes #11072 from rxin/SPARK-13187.
2016-02-04 22:43:44 -08:00
Jakob Odersky 352102ed0b [SPARK-13208][CORE] Replace use of Pairs with Tuple2s
Another trivial deprecation fix for Scala 2.11

Author: Jakob Odersky <jakob@odersky.com>

Closes #11089 from jodersky/SPARK-13208.
2016-02-04 22:22:41 -08:00
gatorsmile e3c75c6398 [SPARK-12850][SQL] Support Bucket Pruning (Predicate Pushdown for Bucketed Tables)
JIRA: https://issues.apache.org/jira/browse/SPARK-12850

This PR is to support bucket pruning when the predicates are `EqualTo`, `EqualNullSafe`, `IsNull`, `In`, and `InSet`.

Like HIVE, in this PR, the bucket pruning works when the bucketing key has one and only one column.

So far, I do not find a way to verify how many buckets are actually scanned. However, I did verify it when doing the debug. Could you provide a suggestion how to do it properly? Thank you! cloud-fan yhuai rxin marmbrus

BTW, we can add more cases to support complex predicate including `Or` and `And`. Please let me know if I should do it in this PR.

Maybe we also need to add test cases to verify if bucket pruning works well for each data type.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10942 from gatorsmile/pruningBuckets.
2016-02-04 18:37:58 -08:00
Raafat Akkad 6dbfc40776 [SPARK-13052] waitingApps metric doesn't show the number of apps currently in the WAITING state
Author: Raafat Akkad <raafat.akkad@gmail.com>

Closes #10959 from RaafatAkkad/master.
2016-02-04 16:09:31 -08:00
Andrew Or 7a4b37f02c [HOTFIX] Fix style violation caused by c756bda 2016-02-04 12:47:32 -08:00
Shixiong Zhu 8e2f296306 [SPARK-13195][STREAMING] Fix NoSuchElementException when a state is not set but timeoutThreshold is defined
Check the state Existence before calling get.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11081 from zsxwing/SPARK-13195.
2016-02-04 12:43:16 -08:00
Andrew Or bd38dd6f75 [SPARK-13079][SQL] InMemoryCatalog follow-ups
This patch incorporates review feedback from #11069, which is already merged.

Author: Andrew Or <andrew@databricks.com>

Closes #11080 from andrewor14/catalog-follow-ups.
2016-02-04 12:20:18 -08:00
Andrew Or c756bda477 [SPARK-12330][MESOS][HOTFIX] Rename timeout config
The config already describes time and accepts a general format
that is not restricted to ms. This commit renames the internal
config to use a format that's consistent in Spark.
2016-02-04 12:04:54 -08:00
Josh Rosen 33212cb9a1 [SPARK-13168][SQL] Collapse adjacent repartition operators
Spark SQL should collapse adjacent `Repartition` operators and only keep the last one.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11064 from JoshRosen/collapse-repartition.
2016-02-04 11:08:50 -08:00
Andrew Or 085f510ae5 MAINTENANCE: Automated closing of pull requests.
This commit exists to close the following pull requests on Github:

Closes #7971 (requested by yhuai)
Closes #8539 (requested by srowen)
Closes #8746 (requested by yhuai)
Closes #9288 (requested by andrewor14)
Closes #9321 (requested by andrewor14)
Closes #9935 (requested by JoshRosen)
Closes #10442 (requested by andrewor14)
Closes #10585 (requested by srowen)
Closes #10785 (requested by srowen)
Closes #10832 (requested by andrewor14)
Closes #10941 (requested by marmbrus)
Closes #11024 (requested by andrewor14)
2016-02-04 11:07:06 -08:00
Andrew Or 15205da817 [SPARK-13053][TEST] Unignore tests in InternalAccumulatorSuite
These were ignored because they are incorrectly written; they don't actually trigger stage retries, which is what the tests are testing. These tests are now rewritten to induce stage retries through fetch failures.

Note: there were 2 tests before and now there's only 1. What happened? It turns out that the case where we only resubmit a subset of of the original missing partitions is very difficult to simulate in tests without potentially introducing flakiness. This is because the `DAGScheduler` removes all map outputs associated with a given executor when this happens, and we will need multiple executors to trigger this case, and sometimes the scheduler still removes map outputs from all executors.

Author: Andrew Or <andrew@databricks.com>

Closes #10969 from andrewor14/unignore-accum-test.
2016-02-04 10:34:43 -08:00
Andrew Or 4120bcbaff [SPARK-13162] Standalone mode does not respect initial executors
Currently the Master would always set an application's initial executor limit to infinity. If the user specified `spark.dynamicAllocation.initialExecutors`, the config would not take effect. This is similar to #11047 but for standalone mode.

Author: Andrew Or <andrew@databricks.com>

Closes #11054 from andrewor14/standalone-da-initial.
2016-02-04 10:32:32 -08:00
Holden Karau 62a7c28388 [SPARK-13164][CORE] Replace deprecated synchronized buffer in core
Building with scala 2.11 results in the warning trait SynchronizedBuffer in package mutable is deprecated: Synchronization via traits is deprecated as it is inherently unreliable. Consider java.util.concurrent.ConcurrentLinkedQueue as an alternative. Investigation shows we are already using ConcurrentLinkedQueue in other locations so switch our uses of SynchronizedBuffer to ConcurrentLinkedQueue.

Author: Holden Karau <holden@us.ibm.com>

Closes #11059 from holdenk/SPARK-13164-replace-deprecated-synchronized-buffer-in-core.
2016-02-04 10:29:38 -08:00
Charles Allen 2eaeafe8a2 [SPARK-12330][MESOS] Fix mesos coarse mode cleanup
In the current implementation the mesos coarse scheduler does not wait for the mesos tasks to complete before ending the driver. This causes a race where the task has to finish cleaning up before the mesos driver terminates it with a SIGINT (and SIGKILL after 3 seconds if the SIGINT doesn't work).

This PR causes the mesos coarse scheduler to wait for the mesos tasks to finish (with a timeout defined by `spark.mesos.coarse.shutdown.ms`)

This PR also fixes a regression caused by [SPARK-10987] whereby submitting a shutdown causes a race between the local shutdown procedure and the notification of the scheduler driver disconnection. If the scheduler driver disconnection wins the race, the coarse executor incorrectly exits with status 1 (instead of the proper status 0)

With this patch the mesos coarse scheduler terminates properly, the executors clean up, and the tasks are reported as `FINISHED` in the Mesos console (as opposed to `KILLED` in < 1.6 or `FAILED` in 1.6 and later)

Author: Charles Allen <charles@allen-net.com>

Closes #10319 from drcrallen/SPARK-12330.
2016-02-04 10:27:25 -08:00
Reynold Xin dee801adb7 [SPARK-12828][SQL] Natural join follow-up
This is a small addendum to #10762 to make the code more robust again future changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #11070 from rxin/SPARK-12828-natural-join.
2016-02-03 23:43:48 -08:00
Liang-Chi Hsieh d39087147f [SPARK-13113] [CORE] Remove unnecessary bit operation when decoding page number
JIRA: https://issues.apache.org/jira/browse/SPARK-13113

As we shift bits right, looks like the bitwise AND operation is unnecessary.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #11002 from viirya/improve-decodepagenumber.
2016-02-03 23:17:51 -08:00
Yuhao Yang c2c956bcd1 [ML][DOC] fix wrong api link in ml onevsrest
minor fix for api link in ml onevsrest

Author: Yuhao Yang <hhbyyh@gmail.com>

Closes #11068 from hhbyyh/onevsrestDoc.
2016-02-03 21:19:44 -08:00
Daoyuan Wang 0f81318ae2 [SPARK-12828][SQL] add natural join support
Jira:
https://issues.apache.org/jira/browse/SPARK-12828

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #10762 from adrian-wang/naturaljoin.
2016-02-03 21:05:53 -08:00
Andrew Or a64831124c [SPARK-13079][SQL] Extend and implement InMemoryCatalog
This is a step towards consolidating `SQLContext` and `HiveContext`.

This patch extends the existing Catalog API added in #10982 to include methods for handling table partitions. In particular, a partition is identified by `PartitionSpec`, which is just a `Map[String, String]`. The Catalog is still not used by anything yet, but its API is now more or less complete and an implementation is fully tested.

About 200 lines are test code.

Author: Andrew Or <andrew@databricks.com>

Closes #11069 from andrewor14/catalog.
2016-02-03 19:32:41 -08:00
Holden Karau a8e2ba776b [SPARK-13152][CORE] Fix task metrics deprecation warning
Make an internal non-deprecated version of incBytesRead and incRecordsRead so we don't have unecessary deprecation warnings in our build.

Right now incBytesRead and incRecordsRead are marked as deprecated and for internal use only. We should make private[spark] versions which are not deprecated and switch to those internally so as to not clutter up the warning messages when building.

cc andrewor14 who did the initial deprecation

Author: Holden Karau <holden@us.ibm.com>

Closes #11056 from holdenk/SPARK-13152-fix-task-metrics-deprecation-warnings.
2016-02-03 17:43:14 -08:00
Davies Liu de0914522f [SPARK-13131] [SQL] Use best and average time in benchmark
Best time is stabler than average time, also added a column for nano seconds per row (which could be used to estimate contributions of each components in a query).

Having best time and average time together for more information (we can see kind of variance).

rate, time per row and relative are all calculated using best time.

The result looks like this:
```
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
rang/filter/sum:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
rang/filter/sum codegen=false          14332 / 16646         36.0          27.8       1.0X
rang/filter/sum codegen=true              845 /  940        620.0           1.6      17.0X
```

Author: Davies Liu <davies@databricks.com>

Closes #11018 from davies/gen_bench.
2016-02-03 17:07:27 -08:00
Reynold Xin 915a75398e [SPARK-13166][SQL] Remove DataStreamReader/Writer
They seem redundant and we can simply use DataFrameReader/Writer. The new usage looks like:

```scala
val df = sqlContext.read.stream("...")
val handle = df.write.stream("...")
handle.stop()
```

Author: Reynold Xin <rxin@databricks.com>

Closes #11062 from rxin/SPARK-13166.
2016-02-03 16:10:11 -08:00
Alex Bozarth 3221eddb8f [SPARK-3611][WEB UI] Show number of cores for each executor in application web UI
Added a Cores column in the Executors UI

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #11039 from ajbozarth/spark3611.
2016-02-03 15:53:10 -08:00
Herman van Hovell 9dd2741ebe [SPARK-13157] [SQL] Support any kind of input for SQL commands.
The ```SparkSqlLexer``` currently swallows characters which have not been defined in the grammar. This causes problems with SQL commands, such as: ```add jar file:///tmp/ab/TestUDTF.jar```. In this example the `````` is swallowed.

This PR adds an extra Lexer rule to handle such input, and makes a tiny modification to the ```ASTNode```.

cc davies liancheng

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #11052 from hvanhovell/SPARK-13157.
2016-02-03 12:31:30 -08:00
Davies Liu c4feec26eb [SPARK-12798] [SQL] generated BroadcastHashJoin
A row from stream side could match multiple rows on build side, the loop for these matched rows should not be interrupted when emitting a row, so we buffer the output rows in a linked list, check the termination condition on producer loop (for example, Range or Aggregate).

Author: Davies Liu <davies@databricks.com>

Closes #10989 from davies/gen_join.
2016-02-03 10:38:53 -08:00
Mario Briggs e9eb248edf [SPARK-12739][STREAMING] Details of batch in Streaming tab uses two Duration columns
I have clearly prefix the two 'Duration' columns in 'Details of Batch' Streaming tab as 'Output Op Duration' and 'Job Duration'

Author: Mario Briggs <mario.briggs@in.ibm.com>
Author: mariobriggs <mariobriggs@in.ibm.com>

Closes #11022 from mariobriggs/spark-12739.
2016-02-03 09:50:28 -08:00
Sameer Agarwal 138c300f97 [SPARK-12957][SQL] Initial support for constraint propagation in SparkSQL
Based on the semantics of a query, we can derive a number of data constraints on output of each (logical or physical) operator. For instance, if a filter defines `‘a > 10`, we know that the output data of this filter satisfies 2 constraints:

1. `‘a > 10`
2. `isNotNull(‘a)`

This PR proposes a possible way of keeping track of these constraints and propagating them in the logical plan, which can then help us build more advanced optimizations (such as pruning redundant filters, optimizing joins, among others). We define constraints as a set of (implicitly conjunctive) expressions. For e.g., if a filter operator has constraints = `Set(‘a > 10, ‘b < 100)`, it’s implied that the outputs satisfy both individual constraints (i.e., `‘a > 10` AND `‘b < 100`).

Design Document: https://docs.google.com/a/databricks.com/document/d/1WQRgDurUBV9Y6CWOBS75PQIqJwT-6WftVa18xzm7nCo/edit?usp=sharing

Author: Sameer Agarwal <sameer@databricks.com>

Closes #10844 from sameeragarwal/constraints.
2016-02-02 22:22:50 -08:00
Davies Liu e86f8f63bf [SPARK-13147] [SQL] improve readability of generated code
1. try to avoid the suffix (unique id)
2. remove the comment if there is no code generated.
3. re-arrange the order of functions
4. trop the new line for inlined blocks.

Author: Davies Liu <davies@databricks.com>

Closes #11032 from davies/better_suffix.
2016-02-02 22:13:10 -08:00
Shixiong Zhu 335f10edad [SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnv
`rpcEnv.awaitTermination()` was not added in #10854 because some Streaming Python tests hung forever.

This patch fixed the hung issue and added rpcEnv.awaitTermination() back to SparkEnv.

Previously, Streaming Kafka Python tests shutdowns the zookeeper server before stopping StreamingContext. Then when stopping StreamingContext, KafkaReceiver may be hung due to https://issues.apache.org/jira/browse/KAFKA-601, hence, some thread of RpcEnv's Dispatcher cannot exit and rpcEnv.awaitTermination is hung.The patch just changed the shutdown order to fix it.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11031 from zsxwing/awaitTermination.
2016-02-02 21:13:54 -08:00
Imran Younus 0557146619 [SPARK-12732][ML] bug fix in linear regression train
Fixed the bug in linear regression train for the case when the target variable is constant. The two cases for `fitIntercept=true` or `fitIntercept=false` should be treated differently.

Author: Imran Younus <iyounus@us.ibm.com>

Closes #10702 from iyounus/SPARK-12732_bug_fix_in_linear_regression_train.
2016-02-02 20:38:53 -08:00
Davies Liu 99a6e3c1e8 [SPARK-12951] [SQL] support spilling in generated aggregate
This PR add spilling support for generated TungstenAggregate.

If spilling happened, it's not that bad to do the iterator based sort-merge-aggregate (not generated).

The changes will be covered by TungstenAggregationQueryWithControlledFallbackSuite

Author: Davies Liu <davies@databricks.com>

Closes #10998 from davies/gen_spilling.
2016-02-02 19:47:44 -08:00
Adam Budde ff71261b65 [SPARK-13122] Fix race condition in MemoryStore.unrollSafely()
https://issues.apache.org/jira/browse/SPARK-13122

A race condition can occur in MemoryStore's unrollSafely() method if two threads that
return the same value for currentTaskAttemptId() execute this method concurrently. This
change makes the operation of reading the initial amount of unroll memory used, performing
the unroll, and updating the associated memory maps atomic in order to avoid this race
condition.

Initial proposed fix wraps all of unrollSafely() in a memoryManager.synchronized { } block. A cleaner approach might be introduce a mechanism that synchronizes based on task attempt ID. An alternative option might be to track unroll/pending unroll memory based on block ID rather than task attempt ID.

Author: Adam Budde <budde@amazon.com>

Closes #11012 from budde/master.
2016-02-02 19:35:33 -08:00
Nong Li 21112e8a14 [SPARK-12992] [SQL] Update parquet reader to support more types when decoding to ColumnarBatch.
This patch implements support for more types when doing the vectorized decode. There are
a few more types remaining but they should be very straightforward after this. This code
has a few copy and paste pieces but they are difficult to eliminate due to performance
considerations.

Specifically, this patch adds support for:
  - String, Long, Byte types
  - Dictionary encoding for those types.

Author: Nong Li <nong@databricks.com>

Closes #10908 from nongli/spark-12992.
2016-02-02 16:33:21 -08:00
Wenchen Fan 672032d0ab [SPARK-13020][SQL][TEST] fix random generator for map type
when we generate map, we first randomly pick a length, then create a seq of key value pair with the expected length, and finally call `toMap`. However, `toMap` will remove all duplicated keys, which makes the actual map size much less than we expected.

This PR fixes this problem by put keys in a set first, to guarantee we have enough keys to build a map with expected length.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10930 from cloud-fan/random-generator.
2016-02-03 08:26:35 +08:00
Davies Liu 6de6a97728 [SPARK-13150] [SQL] disable two flaky tests
Author: Davies Liu <davies@databricks.com>

Closes #11037 from davies/disable_flaky.
2016-02-02 16:24:31 -08:00
Kevin (Sangwoo) Kim b377b03531 [DOCS] Update StructType.scala
The example will throw error like
<console>:20: error: not found: value StructType

Need to add this line:
import org.apache.spark.sql.types._

Author: Kevin (Sangwoo) Kim <sangwookim.me@gmail.com>

Closes #10141 from swkimme/patch-1.
2016-02-02 13:24:21 -08:00
Gabriele Nizzoli d0df2ca409 [SPARK-13121][STREAMING] java mapWithState mishandles scala Option
Already merged into 1.6 branch, this PR is to commit to master the same change

Author: Gabriele Nizzoli <mail@nizzoli.net>

Closes #11028 from gabrielenizzoli/patch-1.
2016-02-02 13:20:01 -08:00
Davies Liu be5dd881f1 [SPARK-12913] [SQL] Improve performance of stat functions
As benchmarked and discussed here: https://github.com/apache/spark/pull/10786/files#r50038294, benefits from codegen, the declarative aggregate function could be much faster than imperative one.

Author: Davies Liu <davies@databricks.com>

Closes #10960 from davies/stddev.
2016-02-02 11:50:14 -08:00
Reynold Xin 7f6e3ec79b [SPARK-13138][SQL] Add "logical" package prefix for ddl.scala
ddl.scala is defined in the execution package, and yet its reference of "UnaryNode" and "Command" are logical. This was fairly confusing when I was trying to understand the ddl code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11021 from rxin/SPARK-13138.
2016-02-02 11:29:20 -08:00
Grzegorz Chilkiewicz b1835d7272 [SPARK-12711][ML] ML StopWordsRemover does not protect itself from column name duplication
Fixes problem and verifies fix by test suite.
Also - adds optional parameter: nullable (Boolean) to: SchemaUtils.appendColumn
and deduplicates SchemaUtils.appendColumn functions.

Author: Grzegorz Chilkiewicz <grzegorz.chilkiewicz@codilime.com>

Closes #10741 from grzegorz-chilkiewicz/master.
2016-02-02 11:16:24 -08:00
Daoyuan Wang 358300c795 [SPARK-13056][SQL] map column would throw NPE if value is null
Jira:
https://issues.apache.org/jira/browse/SPARK-13056

Create a map like
{ "a": "somestring", "b": null}
Query like
SELECT col["b"] FROM t1;
NPE would be thrown.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #10964 from adrian-wang/npewriter.
2016-02-02 11:09:40 -08:00
Bryan Cutler cba1d6b659 [SPARK-12631][PYSPARK][DOC] PySpark clustering parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the clustering module.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #10610 from BryanCutler/param-desc-consistent-cluster-SPARK-12631.
2016-02-02 10:50:22 -08:00
hyukjinkwon b93830126c [SPARK-13114][SQL] Add a test for tokens more than the fields in schema
https://issues.apache.org/jira/browse/SPARK-13114

This PR adds a test for tokens more than the fields in schema.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11020 from HyukjinKwon/SPARK-13114.
2016-02-02 10:41:06 -08:00
Michael Armbrust 29d92181d0 [SPARK-13094][SQL] Add encoders for seq/array of primitives
Author: Michael Armbrust <michael@databricks.com>

Closes #11014 from marmbrus/seqEncoders.
2016-02-02 10:15:40 -08:00