Commit graph

55 commits

Author SHA1 Message Date
Michael Armbrust c6e041d171 [SQL] Simple framework for debugging query execution
Only records number of tuples and unique dataTypes output right now...

Example:
```scala
scala> import org.apache.spark.sql.execution.debug._
scala> hql("SELECT value FROM src WHERE key > 10").debug(sparkContext)

Results returned: 489
== Project [value#1:0] ==
Tuples output: 489
 value StringType: {java.lang.String}
== Filter (key#0:1 > 10) ==
Tuples output: 489
 value StringType: {java.lang.String}
 key IntegerType: {java.lang.Integer}
== HiveTableScan [value#1,key#0], (MetastoreRelation default, src, None), None ==
Tuples output: 500
 value StringType: {java.lang.String}
 key IntegerType: {java.lang.Integer}
```

Author: Michael Armbrust <michael@databricks.com>

Closes #1005 from marmbrus/debug and squashes the following commits:

dcc3ca6 [Michael Armbrust] Add comments.
c9dded2 [Michael Armbrust] Simple framework for debugging query execution
2014-06-09 14:24:19 -07:00
Daoyuan 0cf6002801 [SPARK-1495][SQL]add support for left semi join
Just submit another solution for #395

Author: Daoyuan <daoyuan.wang@intel.com>
Author: Michael Armbrust <michael@databricks.com>
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #837 from adrian-wang/left-semi-join-support and squashes the following commits:

d39cd12 [Daoyuan Wang] Merge pull request #1 from marmbrus/pr/837
6713c09 [Michael Armbrust] Better debugging for failed query tests.
035b73e [Michael Armbrust] Add test for left semi that can't be done with a hash join.
5ec6fa4 [Michael Armbrust] Add left semi to SQL Parser.
4c726e5 [Daoyuan] improvement according to Michael
8d4a121 [Daoyuan] add golden files for leftsemijoin
83a3c8a [Daoyuan] scala style fix
14cff80 [Daoyuan] add support for left semi join
2014-06-09 11:31:36 -07:00
Michael Armbrust a6c72ab16e [SPARK-1994][SQL] Weird data corruption bug when running Spark SQL on data in HDFS
Basically there is a race condition (possibly a scala bug?) when these values are recomputed on all of the slaves that results in an incorrect projection being generated (possibly because the GUID uniqueness contract is broken?).

In general we should probably enforce that all expression planing occurs on the driver, as is now occurring here.

Author: Michael Armbrust <michael@databricks.com>

Closes #1004 from marmbrus/fixAggBug and squashes the following commits:

e0c116c [Michael Armbrust] Compute aggregate expression during planning instead of lazily on workers.
2014-06-07 14:20:33 -07:00
Cheng Lian d000ca98a8 [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan.
In cases like `Limit` and `TakeOrdered`, `executeCollect()` makes optimizations that `execute().collect()` will not.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #939 from liancheng/spark-1958 and squashes the following commits:

bdc4a14 [Cheng Lian] Copy rows to present immutable data to users
8250976 [Cheng Lian] Added return type explicitly for public API
192a25c [Cheng Lian] [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan.
2014-06-02 12:09:43 -07:00
Cheng Lian 8f7141fbc0 [SPARK-1368][SQL] Optimized HiveTableScan
JIRA issue: [SPARK-1368](https://issues.apache.org/jira/browse/SPARK-1368)

This PR introduces two major updates:

- Replaced FP style code with `while` loop and reusable `GenericMutableRow` object in critical path of `HiveTableScan`.
- Using `ColumnProjectionUtils` to help optimizing RCFile and ORC column pruning.

My quick micro benchmark suggests these two optimizations made the optimized version 2x and 2.5x faster when scanning CSV table and RCFile table respectively:

```
Original:

[info] CSV: 27676 ms, RCFile: 26415 ms
[info] CSV: 27703 ms, RCFile: 26029 ms
[info] CSV: 27511 ms, RCFile: 25962 ms

Optimized:

[info] CSV: 13820 ms, RCFile: 10402 ms
[info] CSV: 14158 ms, RCFile: 10691 ms
[info] CSV: 13606 ms, RCFile: 10346 ms
```

The micro benchmark loads a 609MB CVS file (structurally similar to the `src` test table) into a normal Hive table with `LazySimpleSerDe` and a RCFile table, then scans these tables respectively.

Preparation code:

```scala
package org.apache.spark.examples.sql.hive

import org.apache.spark.sql.hive.LocalHiveContext
import org.apache.spark.{SparkConf, SparkContext}

object HiveTableScanPrepare extends App {
  val sparkContext = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))

  val hiveContext = new LocalHiveContext(sparkContext)

  import hiveContext._

  hql("drop table scan_csv")
  hql("drop table scan_rcfile")

  hql("""create table scan_csv (key int, value string)
        |  row format serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
        |  with serdeproperties ('field.delim'=',')
      """.stripMargin)

  hql(s"""load data local inpath "${args(0)}" into table scan_csv""")

  hql("""create table scan_rcfile (key int, value string)
        |  row format serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'
        |stored as
        |  inputformat 'org.apache.hadoop.hive.ql.io.RCFileInputFormat'
        |  outputformat 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat'
      """.stripMargin)

  hql(
    """
      |from scan_csv
      |insert overwrite table scan_rcfile
      |select scan_csv.key, scan_csv.value
    """.stripMargin)
}
```

Benchmark code:

```scala
package org.apache.spark.examples.sql.hive

import org.apache.spark.sql.hive.LocalHiveContext
import org.apache.spark.{SparkConf, SparkContext}

object HiveTableScanBenchmark extends App {
  val sparkContext = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))

  val hiveContext = new LocalHiveContext(sparkContext)

  import hiveContext._

  val scanCsv = hql("select key from scan_csv")
  val scanRcfile = hql("select key from scan_rcfile")

  val csvDuration = benchmark(scanCsv.count())
  val rcfileDuration = benchmark(scanRcfile.count())

  println(s"CSV: $csvDuration ms, RCFile: $rcfileDuration ms")

  def benchmark(f: => Unit) = {
    val begin = System.currentTimeMillis()
    f
    val end = System.currentTimeMillis()
    end - begin
  }
}
```

@marmbrus Please help review, thanks!

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #758 from liancheng/fastHiveTableScan and squashes the following commits:

4241a19 [Cheng Lian] Distinguishes sorted and possibly not sorted operations more accurately in HiveComparisonTest
cf640d8 [Cheng Lian] More HiveTableScan optimisations:
bf0e7dc [Cheng Lian] Added SortedOperation pattern to match *some* definitely sorted operations and avoid some sorting cost in HiveComparisonTest.
6d1c642 [Cheng Lian] Using ColumnProjectionUtils to optimise RCFile and ORC column pruning
eb62fd3 [Cheng Lian] [SPARK-1368] Optimized HiveTableScan
2014-05-29 15:24:03 -07:00
Aaron Davidson c3576ffcd7 [SQL] Minor: Introduce SchemaRDD#aggregate() for simple aggregations
```scala
rdd.aggregate(Sum('val))
```
is just shorthand for

```scala
rdd.groupBy()(Sum('val))
```

but seems be more natural than doing a groupBy with no grouping expressions when you really just want an aggregation over all rows.

Did not add a JavaSchemaRDD or Python API, as these seem to be lacking several other methods like groupBy() already -- leaving that cleanup for future patches.

Author: Aaron Davidson <aaron@databricks.com>

Closes #874 from aarondav/schemardd and squashes the following commits:

e9e68ee [Aaron Davidson] Add comment
db6afe2 [Aaron Davidson] Introduce SchemaRDD#aggregate() for simple aggregations
2014-05-25 18:37:44 -07:00
Reynold Xin d66642e397 SPARK-1822: Some minor cleanup work on SchemaRDD.count()
Minor cleanup following #841.

Author: Reynold Xin <rxin@apache.org>

Closes #868 from rxin/schema-count and squashes the following commits:

5442651 [Reynold Xin] SPARK-1822: Some minor cleanup work on SchemaRDD.count()
2014-05-25 01:44:49 -07:00
Kan Zhang 6052db9dc1 [SPARK-1822] SchemaRDD.count() should use query optimizer
Author: Kan Zhang <kzhang@apache.org>

Closes #841 from kanzhang/SPARK-1822 and squashes the following commits:

2f8072a [Kan Zhang] [SPARK-1822] Minor style update
cf4baa4 [Kan Zhang] [SPARK-1822] Adding Scaladoc
e67c910 [Kan Zhang] [SPARK-1822] SchemaRDD.count() should use optimizer
2014-05-25 00:06:42 -07:00
Cheng Lian 5afe6af0b1 [SPARK-1913][SQL] Bug fix: column pruning error in Parquet support
JIRA issue: [SPARK-1913](https://issues.apache.org/jira/browse/SPARK-1913)

When scanning Parquet tables, attributes referenced only in predicates that are pushed down are not passed to the `ParquetTableScan` operator and causes exception.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #863 from liancheng/spark-1913 and squashes the following commits:

f976b73 [Cheng Lian] Addessed the readability issue commented by @rxin
f5b257d [Cheng Lian] Added back comments deleted by mistake
ae60ab3 [Cheng Lian] [SPARK-1913] Attributes referenced only in predicates pushed down should remain in ParquetTableScan operator
2014-05-24 20:42:01 -07:00
Andre Schumacher 40d6acd6ba SPARK-1487 [SQL] Support record filtering via predicate pushdown in Parquet
Simple filter predicates such as LessThan, GreaterThan, etc., where one side is a literal and the other one a NamedExpression are now pushed down to the underlying ParquetTableScan. Here are some results for a microbenchmark with a simple schema of six fields of different types where most records failed the test:

             | Uncompressed  | Compressed
-------------| ------------- | -------------
File size  |     10 GB  | 2 GB
Speedup |      2         | 1.8

Since mileage may vary I added a new option to SparkConf:

`org.apache.spark.sql.parquet.filter.pushdown`

Default value would be `true` and setting it to `false` disables the pushdown. When most rows are expected to pass the filter or when there are few fields performance can be better when pushdown is disabled. The default should fit situations with a reasonable number of (possibly nested) fields where not too many records on average pass the filter.

Because of an issue with Parquet ([see here](https://github.com/Parquet/parquet-mr/issues/371])) currently only predicates on non-nullable attributes are pushed down. If one would know that for a given table no optional fields have missing values one could also allow overriding this.

Author: Andre Schumacher <andre.schumacher@iki.fi>

Closes #511 from AndreSchumacher/parquet_filter and squashes the following commits:

16bfe83 [Andre Schumacher] Removing leftovers from merge during rebase
7b304ca [Andre Schumacher] Fixing formatting
c36d5cb [Andre Schumacher] Scalastyle
3da98db [Andre Schumacher] Second round of review feedback
7a78265 [Andre Schumacher] Fixing broken formatting in ParquetFilter
a86553b [Andre Schumacher] First round of code review feedback
b0f7806 [Andre Schumacher] Optimizing imports in ParquetTestData
85fea2d [Andre Schumacher] Adding SparkConf setting to disable filter predicate pushdown
f0ad3cf [Andre Schumacher] Undoing changes not needed for this PR
210e9cb [Andre Schumacher] Adding disjunctive filter predicates
a93a588 [Andre Schumacher] Adding unit test for filtering
6d22666 [Andre Schumacher] Extending ParquetFilters
93e8192 [Andre Schumacher] First commit Parquet record filtering
2014-05-16 13:41:41 -07:00
Takuya UESHIN db8cc6f28a [SPARK-1845] [SQL] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of ...
...Scala collections.

When I execute `orderBy` or `limit` for `SchemaRDD` including `ArrayType` or `MapType`, `SparkSqlSerializer` throws the following exception:

```
com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.$colon$colon
```

or

```
com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.Vector
```

or

```
com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.HashMap$HashTrieMap
```

and so on.

This is because registrations of serializers for each concrete collections are missing in `SparkSqlSerializer`.
I believe it should use `AllScalaRegistrar`.
`AllScalaRegistrar` covers a lot of serializers for concrete classes of `Seq`, `Map` for `ArrayType`, `MapType`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #790 from ueshin/issues/SPARK-1845 and squashes the following commits:

d1ed992 [Takuya UESHIN] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of Scala collections.
2014-05-15 11:20:21 -07:00
larvaboy c33b8dcbf6 Implement ApproximateCountDistinct for SparkSql
Add the implementation for ApproximateCountDistinct to SparkSql. We use the HyperLogLog algorithm implemented in stream-lib, and do the count in two phases: 1) counting the number of distinct elements in each partitions, and 2) merge the HyperLogLog results from different partitions.

A simple serializer and test cases are added as well.

Author: larvaboy <larvaboy@gmail.com>

Closes #737 from larvaboy/master and squashes the following commits:

bd8ef3f [larvaboy] Add support of user-provided standard deviation to ApproxCountDistinct.
9ba8360 [larvaboy] Fix alignment and null handling issues.
95b4067 [larvaboy] Add a test case for count distinct and approximate count distinct.
f57917d [larvaboy] Add the parser for the approximate count.
a2d5d10 [larvaboy] Add ApproximateCountDistinct aggregates and functions.
7ad273a [larvaboy] Add SparkSql serializer for HyperLogLog.
1d9aacf [larvaboy] Fix a minor typo in the toString method of the Count case class.
653542b [larvaboy] Fix a couple of minor typos.
2014-05-13 21:26:08 -07:00
Michael Armbrust 44233865cf [SQL] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext.
Author: Michael Armbrust <michael@databricks.com>

Closes #761 from marmbrus/existingContext and squashes the following commits:

4651051 [Michael Armbrust] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext.
2014-05-13 21:23:51 -07:00
Matei Zaharia 7eefc9d2b3 SPARK-1708. Add a ClassTag on Serializer and things that depend on it
This pull request contains a rebased patch from @heathermiller (https://github.com/heathermiller/spark/pull/1) to add ClassTags on Serializer and types that depend on it (Broadcast and AccumulableCollection). Putting these in the public API signatures now will allow us to use Scala Pickling for serialization down the line without breaking binary compatibility.

One question remaining is whether we also want them on Accumulator -- Accumulator is passed as part of a bigger Task or TaskResult object via the closure serializer so it doesn't seem super useful to add the ClassTag there. Broadcast and AccumulableCollection in contrast were being serialized directly.

CC @rxin, @pwendell, @heathermiller

Author: Matei Zaharia <matei@databricks.com>

Closes #700 from mateiz/spark-1708 and squashes the following commits:

1a3d8b0 [Matei Zaharia] Use fake ClassTag in Java
3b449ed [Matei Zaharia] test fix
2209a27 [Matei Zaharia] Code style fixes
9d48830 [Matei Zaharia] Add a ClassTag on Serializer and things that depend on it
2014-05-10 12:10:24 -07:00
Takuya UESHIN 8e94d2721a [SPARK-1778] [SQL] Add 'limit' transformation to SchemaRDD.
Add `limit` transformation to `SchemaRDD`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #711 from ueshin/issues/SPARK-1778 and squashes the following commits:

33169df [Takuya UESHIN] Add 'limit' transformation to SchemaRDD.
2014-05-10 12:03:27 -07:00
Kan Zhang 967635a242 [SPARK-1460] Returning SchemaRDD instead of normal RDD on Set operations...
... that do not change schema

Author: Kan Zhang <kzhang@apache.org>

Closes #448 from kanzhang/SPARK-1460 and squashes the following commits:

111e388 [Kan Zhang] silence MiMa errors in EdgeRDD and VertexRDD
91dc787 [Kan Zhang] Taking into account newly added Ordering param
79ed52a [Kan Zhang] [SPARK-1460] Returning SchemaRDD on Set operations that do not change schema
2014-05-07 09:41:31 -07:00
Michael Armbrust 3c64750bdd [SQL] SPARK-1732 - Support for null primitive values.
I also removed a println that I bumped into.

Author: Michael Armbrust <michael@databricks.com>

Closes #658 from marmbrus/nullPrimitives and squashes the following commits:

a3ec4f3 [Michael Armbrust] Remove println.
695606b [Michael Armbrust] Support for null primatives from using scala and java reflection.
2014-05-05 22:59:42 -07:00
Cheng Lian 6d721c5f71 [SPARK-1678][SPARK-1679] In-memory compression bug fix and made compression configurable, disabled by default
In-memory compression is now configurable in `SparkConf` by the `spark.sql.inMemoryCompression.enabled` property, and is disabled by default.

To help code review, the bug fix is in [the first commit](d537a367ed), compression configuration is in [the second one](4ce09aa8aa).

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #608 from liancheng/spark-1678 and squashes the following commits:

66c3a8d [Cheng Lian] Renamed in-memory compression configuration key
f8fb3a0 [Cheng Lian] Added assertion for testing .hasNext of various decoder
4ce09aa [Cheng Lian] Made in-memory compression configurable via SparkConf
d537a36 [Cheng Lian] Fixed SPARK-1678
2014-05-05 19:38:59 -07:00
ArcherShao 9347565f41 Update SchemaRDD.scala
Modify spelling errors

Author: ArcherShao <ArcherShao@users.noreply.github.com>

Closes #619 from ArcherShao/patch-1 and squashes the following commits:

2957195 [ArcherShao] Update SchemaRDD.scala
2014-05-03 00:17:36 -07:00
Michael Armbrust 86ff8b1027 Generalize pattern for planning hash joins.
This will be helpful for [SPARK-1495](https://issues.apache.org/jira/browse/SPARK-1495) and other cases where we want to have custom hash join implementations but don't want to repeat the logic for finding the join keys.

Author: Michael Armbrust <michael@databricks.com>

Closes #418 from marmbrus/hashFilter and squashes the following commits:

d5cc79b [Michael Armbrust] Address @rxin 's comments.
366b6d9 [Michael Armbrust] style fixes
14560eb [Michael Armbrust] Generalize pattern for planning hash joins.
f4809c1 [Michael Armbrust] Move common functions to PredicateHelper.
2014-04-24 21:42:33 -07:00
Mridul Muralidharan 968c0187a1 SPARK-1586 Windows build fixes
Unfortunately, this is not exhaustive - particularly hive tests still fail due to path issues.

Author: Mridul Muralidharan <mridulm80@apache.org>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #505 from mridulm/windows_fixes and squashes the following commits:

ef12283 [Mridul Muralidharan] Move to org.apache.commons.lang3 for StringEscapeUtils. Earlier version was buggy appparently
cdae406 [Mridul Muralidharan] Remove leaked changes from > 2G fix branch
3267f4b [Mridul Muralidharan] Fix build failures
35b277a [Mridul Muralidharan] Fix Scalastyle failures
bc69d14 [Mridul Muralidharan] Change from hardcoded path separator
10c4d78 [Mridul Muralidharan] Use explicit encoding while using getBytes
1337abd [Mridul Muralidharan] fix classpath while running in windows
2014-04-24 20:48:33 -07:00
Arun Ramakrishnan 35e3d199f0 SPARK-1438 RDD.sample() make seed param optional
copying form previous pull request https://github.com/apache/spark/pull/462

Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None.

In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention.

Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params.
sample(fraction, withReplacement=false, seed=math.random)
Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it.

If backward compatible is important, 3 new method can be introduced (without default params) like this
sample(fraction)
sample(fraction, withReplacement)
sample(fraction, withReplacement, seed)

Added some tests for the scala RDD takeSample method.

Author: Arun Ramakrishnan <smartnut007@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #477 from smartnut007/master and squashes the following commits:

07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues
b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler
8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance.
69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue
0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample
2014-04-24 17:27:16 -07:00
Sandeep a03ac222d8 Fix Scala Style
Any comments are welcome

Author: Sandeep <sandeep@techaddict.me>

Closes #531 from techaddict/stylefix-1 and squashes the following commits:

7492730 [Sandeep] Pass 4
98b2428 [Sandeep] fix rxin suggestions
b5e2e6f [Sandeep] Pass 3
05932d7 [Sandeep] fix if else styling 2
08690e5 [Sandeep] fix if else styling
2014-04-24 15:07:23 -07:00
Michael Armbrust aa77f8a6a6 SPARK-1562 Fix visibility / annotation of Spark SQL APIs
Author: Michael Armbrust <michael@databricks.com>

Closes #489 from marmbrus/sqlDocFixes and squashes the following commits:

acee4f3 [Michael Armbrust] Fix visibility / annotation of Spark SQL APIs
2014-04-22 20:02:33 -07:00
Kan Zhang ea8cea82a0 [SPARK-1570] Fix classloading in JavaSQLContext.applySchema
I think I hit a class loading issue when running JavaSparkSQL example using spark-submit in local mode.

Author: Kan Zhang <kzhang@apache.org>

Closes #484 from kanzhang/SPARK-1570 and squashes the following commits:

feaaeba [Kan Zhang] [SPARK-1570] Fix classloading in JavaSQLContext.applySchema
2014-04-22 15:05:12 -07:00
Andrew Or b3e5366f69 [Fix #274] Document + fix annotation usages
... so that we don't follow an unspoken set of forbidden rules for adding **@AlphaComponent**, **@DeveloperApi**, and **@Experimental** annotations in the code.

In addition, this PR
(1) removes unnecessary `:: * ::` tags,
(2) adds missing `:: * ::` tags, and
(3) removes annotations for internal APIs.

Author: Andrew Or <andrewor14@gmail.com>

Closes #470 from andrewor14/annotations-fix and squashes the following commits:

92a7f42 [Andrew Or] Document + fix annotation usages
2014-04-21 22:24:44 -07:00
Cheng Lian 89f47434e2 Reuses Row object in ExistingRdd.productToRowRdd()
Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #432 from liancheng/reuseRow and squashes the following commits:

9e6d083 [Cheng Lian] Simplified code with BufferedIterator
52acec9 [Cheng Lian] Reuses Row object in ExistingRdd.productToRowRdd()
2014-04-18 10:02:27 -07:00
Michael Armbrust 273c2fd08d [SQL] SPARK-1424 Generalize insertIntoTable functions on SchemaRDDs
This makes it possible to create tables and insert into them using the DSL and SQL for the scala and java apis.

Author: Michael Armbrust <michael@databricks.com>

Closes #354 from marmbrus/insertIntoTable and squashes the following commits:

6c6f227 [Michael Armbrust] Create random temporary files in python parquet unit tests.
f5e6d5c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into insertIntoTable
765c506 [Michael Armbrust] Add to JavaAPI.
77b512c [Michael Armbrust] typos.
5c3ef95 [Michael Armbrust] use names for boolean args.
882afdf [Michael Armbrust] Change createTableAs to saveAsTable.  Clean up api annotations.
d07d94b [Michael Armbrust] Add tests, support for creating parquet files and hive tables.
fa3fe81 [Michael Armbrust] Make insertInto available on JavaSchemaRDD as well.  Add createTableAs function.
2014-04-15 20:40:40 -07:00
Ahir Reddy c99bcb7fea SPARK-1374: PySpark API for SparkSQL
An initial API that exposes SparkSQL functionality in PySpark. A PythonRDD composed of dictionaries, with string keys and primitive values (boolean, float, int, long, string) can be converted into a SchemaRDD that supports sql queries.

```
from pyspark.context import SQLContext
sqlCtx = SQLContext(sc)
rdd = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}])
srdd = sqlCtx.applySchema(rdd)
sqlCtx.registerRDDAsTable(srdd, "table1")
srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1")
srdd2.collect()
```
The last line yields ```[{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, {"f1" : 3, "f2": "row3"}]```

Author: Ahir Reddy <ahirreddy@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #363 from ahirreddy/pysql and squashes the following commits:

0294497 [Ahir Reddy] Updated log4j properties to supress Hive Warns
307d6e0 [Ahir Reddy] Style fix
6f7b8f6 [Ahir Reddy] Temporary fix MIMA checker. Since we now assemble Spark jar with Hive, we don't want to check the interfaces of all of our hive dependencies
3ef074a [Ahir Reddy] Updated documentation because classes moved to sql.py
29245bf [Ahir Reddy] Cache underlying SchemaRDD instead of generating and caching PythonRDD
f2312c7 [Ahir Reddy] Moved everything into sql.py
a19afe4 [Ahir Reddy] Doc fixes
6d658ba [Ahir Reddy] Remove the metastore directory created by the HiveContext tests in SparkSQL
521ff6d [Ahir Reddy] Trying to get spark to build with hive
ab95eba [Ahir Reddy] Set SPARK_HIVE=true on jenkins
ded03e7 [Ahir Reddy] Added doc test for HiveContext
22de1d4 [Ahir Reddy] Fixed maven pyrolite dependency
e4da06c [Ahir Reddy] Display message if hive is not built into spark
227a0be [Michael Armbrust] Update API links. Fix Hive example.
58e2aa9 [Michael Armbrust] Build Docs for pyspark SQL Api.  Minor fixes.
4285340 [Michael Armbrust] Fix building of Hive API Docs.
38a92b0 [Michael Armbrust] Add note to future non-python developers about python docs.
337b201 [Ahir Reddy] Changed com.clearspring.analytics stream version from 2.4.0 to 2.5.1 to match SBT build, and added pyrolite to maven build
40491c9 [Ahir Reddy] PR Changes + Method Visibility
1836944 [Michael Armbrust] Fix comments.
e00980f [Michael Armbrust] First draft of python sql programming guide.
b0192d3 [Ahir Reddy] Added Long, Double and Boolean as usable types + unit test
f98a422 [Ahir Reddy] HiveContexts
79621cf [Ahir Reddy] cleaning up cruft
b406ba0 [Ahir Reddy] doctest formatting
20936a5 [Ahir Reddy] Added tests and documentation
e4d21b4 [Ahir Reddy] Added pyrolite dependency
79f739d [Ahir Reddy] added more tests
7515ba0 [Ahir Reddy] added more tests :)
d26ec5e [Ahir Reddy] added test
e9f5b8d [Ahir Reddy] adding tests
906d180 [Ahir Reddy] added todo explaining cost of creating Row object in python
251f99d [Ahir Reddy] for now only allow dictionaries as input
09b9980 [Ahir Reddy] made jrdd explicitly lazy
c608947 [Ahir Reddy] SchemaRDD now has all RDD operations
725c91e [Ahir Reddy] awesome row objects
55d1c76 [Ahir Reddy] return row objects
4fe1319 [Ahir Reddy] output dictionaries correctly
be079de [Ahir Reddy] returning dictionaries works
cd5f79f [Ahir Reddy] Switched to using Scala SQLContext
e948bd9 [Ahir Reddy] yippie
4886052 [Ahir Reddy] even better
c0fb1c6 [Ahir Reddy] more working
043ca85 [Ahir Reddy] working
5496f9f [Ahir Reddy] doesn't crash
b8b904b [Ahir Reddy] Added schema rdd class
67ba875 [Ahir Reddy] java to python, and python to java
bcc0f23 [Ahir Reddy] Java to python
ab6025d [Ahir Reddy] compiling
2014-04-15 00:07:55 -07:00
Cheng Lian 7dbca68e92 [BUGFIX] In-memory columnar storage bug fixes
Fixed several bugs of in-memory columnar storage to make `HiveInMemoryCompatibilitySuite` pass.

@rxin @marmbrus It is reasonable to include `HiveInMemoryCompatibilitySuite` in this PR, but I didn't, since it significantly increases test execution time. What do you think?

**UPDATE** `HiveCompatibilitySuite` has been made to cache tables in memory. `HiveInMemoryCompatibilitySuite` was removed.

Author: Cheng Lian <lian.cs.zju@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #374 from liancheng/inMemBugFix and squashes the following commits:

6ad6d9b [Cheng Lian] Merged HiveCompatibilitySuite and HiveInMemoryCompatibilitySuite
5bdbfe7 [Cheng Lian] Revert 882c538 & 8426ddc, which introduced regression
882c538 [Cheng Lian] Remove attributes field from InMemoryColumnarTableScan
32cc9ce [Cheng Lian] Code style cleanup
99382bf [Cheng Lian] Enable compression by default
4390bcc [Cheng Lian] Report error for any Throwable in HiveComparisonTest
d1df4fd [Michael Armbrust] Remove test tables that might always get created anyway?
ab9e807 [Michael Armbrust] Fix the logged console version of failed test cases to use the new syntax.
1965123 [Michael Armbrust] Don't use coalesce for gathering all data to a single partition, as it does not work correctly with mutable rows.
e36cdd0 [Michael Armbrust] Spelling.
2d0e168 [Michael Armbrust] Run Hive tests in-memory too.
6360723 [Cheng Lian] Made PreInsertionCasts support SparkLogicalPlan and InMemoryColumnarTableScan
c9b0f6f [Cheng Lian] Let InsertIntoTable support InMemoryColumnarTableScan
9c8fc40 [Cheng Lian] Disable compression by default
e619995 [Cheng Lian] Bug fix: incorrect byte order in CompressionScheme.columnHeaderSize
8426ddc [Cheng Lian] Bug fix: InMemoryColumnarTableScan should cache columns specified by the attributes argument
036cd09 [Cheng Lian] Clean up unused imports
44591a5 [Cheng Lian] Bug fix: NullableColumnAccessor.hasNext must take nulls into account
052bf41 [Cheng Lian] Bug fix: should only gather compressibility info for non-null values
95b3301 [Cheng Lian] Fixed bugs in IntegralDelta
2014-04-14 15:22:43 -07:00
Patrick Wendell 4bc07eebbf SPARK-1480: Clean up use of classloaders
The Spark codebase is a bit fast-and-loose when accessing classloaders and this has caused a few bugs to surface in master.

This patch defines some utility methods for accessing classloaders. This makes the intention when accessing a classloader much more explicit in the code and fixes a few cases where the wrong one was chosen.

case (a) -> We want the classloader that loaded Spark
case (b) -> We want the context class loader, or if not present, we want (a)

This patch provides a better fix for SPARK-1403 (https://issues.apache.org/jira/browse/SPARK-1403) than the current work around, which it reverts. It also fixes a previously unreported bug that the `./spark-submit` script did not work for running with `local` master. It didn't work because the executor classloader did not properly delegate to the context class loader (if it is defined) and in local mode the context class loader is set by the `./spark-submit` script. A unit test is added for that case.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #398 from pwendell/class-loaders and squashes the following commits:

b4a1a58 [Patrick Wendell] Minor clean up
14f1272 [Patrick Wendell] SPARK-1480: Clean up use of classloaders
2014-04-13 08:58:37 -07:00
witgo a74fbbbca8 Fix SPARK-1413: Parquet messes up stdout and stdin when used in Spark REPL
Author: witgo <witgo@qq.com>

Closes #325 from witgo/SPARK-1413 and squashes the following commits:

e57cd8e [witgo] use scala reflection to access and call the SLF4JBridgeHandler  methods
45c8f40 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
5e35d87 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
0d5f819 [witgo] review commit
45e5b70 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
fa69dcf [witgo] Merge branch 'master' into SPARK-1413
3c98dc4 [witgo] Merge branch 'master' into SPARK-1413
38160cb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
ba09bcd [witgo] remove set the parquet log level
a63d574 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
5231ecd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
3feb635 [witgo] parquet logger use parent handler
fa00d5d [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
8bb6ffd [witgo] enableLogForwarding note fix
edd9630 [witgo]  move to
f447f50 [witgo] merging master
5ad52bd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
76670c1 [witgo] review commit
70f3c64 [witgo] Fix SPARK-1413
2014-04-10 10:36:20 -07:00
Patrick Wendell 87bd1f9ef7 SPARK-1093: Annotate developer and experimental API's
This patch marks some existing classes as private[spark] and adds two types of API annotations:
- `EXPERIMENTAL API` = experimental user-facing module
- `DEVELOPER API - UNSTABLE` = developer-facing API that might change

There is some discussion of the different mechanisms for doing this here:
https://issues.apache.org/jira/browse/SPARK-1081

I was pretty aggressive with marking things private. Keep in mind that if we want to open something up in the future we can, but we can never reduce visibility.

A few notes here:
- In the past we've been inconsistent with the visiblity of the X-RDD classes. This patch marks them private whenever there is an existing function in RDD that can directly creat them (e.g. CoalescedRDD and rdd.coalesce()). One trade-off here is users can't subclass them.
- Noted that compression and serialization formats don't have to be wire compatible across versions.
- Compression codecs and serialization formats are semi-private as users typically don't instantiate them directly.
- Metrics sources are made private - user only interacts with them through Spark's reflection

Author: Patrick Wendell <pwendell@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>

Closes #274 from pwendell/private-apis and squashes the following commits:

44179e4 [Patrick Wendell] Merge remote-tracking branch 'apache-github/master' into private-apis
042c803 [Patrick Wendell] spark.annotations -> spark.annotation
bfe7b52 [Patrick Wendell] Adding experimental for approximate counts
8d0c873 [Patrick Wendell] Warning in SparkEnv
99b223a [Patrick Wendell] Cleaning up annotations
e849f64 [Patrick Wendell] Merge pull request #2 from andrewor14/annotations
982a473 [Andrew Or] Generalize jQuery matching for non Spark-core API docs
a01c076 [Patrick Wendell] Merge pull request #1 from andrewor14/annotations
c1bcb41 [Andrew Or] DeveloperAPI -> DeveloperApi
0d48908 [Andrew Or] Comments and new lines (minor)
f3954e0 [Andrew Or] Add identifier tags in comments to work around scaladocs bug
99192ef [Andrew Or] Dynamically add badges based on annotations
824011b [Andrew Or] Add support for injecting arbitrary JavaScript to API docs
037755c [Patrick Wendell] Some changes after working with andrew or
f7d124f [Patrick Wendell] Small fixes
c318b24 [Patrick Wendell] Use CSS styles
e4c76b9 [Patrick Wendell] Logging
f390b13 [Patrick Wendell] Better visibility for workaround constructors
d6b0afd [Patrick Wendell] Small chang to existing constructor
403ba52 [Patrick Wendell] Style fix
870a7ba [Patrick Wendell] Work around for SI-8479
7fb13b2 [Patrick Wendell] Changes to UnionRDD and EmptyRDD
4a9e90c [Patrick Wendell] EXPERIMENTAL API --> EXPERIMENTAL
c581dce [Patrick Wendell] Changes after building against Shark.
8452309 [Patrick Wendell] Style fixes
1ed27d2 [Patrick Wendell] Formatting and coloring of badges
cd7a465 [Patrick Wendell] Code review feedback
2f706f1 [Patrick Wendell] Don't use floats
542a736 [Patrick Wendell] Small fixes
cf23ec6 [Patrick Wendell] Marking GraphX as alpha
d86818e [Patrick Wendell] Another naming change
5a76ed6 [Patrick Wendell] More visiblity clean-up
42c1f09 [Patrick Wendell] Using better labels
9d48cbf [Patrick Wendell] Initial pass
2014-04-09 01:14:46 -07:00
Cheng Lian 0d0493fcf7 [SPARK-1402] Added 3 more compression schemes
JIRA issue: [SPARK-1402](https://issues.apache.org/jira/browse/SPARK-1402)

This PR provides 3 more compression schemes for Spark SQL in-memory columnar storage:

* `BooleanBitSet`
* `IntDelta`
* `LongDelta`

Now there are 6 compression schemes in total, including the no-op `PassThrough` scheme.

Also fixed a bug in PR #286: not all compression schemes are added as available schemes when accessing an in-memory column, and when a column is compressed with an unrecognised scheme, `ColumnAccessor` throws exception.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #330 from liancheng/moreCompressionSchemes and squashes the following commits:

1d037b8 [Cheng Lian] Fixed SPARK-1436: in-memory column byte buffer must be able to be accessed multiple times
d7c0e8f [Cheng Lian] Added test suite for IntegralDelta (IntDelta & LongDelta)
3c1ad7a [Cheng Lian] Added test suite for BooleanBitSet, refactored other test suites
44fe4b2 [Cheng Lian] Refactored CompressionScheme, added 3 more compression schemes.
2014-04-07 22:24:12 -07:00
Reynold Xin 14c9238aa7 [sql] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables.
Author: Reynold Xin <rxin@apache.org>

Closes #348 from rxin/aggregate and squashes the following commits:

f4bc36f [Reynold Xin] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables.
2014-04-07 18:38:44 -07:00
Reynold Xin 83f2a2f14e [sql] Rename Expression.apply to eval for better readability.
Also used this opportunity to add a bunch of override's and made some members private.

Author: Reynold Xin <rxin@apache.org>

Closes #340 from rxin/eval and squashes the following commits:

a7c7ca7 [Reynold Xin] Fixed conflicts in merge.
9069de6 [Reynold Xin] Merge branch 'master' into eval
3ccc313 [Reynold Xin] Merge branch 'master' into eval
1a47e10 [Reynold Xin] Renamed apply to eval for generators and added a bunch of override's.
ea061de [Reynold Xin] Rename Expression.apply to eval for better readability.
2014-04-07 10:45:31 -07:00
Michael Armbrust b5bae849db [SQL] SPARK-1427 Fix toString for SchemaRDD NativeCommands.
Author: Michael Armbrust <michael@databricks.com>

Closes #343 from marmbrus/toStringFix and squashes the following commits:

37198fe [Michael Armbrust] Fix toString for SchemaRDD NativeCommands.
2014-04-07 01:46:50 -07:00
Michael Armbrust accd0999f9 [SQL] SPARK-1371 Hash Aggregation Improvements
Given:
```scala
case class Data(a: Int, b: Int)
val rdd =
  sparkContext
    .parallelize(1 to 200)
    .flatMap(_ => (1 to 50000).map(i => Data(i % 100, i)))
rdd.registerAsTable("data")
cacheTable("data")
```
Before:
```
SELECT COUNT(*) FROM data:[10000000]
16795.567ms
SELECT a, SUM(b) FROM data GROUP BY a
7536.436ms
SELECT SUM(b) FROM data
10954.1ms
```

After:
```
SELECT COUNT(*) FROM data:[10000000]
1372.175ms
SELECT a, SUM(b) FROM data GROUP BY a
2070.446ms
SELECT SUM(b) FROM data
958.969ms
```

Author: Michael Armbrust <michael@databricks.com>

Closes #295 from marmbrus/hashAgg and squashes the following commits:

ec63575 [Michael Armbrust] Add comment.
d0495a9 [Michael Armbrust] Use scaladoc instead.
b4a6887 [Michael Armbrust] Address review comments.
a2d90ba [Michael Armbrust] Capture child output statically to avoid issues with generators and serialization.
7c13112 [Michael Armbrust] Rewrite Aggregate operator to stream input and use projections.  Remove unused local RDD functions implicits.
5096f99 [Michael Armbrust] Make HiveUDAF fields transient since object inspectors are not serializable.
6a4b671 [Michael Armbrust] Add option to avoid binding operators expressions automatically.
92cca08 [Michael Armbrust] Always include serialization debug info when running tests.
1279df2 [Michael Armbrust] Increase default number of partitions.
2014-04-07 00:14:00 -07:00
Sean Owen 890d63bd4e Fix for PR #195 for Java 6
Use Java 6's recommended equivalent of Java 7's Logger.getGlobal() to retain Java 6 compatibility. See PR #195

Author: Sean Owen <sowen@cloudera.com>

Closes #334 from srowen/FixPR195ForJava6 and squashes the following commits:

f92fbd3 [Sean Owen] Use Java 6's recommended equivalent of Java 7's Logger.getGlobal() to retain Java 6 compatibility
2014-04-05 19:08:24 -07:00
Michael Armbrust d956cc2516 [SQL] Minor fixes.
Author: Michael Armbrust <michael@databricks.com>

Closes #315 from marmbrus/minorFixes and squashes the following commits:

b23a15d [Michael Armbrust] fix scaladoc
11062ac [Michael Armbrust] Fix registering "SELECT *" queries as tables and caching them.  As some tests for this and self-joins.
3997dc9 [Michael Armbrust] Move Row extractor to catalyst.
208bf5e [Michael Armbrust] More idiomatic naming of DSL functions. * subquery => as * for join condition => on, i.e., `r.join(s, condition = 'a == 'b)` =>`r.join(s, on = 'a == 'b)`
87211ce [Michael Armbrust] Correctly handle self joins of in-memory cached tables.
69e195e [Michael Armbrust] Change != to !== in the DSL since != will always translate to != on Any.
01f2dd5 [Michael Armbrust] Correctly assign aliases to tables in SqlParser.
2014-04-04 17:23:17 -07:00
Michael Armbrust d94826be6d [BUILD FIX] Fix compilation of Spark SQL Java API.
The JavaAPI and the Parquet improvements PRs didn't conflict, but broke the build.

Author: Michael Armbrust <michael@databricks.com>

Closes #316 from marmbrus/hotFixJavaApi and squashes the following commits:

0b84c2d [Michael Armbrust] Fix compilation of Spark SQL Java API.
2014-04-03 16:12:08 -07:00
Michael Armbrust b8f534196f [SQL] SPARK-1333 First draft of java API
WIP: Some work remains...
 * [x] Hive support
 * [x] Tests
 * [x] Update docs

Feedback welcome!

Author: Michael Armbrust <michael@databricks.com>

Closes #248 from marmbrus/javaSchemaRDD and squashes the following commits:

b393913 [Michael Armbrust] @srowen 's java style suggestions.
f531eb1 [Michael Armbrust] Address matei's comments.
33a1b1a [Michael Armbrust] Ignore JavaHiveSuite.
822f626 [Michael Armbrust] improve docs.
ab91750 [Michael Armbrust] Improve Java SQL API: * Change JavaRow => Row * Add support for querying RDDs of JavaBeans * Docs * Tests * Hive support
0b859c8 [Michael Armbrust] First draft of java API.
2014-04-03 15:45:34 -07:00
Andre Schumacher fbebaedf26 Spark parquet improvements
A few improvements to the Parquet support for SQL queries:
- Instead of files a ParquetRelation is now backed by a directory, which simplifies importing data from other
  sources
- InsertIntoParquetTable operation now supports switching between overwriting or appending (at least in
  HiveQL)
- tests now use the new API
- Parquet logging can be set to WARNING level (Default)
- Default compression for Parquet files (GZIP, as in parquet-mr)

Author: Andre Schumacher <andre.schumacher@iki.fi>

Closes #195 from AndreSchumacher/spark_parquet_improvements and squashes the following commits:

54df314 [Andre Schumacher] SPARK-1383 [SQL] Improvements to ParquetRelation
2014-04-03 15:31:47 -07:00
Reynold Xin ed730c9502 StopAfter / TopK related changes
1. Renamed StopAfter to Limit to be more consistent with naming in other relational databases.
2. Renamed TopK to TakeOrdered to be more consistent with Spark RDD API.
3. Avoid breaking lineage in Limit.
4. Added a bunch of override's to execution/basicOperators.scala.

@marmbrus @liancheng

Author: Reynold Xin <rxin@apache.org>
Author: Michael Armbrust <michael@databricks.com>

Closes #233 from rxin/limit and squashes the following commits:

13eb12a [Reynold Xin] Merge pull request #1 from marmbrus/limit
92b9727 [Michael Armbrust] More hacks to make Maps serialize with Kryo.
4fc8b4e [Reynold Xin] Merge branch 'master' of github.com:apache/spark into limit
87b7d37 [Reynold Xin] Use the proper serializer in limit.
9b79246 [Reynold Xin] Updated doc for Limit.
47d3327 [Reynold Xin] Copy tuples in Limit before shuffle.
231af3a [Reynold Xin] Limit/TakeOrdered: 1. Renamed StopAfter to Limit to be more consistent with naming in other relational databases. 2. Renamed TopK to TakeOrdered to be more consistent with Spark RDD API. 3. Avoid breaking lineage in Limit. 4. Added a bunch of override's to execution/basicOperators.scala.
2014-04-02 12:48:04 -07:00
Cheng Lian 1faa579711 [SPARK-1371][WIP] Compression support for Spark SQL in-memory columnar storage
JIRA issue: [SPARK-1373](https://issues.apache.org/jira/browse/SPARK-1373)

(Although tagged as WIP, this PR is structurally complete. The only things left unimplemented are 3 more compression algorithms: `BooleanBitSet`, `IntDelta` and `LongDelta`, which are trivial to add later in this or another separate PR.)

This PR contains compression support for Spark SQL in-memory columnar storage. Main interfaces include:

*   `CompressionScheme`

    Each `CompressionScheme` represents a concrete compression algorithm, which basically consists of an `Encoder` for compression and a `Decoder` for decompression. Algorithms implemented include:

    * `RunLengthEncoding`
    * `DictionaryEncoding`

    Algorithms to be implemented include:

    * `BooleanBitSet`
    * `IntDelta`
    * `LongDelta`

*   `CompressibleColumnBuilder`

    A stackable `ColumnBuilder` trait used to build byte buffers for compressible columns.  A best `CompressionScheme` that exhibits lowest compression ratio is chosen for each column according to statistical information gathered while elements are appended into the `ColumnBuilder`. However, if no `CompressionScheme` can achieve a compression ratio better than 80%, no compression will be done for this column to save CPU time.

    Memory layout of the final byte buffer is showed below:

    ```
     .--------------------------- Column type ID (4 bytes)
     |   .----------------------- Null count N (4 bytes)
     |   |   .------------------- Null positions (4 x N bytes, empty if null count is zero)
     |   |   |     .------------- Compression scheme ID (4 bytes)
     |   |   |     |   .--------- Compressed non-null elements
     V   V   V     V   V
    +---+---+-----+---+---------+
    |   |   | ... |   | ... ... |
    +---+---+-----+---+---------+
     \-----------/ \-----------/
        header         body
    ```

*   `CompressibleColumnAccessor`

    A stackable `ColumnAccessor` trait used to iterate (possibly) compressed data column.

*   `ColumnStats`

    Used to collect statistical information while loading data into in-memory columnar table. Optimizations like partition pruning rely on this information.

    Strictly speaking, `ColumnStats` related code is not part of the compression support. It's contained in this PR to ensure and validate the row-based API design (which is used to avoid boxing/unboxing cost whenever possible).

A major refactoring change since PR #205 is:

* Refactored all getter/setter methods for primitive types in various places into `ColumnType` classes to remove duplicated code.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #285 from liancheng/memColumnarCompression and squashes the following commits:

ed71bbd [Cheng Lian] Addressed all PR comments by @marmbrus
d3a4fa9 [Cheng Lian] Removed Ordering[T] in ColumnStats for better performance
5034453 [Cheng Lian] Bug fix, more tests, and more refactoring
c298b76 [Cheng Lian] Test suites refactored
2780d6a [Cheng Lian] [WIP] in-memory columnar compression support
211331c [Cheng Lian] WIP: in-memory columnar compression support
85cc59b [Cheng Lian] Refactored ColumnAccessors & ColumnBuilders to remove duplicate code
2014-04-02 12:47:22 -07:00
Michael Armbrust f5c418da04 [SQL] SPARK-1372 Support for caching and uncaching tables in a SQLContext.
This doesn't yet support different databases in Hive (though you can probably workaround this by calling `USE <dbname>`).  However, given the time constraints for 1.0 I think its probably worth including this now and extending the functionality in the next release.

Author: Michael Armbrust <michael@databricks.com>

Closes #282 from marmbrus/cacheTables and squashes the following commits:

83785db [Michael Armbrust] Support for caching and uncaching tables in a SQLContext.
2014-04-01 14:45:44 -07:00
Michael Armbrust 5731af5be6 [SQL] Rewrite join implementation to allow streaming of one relation.
Before we were materializing everything in memory.  This also uses the projection interface so will be easier to plug in code gen (its ported from that branch).

@rxin @liancheng

Author: Michael Armbrust <michael@databricks.com>

Closes #250 from marmbrus/hashJoin and squashes the following commits:

1ad873e [Michael Armbrust] Change hasNext logic back to the correct version.
8e6f2a2 [Michael Armbrust] Review comments.
1e9fb63 [Michael Armbrust] style
bc0cb84 [Michael Armbrust] Rewrite join implementation to allow streaming of one relation.
2014-03-31 15:23:46 -07:00
Prashant Sharma d666053679 SPARK-1352 - Comment style single space before ending */ check.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #261 from ScrapCodes/comment-style-check2 and squashes the following commits:

6cde61e [Prashant Sharma] comment style space before ending */ check.
2014-03-30 10:06:56 -07:00
Michael Armbrust 2861b07bb0 [SQL] SPARK-1354 Fix self-joins of parquet relations
@AndreSchumacher, please take a look.

https://spark-project.atlassian.net/browse/SPARK-1354

Author: Michael Armbrust <michael@databricks.com>

Closes #269 from marmbrus/parquetJoin and squashes the following commits:

4081e77 [Michael Armbrust] Create new instances of Parquet relation when multiple copies are in a single plan.
2014-03-29 22:02:53 -07:00
Michael Armbrust e15e57413e [SQL] Add a custom serializer for maps since they do not have a no-arg constructor.
Author: Michael Armbrust <michael@databricks.com>

Closes #243 from marmbrus/mapSer and squashes the following commits:

54045f7 [Michael Armbrust] Add a custom serializer for maps since they do not have a no-arg constructor.
2014-03-26 18:19:49 -07:00