SPARK-1251 Support for optimizing and executing structured queries

This pull request adds support to Spark for working with structured data using a simple SQL dialect, HiveQL and a Scala Query DSL.

*This is being contributed as a new __alpha component__ to Spark and does not modify Spark core or other components.*

The code is broken into three primary components:
 - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions.
 - Execution (sql/core) - A query planner / execution engine for translating Catalyst’s logical query plans into Spark RDDs.  This component also includes a new public interface, SqlContext, that allows users to execute SQL or structured scala queries against existing RDDs and Parquet files.
 - Hive Metastore Support (sql/hive) - An extension of SqlContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes.  There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs.

A more complete design of this new component can be found in [the associated JIRA](https://spark-project.atlassian.net/browse/SPARK-1251).

[An updated version of the Spark documentation, including API Docs for all three sub-components,](http://people.apache.org/~pwendell/catalyst-docs/sql-programming-guide.html) is also available for review.

With this PR comes support for inferring the schema of existing RDDs that contain case classes.  Using this information, developers can now express structured queries that are automatically compiled into RDD operations.

```scala
// Define the schema using a case class.
case class Person(name: String, age: Int)
val people: RDD[Person] =
  sc.textFile("people.txt").map(_.split(",")).map(p => Person(p(0), p(1).toInt))

// The following is the same as 'SELECT name FROM people WHERE age >= 10 && age <= 19'
val teenagers = people.where('age >= 10).where('age <= 19).select('name).toRdd
```

RDDs can also be registered as Tables, allowing SQL queries to be written over them.
```scala
people.registerAsTable("people")
val teenagers = sql("SELECT name FROM people WHERE age >= 10 && age <= 19")
```

The results of queries are themselves RDDs and support standard RDD operations:
```scala
teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
```

Finally, with the optional Hive support, users can read and write data located in existing Apache Hive deployments using HiveQL.
```scala
sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src")

// Queries are expressed in HiveQL
sql("SELECT key, value FROM src").collect().foreach(println)
```

## Relationship to Shark

Unlike Shark, Spark SQL does not act as a drop in replacement for Hive or the HiveServer. Instead this new feature is intended to make it easier for Spark developers to run queries over structured data, using either SQL or the query DSL. After this sub-project graduates from Alpha status it will likely become a new optimizer/backend for the Shark project.

Author: Michael Armbrust <michael@databricks.com>
Author: Yin Huai <huaiyin.thu@gmail.com>
Author: Reynold Xin <rxin@apache.org>
Author: Lian, Cheng <rhythm.mail@gmail.com>
Author: Andre Schumacher <andre.schumacher@iki.fi>
Author: Yin Huai <huai@cse.ohio-state.edu>
Author: Timothy Chen <tnachen@gmail.com>
Author: Cheng Lian <lian.cs.zju@gmail.com>
Author: Timothy Chen <tnachen@apache.org>
Author: Henry Cook <henry.m.cook+github@gmail.com>
Author: Mark Hamstra <markhamstra@gmail.com>

Closes #146 from marmbrus/catalyst and squashes the following commits:

458bd1b [Michael Armbrust] Update people.txt
0d638c3 [Michael Armbrust] Typo fix from @ash211.
bdab185 [Michael Armbrust] Address another round of comments: * Doc examples can now copy/paste into spark-shell. * SQLContext is serializable * Minor parser bugs fixed * Self-joins of RDDs now handled correctly. * Removed deprecated examples * Removed deprecated parquet docs * Made more of the API private * Copied all the DSLQuery tests and rewrote them as SQLQueryTests
778299a [Michael Armbrust] Fix some old links to spark-project.org
fead0b6 [Michael Armbrust] Create a new RDD type, SchemaRDD, that is now the return type for all SQL operations.  This improves the old API by reducing the number of implicits that are required, and avoids throwing away schema information when returning an RDD to the user.  This change also makes it slightly less verbose to run language integrated queries.
fee847b [Michael Armbrust] Merge remote-tracking branch 'origin/master' into catalyst, integrating changes to serialization for ShuffledRDD.
48a99bc [Michael Armbrust] Address first round of feedback.
461581c [Michael Armbrust] Blacklist test that depends on JVM specific rounding behaviour
adcf1a4 [Henry Cook] Update sql-programming-guide.md
9dffbfa [Michael Armbrust] Style fixes. Add downloading of test cases to jenkins.
6978dd8 [Michael Armbrust] update docs, add apache license
1d0eb63 [Michael Armbrust] update changes with spark core
e5e1d6b [Michael Armbrust] Remove travis configuration.
c2efad6 [Michael Armbrust] First draft of SQL documentation.
013f62a [Michael Armbrust] Fix documentation / code style.
c01470f [Michael Armbrust] Clean up example
2f22454 [Michael Armbrust] WIP: Parquet example.
ce8073b [Michael Armbrust] clean up implicits.
f7d992d [Michael Armbrust] Naming / spelling.
9eb0294 [Michael Armbrust] Bring expressions implicits into SqlContext.
d2d9678 [Michael Armbrust] Make sure hive isn't in the assembly jar.  Create a separate, optional Hive assembly that is used when present.
8b35e0a [Michael Armbrust] address feedback, work on DSL
5d71074 [Michael Armbrust] Merge pull request #62 from AndreSchumacher/parquet_file_fixes
f93aa39 [Andre Schumacher] Better handling of path names in ParquetRelation
1a4bbd9 [Michael Armbrust] Merge pull request #60 from marmbrus/maven
3386e4f [Michael Armbrust] Merge pull request #58 from AndreSchumacher/parquet_fixes
3447c3e [Michael Armbrust] Don't override the metastore / warehouse in non-local/test hive context.
7233a74 [Michael Armbrust] initial support for maven builds
f0ba39e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into maven
7386a9f [Michael Armbrust] Initial example programs using spark sql.
aeaef54 [Andre Schumacher] Removing unnecessary Row copying and reverting some changes to MutableRow
7ca4b4e [Andre Schumacher] Improving checks in Parquet tests
5bacdc0 [Andre Schumacher] Moving towards mutable rows inside ParquetRowSupport
54637ec [Andre Schumacher] First part of second round of code review feedback
c2a658d [Michael Armbrust] Merge pull request #55 from marmbrus/mutableRows
ba28849 [Michael Armbrust] code review comments.
d994333 [Michael Armbrust] Remove copies before shuffle, this required changing the default shuffle serialization.
9049cf0 [Michael Armbrust] Extend MutablePair interface to support easy syntax for in-place updates.  Also add a constructor so that it can be serialized out-of-the-box.
959bdf0 [Michael Armbrust] Don't silently swallow all KryoExceptions, only the one that indicates the end of a stream.
d371393 [Michael Armbrust] Add a framework for dealing with mutable rows to reduce the number of object allocations that occur in the critical path.
c9f8fb3 [Michael Armbrust] Merge pull request #53 from AndreSchumacher/parquet_support
3c3f962 [Michael Armbrust] Fix a bug due to array reuse.  This will need to be revisited after we merge the mutable row PR.
7d0f13e [Michael Armbrust] Update parquet support with master.
9d419a6 [Michael Armbrust] Merge remote-tracking branch 'catalyst/catalystIntegration' into parquet_support
0040ae6 [Andre Schumacher] Feedback from code review
1ce01c7 [Michael Armbrust] Merge pull request #56 from liancheng/unapplySeqForRow
70e489d [Cheng Lian] Fixed a spelling typo
6d315bb [Cheng Lian] Added Row.unapplySeq to extract fields from a Row object.
8d5da5e [Michael Armbrust] modify compute-classpath.sh to include datanucleus jars explicitly
99e61fb [Michael Armbrust] Merge pull request #51 from marmbrus/expressionEval
7b9d142 [Michael Armbrust] Update travis to increase permgen size.
da9afbd [Michael Armbrust] Add byte wrappers for hive UDFS.
6fdefe6 [Michael Armbrust] Port sbt improvements from master.
296fe50 [Michael Armbrust] Address review feedback.
d7fbc3a [Michael Armbrust] Several performance enhancements and simplifications of the expression evaluation framework.
3bda72d [Andre Schumacher] Adding license banner to new files
3ac9eb0 [Andre Schumacher] Rebasing to new main branch
c863bed [Andre Schumacher] Codestyle checks
61e3bfb [Andre Schumacher] Adding WriteToFile operator and rewriting ParquetQuerySuite
3321195 [Andre Schumacher] Fixing one import in ParquetQueryTests.scala
3a0a552 [Andre Schumacher] Reorganizing Parquet table operations
18fdc44 [Andre Schumacher] Reworking Parquet metadata in relation and adding CREATE TABLE AS for Parquet tables
75262ee [Andre Schumacher] Integrating operations on Parquet files into SharkStrategies
f347273 [Andre Schumacher] Adding ParquetMetaData extraction, fixing schema projection
6a6bf98 [Andre Schumacher] Added column projections to ParquetTableScan
0f17d7b [Andre Schumacher] Rewriting ParquetRelation tests with RowWriteSupport
a11e364 [Andre Schumacher] Adding Parquet RowWriteSupport
6ad05b3 [Andre Schumacher] Moving ParquetRelation to spark.sql core
eb0e521 [Andre Schumacher] Fixing package names and other problems that came up after the rebase
99a9209 [Andre Schumacher] Expanding ParquetQueryTests to cover all primitive types
b33e47e [Andre Schumacher] First commit of Parquet import of primitive column types
c334386 [Michael Armbrust] Initial support for generating schema's based on case classes.
608a29e [Michael Armbrust] Add hive as a repl dependency
7413ac2 [Michael Armbrust] make test downloading quieter.
4d57d0e [Michael Armbrust] Fix test execution on travis.
5f2963c [Michael Armbrust] naming and continuous compilation fixes.
f5e7492 [Michael Armbrust] Add Apache license.  Make naming more consistent.
3ac9416 [Michael Armbrust] Merge support for working with schema-ed RDDs using catalyst in as a spark subproject.
2225431 [Michael Armbrust] Merge pull request #48 from marmbrus/minorFixes
d393d2a [Michael Armbrust] Review Comments: Add comment to map that adds a sub query.
24eaa79 [Michael Armbrust] fix > 100 chars
6e04e5b [Michael Armbrust] Add insertIntoTable to the DSL.
df88f01 [Michael Armbrust] add a simple test for aggregation
18a861b [Michael Armbrust] Correctly convert nested products into nested rows when turning scala data into catalyst data.
b922511 [Michael Armbrust] Fix insertion of nested types into hive tables.
5fe7de4 [Michael Armbrust] Move table creation out of rule into a separate function.
a430895 [Michael Armbrust] Planning for logical Repartition operators.
532dd37 [Michael Armbrust] Allow the local warehouse path to be specified.
4905b2b [Michael Armbrust] Add more efficient TopK that avoids global sort for logical Sort => StopAfter.
8c01c24 [Michael Armbrust] Move definition of Row out of execution to top level sql package.
c9116a6 [Michael Armbrust] Add combiner to avoid NPE when spark performs external aggregation.
29effad [Michael Armbrust] Include alias in attributes that are produced by overridden tables.
9990ec7 [Michael Armbrust] Merge pull request #28 from liancheng/columnPruning
f22df3a [Michael Armbrust] Merge pull request #37 from yhuai/SerDe
cf4db59 [Lian, Cheng] Added golden answers for PruningSuite
54f165b [Lian, Cheng] Fixed spelling typo in two golden answer file names
2682f72 [Lian, Cheng] Merge remote-tracking branch 'origin/master' into columnPruning
c5a4fab [Lian, Cheng] Merge branch 'master' into columnPruning
f670c8c [Yin Huai] Throw a NotImplementedError for not supported clauses in a CTAS query.
128a9f8 [Yin Huai] Minor changes.
017872c [Yin Huai] Remove stats20 from whitelist.
a1a4776 [Yin Huai] Update comments.
feb022c [Yin Huai] Partitioning key should be case insensitive.
555fb1d [Yin Huai] Correctly set the extension for a text file.
d00260b [Yin Huai] Strips backticks from partition keys.
334aace [Yin Huai] New golden files.
a40d6d6 [Yin Huai] Loading the static partition specified in a INSERT INTO/OVERWRITE query.
428aff5 [Yin Huai] Distinguish `INSERT INTO` and `INSERT OVERWRITE`.
eea75c5 [Yin Huai] Correctly set codec.
45ffb86 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew
e089627 [Yin Huai] Code style.
563bb22 [Yin Huai] Set compression info in FileSinkDesc.
35c9a8a [Michael Armbrust] Merge pull request #46 from marmbrus/reviewFeedback
bdab5ed [Yin Huai] Add a TODO for loading data into partitioned tables.
5495fab [Yin Huai] Remove cloneRecords which is no longer needed.
1596e1b [Yin Huai] Cleanup imports to make IntelliJ happy.
3bb272d [Michael Armbrust] move org.apache.spark.sql package.scala to the correct location.
8506c17 [Michael Armbrust] Address review feedback.
3cb4f2e [Michael Armbrust] Merge pull request #45 from tnachen/master
9ad474d [Michael Armbrust] Merge pull request #44 from marmbrus/sampling
566fd66 [Timothy Chen] Whitelist tests and add support for Binary type
69adf72 [Yin Huai] Set cloneRecords to false.
a9c3188 [Timothy Chen] Fix udaf struct return
346f828 [Yin Huai] Move SharkHadoopWriter to the correct location.
59e37a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew
ed3a1d1 [Yin Huai] Load data directly into Hive.
7f206b5 [Michael Armbrust] Add support for hive TABLESAMPLE PERCENT.
b6de691 [Michael Armbrust] Merge pull request #43 from liancheng/fixMakefile
1f6260d [Lian, Cheng] Fixed package name and test suite name in Makefile
5ae010f [Michael Armbrust] Merge pull request #42 from markhamstra/non-ascii
678341a [Mark Hamstra] Replaced non-ascii text
887f928 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew
1f7d00a [Reynold Xin] Merge pull request #41 from marmbrus/splitComponents
7588a57 [Michael Armbrust] Break into 3 major components and move everything into the org.apache.spark.sql package.
bc9a12c [Michael Armbrust] Move hive test files.
5720d2b [Lian, Cheng] Fixed comment typo
f0c3742 [Lian, Cheng] Refactored PhysicalOperation
f235914 [Lian, Cheng] Test case udf_regex and udf_like need BooleanWritable registered
cf691df [Lian, Cheng] Added the PhysicalOperation to generalize ColumnPrunings
2407a21 [Lian, Cheng] Added optimized logical plan to debugging output
a7ad058 [Michael Armbrust] Merge pull request #40 from marmbrus/includeGoldens
9329820 [Michael Armbrust] add golden answer files to repository
dce0593 [Michael Armbrust] move golden answer to the source code directory.
964368f [Michael Armbrust] Merge pull request #39 from marmbrus/lateralView
7785ee6 [Michael Armbrust] Tighten visibility based on comments.
341116c [Michael Armbrust] address comments.
0e6c1d7 [Reynold Xin] Merge pull request #38 from yhuai/parseDBNameInCTAS
2897deb [Michael Armbrust] fix scaladoc
7123225 [Yin Huai] Correctly parse the db name and table name in INSERT queries.
b376d15 [Michael Armbrust] fix newlines at EOF
5cc367c [Michael Armbrust] use berkeley instead of cloudbees
ff5ea3f [Michael Armbrust] new golden
db92adc [Michael Armbrust] more tests passing. clean up logging.
740febb [Michael Armbrust] Tests for tgfs.
0ce61b0 [Michael Armbrust] Docs for GenericHiveUdtf.
ba8897f [Michael Armbrust] Merge remote-tracking branch 'yin/parseDBNameInCTAS' into lateralView
dd00b7e [Michael Armbrust] initial implementation of generators.
ea76cf9 [Michael Armbrust] Add NoRelation to planner.
bea4b7f [Michael Armbrust] Add SumDistinct.
016b489 [Michael Armbrust] fix typo.
acb9566 [Michael Armbrust] Correctly type attributes of CTAS.
8841eb8 [Michael Armbrust] Rename Transform -> ScriptTransformation.
02ff8e4 [Yin Huai] Correctly parse the db name and table name in a CTAS query.
5e4d9b4 [Michael Armbrust] Merge pull request #35 from marmbrus/smallFixes
5479066 [Reynold Xin] Merge pull request #36 from marmbrus/partialAgg
8017afb [Michael Armbrust] fix copy paste error.
dc6353b [Michael Armbrust] turn off deprecation
cab1a84 [Michael Armbrust] Fix PartialAggregate inheritance.
883006d [Michael Armbrust] improve tests.
32b615b [Michael Armbrust] add override to asPartial.
e1999f9 [Yin Huai] Use Deserializer and Serializer instead of AbstractSerDe.
f94345c [Michael Armbrust] fix doc link
d8cb805 [Michael Armbrust] Implement partial aggregation.
ccdb07a [Michael Armbrust] Fix bug where averages of strings are turned into sums of strings.  Remove a blank line.
b4be6a5 [Michael Armbrust] better logging when applying rules.
67128b8 [Reynold Xin] Merge pull request #30 from marmbrus/complex
cb57459 [Michael Armbrust] blacklist machine specific test.
2f27604 [Michael Armbrust] Address comments / style errors.
389525d [Michael Armbrust] update golden, blacklist mr.
e3c10bd [Michael Armbrust] update whitelist.
44d343c [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into complex
42ec4af [Michael Armbrust] improve complex type support in hive udfs/udafs.
ab5bff3 [Michael Armbrust] Support for get item of map types.
1679554 [Michael Armbrust] add toString for if and IS NOT NULL.
ab9a131 [Michael Armbrust] when UDFs fail they should return null.
25288d0 [Michael Armbrust] Implement [] for arrays and maps.
e7933e9 [Michael Armbrust] fix casting bug when working with fractional expressions.
010accb [Michael Armbrust] add tinyint to metastore type parser.
7a0f543 [Michael Armbrust] Avoid propagating types from unresolved nodes.
ac9d7de [Michael Armbrust] Resolve *s in Transform clauses.
692a477 [Michael Armbrust] Support for wrapping arrays to be written into hive tables.
92e4158 [Reynold Xin] Merge pull request #32 from marmbrus/tooManyProjects
9c06778 [Michael Armbrust] fix serialization issues, add JavaStringObjectInspector.
72a003d [Michael Armbrust] revert regex change
7661b6c [Michael Armbrust] blacklist machines specific tests
aa430e7 [Michael Armbrust] Update .travis.yml
e4def6b [Michael Armbrust] set dataType for HiveGenericUdfs.
5e54aa6 [Michael Armbrust] quotes for struct field names.
bbec500 [Michael Armbrust] update test coverage, new golden
3734a94 [Michael Armbrust] only quote string types.
3f9e519 [Michael Armbrust] use names w/ boolean args
5b3d2c8 [Michael Armbrust] implement distinct.
5b33216 [Michael Armbrust] work on decimal support.
2c6deb3 [Michael Armbrust] improve printing compatibility.
35a70fb [Michael Armbrust] multi-letter field names.
a9388fb [Michael Armbrust] printing for map types.
c3feda7 [Michael Armbrust] use toArray.
c654f19 [Michael Armbrust] Support for list and maps in hive table scan.
cf8d992 [Michael Armbrust] Use built in functions for creating temp directory.
1579eec [Michael Armbrust] Only cast unresolved inserts.
6420c7c [Michael Armbrust] Memoize the ordinal in the GetField expression.
da7ae9d [Michael Armbrust] Add boolean writable that was breaking udf_regexp test.  Not sure how this was passing before...
6709441 [Michael Armbrust] Evaluation for accessing nested fields.
dc6463a [Michael Armbrust] Support for resolving access to nested fields using "." notation.
d670e41 [Michael Armbrust] Print nested fields like hive does.
efa7217 [Michael Armbrust] Support for reading structs in HiveTableScan.
9c22b4e [Michael Armbrust] Support for parsing nested types.
82163e3 [Michael Armbrust] special case handling of partitionKeys when casting insert into tables
ea6f37f [Michael Armbrust] fix style.
7845364 [Michael Armbrust] deactivate concurrent test.
b649c20 [Michael Armbrust] fix test logging / caching.
1590568 [Michael Armbrust] add log4j.properties
19bfd74 [Michael Armbrust] store hive output in circular buffer
dfb67aa [Michael Armbrust] add test case
cb775ac [Michael Armbrust] get rid of SharkContext singleton
2de89d0 [Michael Armbrust] Merge pull request #13 from tnachen/master
63003e9 [Michael Armbrust] Fix spacing.
41b41f3 [Michael Armbrust] Only cast unresolved inserts.
6eb5960 [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into udafs
5b7afd8 [Michael Armbrust] Merge pull request #10 from yhuai/exchangeOperator
b1151a8 [Timothy Chen] Fix load data regex
8e0931f [Michael Armbrust] Cast to avoid using deprecated hive API.
e079f2b [Timothy Chen] Add GenericUDAF wrapper and HiveUDAFFunction
45b334b [Yin Huai] fix comments
235cbb4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator
fc67b50 [Yin Huai] Check for a Sort operator with the global flag set instead of an Exchange operator with a RangePartitioning.
6015f93 [Michael Armbrust] Merge pull request #29 from rxin/style
271e483 [Michael Armbrust] Update build status icon.
d3a3d48 [Michael Armbrust] add testing to travis
807b2d7 [Michael Armbrust] check style and publish docs with travis
d20b565 [Michael Armbrust] fix if style
bce024d [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into style Disable if brace checking as it errors in single line functional cases unlike the style guide.
d91e276 [Michael Armbrust] Remove dependence on HIVE_HOME for running tests.  This was done by moving all the hive query test (from branch-0.12) and data files into src/test/hive.  These are used by default when HIVE_HOME is not set.
f47c2f6 [Yin Huai] set outputPartitioning in BroadcastNestedLoopJoin
41bbee6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator
7e24436 [Reynold Xin] Removed dependency on JDK 7 (nio.file).
5c1e600 [Reynold Xin] Added hash code implementation for AttributeReference
7213a2c [Reynold Xin] style fix for Hive.scala.
08e4d05 [Reynold Xin] First round of style cleanup.
605255e [Reynold Xin] Added scalastyle checker.
61e729c [Lian, Cheng] Added ColumnPrunings strategy and test cases
2486fb7 [Lian, Cheng] Fixed spelling
8ee41be [Lian, Cheng] Minor refactoring
ebb56fa [Michael Armbrust] add travis config
4c89d6e [Reynold Xin] Merge pull request #27 from marmbrus/moreTests
d4f539a [Michael Armbrust] blacklist mr and user specific tests.
677eb07 [Michael Armbrust] Update test whitelist.
5dab0bc [Michael Armbrust] Merge pull request #26 from liancheng/serdeAndPartitionPruning
c263c84 [Michael Armbrust] Only push predicates into partitioned table scans.
ab77882 [Michael Armbrust] upgrade spark to RC5.
c98ede5 [Lian, Cheng] Response to comments from @marmbrus
83d4520 [Yin Huai] marmbrus's comments
70994a3 [Lian, Cheng] Revert unnecessary Scaladoc changes
9ebff47 [Yin Huai] remove unnecessary .toSeq
e811d1a [Yin Huai] markhamstra's comments
4802f69 [Yin Huai] The outputPartitioning of a UnaryNode inherits its child's outputPartitioning by default. Also, update the logic in AddExchange to avoid unnecessary shuffling operations.
040fbdf [Yin Huai] AddExchange is the only place to add Exchange operators.
9fb357a [Yin Huai] use getSpecifiedDistribution to create Distribution. ClusteredDistribution and OrderedDistribution do not take Nil as inptu expressions.
e9347fc [Michael Armbrust] Remove broken scaladoc links.
99c6707 [Michael Armbrust] upgrade spark
57799ad [Lian, Cheng] Added special treat for HiveVarchar in InsertIntoHiveTable
cb49af0 [Lian, Cheng] Fixed Scaladoc links
4e5e4d4 [Lian, Cheng] Added PreInsertionCasts to do necessary casting before insertion
111ffdc [Lian, Cheng] More comments and minor reformatting
9e0d840 [Lian, Cheng] Added partition pruning optimization
761bbb8 [Lian, Cheng] Generalized BindReferences to run against any query plan
04eb5da [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator
9dd3b26 [Michael Armbrust] Fix scaladoc.
6f44cac [Lian, Cheng] Made TableReader & HadoopTableReader private to catalyst
7c92a41 [Lian, Cheng] Added Hive SerDe support
ce5fdd6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator
2957f31 [Yin Huai] addressed comments on PR
907db68 [Michael Armbrust] Space after while.
04573a0 [Reynold Xin] Merge pull request #24 from marmbrus/binaryCasts
4e50679 [Reynold Xin] Merge pull request #25 from marmbrus/rowOrderingWhile
5bc1dc2 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator
be1fff7 [Michael Armbrust] Replace foreach with while in RowOrdering. Fixes #23
fd084a4 [Michael Armbrust] implement casts binary <=> string.
0b31176 [Michael Armbrust] Merge pull request #22 from rxin/type
548e479 [Yin Huai] merge master into exchangeOperator and fix code style
5b11db0 [Reynold Xin] Added Void to Boolean type widening.
9e3d989 [Reynold Xin] Made HiveTypeCoercion.WidenTypes more clear.
9bb1979 [Reynold Xin] Merge pull request #19 from marmbrus/variadicUnion
a2beb38 [Michael Armbrust] Merge pull request #21 from liancheng/fixIssue20
b20a4d4 [Lian, Cheng] Fix issue #20
6d6cb58 [Michael Armbrust] add source links that point to github to the scala doc.
4285962 [Michael Armbrust] Remove temporary test cases
167162f [Michael Armbrust] more merge errors, cleanup.
e170ccf [Michael Armbrust] Improve documentation and remove some spurious changes that were introduced by the merge.
6377d0b [Michael Armbrust] Drop empty files, fix if ().
c0b0e60 [Michael Armbrust] cleanup broken doc links.
330a88b [Michael Armbrust] Fix bugs in AddExchange.
4f345f2 [Michael Armbrust] Remove SortKey, use RowOrdering.
043e296 [Michael Armbrust] Make physical union nodes variadic.
ece15e1 [Michael Armbrust] update unit tests
5c89d2e [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into exchangeOperator Fix deprecated use of combineValuesByKey. Get rid of test where the answer is dependent on the plan execution width.
9804eb5 [Michael Armbrust] upgrade spark
053a371 [Michael Armbrust] Merge pull request #15 from marmbrus/orderedRow
5ab18be [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into orderedRow
ca2ff68 [Michael Armbrust] Merge pull request #17 from marmbrus/unionTypes
bf9161c [Michael Armbrust] Merge pull request #18 from marmbrus/noSparkAgg
563053f [Michael Armbrust] Address @rxin's comments.
6537c66 [Michael Armbrust] Address @rxin's comments.
2a76fc6 [Michael Armbrust] add notes from @rxin.
685bfa1 [Michael Armbrust] fix spelling
69ed98f [Michael Armbrust] Output a single row for empty Aggregations with no grouping expressions.
7859a86 [Michael Armbrust] Remove SparkAggregate.  Its kinda broken and breaks RDD lineage.
fc22e01 [Michael Armbrust] whitelist newly passing union test.
3f547b8 [Michael Armbrust] Add support for widening types in unions.
53b95f8 [Michael Armbrust] coercion should not occur until children are resolved.
b892e32 [Michael Armbrust] Union is not resolved until the types match up.
95ab382 [Michael Armbrust] Use resolved instead of custom function.  This is better because some nodes override the notion of resolved.
81a109d [Michael Armbrust] fix link.
f143f61 [Michael Armbrust] Implement sampling.  Fixes a flaky test where the JVM notices that RAND as a Comparison method "violates its general contract!"
6cd442b [Michael Armbrust] Use numPartitions variable, fix grammar.
c800798 [Michael Armbrust] Add build status icon.
0cf5a75 [Michael Armbrust] Merge pull request #16 from marmbrus/filterPushDown
05d3a0d [Michael Armbrust] Refactor to avoid serializing ordering details with every row.
f2fdd77 [Michael Armbrust] fix required distribtion for aggregate.
658866e [Michael Armbrust] Pull back in changes made by @yhuai eliminating CoGroupedLocallyRDD.scala
583a337 [Michael Armbrust] break apart distribution and partitioning.
e8d41a9 [Michael Armbrust] Merge remote-tracking branch 'yin/exchangeOperator' into exchangeOperator
0ff8be7 [Michael Armbrust] Cleanup spurious changes and fix doc links.
73c70de [Yin Huai] add a first set of unit tests for data properties.
fbfa437 [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into filterPushDown Minor doc improvements.
2b9d80f [Yin Huai] initial commit of adding exchange operators to physical plans.
fcbc03b [Michael Armbrust] Fix if ().
7b9080c [Michael Armbrust] Create OrderedRow class to allow ordering to be used by multiple operators.
b4adb0f [Michael Armbrust] Merge pull request #14 from marmbrus/castingAndTypes
b2a1ec5 [Michael Armbrust] add comment on how using numeric implicitly complicates spark serialization.
e286d20 [Michael Armbrust] address code review comments.
80d0681 [Michael Armbrust] fix scaladoc links.
de0c248 [Michael Armbrust] Print the executed plan in SharkQuery toString.
3413e61 [Michael Armbrust] Add mapChildren and withNewChildren methods to TreeNode.
404d552 [Michael Armbrust] Better exception when unbound attributes make it to evaluation.
fb84ae4 [Michael Armbrust] Refactor DataProperty into Distribution.
2abb0bc [Michael Armbrust] better debug messages, use exists.
098dfc4 [Michael Armbrust] Implement Long sorting again.
60f3a9a [Michael Armbrust] More aggregate functions out of the aggregate class to make things more readable.
a1ef62e [Michael Armbrust] Print the executed plan in SharkQuery toString.
dfce426 [Michael Armbrust] Add mapChildren and withNewChildren methods to TreeNode.
037a2ed [Michael Armbrust] Better exception when unbound attributes make it to evaluation.
ec90620 [Michael Armbrust] Support for Sets as arguments to TreeNode classes.
b21f803 [Michael Armbrust] Merge pull request #11 from marmbrus/goldenGen
83adb9d [Yin Huai] add DataProperty
5a26292 [Michael Armbrust] Rules to bring casting more inline with Hive semantics.
f0e0161 [Michael Armbrust] Move numeric types into DataTypes simplifying evaluator.  This can probably also be use for codegen...
6d2924d [Michael Armbrust] add support for If. Not integrated in HiveQL yet.
ccc4dbf [Michael Armbrust] Add optimization rule to simplify casts.
058ec15 [Michael Armbrust] handle more writeables.
ffa9f25 [Michael Armbrust] blacklist some more MR tests.
aa2239c [Michael Armbrust] filter test lines containing Owner:
f71a325 [Michael Armbrust] Update golden jar.
a3003ae [Michael Armbrust] Update makefile to use better sharding support.
568d150 [Michael Armbrust] Updates to white/blacklist.
8351f25 [Michael Armbrust] Add an ignored test to remind us we don't do empty aggregations right.
c4104ec [Michael Armbrust] Numerous improvements to testing infrastructure.  See comments for details.
09c6300 [Michael Armbrust] Add nullability information to StructFields.
5460b2d [Michael Armbrust] load srcpart by default.
3695141 [Michael Armbrust] Lots of parser improvements.
965ac9a [Michael Armbrust] Add expressions that allow access into complex types.
3ba53c9 [Michael Armbrust] Output type suffixes on AttributeReferences.
8777489 [Michael Armbrust] Initial support for operators that allow the user to specify partitioning.
e57f97a [Michael Armbrust] more decimal/null support.
e1440ed [Michael Armbrust] Initial support for function specific type conversions.
1814ed3 [Michael Armbrust] use childrenResolved function.
f2ec57e [Michael Armbrust] Begin supporting decimal.
6924e6e [Michael Armbrust] Handle NullTypes when resolving HiveUDFs
7fcfa8a [Michael Armbrust] Initial support for parsing unspecified partition parameters.
d0124f3 [Michael Armbrust] Correctly type null literals.
b65626e [Michael Armbrust] Initial support for parsing BigDecimal.
a90efda [Michael Armbrust] utility function for outputing string stacktraces.
7102f33 [Michael Armbrust] methods with side-effects should use ().
3ccaef7 [Michael Armbrust] add renaming TODO.
bc282c7 [Michael Armbrust] fix bug in getNodeNumbered
c8e89d5 [Michael Armbrust] memoize inputSet calculation.
6aefa46 [Michael Armbrust] Skip folding literals.
a72e540 [Michael Armbrust] Add IN operator.
04f885b [Michael Armbrust] literals are only non-nullable if they are not null.
35d2948 [Michael Armbrust] correctly order partition and normal attributes in hive relation output.
12fd52d [Michael Armbrust] support for sorting longs.
0606520 [Michael Armbrust] drop old comment.
859200a [Michael Armbrust] support for reading more types from the metastore.
1fedd18 [Michael Armbrust] coercion from null to numeric types
71e902d [Michael Armbrust] fix test cases.
cc06b6c [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into interviewAnswer
8a8b521 [Reynold Xin] Merge pull request #8 from marmbrus/testImprovment
86355a6 [Michael Armbrust] throw error if there are unexpected join clauses.
c5842d2 [Michael Armbrust] don't throw an error when a select clause outputs multiple copies of the same attribute.
0e975ea [Michael Armbrust] parse bucket sampling as percentage sampling
a92919d [Michael Armbrust] add alter view as to native commands
f58d5a5 [Michael Armbrust] support for parsing SELECT DISTINCT
f0faa26 [Michael Armbrust] add sample and distinct operators.
ef7b943 [Michael Armbrust] add metastore support for float
e9f4588 [Michael Armbrust] fix > 100 char.
755b229 [Michael Armbrust] blacklist some ddl tests.
9ae740a [Michael Armbrust] blacklist more tests that require MR.
4cfc11a [Michael Armbrust] more test coverage.
0d9d56a [Michael Armbrust] add more native commands to parser
78d730d [Michael Armbrust] Load src test table on RESET.
8364ec2 [Michael Armbrust] whitelist all possible partition values.
b01468d [Michael Armbrust] support path rewrites when the query begins with a comment.
4c6b454 [Michael Armbrust] add option for recomputing the cached golden answer when tests fail.
4c5fb0f [Michael Armbrust] makefile target for building new whitelist.
4b6fed8 [Michael Armbrust] support for parsing both DESTINATION and INSERT_INTO.
516481c [Michael Armbrust] Ignore requests to explain native commands.
68aa2e6 [Michael Armbrust] Stronger type for Token extractor.
ca4ea26 [Michael Armbrust] Support for parsing UDF(*).
1aafea3 [Michael Armbrust] Configure partition whitelist in TestShark reset.
9627616 [Michael Armbrust] Use current database as default database.
9b02b44 [Michael Armbrust] Fix spelling error. Add failFast mode.
6f64cee [Michael Armbrust] don't line wrap string literal
eafaeed [Michael Armbrust] add type documentation
f54c94c [Michael Armbrust] make golden answers file a test dependency
5362365 [Michael Armbrust] push conditions into join
0d2388b [Michael Armbrust] Point at databricks hosted scaladoc.
73b29cd [Michael Armbrust] fix bad casting
9aa06c5 [Michael Armbrust] Merge pull request #7 from marmbrus/docFixes
7eff191 [Michael Armbrust] link all the expression names.
83227e4 [Michael Armbrust] fix scaladoc list syntax, add docs for some rules
9de6b74 [Michael Armbrust] fix language feature and deprecation warnings.
0b1960a [Michael Armbrust] Fix broken scala doc links / warnings.
b1acb36 [Michael Armbrust] Merge pull request #3 from yhuai/evalauteLiteralsInExpressions
01c00c2 [Michael Armbrust] new golden
5c14857 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
b749b51 [Michael Armbrust] Merge pull request #5 from marmbrus/testCaching
66adceb [Michael Armbrust] Merge pull request #6 from marmbrus/joinWork
1a393da [Yin Huai] folded -> foldable
1e964ea [Yin Huai] update
a43d41c [Michael Armbrust] more tests passing!
8ca38d0 [Michael Armbrust] begin support for varchar / binary types.
ab8bbd1 [Michael Armbrust] parsing % operator
c16c8b5 [Michael Armbrust] case insensitive checking for hooks in tests.
3a90a5f [Michael Armbrust] simpler output when running a single test from the commandline.
5332fee [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
367fb9e [Yin Huai] update
0cd5cc6 [Michael Armbrust] add BIGINT cast parsing
61b266f [Michael Armbrust] comment for eliminate subqueries.
d72a5a2 [Michael Armbrust] add long to literal factory object.
b3bd15f [Michael Armbrust] blacklist more mr requiring tests.
e06fd38 [Michael Armbrust] black list map reduce tests.
8e7ce30 [Michael Armbrust] blacklist some env specific tests.
6250cbd [Michael Armbrust] Do not exit on test failure
b22b220 [Michael Armbrust] also look for cached hive test answers on the classpath.
b6e4899 [Yin Huai] formatting
e75c90d [Reynold Xin] Merge pull request #4 from marmbrus/hive12
5fabbec [Michael Armbrust] ignore partitioned scan test. scan seems to be working but there is some error about the table already existing?
9e190f5 [Michael Armbrust] drop unneeded ()
68b58c1 [Michael Armbrust] drop a few more tests.
b0aa400 [Michael Armbrust] update whitelist.
c99012c [Michael Armbrust] skip tests with hooks
db00ebf [Michael Armbrust] more types for hive udfs
dbc3678 [Michael Armbrust] update ghpages repo
138f53d [Yin Huai] addressed comments and added a space after a space after the defining keyword of every control structure.
6f954ee [Michael Armbrust] export the hadoop classpath when starting sbt, required to invoke hive during tests.
46bf41b [Michael Armbrust] add a makefile for priming the test answer cache in parallel.  usage: "make -j 8 -i"
8d47ed4 [Yin Huai] comment
2795f05 [Yin Huai] comment
e003728 [Yin Huai] move OptimizerSuite to the package of catalyst.optimizer
2941d3a [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
0bd1688 [Yin Huai] update
6a7bd75 [Michael Armbrust] fix partition column delimiter configuration.
e942da1 [Michael Armbrust] Begin upgrade to Hive 0.12.0.
b8cd7e3 [Michael Armbrust] Merge pull request #7 from rxin/moreclean
52864da [Reynold Xin] Added executeCollect method to SharkPlan.
f0e1cbf [Reynold Xin] Added resolved lazy val to LogicalPlan.
b367e36 [Reynold Xin] Replaced the use of ??? with UnsupportedOperationException.
38124bd [Yin Huai] formatting
2924468 [Yin Huai] add two tests for testing pre-order and post-order tree traversal, respectively
555d839 [Reynold Xin] More cleaning ...
d48d0e1 [Reynold Xin] Code review feedback.
aa2e694 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
5c421ac [Reynold Xin] Imported SharkEnv, SharkContext, and HadoopTableReader to remove Shark dependency.
479e055 [Reynold Xin] A set of minor changes, including: - import order - limit some lines to 100 character wide - inline code comment - more scaladocs - minor spacing (i.e. add a space after if)
da16e45 [Reynold Xin] Merge pull request #3 from rxin/packagename
e36caf5 [Reynold Xin] Renamed Rule.name to Rule.ruleName since name is used too frequently in the code base and is shadowed often by local scope.
72426ed [Reynold Xin] Rename shark2 package to execution.
0892153 [Reynold Xin] Merge pull request #2 from rxin/packagename
e58304a [Reynold Xin] Merge pull request #1 from rxin/gitignore
3f9fee1 [Michael Armbrust] rewrite push filter through join optimization.
c6527f5 [Reynold Xin] Moved the test src files into the catalyst directory.
c9777d8 [Reynold Xin] Put all source files in a catalyst directory.
019ea74 [Reynold Xin] Updated .gitignore to include IntelliJ files.
80ca4be [Timothy Chen] Address comments
0079392 [Michael Armbrust] support for multiple insert commands in a single query
75b5a01 [Michael Armbrust] remove space.
4283400 [Timothy Chen] Add limited predicate push down
e547e50 [Michael Armbrust] implement First.
e77c9b6 [Michael Armbrust] more work on unique join.
c795e06 [Michael Armbrust] improve star expansion
a26494e [Michael Armbrust] allow aliases to have qualifiers
d078333 [Michael Armbrust] remove extra space
a75c023 [Michael Armbrust] implement Coalesce
3a018b6 [Michael Armbrust] fix up docs.
ab6f67d [Michael Armbrust] import the string "null" as actual null.
5377c04 [Michael Armbrust] don't call dataType until checking if children are resolved.
191ce3e [Michael Armbrust] analyze rewrite test query.
60b1526 [Michael Armbrust] don't call dataType until checking if children are resolved.
2ab5a32 [Michael Armbrust] stop using uberjar as it has its own set of issues.
e42f75a [Michael Armbrust] Merge remote-tracking branch 'origin/master' into HEAD
c086a35 [Michael Armbrust] docs, spacing
c4060e4 [Michael Armbrust] cleanup
3b85462 [Michael Armbrust] more tests passing
bcfc8c5 [Michael Armbrust] start supporting partition attributes when inserting data.
c944a95 [Michael Armbrust] First aggregate expression.
1e28311 [Michael Armbrust] make tests execute in alpha order again
a287481 [Michael Armbrust] spelling
8492548 [Michael Armbrust] beginning of UNIQUEJOIN parsing.
a6ab6c7 [Michael Armbrust] add !=
4529594 [Michael Armbrust] draft of coalesce
70f253f [Michael Armbrust] more tests passing!
7349e7b [Michael Armbrust] initial support for test thrift table
d3c9305 [Michael Armbrust] fix > 100 char line
93b64b0 [Michael Armbrust] load test tables that are args to "DESCRIBE"
06b2aba [Michael Armbrust] don't be case sensitive when fixing load paths
6355d0e [Michael Armbrust] match actual return type of count with expected
cda43ab [Michael Armbrust] don't throw an exception when one of the join tables is empty.
fd4b096 [Michael Armbrust] fix casing of null strings as well.
4632695 [Michael Armbrust] support for megastore bigint
67b88cf [Michael Armbrust] more verbose debugging of evaluation return types
c680e0d [Michael Armbrust] Failed string => number conversion should return null.
2326be1 [Michael Armbrust] make getClauses case insensitive.
dac2786 [Michael Armbrust] correctly handle null values when going from string to numeric types.
045ac4b [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
fb5ddfd [Michael Armbrust] move ViewExamples to examples/
83833e8 [Michael Armbrust] more tests passing!
47c98d6 [Michael Armbrust] add query tests for like and hash.
1724c16 [Michael Armbrust] clear lines that contain last updated times.
cfd6bbc [Michael Armbrust] Quick skipping of tests that we can't even parse.
9b2642b [Michael Armbrust] make the blacklist support regexes
1d50af6 [Michael Armbrust] more datatypes, fix nonserializable instance variables in udfs
910e33e [Michael Armbrust] basic support for building an assembly jar.
d55bb52 [Michael Armbrust] add local warehouse/metastore to gitignore.
495d9dc [Michael Armbrust] Add an expression for when we decide to support LIKE natively instead of using the HIVE udf.
65f4e69 [Michael Armbrust] remove incorrect comments
0831a3c [Michael Armbrust] support for parsing some operator udfs.
6c27aa7 [Michael Armbrust] more cast parsing.
43db061 [Michael Armbrust] significant generalization of hive udf functionality.
3fe24ec [Michael Armbrust] better implementation of 3vl in Evaluate, fix some > 100 char lines.
e5690a6 [Michael Armbrust] add BinaryType
adab892 [Michael Armbrust] Clear out functions that are created during tests when reset is called.
d408021 [Michael Armbrust] support for printing out arrays in the output in the same form as hive (e.g., [e1, e1]).
8d5f504 [Michael Armbrust] Example of schema RDD using scala's dynamic trait, resulting in a more standard ORM style of usage.
21f0d91 [Michael Armbrust] Simple example of schemaRdd with scala filter function.
0daaa0e [Michael Armbrust] Promote booleans that appear in comparisons.
2b70abf [Michael Armbrust] true and false literals.
ef8b0a5 [Michael Armbrust] more tests.
14d070f [Michael Armbrust] add support for correctly extracting partition keys.
0afbe73 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
69a0bd4 [Michael Armbrust] promote strings in predicates with number too.
3946e31 [Michael Armbrust] don't build strings unless assertion fails.
90c453d [Michael Armbrust] more tests passing!
6e6417a [Michael Armbrust] correct handling of nulls in boolean logic and sorting.
8000504 [Michael Armbrust] Improve type coercion.
9087152 [Michael Armbrust] fix toString of Not.
58b111c [Michael Armbrust] fix bad scaladoc tag.
d5c05c6 [Michael Armbrust] For now, ignore the big data benchmark tests when the data isn't there.
ac6376d [Michael Armbrust] Split out general shark query execution driver from test harness.
1d0ae1e [Michael Armbrust] Switch from IndexSeq[Any] to Row interface that will allow us unboxed access to primitive types.
d873b2b [Yin Huai] Remove numbers associated with test cases.
8545675 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions
b34a9eb [Michael Armbrust] Merge branch 'master' into filterPushDown
d1e7b8e [Michael Armbrust] Update README.md
c8b1553 [Michael Armbrust] Update README.md
9307ef9 [Michael Armbrust] update list of passing tests.
934c18c [Michael Armbrust] Filter out non-deterministic lines when comparing test answers.
a045c9c [Michael Armbrust] SparkAggregate doesn't actually support sum right now.
ae0024a [Yin Huai] update
cf80545 [Yin Huai] Merge remote-tracking branch 'origin/evalauteLiteralsInExpressions' into evalauteLiteralsInExpressions
21976ae [Yin Huai] update
b4999fe [Yin Huai] Merge remote-tracking branch 'upstream/filterPushDown' into evalauteLiteralsInExpressions
dedbf0c [Yin Huai] support Boolean literals
eaac9e2 [Yin Huai] explain the limitation of the current EvaluateLiterals
37817b5 [Yin Huai] add a comment to EvaluateLiterals.
468667f [Yin Huai] First draft of literal evaluation in the optimization phase. TreeNode has been extended to support transform in the post order. So, for an expression, we can evaluate literal from the leaf nodes of this expression tree. For an attribute reference in the expression node, we just leave it as is.
b1d1843 [Michael Armbrust] more work on big data benchmark tests.
cc9a957 [Michael Armbrust] support for creating test tables outside of TestShark
7d7fa9f [Michael Armbrust] support for create table as
5f54f03 [Michael Armbrust] parsing for ASC
d42b725 [Michael Armbrust] Sum of strings requires cast
34b30fa [Michael Armbrust] not all attributes need to be bound (e.g. output attributes that are contained in non-leaf operators.)
81659cb [Michael Armbrust] implement transform operator.
5cd76d6 [Michael Armbrust] break up the file based test case code for reuse
1031b65 [Michael Armbrust] support for case insensitive resolution.
320df04 [Michael Armbrust] add snapshot repo for databricks (has shark/spark snapshots)
b6f083e [Michael Armbrust] support for publishing scala doc to github from sbt
d9d18b4 [Michael Armbrust] debug logging implicit.
669089c [Yin Huai] support Boolean literals
ef3321e [Yin Huai] explain the limitation of the current EvaluateLiterals
73a05fd [Yin Huai] add a comment to EvaluateLiterals.
191eb7d [Yin Huai] First draft of literal evaluation in the optimization phase. TreeNode has been extended to support transform in the post order. So, for an expression, we can evaluate literal from the leaf nodes of this expression tree. For an attribute reference in the expression node, we just leave it as is.
80039cc [Yin Huai] Merge pull request #1 from yhuai/master
cbe1ca1 [Yin Huai] add explicit result type to the overloaded sideBySide
5c518e4 [Michael Armbrust] fix bug in test.
b50dd0e [Michael Armbrust] fix return type of overloaded method
05679b7 [Michael Armbrust] download assembly jar for easy compiling during interview.
8c60cc0 [Michael Armbrust] Update README.md
03b9526 [Michael Armbrust] First draft of optimizer tests.
f392755 [Michael Armbrust] Add flatMap to TreeNode
6cbe8d1 [Michael Armbrust] fix bug in side by side, add support for working with unsplit strings
15a53fc [Michael Armbrust] more generic sum calculation and better binding of grouping expressions.
06749d0 [Michael Armbrust] add expression enumerations for query plan operators and recursive version of transform expression.
4b0a888 [Michael Armbrust] implement string comparison and more casts.
356b321 [Michael Armbrust] Update README.md
3776395 [Michael Armbrust] Update README.md
304d17d [Michael Armbrust] Create README.md
b7d8be0 [Michael Armbrust] more tests passing.
b82481f [Michael Armbrust] add todo comment.
02e6dee [Michael Armbrust] add another test that breaks the harness to the blacklist.
cc5efe3 [Michael Armbrust] First draft of broadcast nested loop join with full outer support.
c43a259 [Michael Armbrust] comments
15ff448 [Michael Armbrust] better error message when a dsl test throws an exception
76ec650 [Michael Armbrust] fix join conditions
e10df99 [Michael Armbrust] Create new expr ids for local relations that exist more than once in a query plan.
91573a4 [Michael Armbrust] initial type promotion
e2ef4a5 [Michael Armbrust] logging
e43dc1e [Michael Armbrust] add string => int cast evaluation
f1f7e96 [Michael Armbrust] fix incorrect generation of join keys
2b27230 [Michael Armbrust] add depth based subtree access
0f6279f [Michael Armbrust] broken tests.
389bc0b [Michael Armbrust] support for partitioned columns in output.
12584f4 [Michael Armbrust] better errors for missing clauses. support for matching multiple clauses with the same name.
b67a225 [Michael Armbrust] better errors when types don't match up.
9e74808 [Michael Armbrust] add children resolved.
6d03ce9 [Michael Armbrust] defaults for unresolved relation
2469b00 [Michael Armbrust] skip nodes with unresolved children when doing coersions
be5ae2c [Michael Armbrust] better resolution logging
cb7b5af [Michael Armbrust] views example
420e05b [Michael Armbrust] more tests passing!
6916c63 [Michael Armbrust] Reading from partitioned hive tables.
a1245f9 [Michael Armbrust] more tests passing
956e760 [Michael Armbrust] extended explain
5f14c35 [Michael Armbrust] more test tables supported
175c43e [Michael Armbrust] better errors for parse exceptions
480ade5 [Michael Armbrust] don't use partial cached results.
8a9d21c [Michael Armbrust] fix evaluation
7aee69c [Michael Armbrust] parsing for joins, boolean logic
7fcf480 [Michael Armbrust] test for and logic
3ea9b00 [Michael Armbrust] don't use simpleString if there are no new lines.
6902490 [Michael Armbrust] fix boolean logic evaluation
4d5eba7 [Michael Armbrust] add more dsl for expression arithmetic and boolean logic
8b2a2ee [Michael Armbrust] more tests passing!
ad1f3b4 [Michael Armbrust] toString for null literals
a5c0a1b [Michael Armbrust] more test harness improvements: * regex whitelist * side by side answer comparison (still needs formatting work)
60ec19d [Michael Armbrust] initial support for udfs
c45b440 [Michael Armbrust] support for is (not) null and boolean logic
7f4a1dc [Michael Armbrust] add NoRelation logical operator
72e183b [Michael Armbrust] support for null values in tree node args.
ad596d2 [Michael Armbrust] add sc to Union's otherCopyArgs
e5c9d1a [Michael Armbrust] use nonEmpty
dcc4fe1 [Michael Armbrust] support for src1 test table.
c78b587 [Michael Armbrust] casting.
75c3f3f [Michael Armbrust] add support for logging with scalalogging.
da2c011 [Michael Armbrust] make it more obvious when results are being truncated.
96b73ba [Michael Armbrust] more docs in TestShark
18524fd [Michael Armbrust] add method to SharkSqlQuery for directly executing the same query on hive.
e6d063b [Michael Armbrust] more join tests.
664c1c3 [Michael Armbrust] make parsing of function names case insensitive.
0967d4e [Michael Armbrust] fix hardcoded path to hiveDevHome.
1a6db68 [Michael Armbrust] spelling
7638cb4 [Michael Armbrust] simple join execution with dsl tests.  no hive tests yes.
859d4c9 [Michael Armbrust] better argString printing of nested trees.
fc53615 [Michael Armbrust] add same instance comparisons for tree nodes.
a026e6b [Michael Armbrust] move out hive specific operators
fff4d1c [Michael Armbrust] add simple query execution debugging
e2120ab [Michael Armbrust] sorting for strings
da06eb6 [Michael Armbrust] Parsing for sortby and joins
9eb5c5e [Michael Armbrust] override equality in Attribute references to compare exprId.
8eb2460 [Michael Armbrust] add system property to override whitelist.
88124bb [Michael Armbrust] make strategy evaluation lazy.
74a3a21 [Michael Armbrust] implement outputSet
d25b171 [Michael Armbrust] Add AND and OR expressions
67f0a4a [Michael Armbrust] dsl improvements: string to attribute, subquery, unionAll
12acf0a [Michael Armbrust] add .DS_Store for macs
f7da6ce [Michael Armbrust] add agg with grouping expr in select test
36805b3 [Michael Armbrust] pull out and improve aggregation
75613e1 [Michael Armbrust] better evaluations failure messages.
4789a35 [Michael Armbrust] weaken type since its hard to create pure references.
e89dd36 [Michael Armbrust] no newline for online trees
d0590d4 [Michael Armbrust] include stack trace for catalyst failures.
081c0d9 [Michael Armbrust] more generic computation of agg functions.
31af3a0 [Michael Armbrust] fail when clauses are unhandeled in the parser
ecd45b2 [Michael Armbrust] Add more passing tests.
97d5419 [Michael Armbrust] fix alignment.
565cc13 [Michael Armbrust] make the canary query optional.
a95e65c [Michael Armbrust] support for resolving qualified attribute references.
e1dfa0c [Michael Armbrust] better error reporting for comparison tests when hive works but catalyst fails.
4640a0b [Michael Armbrust] handle test tables when database is specified.
bef12e3 [Michael Armbrust] Add Subquery node and trivial optimizer to remove it after analysis.
fec5158 [Michael Armbrust] add hive / idea files to .gitignore
3f97ffe [Michael Armbrust] Rename Hive => HiveQl
656b836 [Michael Armbrust] Support for parsing select clause aliases.
3ca7414 [Michael Armbrust] StopAfter needs otherCopyArgs.
3ffde66 [Michael Armbrust] When the child of an alias is unresolved it should return an unresolved attribute instead of throwing an exception.
8cbef8a [Michael Armbrust] spelling
aa8c37c [Michael Armbrust] Better toString for SortOrder
1bb8b45 [Michael Armbrust] fix error message for UnresolvedExceptions
a2e0327 [Michael Armbrust] add a bunch of tests.
4a3e1ea [Michael Armbrust] docs and use shark for data loading.
339bb8f [Michael Armbrust] better docs, Not support
1d7b2d9 [Michael Armbrust] Add NaN conversions.
46a2534 [Michael Armbrust] only run canary query on failure.
8996066 [Michael Armbrust] remove protected from makeCopy
53bcf41 [Michael Armbrust] testing improvements: * reset hive vars * delete indexes and tables * delete database * reset to use default database * record tests that pass
04a372a [Michael Armbrust] add a flag for running all tests.
3b2235b [Michael Armbrust] More general implementation of arithmetic.
edd7795 [Michael Armbrust] More testing improvements: * Check that results match for native commands * Ensure explain commands can be planned * Cache hive "golden" results
da6c577 [Michael Armbrust] add string <==> file utility functions.
3adf5ca [Michael Armbrust] Initial support for groupBy and count.
7bcd8a4 [Michael Armbrust] Improvements to comparison tests: * Sort answer when query doesn't contain an order by. * Display null values the same as Hive. * Print full query results in easy to read format when they differ.
a52e7c9 [Michael Armbrust] Transform children that are present in sequences of the product.
d66ba7e [Michael Armbrust] drop printlns.
88f2efd [Michael Armbrust] Add sum / count distinct expressions.
05adedc [Michael Armbrust] rewrite relative paths when loading data in TestShark
07784b3 [Michael Armbrust] add support for rewriting paths and running 'set' commands.
b8a9910 [Michael Armbrust] quote tests passing.
8e5e267 [Michael Armbrust] handle aliased select expressions.
4286a96 [Michael Armbrust] drop debugging println
ac34aeb [Michael Armbrust] proof of concept for hive ast transformations.
2238b00 [Michael Armbrust] better error when makeCopy functions fails due to incorrect arguments
ff1eab8 [Michael Armbrust] start trying to make insert into hive table more general.
74a6337 [Michael Armbrust] use fastEquals when doing transformations.
1184a23 [Michael Armbrust] add native test for escapes.
b972b18 [Michael Armbrust] create BaseRelation class
fa6bce9 [Michael Armbrust] implement union
6391a87 [Michael Armbrust] count aggregate.
d47c317 [Michael Armbrust] add unary minus, more tests passing.
c7114e4 [Michael Armbrust] first draft of star expansion.
044c43d [Michael Armbrust] better support for numeric literal parsing.
1d0f072 [Michael Armbrust] use native drop table as it doesn't appear to fail when the "table" is actually a view.
61503c5 [Michael Armbrust] add cached toRdd
2036883 [Michael Armbrust] skip explain queries when testing.
ebac4b1 [Michael Armbrust] fix bug in sort reference calculation
ca0dee0 [Michael Armbrust] docs.
1ee0471 [Michael Armbrust] string literal parsing.
357278b [Michael Armbrust] add limit support
9b3e479 [Michael Armbrust] creation of string literals.
02efa30 [Michael Armbrust] alias evaluation
cb68b33 [Michael Armbrust] parsing for random sample in hive ql.
126dd36 [Michael Armbrust] include query plans in failure output
bb59ae9 [Michael Armbrust] doc fixes
7e68286 [Michael Armbrust] fix confusing naming
768bb25 [Michael Armbrust] handle errors in shark query toString
829c3ce [Michael Armbrust] Auto loading of test data on demand. Add reset method to test shark.  Make test shark a singleton to avoid weirdness with the hive megastore.
ad02e41 [Michael Armbrust] comment jdo dependency
7bc89fe [Michael Armbrust] add collect to TreeNode.
438cf74 [Michael Armbrust] create explicit treeString function in addition to toString override. docs.
09679ee [Michael Armbrust] fix bug in TreeNode foreach
2930b27 [Michael Armbrust] more specific name for del query tests.
8842549 [Michael Armbrust] docs.
da81f81 [Michael Armbrust] Implementation and tests for simple AVG query in Hive SQL.
a8969b9 [Michael Armbrust] Factor out hive query comparison test framework.
1a7efb0 [Michael Armbrust] specialize spark aggregate for global aggregations.
a36dd9a [Michael Armbrust] evaluation for other > data types.
cae729b [Michael Armbrust] remove unnecessary lazy vals.
d8e12af [Michael Armbrust] docs
3a60d67 [Michael Armbrust] implement average, placeholder for count
f05c106 [Michael Armbrust] checkAnswer handles single row results.
2730534 [Michael Armbrust] implement inputSet
a9aa79d [Michael Armbrust] debugging for sort exec
8bec3c9 [Michael Armbrust] better tree makeCopy when there are two constructors.
554b4b2 [Michael Armbrust] BoundAttribute pretty printing.
754f5fa [Michael Armbrust] dsl for setting nullability
a206d7a [Michael Armbrust] clean up query tests.
84ad6ef [Michael Armbrust] better sort implementation and tests.
de24923 [Michael Armbrust] add double type.
9611a2c [Michael Armbrust] literal creation for doubles.
7358313 [Michael Armbrust] sort order returns child type.
b544715 [Michael Armbrust] implement eval for rand, and > for doubles
7013bad [Michael Armbrust] asc, desc should work for expressions and unresolved attributes (symbols)
1c1a35e [Michael Armbrust] add simple Rand expression.
3ca51de [Michael Armbrust] add orderBy to dsl
7ae41ab [Michael Armbrust] more literal implicit conversions
b18b675 [Michael Armbrust] First cut at native query tests for shark.
d392e29 [Michael Armbrust] add toRdd implicit conversion for logical plans in TestShark.
5eac895 [Michael Armbrust] better error when descending is specified.
2b16f86 [Michael Armbrust] add todo
e527bb8 [Michael Armbrust] remove arguments to binary predicate constructor as they seem to break serialization
9dde3c8 [Michael Armbrust] add project and filter operations.
ad9037b [Michael Armbrust] Add support for local relations.
6227143 [Michael Armbrust] evaluation of Equals.
7526290 [Michael Armbrust] BoundReference should also be an Attribute.
bd33e26 [Michael Armbrust] more documentation
5de0ea3 [Michael Armbrust] Move all shark specific into a separate package.  Lots of documentation improvements.
0ae292b [Michael Armbrust] implement calculation of sort expressions.
9fd5011 [Michael Armbrust] First cut at expression evaluation.
6259e3a [Michael Armbrust] cleanup
787e5a2 [Michael Armbrust] use fastEquals
f90da36 [Michael Armbrust] better printing of optimization exceptions
b05dd67 [Michael Armbrust] Application of rules to fixed point.
bb2e0db [Michael Armbrust] pretty print for literals.
1ec3287 [Michael Armbrust] Add extractor for IntegerLiterals.
d3a3687 [Michael Armbrust] add fastEquals
2b4935b [Michael Armbrust] set sbt.version explicitly
46dfd7f [Michael Armbrust] first cut at checking answer for HiveCompatability tests.
c79f2fd [Michael Armbrust] insert operator should return an empty rdd.
14c22ec [Michael Armbrust] implement sorting when the sort expression is the first attribute of the input.
ae7b4c3 [Michael Armbrust] remove implicit dependencies.  now compiles without copying things into lib/ manually.
84082f9 [Michael Armbrust] add sbt binaries and scripts
15371a8 [Michael Armbrust] First draft of simple Hive DDL parser.
063bf44 [Michael Armbrust] Periods should end all comments.
e1f7f4c [Michael Armbrust] Remove "NativePlaceholder" hack.
ed3633e [Michael Armbrust] start consolidating Hive/Shark specific code. first hive compatibility test case passing!
b34a770 [Michael Armbrust] Add data sink strategy, make strategy application a little more robust.
e7174ec [Michael Armbrust] fix schema, add docs, make helper method protected.
26f410a [Michael Armbrust] physical traits should extend PhysicalPlan.
dc72469 [Michael Armbrust] beginning of hive compatibility testing framework.
0763490 [Michael Armbrust] support for hive native command pass-through.
d8a924f [Michael Armbrust] scaladoc
29a7163 [Michael Armbrust] Insert into hive table physical operator.
633cebc [Michael Armbrust] better error message when there is no appropriate planning strategy.
59ac444 [Michael Armbrust] add unary expression
3aa1b28 [Michael Armbrust] support for table names in the form 'database.tableName'
665f7d0 [Michael Armbrust] add logical nodes for hive data sinks.
64d2923 [Michael Armbrust] Add classes for representing sorts.
f72b7ce [Michael Armbrust] first trivial end to end query execution.
5c7d244 [Michael Armbrust] first draft of references implementation.
7bff274 [Michael Armbrust] point at new shark.
c7cd57f [Michael Armbrust] docs for util function.
910811c [Michael Armbrust] check each item of the sequence
ef21a0b [Michael Armbrust] line up comments.
4b765d5 [Michael Armbrust] docs, drop println
6f9bafd [Michael Armbrust] empty output for unresolved relation to avoid exception in resolution.
a703c49 [Michael Armbrust] this order works better until fixed point is implemented.
ec1d7c0 [Michael Armbrust] Simple attribute resolution.
069df02 [Michael Armbrust] parsing binary predicates
a1cf754 [Michael Armbrust] add joins and equality.
3f5bc98 [Michael Armbrust] add optiq to sbt.
54f3460 [Michael Armbrust] initial optiq parsing.
d9161ce [Michael Armbrust] add join operator
1e423eb [Michael Armbrust] placeholders in LogicalPlan, docs
24ef6fb [Michael Armbrust] toString for alias.
ae7d776 [Michael Armbrust] add nullability changing function
d49dc02 [Michael Armbrust] scaladoc for named exprs
7c45dd7 [Michael Armbrust] pretty printing of trees.
78e34bf [Michael Armbrust] simple git ignore.
7ba19be [Michael Armbrust] First draft of interface to hive metastore.
7e7acf0 [Michael Armbrust] physical placeholder.
1c11136 [Michael Armbrust] first draft of error handling / plans for debugging.
3766a41 [Michael Armbrust] rearrange utility functions.
7fb3d5e [Michael Armbrust] docs and equality improvements.
45da47b [Michael Armbrust] flesh out plans and expressions a little. first cut at named expressions.
002d4d4 [Michael Armbrust] default to no alias.
be25003 [Michael Armbrust] add repl initialization to sbt.
0608a00 [Michael Armbrust] tighten public interface
a1a8b38 [Michael Armbrust] test that ids don't change for no-op transforms.
daa71ca [Michael Armbrust] foreach, maps, and scaladoc
6a158cb [Michael Armbrust] simple transform working.
db0299f [Michael Armbrust] basic analysis of relations minus transform function.
f74c4ee [Michael Armbrust] parsing a simple query.
08e4f57 [Michael Armbrust] upgrade scala include shark.
d3c6404 [Michael Armbrust] initial commit
This commit is contained in:
Michael Armbrust 2014-03-20 18:03:20 -07:00 committed by Reynold Xin
parent ca76423e23
commit 9aadcffabd
131 changed files with 16398 additions and 15 deletions

View file

@ -79,6 +79,11 @@
<artifactId>spark-graphx_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>net.sf.py4j</groupId>
<artifactId>py4j</artifactId>

View file

@ -33,23 +33,43 @@ fi
# Build up classpath
CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
# Support for interacting with Hive. Since hive pulls in a lot of dependencies that might break
# existing Spark applications, it is not included in the standard spark assembly. Instead, we only
# include it in the classpath if the user has explicitly requested it by running "sbt hive/assembly"
# Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in
# the future.
if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then
echo "Hive assembly found, including hive support. If this isn't desired run sbt hive/clean."
# Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost.
DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}")
CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS
ASSEMBLY_DIR="$FWDIR/sql/hive/target/scala-$SCALA_VERSION/"
else
ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION/"
fi
# First check if we have a dependencies jar. If so, include binary classes with the deps jar
if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then
if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then
CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes"
DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar`
DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*-deps.jar`
CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR"
else
# Else use spark-assembly jar from either RELEASE or assembly directory
if [ -f "$FWDIR/RELEASE" ]; then
ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark*-assembly*.jar`
else
ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*.jar`
fi
CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
fi
@ -62,6 +82,9 @@ if [[ $SPARK_TESTING == 1 ]]; then
CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/test-classes"
CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/test-classes"
fi
# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !

4
dev/download-hive-tests.sh Executable file
View file

@ -0,0 +1,4 @@
#!/bin/sh
wget -O hiveTests.tgz http://cs.berkeley.edu/~marmbrus/tmp/hiveTests.tgz
tar zxf hiveTests.tgz

View file

@ -21,6 +21,9 @@
FWDIR="$(cd `dirname $0`/..; pwd)"
cd $FWDIR
# Download Hive Compatability Files
dev/download-hive-tests.sh
# Remove work directory
rm -rf ./work

View file

@ -66,6 +66,7 @@
<li><a href="python-programming-guide.html">Spark in Python</a></li>
<li class="divider"></li>
<li><a href="streaming-programming-guide.html">Spark Streaming</a></li>
<li><a href="sql-programming-guide.html">Spark SQL</a></li>
<li><a href="mllib-guide.html">MLlib (Machine Learning)</a></li>
<li><a href="bagel-programming-guide.html">Bagel (Pregel on Spark)</a></li>
<li><a href="graphx-programming-guide.html">GraphX (Graph Processing)</a></li>
@ -79,6 +80,14 @@
<li><a href="api/pyspark/index.html">Spark Core for Python</a></li>
<li class="divider"></li>
<li><a href="api/streaming/index.html#org.apache.spark.streaming.package">Spark Streaming</a></li>
<li class="dropdown-submenu">
<a tabindex="-1" href="#">Spark SQL</a>
<ul class="dropdown-menu">
<li><a href="api/sql/core/org/apache/spark/sql/SQLContext.html">Spark SQL Core</a></li>
<li><a href="api/sql/hive/org/apache/spark/sql/hive/package.html">Hive Support</a></li>
<li><a href="api/sql/catalyst/org/apache/spark/sql/catalyst/package.html">Catalyst (Optimization)</a></li>
</ul>
</li>
<li><a href="api/mllib/index.html#org.apache.spark.mllib.package">MLlib (Machine Learning)</a></li>
<li><a href="api/bagel/index.html#org.apache.spark.bagel.package">Bagel (Pregel on Spark)</a></li>
<li><a href="api/graphx/index.html#org.apache.spark.graphx.package">GraphX (Graph Processing)</a></li>

View file

@ -22,6 +22,7 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1')
# Build Scaladoc for Java/Scala
core_projects = ["core", "examples", "repl", "bagel", "graphx", "streaming", "mllib"]
external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"]
sql_projects = ["catalyst", "core", "hive"]
projects = core_projects + external_projects.map { |project_name| "external/" + project_name }
@ -49,6 +50,18 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1')
cp_r(source + "/.", dest)
end
sql_projects.each do |project_name|
source = "../sql/" + project_name + "/target/scala-2.10/api/"
dest = "api/sql/" + project_name
puts "echo making directory " + dest
mkdir_p dest
# From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't.
puts "cp -r " + source + "/. " + dest
cp_r(source + "/.", dest)
end
# Build Epydoc for Python
puts "Moving to python directory and building epydoc."
cd("../python")

View file

@ -78,6 +78,7 @@ For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to bui
* [Java Programming Guide](java-programming-guide.html): using Spark from Java
* [Python Programming Guide](python-programming-guide.html): using Spark from Python
* [Spark Streaming](streaming-programming-guide.html): Spark's API for processing data streams
* [Spark SQL](sql-programming-guide.html): Support for running relational queries on Spark
* [MLlib (Machine Learning)](mllib-guide.html): Spark's built-in machine learning library
* [Bagel (Pregel on Spark)](bagel-programming-guide.html): simple graph processing model
* [GraphX (Graphs on Spark)](graphx-programming-guide.html): Spark's new API for graphs

View file

@ -0,0 +1,143 @@
---
layout: global
title: Spark SQL Programming Guide
---
**Spark SQL is currently an Alpha component. Therefore, the APIs may be changed in future releases.**
* This will become a table of contents (this text will be scraped).
{:toc}
# Overview
Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using
Spark. At the core of this component is a new type of RDD,
[SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed
[Row](api/sql/catalyst/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with
a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table
in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet
file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
**All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.**
***************************************************************************************************
# Getting Started
The entry point into all relational functionallity in Spark is the
[SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its
decendents. To create a basic SQLContext, all you need is a SparkContext.
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
// Importing the SQL context gives access to all the public SQL functions and implicit conversions.
import sqlContext._
{% endhighlight %}
## Running SQL on RDDs
One type of table that is supported by Spark SQL is an RDD of Scala case classetees. The case class
defines the schema of the table. The names of the arguments to the case class are read using
reflection and become the names of the columns. Case classes can also be nested or contain complex
types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be
registered as a table. Tables can used in subsequent SQL statements.
{% highlight scala %}
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._
// Define the schema using a case class.
case class Person(name: String, age: Int)
// Create an RDD of Person objects and register it as a table.
val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt))
people.registerAsTable("people")
// SQL statements can be run by using the sql methods provided by sqlContext.
val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
{% endhighlight %}
**Note that Spark SQL currently uses a very basic SQL parser, and the keywords are case sensitive.**
Users that want a more complete dialect of SQL should look at the HiveQL support provided by
`HiveContext`.
## Using Parquet
Parquet is a columnar format that is supported by many other data processing systems. Spark SQL
provides support for both reading and writing parquet files that automatically preserves the schema
of the original data. Using the data from the above example:
{% highlight scala %}
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._
val people: RDD[Person] // An RDD of case class objects, from the previous example.
// The RDD is implicitly converted to a SchemaRDD, allowing it to be stored using parquet.
people.saveAsParquetFile("people.parquet")
// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
// The result of loading a parquet file is also a SchemaRDD.
val parquetFile = sqlContext.parquetFile("people.parquet")
//Parquet files can also be registered as tables and then used in SQL statements.
parquetFile.registerAsTable("parquetFile")
val teenagers = sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19")
teenagers.collect().foreach(println)
{% endhighlight %}
## Writing Language-Integrated Relational Queries
Spark SQL also supports a domain specific language for writing queries. Once again,
using the data from the above examples:
{% highlight scala %}
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._
val people: RDD[Person] // An RDD of case class objects, from the first example.
// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19'
val teenagers = people.where('age >= 10).where('age <= 19).select('name)
{% endhighlight %}
The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers
prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are
evaluated by the SQL execution engine. A full list of the functions supported can be found in the
[ScalaDoc](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD).
<!-- TODO: Include the table of operations here. -->
# Hive Support
Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/).
However, since Hive has a large number of dependencies, it is not included in the default Spark assembly.
In order to use Hive you must first run '`sbt/sbt hive/assembly`'. This command builds a new assembly
jar that includes Hive. When this jar is present, Spark will use the Hive
assembly instead of the normal Spark assembly. Note that this Hive assembly jar must also be present
on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries
(SerDes) in order to acccess data stored in Hive.
Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and
adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do
not have an existing Hive deployment can also experiment with the `LocalHiveContext`,
which is similar to `HiveContext`, but creates a local copy of the `metastore` and `warehouse`
automatically.
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc)
// Importing the SQL context gives access to all the public SQL functions and implicit conversions.
import hiveContext._
sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
// Queries are expressed in HiveQL
sql("SELECT key, value FROM src").collect().foreach(println)
{% endhighlight %}

View file

@ -70,6 +70,12 @@
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-graphx_${scala.binary.version}</artifactId>

View file

@ -0,0 +1,500 @@
238val_238
86val_86
311val_311
27val_27
165val_165
409val_409
255val_255
278val_278
98val_98
484val_484
265val_265
193val_193
401val_401
150val_150
273val_273
224val_224
369val_369
66val_66
128val_128
213val_213
146val_146
406val_406
429val_429
374val_374
152val_152
469val_469
145val_145
495val_495
37val_37
327val_327
281val_281
277val_277
209val_209
15val_15
82val_82
403val_403
166val_166
417val_417
430val_430
252val_252
292val_292
219val_219
287val_287
153val_153
193val_193
338val_338
446val_446
459val_459
394val_394
237val_237
482val_482
174val_174
413val_413
494val_494
207val_207
199val_199
466val_466
208val_208
174val_174
399val_399
396val_396
247val_247
417val_417
489val_489
162val_162
377val_377
397val_397
309val_309
365val_365
266val_266
439val_439
342val_342
367val_367
325val_325
167val_167
195val_195
475val_475
17val_17
113val_113
155val_155
203val_203
339val_339
0val_0
455val_455
128val_128
311val_311
316val_316
57val_57
302val_302
205val_205
149val_149
438val_438
345val_345
129val_129
170val_170
20val_20
489val_489
157val_157
378val_378
221val_221
92val_92
111val_111
47val_47
72val_72
4val_4
280val_280
35val_35
427val_427
277val_277
208val_208
356val_356
399val_399
169val_169
382val_382
498val_498
125val_125
386val_386
437val_437
469val_469
192val_192
286val_286
187val_187
176val_176
54val_54
459val_459
51val_51
138val_138
103val_103
239val_239
213val_213
216val_216
430val_430
278val_278
176val_176
289val_289
221val_221
65val_65
318val_318
332val_332
311val_311
275val_275
137val_137
241val_241
83val_83
333val_333
180val_180
284val_284
12val_12
230val_230
181val_181
67val_67
260val_260
404val_404
384val_384
489val_489
353val_353
373val_373
272val_272
138val_138
217val_217
84val_84
348val_348
466val_466
58val_58
8val_8
411val_411
230val_230
208val_208
348val_348
24val_24
463val_463
431val_431
179val_179
172val_172
42val_42
129val_129
158val_158
119val_119
496val_496
0val_0
322val_322
197val_197
468val_468
393val_393
454val_454
100val_100
298val_298
199val_199
191val_191
418val_418
96val_96
26val_26
165val_165
327val_327
230val_230
205val_205
120val_120
131val_131
51val_51
404val_404
43val_43
436val_436
156val_156
469val_469
468val_468
308val_308
95val_95
196val_196
288val_288
481val_481
457val_457
98val_98
282val_282
197val_197
187val_187
318val_318
318val_318
409val_409
470val_470
137val_137
369val_369
316val_316
169val_169
413val_413
85val_85
77val_77
0val_0
490val_490
87val_87
364val_364
179val_179
118val_118
134val_134
395val_395
282val_282
138val_138
238val_238
419val_419
15val_15
118val_118
72val_72
90val_90
307val_307
19val_19
435val_435
10val_10
277val_277
273val_273
306val_306
224val_224
309val_309
389val_389
327val_327
242val_242
369val_369
392val_392
272val_272
331val_331
401val_401
242val_242
452val_452
177val_177
226val_226
5val_5
497val_497
402val_402
396val_396
317val_317
395val_395
58val_58
35val_35
336val_336
95val_95
11val_11
168val_168
34val_34
229val_229
233val_233
143val_143
472val_472
322val_322
498val_498
160val_160
195val_195
42val_42
321val_321
430val_430
119val_119
489val_489
458val_458
78val_78
76val_76
41val_41
223val_223
492val_492
149val_149
449val_449
218val_218
228val_228
138val_138
453val_453
30val_30
209val_209
64val_64
468val_468
76val_76
74val_74
342val_342
69val_69
230val_230
33val_33
368val_368
103val_103
296val_296
113val_113
216val_216
367val_367
344val_344
167val_167
274val_274
219val_219
239val_239
485val_485
116val_116
223val_223
256val_256
263val_263
70val_70
487val_487
480val_480
401val_401
288val_288
191val_191
5val_5
244val_244
438val_438
128val_128
467val_467
432val_432
202val_202
316val_316
229val_229
469val_469
463val_463
280val_280
2val_2
35val_35
283val_283
331val_331
235val_235
80val_80
44val_44
193val_193
321val_321
335val_335
104val_104
466val_466
366val_366
175val_175
403val_403
483val_483
53val_53
105val_105
257val_257
406val_406
409val_409
190val_190
406val_406
401val_401
114val_114
258val_258
90val_90
203val_203
262val_262
348val_348
424val_424
12val_12
396val_396
201val_201
217val_217
164val_164
431val_431
454val_454
478val_478
298val_298
125val_125
431val_431
164val_164
424val_424
187val_187
382val_382
5val_5
70val_70
397val_397
480val_480
291val_291
24val_24
351val_351
255val_255
104val_104
70val_70
163val_163
438val_438
119val_119
414val_414
200val_200
491val_491
237val_237
439val_439
360val_360
248val_248
479val_479
305val_305
417val_417
199val_199
444val_444
120val_120
429val_429
169val_169
443val_443
323val_323
325val_325
277val_277
230val_230
478val_478
178val_178
468val_468
310val_310
317val_317
333val_333
493val_493
460val_460
207val_207
249val_249
265val_265
480val_480
83val_83
136val_136
353val_353
172val_172
214val_214
462val_462
233val_233
406val_406
133val_133
175val_175
189val_189
454val_454
375val_375
401val_401
421val_421
407val_407
384val_384
256val_256
26val_26
134val_134
67val_67
384val_384
379val_379
18val_18
462val_462
492val_492
100val_100
298val_298
9val_9
341val_341
498val_498
146val_146
458val_458
362val_362
186val_186
285val_285
348val_348
167val_167
18val_18
273val_273
183val_183
281val_281
344val_344
97val_97
469val_469
315val_315
84val_84
28val_28
37val_37
448val_448
152val_152
348val_348
307val_307
194val_194
414val_414
477val_477
222val_222
126val_126
90val_90
169val_169
403val_403
400val_400
200val_200
97val_97

View file

@ -0,0 +1,3 @@
Michael, 29
Andy, 30
Justin, 19

View file

@ -0,0 +1,64 @@
/*
* 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.hive.examples
import org.apache.spark.SparkContext
import org.apache.spark.sql._
import org.apache.spark.sql.hive.LocalHiveContext
object HiveFromSpark {
case class Record(key: Int, value: String)
def main(args: Array[String]) {
val sc = new SparkContext("local", "HiveFromSpark")
// A local hive context creates an instance of the Hive Metastore in process, storing the
// the warehouse data in the current directory. This location can be overridden by
// specifying a second parameter to the constructor.
val hiveContext = new LocalHiveContext(sc)
import hiveContext._
sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src")
// Queries are expressed in HiveQL
println("Result of 'SELECT *': ")
sql("SELECT * FROM src").collect.foreach(println)
// Aggregation queries are also supported.
val count = sql("SELECT COUNT(*) FROM src").collect().head.getInt(0)
println(s"COUNT(*): $count")
// The results of SQL queries are themselves RDDs and support all normal RDD functions. The
// items in the RDD are of type Row, which allows you to access each column by ordinal.
val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key")
println("Result of RDD.map:")
val rddAsStrings = rddFromSql.map {
case Row(key: Int, value: String) => s"Key: $key, Value: $value"
}
// You can also register RDDs as temporary tables within a HiveContext.
val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i")))
rdd.registerAsTable("records")
// Queries can then join RDD data with data stored in Hive.
println("Result of SELECT *:")
sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println)
}
}

View file

@ -0,0 +1,71 @@
/*
* 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.examples
import org.apache.spark.SparkContext
import org.apache.spark.sql.SQLContext
// One method for defining the schema of an RDD is to make a case class with the desired column
// names and types.
case class Record(key: Int, value: String)
object RDDRelation {
def main(args: Array[String]) {
val sc = new SparkContext("local", "RDDRelation")
val sqlContext = new SQLContext(sc)
// Importing the SQL context gives access to all the SQL functions and implicit conversions.
import sqlContext._
val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i")))
// Any RDD containing case classes can be registered as a table. The schema of the table is
// automatically inferred using scala reflection.
rdd.registerAsTable("records")
// Once tables have been registered, you can run SQL queries over them.
println("Result of SELECT *:")
sql("SELECT * FROM records").collect().foreach(println)
// Aggregation queries are also supported.
val count = sql("SELECT COUNT(*) FROM records").collect().head.getInt(0)
println(s"COUNT(*): $count")
// The results of SQL queries are themselves RDDs and support all normal RDD functions. The
// items in the RDD are of type Row, which allows you to access each column by ordinal.
val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10")
println("Result of RDD.map:")
rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect.foreach(println)
// Queries can also be written using a LINQ-like Scala DSL.
rdd.where('key === 1).orderBy('value.asc).select('key).collect().foreach(println)
// Write out an RDD as a parquet file.
rdd.saveAsParquetFile("pair.parquet")
// Read in parquet file. Parquet files are self-describing so the schmema is preserved.
val parquetFile = sqlContext.parquetFile("pair.parquet")
// Queries can be run using the DSL on parequet files just like the original RDD.
parquetFile.where('key === 1).select('value as 'a).collect().foreach(println)
// These files can also be registered as tables.
parquetFile.registerAsTable("parquetFile")
sql("SELECT * FROM parquetFile").collect().foreach(println)
}
}

View file

@ -29,7 +29,7 @@
<artifactId>spark-graphx_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project GraphX</name>
<url>http://spark-project.org/</url>
<url>http://spark.apache.org/</url>
<profiles>
<profile>

View file

@ -91,6 +91,9 @@
<module>mllib</module>
<module>tools</module>
<module>streaming</module>
<module>sql/catalyst</module>
<module>sql/core</module>
<module>sql/hive</module>
<module>repl</module>
<module>assembly</module>
<module>external/twitter</module>
@ -118,6 +121,8 @@
<protobuf.version>2.4.1</protobuf.version>
<yarn.version>0.23.7</yarn.version>
<hbase.version>0.94.6</hbase.version>
<hive.version>0.12.0</hive.version>
<parquet.version>1.3.2</parquet.version>
<PermGen>64m</PermGen>
<MaxPermGen>512m</MaxPermGen>

View file

@ -52,7 +52,7 @@ object SparkBuild extends Build {
lazy val core = Project("core", file("core"), settings = coreSettings)
lazy val repl = Project("repl", file("repl"), settings = replSettings)
.dependsOn(core, graphx, bagel, mllib)
.dependsOn(core, graphx, bagel, mllib, sql)
lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming)
@ -60,12 +60,19 @@ object SparkBuild extends Build {
lazy val graphx = Project("graphx", file("graphx"), settings = graphxSettings) dependsOn(core)
lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) dependsOn(core)
lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst)
// Since hive is its own assembly, it depends on all of the modules.
lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql, graphx, bagel, mllib, streaming, repl)
lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core)
lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
.dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*)
.dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*)
lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects")
@ -131,13 +138,13 @@ object SparkBuild extends Build {
lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
.dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
.dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter, hive) dependsOn(allExternal: _*)
// Everything except assembly, tools, java8Tests and examples belong to packageProjects
lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef ++ maybeGangliaRef
// Everything except assembly, hive, tools, java8Tests and examples belong to packageProjects
lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeGangliaRef
lazy val allProjects = packageProjects ++ allExternalRefs ++
Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests
Seq[ProjectReference](examples, tools, assemblyProj, hive) ++ maybeJava8Tests
def sharedSettings = Defaults.defaultSettings ++ Seq(
organization := "org.apache.spark",
@ -164,7 +171,7 @@ object SparkBuild extends Build {
// Show full stack trace and duration in test cases.
testOptions in Test += Tests.Argument("-oDF"),
// Remove certain packages from Scaladoc
scalacOptions in (Compile,doc) := Seq("-skip-packages", Seq(
scalacOptions in (Compile,doc) := Seq("-groups", "-skip-packages", Seq(
"akka",
"org.apache.spark.network",
"org.apache.spark.deploy",
@ -362,6 +369,61 @@ object SparkBuild extends Build {
)
)
def catalystSettings = sharedSettings ++ Seq(
name := "catalyst",
// The mechanics of rewriting expression ids to compare trees in some test cases makes
// assumptions about the the expression ids being contiguious. Running tests in parallel breaks
// this non-deterministically. TODO: FIX THIS.
parallelExecution in Test := false,
libraryDependencies ++= Seq(
"org.scalatest" %% "scalatest" % "1.9.1" % "test",
"com.typesafe" %% "scalalogging-slf4j" % "1.0.1"
)
)
def sqlCoreSettings = sharedSettings ++ Seq(
name := "spark-sql",
libraryDependencies ++= Seq(
"com.twitter" % "parquet-column" % "1.3.2",
"com.twitter" % "parquet-hadoop" % "1.3.2"
)
)
// Since we don't include hive in the main assembly this project also acts as an alternative
// assembly jar.
def hiveSettings = sharedSettings ++ assemblyProjSettings ++ Seq(
name := "spark-hive",
jarName in assembly <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" },
jarName in packageDependency <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + "-deps.jar" },
javaOptions += "-XX:MaxPermSize=1g",
libraryDependencies ++= Seq(
"org.apache.hive" % "hive-metastore" % "0.12.0",
"org.apache.hive" % "hive-exec" % "0.12.0",
"org.apache.hive" % "hive-serde" % "0.12.0"
),
// Multiple queries rely on the TestHive singleton. See comments there for more details.
parallelExecution in Test := false,
// Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
// only for this subproject.
scalacOptions <<= scalacOptions map { currentOpts: Seq[String] =>
currentOpts.filterNot(_ == "-deprecation")
},
initialCommands in console :=
"""
|import org.apache.spark.sql.catalyst.analysis._
|import org.apache.spark.sql.catalyst.dsl._
|import org.apache.spark.sql.catalyst.errors._
|import org.apache.spark.sql.catalyst.expressions._
|import org.apache.spark.sql.catalyst.plans.logical._
|import org.apache.spark.sql.catalyst.rules._
|import org.apache.spark.sql.catalyst.types._
|import org.apache.spark.sql.catalyst.util._
|import org.apache.spark.sql.execution
|import org.apache.spark.sql.hive._
|import org.apache.spark.sql.hive.TestHive._
|import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin
)
def streamingSettings = sharedSettings ++ Seq(
name := "spark-streaming",
libraryDependencies ++= Seq(

80
sql/README.md Normal file
View file

@ -0,0 +1,80 @@
Spark SQL
=========
This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL.
Spark SQL is broken up into three subprojects:
- Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions.
- Execution (sql/core) - A query planner / execution engine for translating Catalysts logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files.
- Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs.
Other dependencies for developers
---------------------------------
In order to create new hive test cases , you will need to set several environmental variables.
```
export HIVE_HOME="<path to>/hive/build/dist"
export HIVE_DEV_HOME="<path to>/hive/"
export HADOOP_HOME="<path to>/hadoop-1.0.4"
```
Using the console
=================
An interactive scala console can be invoked by running `sbt/sbt hive/console`. From here you can execute queries and inspect the various stages of query optimization.
```scala
catalyst$ sbt/sbt hive/console
[info] Starting scala interpreter...
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.dsl._
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.TestHive._
Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45).
Type in expressions to have them evaluated.
Type :help for more information.
scala> val query = sql("SELECT * FROM (SELECT * FROM src) a")
query: org.apache.spark.sql.ExecutedQuery =
SELECT * FROM (SELECT * FROM src) a
=== Query Plan ===
Project [key#6:0.0,value#7:0.1]
HiveTableScan [key#6,value#7], (MetastoreRelation default, src, None), None
```
Query results are RDDs and can be operated as such.
```
scala> query.collect()
res8: Array[org.apache.spark.sql.execution.Row] = Array([238,val_238], [86,val_86], [311,val_311]...
```
You can also build further queries on top of these RDDs using the query DSL.
```
scala> query.where('key === 100).toRdd.collect()
res11: Array[org.apache.spark.sql.execution.Row] = Array([100,val_100], [100,val_100])
```
From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://databricks.github.io/catalyst/latest/api/#catalyst.trees.TreeNode) objects.
```scala
scala> query.logicalPlan
res1: catalyst.plans.logical.LogicalPlan =
Project {key#0,value#1}
Project {key#0,value#1}
MetastoreRelation default, src, None
scala> query.logicalPlan transform {
| case Project(projectList, child) if projectList == child.output => child
| }
res2: catalyst.plans.logical.LogicalPlan =
Project {key#0,value#1}
MetastoreRelation default, src, None
```

66
sql/catalyst/pom.xml Normal file
View file

@ -0,0 +1,66 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
<version>1.0.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-catalyst_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Catalyst</name>
<url>http://spark.apache.org/</url>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>com.typesafe</groupId>
<artifactId>scalalogging-slf4j_${scala.binary.version}</artifactId>
<version>1.0.1</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
<artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>

View file

@ -0,0 +1,328 @@
/*
* 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.catalyst
import scala.util.matching.Regex
import scala.util.parsing.combinator._
import scala.util.parsing.input.CharArrayReader.EofCh
import lexical._
import syntactical._
import token._
import analysis._
import expressions._
import plans._
import plans.logical._
import types._
/**
* A very simple SQL parser. Based loosly on:
* https://github.com/stephentu/scala-sql-parser/blob/master/src/main/scala/parser.scala
*
* Limitations:
* - Only supports a very limited subset of SQL.
* - Keywords must be capital.
*
* This is currently included mostly for illustrative purposes. Users wanting more complete support
* for a SQL like language should checkout the HiveQL support in the sql/hive subproject.
*/
class SqlParser extends StandardTokenParsers {
def apply(input: String): LogicalPlan = {
phrase(query)(new lexical.Scanner(input)) match {
case Success(r, x) => r
case x => sys.error(x.toString)
}
}
protected case class Keyword(str: String)
protected implicit def asParser(k: Keyword): Parser[String] = k.str
protected class SqlLexical extends StdLexical {
case class FloatLit(chars: String) extends Token {
override def toString = chars
}
override lazy val token: Parser[Token] = (
identChar ~ rep( identChar | digit ) ^^
{ case first ~ rest => processIdent(first :: rest mkString "") }
| rep1(digit) ~ opt('.' ~> rep(digit)) ^^ {
case i ~ None => NumericLit(i mkString "")
case i ~ Some(d) => FloatLit(i.mkString("") + "." + d.mkString(""))
}
| '\'' ~ rep( chrExcept('\'', '\n', EofCh) ) ~ '\'' ^^
{ case '\'' ~ chars ~ '\'' => StringLit(chars mkString "") }
| '\"' ~ rep( chrExcept('\"', '\n', EofCh) ) ~ '\"' ^^
{ case '\"' ~ chars ~ '\"' => StringLit(chars mkString "") }
| EofCh ^^^ EOF
| '\'' ~> failure("unclosed string literal")
| '\"' ~> failure("unclosed string literal")
| delim
| failure("illegal character")
)
override def identChar = letter | elem('.') | elem('_')
override def whitespace: Parser[Any] = rep(
whitespaceChar
| '/' ~ '*' ~ comment
| '/' ~ '/' ~ rep( chrExcept(EofCh, '\n') )
| '#' ~ rep( chrExcept(EofCh, '\n') )
| '-' ~ '-' ~ rep( chrExcept(EofCh, '\n') )
| '/' ~ '*' ~ failure("unclosed comment")
)
}
override val lexical = new SqlLexical
protected val ALL = Keyword("ALL")
protected val AND = Keyword("AND")
protected val AS = Keyword("AS")
protected val ASC = Keyword("ASC")
protected val AVG = Keyword("AVG")
protected val BY = Keyword("BY")
protected val CAST = Keyword("CAST")
protected val COUNT = Keyword("COUNT")
protected val DESC = Keyword("DESC")
protected val DISTINCT = Keyword("DISTINCT")
protected val FALSE = Keyword("FALSE")
protected val FIRST = Keyword("FIRST")
protected val FROM = Keyword("FROM")
protected val FULL = Keyword("FULL")
protected val GROUP = Keyword("GROUP")
protected val HAVING = Keyword("HAVING")
protected val IF = Keyword("IF")
protected val IN = Keyword("IN")
protected val INNER = Keyword("INNER")
protected val IS = Keyword("IS")
protected val JOIN = Keyword("JOIN")
protected val LEFT = Keyword("LEFT")
protected val LIMIT = Keyword("LIMIT")
protected val NOT = Keyword("NOT")
protected val NULL = Keyword("NULL")
protected val ON = Keyword("ON")
protected val OR = Keyword("OR")
protected val ORDER = Keyword("ORDER")
protected val OUTER = Keyword("OUTER")
protected val RIGHT = Keyword("RIGHT")
protected val SELECT = Keyword("SELECT")
protected val STRING = Keyword("STRING")
protected val SUM = Keyword("SUM")
protected val TRUE = Keyword("TRUE")
protected val UNION = Keyword("UNION")
protected val WHERE = Keyword("WHERE")
// Use reflection to find the reserved words defined in this class.
protected val reservedWords =
this.getClass
.getMethods
.filter(_.getReturnType == classOf[Keyword])
.map(_.invoke(this).asInstanceOf[Keyword])
lexical.reserved ++= reservedWords.map(_.str)
lexical.delimiters += (
"@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
",", ";", "%", "{", "}", ":"
)
protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = {
exprs.zipWithIndex.map {
case (ne: NamedExpression, _) => ne
case (e, i) => Alias(e, s"c$i")()
}
}
protected lazy val query: Parser[LogicalPlan] =
select * (
UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } |
UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) }
)
protected lazy val select: Parser[LogicalPlan] =
SELECT ~> opt(DISTINCT) ~ projections ~
opt(from) ~ opt(filter) ~
opt(grouping) ~
opt(having) ~
opt(orderBy) ~
opt(limit) <~ opt(";") ^^ {
case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l =>
val base = r.getOrElse(NoRelation)
val withFilter = f.map(f => Filter(f, base)).getOrElse(base)
val withProjection =
g.map {g =>
Aggregate(assignAliases(g), assignAliases(p), withFilter)
}.getOrElse(Project(assignAliases(p), withFilter))
val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection)
val withHaving = h.map(h => Filter(h, withDistinct)).getOrElse(withDistinct)
val withOrder = o.map(o => Sort(o, withHaving)).getOrElse(withHaving)
val withLimit = l.map { l => StopAfter(l, withOrder) }.getOrElse(withOrder)
withLimit
}
protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",")
protected lazy val projection: Parser[Expression] =
expression ~ (opt(AS) ~> opt(ident)) ^^ {
case e ~ None => e
case e ~ Some(a) => Alias(e, a)()
}
protected lazy val from: Parser[LogicalPlan] = FROM ~> relations
// Based very loosly on the MySQL Grammar.
// http://dev.mysql.com/doc/refman/5.0/en/join.html
protected lazy val relations: Parser[LogicalPlan] =
relation ~ "," ~ relation ^^ { case r1 ~ _ ~ r2 => Join(r1, r2, Inner, None) } |
relation
protected lazy val relation: Parser[LogicalPlan] =
joinedRelation |
relationFactor
protected lazy val relationFactor: Parser[LogicalPlan] =
ident ~ (opt(AS) ~> opt(ident)) ^^ {
case ident ~ alias => UnresolvedRelation(alias, ident)
} |
"(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) }
protected lazy val joinedRelation: Parser[LogicalPlan] =
relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ {
case r1 ~ jt ~ _ ~ r2 ~ cond =>
Join(r1, r2, joinType = jt.getOrElse(Inner), cond)
}
protected lazy val joinConditions: Parser[Expression] =
ON ~> expression
protected lazy val joinType: Parser[JoinType] =
INNER ^^^ Inner |
LEFT ~ opt(OUTER) ^^^ LeftOuter |
RIGHT ~ opt(OUTER) ^^^ RightOuter |
FULL ~ opt(OUTER) ^^^ FullOuter
protected lazy val filter: Parser[Expression] = WHERE ~ expression ^^ { case _ ~ e => e }
protected lazy val orderBy: Parser[Seq[SortOrder]] =
ORDER ~> BY ~> ordering
protected lazy val ordering: Parser[Seq[SortOrder]] =
rep1sep(singleOrder, ",") |
rep1sep(expression, ",") ~ opt(direction) ^^ {
case exps ~ None => exps.map(SortOrder(_, Ascending))
case exps ~ Some(d) => exps.map(SortOrder(_, d))
}
protected lazy val singleOrder: Parser[SortOrder] =
expression ~ direction ^^ { case e ~ o => SortOrder(e,o) }
protected lazy val direction: Parser[SortDirection] =
ASC ^^^ Ascending |
DESC ^^^ Descending
protected lazy val grouping: Parser[Seq[Expression]] =
GROUP ~> BY ~> rep1sep(expression, ",")
protected lazy val having: Parser[Expression] =
HAVING ~> expression
protected lazy val limit: Parser[Expression] =
LIMIT ~> expression
protected lazy val expression: Parser[Expression] = orExpression
protected lazy val orExpression: Parser[Expression] =
andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1,e2) })
protected lazy val andExpression: Parser[Expression] =
comparisionExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) })
protected lazy val comparisionExpression: Parser[Expression] =
termExpression ~ "=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Equals(e1, e2) } |
termExpression ~ "<" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThan(e1, e2) } |
termExpression ~ "<=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThanOrEqual(e1, e2) } |
termExpression ~ ">" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThan(e1, e2) } |
termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } |
termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } |
termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } |
termExpression ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ {
case e1 ~ _ ~ _ ~ e2 => In(e1, e2)
} |
termExpression ~ NOT ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ {
case e1 ~ _ ~ _ ~ _ ~ e2 => Not(In(e1, e2))
} |
termExpression <~ IS ~ NULL ^^ { case e => IsNull(e) } |
termExpression <~ IS ~ NOT ~ NULL ^^ { case e => IsNotNull(e) } |
NOT ~> termExpression ^^ {e => Not(e)} |
termExpression
protected lazy val termExpression: Parser[Expression] =
productExpression * (
"+" ^^^ { (e1: Expression, e2: Expression) => Add(e1,e2) } |
"-" ^^^ { (e1: Expression, e2: Expression) => Subtract(e1,e2) } )
protected lazy val productExpression: Parser[Expression] =
baseExpression * (
"*" ^^^ { (e1: Expression, e2: Expression) => Multiply(e1,e2) } |
"/" ^^^ { (e1: Expression, e2: Expression) => Divide(e1,e2) } |
"%" ^^^ { (e1: Expression, e2: Expression) => Remainder(e1,e2) }
)
protected lazy val function: Parser[Expression] =
SUM ~> "(" ~> expression <~ ")" ^^ { case exp => Sum(exp) } |
SUM ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => SumDistinct(exp) } |
COUNT ~> "(" ~ "*" <~ ")" ^^ { case _ => Count(Literal(1)) } |
COUNT ~> "(" ~ expression <~ ")" ^^ { case dist ~ exp => Count(exp) } |
COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } |
FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } |
AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } |
IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ {
case c ~ "," ~ t ~ "," ~ f => If(c,t,f)
} |
ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ {
case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs)
}
protected lazy val cast: Parser[Expression] =
CAST ~> "(" ~> expression ~ AS ~ dataType <~ ")" ^^ { case exp ~ _ ~ t => Cast(exp, t) }
protected lazy val literal: Parser[Literal] =
numericLit ^^ {
case i if i.toLong > Int.MaxValue => Literal(i.toLong)
case i => Literal(i.toInt)
} |
NULL ^^^ Literal(null, NullType) |
floatLit ^^ {case f => Literal(f.toDouble) } |
stringLit ^^ {case s => Literal(s, StringType) }
protected lazy val floatLit: Parser[String] =
elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars)
protected lazy val baseExpression: Parser[Expression] =
TRUE ^^^ Literal(true, BooleanType) |
FALSE ^^^ Literal(false, BooleanType) |
cast |
"(" ~> expression <~ ")" |
function |
"-" ~> literal ^^ UnaryMinus |
ident ^^ UnresolvedAttribute |
"*" ^^^ Star(None) |
literal
protected lazy val dataType: Parser[DataType] =
STRING ^^^ StringType
}

View file

@ -0,0 +1,185 @@
/*
* 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
package catalyst
package analysis
import expressions._
import plans.logical._
import rules._
/**
* A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing
* when all relations are already filled in and the analyser needs only to resolve attribute
* references.
*/
object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true)
/**
* Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and
* [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and
* a [[FunctionRegistry]].
*/
class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean)
extends RuleExecutor[LogicalPlan] with HiveTypeCoercion {
// TODO: pass this in as a parameter.
val fixedPoint = FixedPoint(100)
val batches: Seq[Batch] = Seq(
Batch("MultiInstanceRelations", Once,
NewRelationInstances),
Batch("CaseInsensitiveAttributeReferences", Once,
(if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*),
Batch("Resolution", fixedPoint,
ResolveReferences ::
ResolveRelations ::
NewRelationInstances ::
ImplicitGenerate ::
StarExpansion ::
ResolveFunctions ::
GlobalAggregates ::
typeCoercionRules :_*)
)
/**
* Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
*/
object ResolveRelations extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case UnresolvedRelation(databaseName, name, alias) =>
catalog.lookupRelation(databaseName, name, alias)
}
}
/**
* Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase.
*/
object LowercaseAttributeReferences extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case UnresolvedRelation(databaseName, name, alias) =>
UnresolvedRelation(databaseName, name, alias.map(_.toLowerCase))
case Subquery(alias, child) => Subquery(alias.toLowerCase, child)
case q: LogicalPlan => q transformExpressions {
case s: Star => s.copy(table = s.table.map(_.toLowerCase))
case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase)
case Alias(c, name) => Alias(c, name.toLowerCase)()
}
}
}
/**
* Replaces [[UnresolvedAttribute]]s with concrete
* [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children.
*/
object ResolveReferences extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case q: LogicalPlan if q.childrenResolved =>
logger.trace(s"Attempting to resolve ${q.simpleString}")
q transformExpressions {
case u @ UnresolvedAttribute(name) =>
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
val result = q.resolve(name).getOrElse(u)
logger.debug(s"Resolving $u to $result")
result
}
}
}
/**
* Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]].
*/
object ResolveFunctions extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case q: LogicalPlan =>
q transformExpressions {
case u @ UnresolvedFunction(name, children) if u.childrenResolved =>
registry.lookupFunction(name, children)
}
}
}
/**
* Turns projections that contain aggregate expressions into aggregations.
*/
object GlobalAggregates extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case Project(projectList, child) if containsAggregates(projectList) =>
Aggregate(Nil, projectList, child)
}
def containsAggregates(exprs: Seq[Expression]): Boolean = {
exprs.foreach(_.foreach {
case agg: AggregateExpression => return true
case _ =>
})
false
}
}
/**
* When a SELECT clause has only a single expression and that expression is a
* [[catalyst.expressions.Generator Generator]] we convert the
* [[catalyst.plans.logical.Project Project]] to a [[catalyst.plans.logical.Generate Generate]].
*/
object ImplicitGenerate extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case Project(Seq(Alias(g: Generator, _)), child) =>
Generate(g, join = false, outer = false, None, child)
}
}
/**
* Expands any references to [[Star]] (*) in project operators.
*/
object StarExpansion extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// Wait until children are resolved
case p: LogicalPlan if !p.childrenResolved => p
// If the projection list contains Stars, expand it.
case p @ Project(projectList, child) if containsStar(projectList) =>
Project(
projectList.flatMap {
case s: Star => s.expand(child.output)
case o => o :: Nil
},
child)
case t: ScriptTransformation if containsStar(t.input) =>
t.copy(
input = t.input.flatMap {
case s: Star => s.expand(t.child.output)
case o => o :: Nil
}
)
// If the aggregate function argument contains Stars, expand it.
case a: Aggregate if containsStar(a.aggregateExpressions) =>
a.copy(
aggregateExpressions = a.aggregateExpressions.flatMap {
case s: Star => s.expand(a.child.output)
case o => o :: Nil
}
)
}
/**
* Returns true if `exprs` contains a [[Star]].
*/
protected def containsStar(exprs: Seq[Expression]): Boolean =
exprs.collect { case _: Star => true }.nonEmpty
}
}

View file

@ -0,0 +1,107 @@
/*
* 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
package catalyst
package analysis
import plans.logical.{LogicalPlan, Subquery}
import scala.collection.mutable
/**
* An interface for looking up relations by name. Used by an [[Analyzer]].
*/
trait Catalog {
def lookupRelation(
databaseName: Option[String],
tableName: String,
alias: Option[String] = None): LogicalPlan
def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit
}
class SimpleCatalog extends Catalog {
val tables = new mutable.HashMap[String, LogicalPlan]()
def registerTable(databaseName: Option[String],tableName: String, plan: LogicalPlan): Unit = {
tables += ((tableName, plan))
}
def dropTable(tableName: String) = tables -= tableName
def lookupRelation(
databaseName: Option[String],
tableName: String,
alias: Option[String] = None): LogicalPlan = {
val table = tables.get(tableName).getOrElse(sys.error(s"Table Not Found: $tableName"))
// If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are
// properly qualified with this alias.
alias.map(a => Subquery(a.toLowerCase, table)).getOrElse(table)
}
}
/**
* A trait that can be mixed in with other Catalogs allowing specific tables to be overridden with
* new logical plans. This can be used to bind query result to virtual tables, or replace tables
* with in-memory cached versions. Note that the set of overrides is stored in memory and thus
* lost when the JVM exits.
*/
trait OverrideCatalog extends Catalog {
// TODO: This doesn't work when the database changes...
val overrides = new mutable.HashMap[(Option[String],String), LogicalPlan]()
abstract override def lookupRelation(
databaseName: Option[String],
tableName: String,
alias: Option[String] = None): LogicalPlan = {
val overriddenTable = overrides.get((databaseName, tableName))
// If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are
// properly qualified with this alias.
val withAlias =
overriddenTable.map(r => alias.map(a => Subquery(a.toLowerCase, r)).getOrElse(r))
withAlias.getOrElse(super.lookupRelation(databaseName, tableName, alias))
}
override def registerTable(
databaseName: Option[String],
tableName: String,
plan: LogicalPlan): Unit = {
overrides.put((databaseName, tableName), plan)
}
}
/**
* A trivial catalog that returns an error when a relation is requested. Used for testing when all
* relations are already filled in and the analyser needs only to resolve attribute references.
*/
object EmptyCatalog extends Catalog {
def lookupRelation(
databaseName: Option[String],
tableName: String,
alias: Option[String] = None) = {
throw new UnsupportedOperationException
}
def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = {
throw new UnsupportedOperationException
}
}

View file

@ -0,0 +1,37 @@
/*
* 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
package catalyst
package analysis
import expressions._
/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */
trait FunctionRegistry {
def lookupFunction(name: String, children: Seq[Expression]): Expression
}
/**
* A trivial catalog that returns an error when a function is requested. Used for testing when all
* functions are already filled in and the analyser needs only to resolve attribute references.
*/
object EmptyFunctionRegistry extends FunctionRegistry {
def lookupFunction(name: String, children: Seq[Expression]): Expression = {
throw new UnsupportedOperationException
}
}

View file

@ -0,0 +1,275 @@
/*
* 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
package catalyst
package analysis
import expressions._
import plans.logical._
import rules._
import types._
/**
* A collection of [[catalyst.rules.Rule Rules]] that can be used to coerce differing types that
* participate in operations into compatible ones. Most of these rules are based on Hive semantics,
* but they do not introduce any dependencies on the hive codebase. For this reason they remain in
* Catalyst until we have a more standard set of coercions.
*/
trait HiveTypeCoercion {
val typeCoercionRules =
List(PropagateTypes, ConvertNaNs, WidenTypes, PromoteStrings, BooleanComparisons, BooleanCasts,
StringToIntegralCasts, FunctionArgumentConversion)
/**
* Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes
* that are made by other rules to instances higher in the query tree.
*/
object PropagateTypes extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// No propagation required for leaf nodes.
case q: LogicalPlan if q.children.isEmpty => q
// Don't propagate types from unresolved children.
case q: LogicalPlan if !q.childrenResolved => q
case q: LogicalPlan => q transformExpressions {
case a: AttributeReference =>
q.inputSet.find(_.exprId == a.exprId) match {
// This can happen when a Attribute reference is born in a non-leaf node, for example
// due to a call to an external script like in the Transform operator.
// TODO: Perhaps those should actually be aliases?
case None => a
// Leave the same if the dataTypes match.
case Some(newType) if a.dataType == newType.dataType => a
case Some(newType) =>
logger.debug(s"Promoting $a to $newType in ${q.simpleString}}")
newType
}
}
}
}
/**
* Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to
* the appropriate numeric equivalent.
*/
object ConvertNaNs extends Rule[LogicalPlan] {
val stringNaN = Literal("NaN", StringType)
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case q: LogicalPlan => q transformExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
/* Double Conversions */
case b: BinaryExpression if b.left == stringNaN && b.right.dataType == DoubleType =>
b.makeCopy(Array(b.right, Literal(Double.NaN)))
case b: BinaryExpression if b.left.dataType == DoubleType && b.right == stringNaN =>
b.makeCopy(Array(Literal(Double.NaN), b.left))
case b: BinaryExpression if b.left == stringNaN && b.right == stringNaN =>
b.makeCopy(Array(Literal(Double.NaN), b.left))
/* Float Conversions */
case b: BinaryExpression if b.left == stringNaN && b.right.dataType == FloatType =>
b.makeCopy(Array(b.right, Literal(Float.NaN)))
case b: BinaryExpression if b.left.dataType == FloatType && b.right == stringNaN =>
b.makeCopy(Array(Literal(Float.NaN), b.left))
case b: BinaryExpression if b.left == stringNaN && b.right == stringNaN =>
b.makeCopy(Array(Literal(Float.NaN), b.left))
}
}
}
/**
* Widens numeric types and converts strings to numbers when appropriate.
*
* Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White
*
* The implicit conversion rules can be summarized as follows:
* - Any integral numeric type can be implicitly converted to a wider type.
* - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly
* converted to DOUBLE.
* - TINYINT, SMALLINT, and INT can all be converted to FLOAT.
* - BOOLEAN types cannot be converted to any other type.
*
* Additionally, all types when UNION-ed with strings will be promoted to strings.
* Other string conversions are handled by PromoteStrings.
*/
object WidenTypes extends Rule[LogicalPlan] {
// See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types.
// The conversion for integral and floating point types have a linear widening hierarchy:
val numericPrecedence =
Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType)
// Boolean is only wider than Void
val booleanPrecedence = Seq(NullType, BooleanType)
val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil
def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = {
// Try and find a promotion rule that contains both types in question.
val applicableConversion = allPromotions.find(p => p.contains(t1) && p.contains(t2))
// If found return the widest common type, otherwise None
applicableConversion.map(_.filter(t => t == t1 || t == t2).last)
}
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case u @ Union(left, right) if u.childrenResolved && !u.resolved =>
val castedInput = left.output.zip(right.output).map {
// When a string is found on one side, make the other side a string too.
case (l, r) if l.dataType == StringType && r.dataType != StringType =>
(l, Alias(Cast(r, StringType), r.name)())
case (l, r) if l.dataType != StringType && r.dataType == StringType =>
(Alias(Cast(l, StringType), l.name)(), r)
case (l, r) if l.dataType != r.dataType =>
logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
findTightestCommonType(l.dataType, r.dataType).map { widestType =>
val newLeft =
if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)()
val newRight =
if (r.dataType == widestType) r else Alias(Cast(r, widestType), r.name)()
(newLeft, newRight)
}.getOrElse((l, r)) // If there is no applicable conversion, leave expression unchanged.
case other => other
}
val (castedLeft, castedRight) = castedInput.unzip
val newLeft =
if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) {
logger.debug(s"Widening numeric types in union $castedLeft ${left.output}")
Project(castedLeft, left)
} else {
left
}
val newRight =
if (castedRight.map(_.dataType) != right.output.map(_.dataType)) {
logger.debug(s"Widening numeric types in union $castedRight ${right.output}")
Project(castedRight, right)
} else {
right
}
Union(newLeft, newRight)
// Also widen types for BinaryExpressions.
case q: LogicalPlan => q transformExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
case b: BinaryExpression if b.left.dataType != b.right.dataType =>
findTightestCommonType(b.left.dataType, b.right.dataType).map { widestType =>
val newLeft =
if (b.left.dataType == widestType) b.left else Cast(b.left, widestType)
val newRight =
if (b.right.dataType == widestType) b.right else Cast(b.right, widestType)
b.makeCopy(Array(newLeft, newRight))
}.getOrElse(b) // If there is no applicable conversion, leave expression unchanged.
}
}
}
/**
* Promotes strings that appear in arithmetic expressions.
*/
object PromoteStrings extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
case a: BinaryArithmetic if a.left.dataType == StringType =>
a.makeCopy(Array(Cast(a.left, DoubleType), a.right))
case a: BinaryArithmetic if a.right.dataType == StringType =>
a.makeCopy(Array(a.left, Cast(a.right, DoubleType)))
case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType =>
p.makeCopy(Array(Cast(p.left, DoubleType), p.right))
case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType =>
p.makeCopy(Array(p.left, Cast(p.right, DoubleType)))
case Sum(e) if e.dataType == StringType =>
Sum(Cast(e, DoubleType))
case Average(e) if e.dataType == StringType =>
Average(Cast(e, DoubleType))
}
}
/**
* Changes Boolean values to Bytes so that expressions like true < false can be Evaluated.
*/
object BooleanComparisons extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
// No need to change Equals operators as that actually makes sense for boolean types.
case e: Equals => e
// Otherwise turn them to Byte types so that there exists and ordering.
case p: BinaryComparison
if p.left.dataType == BooleanType && p.right.dataType == BooleanType =>
p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType)))
}
}
/**
* Casts to/from [[catalyst.types.BooleanType BooleanType]] are transformed into comparisons since
* the JVM does not consider Booleans to be numeric types.
*/
object BooleanCasts extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
case Cast(e, BooleanType) => Not(Equals(e, Literal(0)))
case Cast(e, dataType) if e.dataType == BooleanType =>
Cast(If(e, Literal(1), Literal(0)), dataType)
}
}
/**
* When encountering a cast from a string representing a valid fractional number to an integral
* type the jvm will throw a `java.lang.NumberFormatException`. Hive, in contrast, returns the
* truncated version of this number.
*/
object StringToIntegralCasts extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
case Cast(e @ StringType(), t: IntegralType) =>
Cast(Cast(e, DecimalType), t)
}
}
/**
* This ensure that the types for various functions are as expected.
*/
object FunctionArgumentConversion extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
// Promote SUM to largest types to prevent overflows.
case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest.
case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType))
case Sum(e @ FractionalType()) if e.dataType != DoubleType => Sum(Cast(e, DoubleType))
}
}
}

View file

@ -0,0 +1,54 @@
/*
* 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.catalyst
package analysis
import plans.logical.LogicalPlan
import rules._
/**
* A trait that should be mixed into query operators where an single instance might appear multiple
* times in a logical query plan. It is invalid to have multiple copies of the same attribute
* produced by distinct operators in a query tree as this breaks the gurantee that expression
* ids, which are used to differentate attributes, are unique.
*
* Before analysis, all operators that include this trait will be asked to produce a new version
* of itself with globally unique expression ids.
*/
trait MultiInstanceRelation {
def newInstance: this.type
}
/**
* If any MultiInstanceRelation appears more than once in the query plan then the plan is updated so
* that each instance has unique expression ids for the attributes produced.
*/
object NewRelationInstances extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = {
val localRelations = plan collect { case l: MultiInstanceRelation => l}
val multiAppearance = localRelations
.groupBy(identity[MultiInstanceRelation])
.filter { case (_, ls) => ls.size > 1 }
.map(_._1)
.toSet
plan transform {
case l: MultiInstanceRelation if multiAppearance contains l => l.newInstance
}
}
}

View file

@ -0,0 +1,25 @@
/*
* 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 catalyst
/**
* Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis.
* Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s
* into fully typed objects using information in a schema [[Catalog]].
*/
package object analysis

View file

@ -0,0 +1,109 @@
/*
* 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
package catalyst
package analysis
import expressions._
import plans.logical.BaseRelation
import trees.TreeNode
/**
* Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully
* resolved.
*/
class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends
errors.TreeNodeException(tree, s"Invalid call to $function on unresolved object", null)
/**
* Holds the name of a relation that has yet to be looked up in a [[Catalog]].
*/
case class UnresolvedRelation(
databaseName: Option[String],
tableName: String,
alias: Option[String] = None) extends BaseRelation {
def output = Nil
override lazy val resolved = false
}
/**
* Holds the name of an attribute that has yet to be resolved.
*/
case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] {
def exprId = throw new UnresolvedException(this, "exprId")
def dataType = throw new UnresolvedException(this, "dataType")
def nullable = throw new UnresolvedException(this, "nullable")
def qualifiers = throw new UnresolvedException(this, "qualifiers")
override lazy val resolved = false
def newInstance = this
def withQualifiers(newQualifiers: Seq[String]) = this
override def toString: String = s"'$name"
}
case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression {
def exprId = throw new UnresolvedException(this, "exprId")
def dataType = throw new UnresolvedException(this, "dataType")
override def foldable = throw new UnresolvedException(this, "foldable")
def nullable = throw new UnresolvedException(this, "nullable")
def qualifiers = throw new UnresolvedException(this, "qualifiers")
def references = children.flatMap(_.references).toSet
override lazy val resolved = false
override def toString = s"'$name(${children.mkString(",")})"
}
/**
* Represents all of the input attributes to a given relational operator, for example in
* "SELECT * FROM ...".
*
* @param table an optional table that should be the target of the expansion. If omitted all
* tables' columns are produced.
*/
case class Star(
table: Option[String],
mapFunction: Attribute => Expression = identity[Attribute])
extends Attribute with trees.LeafNode[Expression] {
def name = throw new UnresolvedException(this, "exprId")
def exprId = throw new UnresolvedException(this, "exprId")
def dataType = throw new UnresolvedException(this, "dataType")
def nullable = throw new UnresolvedException(this, "nullable")
def qualifiers = throw new UnresolvedException(this, "qualifiers")
override lazy val resolved = false
def newInstance = this
def withQualifiers(newQualifiers: Seq[String]) = this
def expand(input: Seq[Attribute]): Seq[NamedExpression] = {
val expandedAttributes: Seq[Attribute] = table match {
// If there is no table specified, use all input attributes.
case None => input
// If there is a table, pick out attributes that are part of this table.
case Some(table) => input.filter(_.qualifiers contains table)
}
val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map {
case (n: NamedExpression, _) => n
case (e, originalAttribute) =>
Alias(e, originalAttribute.name)(qualifiers = originalAttribute.qualifiers)
}
mappedAttributes
}
override def toString = table.map(_ + ".").getOrElse("") + "*"
}

View file

@ -0,0 +1,224 @@
/*
* 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
package catalyst
import scala.language.implicitConversions
import scala.reflect.runtime.universe.TypeTag
import analysis.UnresolvedAttribute
import expressions._
import plans._
import plans.logical._
import types._
/**
* Provides experimental support for generating catalyst schemas for scala objects.
*/
object ScalaReflection {
import scala.reflect.runtime.universe._
/** Returns a Sequence of attributes for the given case class type. */
def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match {
case s: StructType =>
s.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)())
}
/** Returns a catalyst DataType for the given Scala Type using reflection. */
def schemaFor[T: TypeTag]: DataType = schemaFor(typeOf[T])
/** Returns a catalyst DataType for the given Scala Type using reflection. */
def schemaFor(tpe: `Type`): DataType = tpe match {
case t if t <:< typeOf[Product] =>
val params = t.member("<init>": TermName).asMethod.paramss
StructType(
params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true)))
case t if t <:< typeOf[Seq[_]] =>
val TypeRef(_, _, Seq(elementType)) = t
ArrayType(schemaFor(elementType))
case t if t <:< typeOf[String] => StringType
case t if t <:< definitions.IntTpe => IntegerType
case t if t <:< definitions.LongTpe => LongType
case t if t <:< definitions.DoubleTpe => DoubleType
case t if t <:< definitions.ShortTpe => ShortType
case t if t <:< definitions.ByteTpe => ByteType
}
implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) {
/**
* Implicitly added to Sequences of case class objects. Returns a catalyst logical relation
* for the the data in the sequence.
*/
def asRelation: LocalRelation = {
val output = attributesFor[A]
LocalRelation(output, data)
}
}
}
/**
* A collection of implicit conversions that create a DSL for constructing catalyst data structures.
*
* {{{
* scala> import catalyst.dsl._
*
* // Standard operators are added to expressions.
* scala> Literal(1) + Literal(1)
* res1: catalyst.expressions.Add = (1 + 1)
*
* // There is a conversion from 'symbols to unresolved attributes.
* scala> 'a.attr
* res2: catalyst.analysis.UnresolvedAttribute = 'a
*
* // These unresolved attributes can be used to create more complicated expressions.
* scala> 'a === 'b
* res3: catalyst.expressions.Equals = ('a = 'b)
*
* // SQL verbs can be used to construct logical query plans.
* scala> TestRelation('key.int, 'value.string).where('key === 1).select('value).analyze
* res4: catalyst.plans.logical.LogicalPlan =
* Project {value#1}
* Filter (key#0 = 1)
* TestRelation {key#0,value#1}
* }}}
*/
package object dsl {
trait ImplicitOperators {
def expr: Expression
def + (other: Expression) = Add(expr, other)
def - (other: Expression) = Subtract(expr, other)
def * (other: Expression) = Multiply(expr, other)
def / (other: Expression) = Divide(expr, other)
def && (other: Expression) = And(expr, other)
def || (other: Expression) = Or(expr, other)
def < (other: Expression) = LessThan(expr, other)
def <= (other: Expression) = LessThanOrEqual(expr, other)
def > (other: Expression) = GreaterThan(expr, other)
def >= (other: Expression) = GreaterThanOrEqual(expr, other)
def === (other: Expression) = Equals(expr, other)
def != (other: Expression) = Not(Equals(expr, other))
def asc = SortOrder(expr, Ascending)
def desc = SortOrder(expr, Descending)
def as(s: Symbol) = Alias(expr, s.name)()
}
trait ExpressionConversions {
implicit class DslExpression(e: Expression) extends ImplicitOperators {
def expr = e
}
implicit def intToLiteral(i: Int) = Literal(i)
implicit def longToLiteral(l: Long) = Literal(l)
implicit def floatToLiteral(f: Float) = Literal(f)
implicit def doubleToLiteral(d: Double) = Literal(d)
implicit def stringToLiteral(s: String) = Literal(s)
implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name)
implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name }
implicit class DslString(val s: String) extends ImplicitAttribute
abstract class ImplicitAttribute extends ImplicitOperators {
def s: String
def expr = attr
def attr = analysis.UnresolvedAttribute(s)
/** Creates a new typed attributes of type int */
def int = AttributeReference(s, IntegerType, nullable = false)()
/** Creates a new typed attributes of type string */
def string = AttributeReference(s, StringType, nullable = false)()
}
implicit class DslAttribute(a: AttributeReference) {
def notNull = a.withNullability(false)
def nullable = a.withNullability(true)
// Protobuf terminology
def required = a.withNullability(false)
}
}
object expressions extends ExpressionConversions // scalastyle:ignore
abstract class LogicalPlanFunctions {
def logicalPlan: LogicalPlan
def select(exprs: NamedExpression*) = Project(exprs, logicalPlan)
def where(condition: Expression) = Filter(condition, logicalPlan)
def join(
otherPlan: LogicalPlan,
joinType: JoinType = Inner,
condition: Option[Expression] = None) =
Join(logicalPlan, otherPlan, joinType, condition)
def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, logicalPlan)
def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = {
val aliasedExprs = aggregateExprs.map {
case ne: NamedExpression => ne
case e => Alias(e, e.toString)()
}
Aggregate(groupingExprs, aliasedExprs, logicalPlan)
}
def subquery(alias: Symbol) = Subquery(alias.name, logicalPlan)
def unionAll(otherPlan: LogicalPlan) = Union(logicalPlan, otherPlan)
def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) =
Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)
def sfilter(dynamicUdf: (DynamicRow) => Boolean) =
Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)
def sample(
fraction: Double,
withReplacement: Boolean = true,
seed: Int = (math.random * 1000).toInt) =
Sample(fraction, withReplacement, seed, logicalPlan)
def generate(
generator: Generator,
join: Boolean = false,
outer: Boolean = false,
alias: Option[String] = None) =
Generate(generator, join, outer, None, logicalPlan)
def insertInto(tableName: String, overwrite: Boolean = false) =
InsertIntoTable(
analysis.UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite)
def analyze = analysis.SimpleAnalyzer(logicalPlan)
}
object plans { // scalastyle:ignore
implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions {
def writeToFile(path: String) = WriteToFile(path, logicalPlan)
}
}
}

View file

@ -0,0 +1,57 @@
/*
* 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
package catalyst
import trees._
/**
* Functions for attaching and retrieving trees that are associated with errors.
*/
package object errors {
class TreeNodeException[TreeType <: TreeNode[_]]
(tree: TreeType, msg: String, cause: Throwable) extends Exception(msg, cause) {
// Yes, this is the same as a default parameter, but... those don't seem to work with SBT
// external project dependencies for some reason.
def this(tree: TreeType, msg: String) = this(tree, msg, null)
override def getMessage: String = {
val treeString = tree.toString
s"${super.getMessage}, tree:${if (treeString contains "\n") "\n" else " "}$tree"
}
}
/**
* Wraps any exceptions that are thrown while executing `f` in a
* [[catalyst.errors.TreeNodeException TreeNodeException]], attaching the provided `tree`.
*/
def attachTree[TreeType <: TreeNode[_], A](tree: TreeType, msg: String = "")(f: => A): A = {
try f catch {
case e: Exception => throw new TreeNodeException(tree, msg, e)
}
}
/**
* Executes `f` which is expected to throw a
* [[catalyst.errors.TreeNodeException TreeNodeException]]. The first tree encountered in
* the stack of exceptions of type `TreeType` is returned.
*/
def getTree[TreeType <: TreeNode[_]](f: => Unit): TreeType = ??? // TODO: Implement
}

View file

@ -0,0 +1,83 @@
/*
* 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
package catalyst
package expressions
import rules._
import errors._
import catalyst.plans.QueryPlan
/**
* A bound reference points to a specific slot in the input tuple, allowing the actual value
* to be retrieved more efficiently. However, since operations like column pruning can change
* the layout of intermediate tuples, BindReferences should be run after all such transformations.
*/
case class BoundReference(ordinal: Int, baseReference: Attribute)
extends Attribute with trees.LeafNode[Expression] {
type EvaluatedType = Any
def nullable = baseReference.nullable
def dataType = baseReference.dataType
def exprId = baseReference.exprId
def qualifiers = baseReference.qualifiers
def name = baseReference.name
def newInstance = BoundReference(ordinal, baseReference.newInstance)
def withQualifiers(newQualifiers: Seq[String]) =
BoundReference(ordinal, baseReference.withQualifiers(newQualifiers))
override def toString = s"$baseReference:$ordinal"
override def apply(input: Row): Any = input(ordinal)
}
class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] {
import BindReferences._
def apply(plan: TreeNode): TreeNode = {
plan.transform {
case leafNode if leafNode.children.isEmpty => leafNode
case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e =>
bindReference(e, unaryNode.children.head.output)
}
}
}
}
object BindReferences extends Logging {
def bindReference(expression: Expression, input: Seq[Attribute]): Expression = {
expression.transform { case a: AttributeReference =>
attachTree(a, "Binding attribute") {
val ordinal = input.indexWhere(_.exprId == a.exprId)
if (ordinal == -1) {
// TODO: This fallback is required because some operators (such as ScriptTransform)
// produce new attributes that can't be bound. Likely the right thing to do is remove
// this rule and require all operators to explicitly bind to the input schema that
// they specify.
logger.debug(s"Couldn't find $a in ${input.mkString("[", ",", "]")}")
a
} else {
BoundReference(ordinal, a)
}
}
}
}
}

View file

@ -0,0 +1,79 @@
/*
* 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
package catalyst
package expressions
import types._
/** Cast the child expression to the target data type. */
case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
override def foldable = child.foldable
def nullable = child.nullable
override def toString = s"CAST($child, $dataType)"
type EvaluatedType = Any
lazy val castingFunction: Any => Any = (child.dataType, dataType) match {
case (BinaryType, StringType) => a: Any => new String(a.asInstanceOf[Array[Byte]])
case (StringType, BinaryType) => a: Any => a.asInstanceOf[String].getBytes
case (_, StringType) => a: Any => a.toString
case (StringType, IntegerType) => a: Any => castOrNull(a, _.toInt)
case (StringType, DoubleType) => a: Any => castOrNull(a, _.toDouble)
case (StringType, FloatType) => a: Any => castOrNull(a, _.toFloat)
case (StringType, LongType) => a: Any => castOrNull(a, _.toLong)
case (StringType, ShortType) => a: Any => castOrNull(a, _.toShort)
case (StringType, ByteType) => a: Any => castOrNull(a, _.toByte)
case (StringType, DecimalType) => a: Any => castOrNull(a, BigDecimal(_))
case (BooleanType, ByteType) => a: Any => a match {
case null => null
case true => 1.toByte
case false => 0.toByte
}
case (dt, IntegerType) =>
a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a)
case (dt, DoubleType) =>
a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toDouble(a)
case (dt, FloatType) =>
a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toFloat(a)
case (dt, LongType) =>
a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toLong(a)
case (dt, ShortType) =>
a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a).toShort
case (dt, ByteType) =>
a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a).toByte
case (dt, DecimalType) =>
a: Any =>
BigDecimal(dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toDouble(a))
}
@inline
protected def castOrNull[A](a: Any, f: String => A) =
try f(a.asInstanceOf[String]) catch {
case _: java.lang.NumberFormatException => null
}
override def apply(input: Row): Any = {
val evaluated = child.apply(input)
if (evaluated == null) {
null
} else {
castingFunction(evaluated)
}
}
}

View file

@ -0,0 +1,196 @@
/*
* 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
package catalyst
package expressions
import errors._
import trees._
import types._
abstract class Expression extends TreeNode[Expression] {
self: Product =>
/** The narrowest possible type that is produced when this expression is evaluated. */
type EvaluatedType <: Any
def dataType: DataType
/**
* Returns true when an expression is a candidate for static evaluation before the query is
* executed.
*
* The following conditions are used to determine suitability for constant folding:
* - A [[expressions.Coalesce Coalesce]] is foldable if all of its children are foldable
* - A [[expressions.BinaryExpression BinaryExpression]] is foldable if its both left and right
* child are foldable
* - A [[expressions.Not Not]], [[expressions.IsNull IsNull]], or
* [[expressions.IsNotNull IsNotNull]] is foldable if its child is foldable.
* - A [[expressions.Literal]] is foldable.
* - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its
* child is foldable.
*/
// TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs.
def foldable: Boolean = false
def nullable: Boolean
def references: Set[Attribute]
/** Returns the result of evaluating this expression on a given input Row */
def apply(input: Row = null): EvaluatedType =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
/**
* Returns `true` if this expression and all its children have been resolved to a specific schema
* and `false` if it is still contains any unresolved placeholders. Implementations of expressions
* should override this if the resolution of this type of expression involves more than just
* the resolution of its children.
*/
lazy val resolved: Boolean = childrenResolved
/**
* Returns true if all the children of this expression have been resolved to a specific schema
* and false if any still contains any unresolved placeholders.
*/
def childrenResolved = !children.exists(!_.resolved)
/**
* A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type
* and do any casting necessary of child evaluation.
*/
@inline
def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = {
val evalE = e.apply(i)
if (evalE == null) {
null
} else {
e.dataType match {
case n: NumericType =>
val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType]
castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType])
case other => sys.error(s"Type $other does not support numeric operations")
}
}
}
@inline
protected final def n2(
i: Row,
e1: Expression,
e2: Expression,
f: ((Numeric[Any], Any, Any) => Any)): Any = {
if (e1.dataType != e2.dataType) {
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
}
val evalE1 = e1.apply(i)
if(evalE1 == null) {
null
} else {
val evalE2 = e2.apply(i)
if (evalE2 == null) {
null
} else {
e1.dataType match {
case n: NumericType =>
f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int](
n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType])
case other => sys.error(s"Type $other does not support numeric operations")
}
}
}
}
@inline
protected final def f2(
i: Row,
e1: Expression,
e2: Expression,
f: ((Fractional[Any], Any, Any) => Any)): Any = {
if (e1.dataType != e2.dataType) {
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
}
val evalE1 = e1.apply(i: Row)
if(evalE1 == null) {
null
} else {
val evalE2 = e2.apply(i: Row)
if (evalE2 == null) {
null
} else {
e1.dataType match {
case ft: FractionalType =>
f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType](
ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType])
case other => sys.error(s"Type $other does not support fractional operations")
}
}
}
}
@inline
protected final def i2(
i: Row,
e1: Expression,
e2: Expression,
f: ((Integral[Any], Any, Any) => Any)): Any = {
if (e1.dataType != e2.dataType) {
throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
}
val evalE1 = e1.apply(i)
if(evalE1 == null) {
null
} else {
val evalE2 = e2.apply(i)
if (evalE2 == null) {
null
} else {
e1.dataType match {
case i: IntegralType =>
f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType](
i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType])
case other => sys.error(s"Type $other does not support numeric operations")
}
}
}
}
}
abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] {
self: Product =>
def symbol: String
override def foldable = left.foldable && right.foldable
def references = left.references ++ right.references
override def toString = s"($left $symbol $right)"
}
abstract class LeafExpression extends Expression with trees.LeafNode[Expression] {
self: Product =>
}
abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] {
self: Product =>
def references = child.references
}

View file

@ -0,0 +1,127 @@
/*
* 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.catalyst
package expressions
/**
* Converts a [[Row]] to another Row given a sequence of expression that define each column of the
* new row. If the schema of the input row is specified, then the given expression will be bound to
* that schema.
*/
class Projection(expressions: Seq[Expression]) extends (Row => Row) {
def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) =
this(expressions.map(BindReferences.bindReference(_, inputSchema)))
protected val exprArray = expressions.toArray
def apply(input: Row): Row = {
val outputArray = new Array[Any](exprArray.size)
var i = 0
while (i < exprArray.size) {
outputArray(i) = exprArray(i).apply(input)
i += 1
}
new GenericRow(outputArray)
}
}
/**
* Converts a [[Row]] to another Row given a sequence of expression that define each column of th
* new row. If the schema of the input row is specified, then the given expression will be bound to
* that schema.
*
* In contrast to a normal projection, a MutableProjection reuses the same underlying row object
* each time an input row is added. This significatly reduces the cost of calcuating the
* projection, but means that it is not safe
*/
case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) {
def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) =
this(expressions.map(BindReferences.bindReference(_, inputSchema)))
private[this] val exprArray = expressions.toArray
private[this] val mutableRow = new GenericMutableRow(exprArray.size)
def currentValue: Row = mutableRow
def apply(input: Row): Row = {
var i = 0
while (i < exprArray.size) {
mutableRow(i) = exprArray(i).apply(input)
i += 1
}
mutableRow
}
}
/**
* A mutable wrapper that makes two rows appear appear as a single concatenated row. Designed to
* be instantiated once per thread and reused.
*/
class JoinedRow extends Row {
private[this] var row1: Row = _
private[this] var row2: Row = _
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
def apply(r1: Row, r2: Row): Row = {
row1 = r1
row2 = r2
this
}
def iterator = row1.iterator ++ row2.iterator
def length = row1.length + row2.length
def apply(i: Int) =
if (i < row1.size) row1(i) else row2(i - row1.size)
def isNullAt(i: Int) = apply(i) == null
def getInt(i: Int): Int =
if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size)
def getLong(i: Int): Long =
if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size)
def getDouble(i: Int): Double =
if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size)
def getBoolean(i: Int): Boolean =
if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size)
def getShort(i: Int): Short =
if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size)
def getByte(i: Int): Byte =
if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size)
def getFloat(i: Int): Float =
if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size)
def getString(i: Int): String =
if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size)
def copy() = {
val totalSize = row1.size + row2.size
val copiedValues = new Array[Any](totalSize)
var i = 0
while(i < totalSize) {
copiedValues(i) = apply(i)
i += 1
}
new GenericRow(copiedValues)
}
}

View file

@ -0,0 +1,29 @@
/*
* 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
package catalyst
package expressions
import types.DoubleType
case object Rand extends LeafExpression {
def dataType = DoubleType
def nullable = false
def references = Set.empty
override def toString = "RAND()"
}

View file

@ -0,0 +1,214 @@
/*
* 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
package catalyst
package expressions
import types._
/**
* Represents one row of output from a relational operator. Allows both generic access by ordinal,
* which will incur boxing overhead for primitives, as well as native primitive access.
*
* It is invalid to use the native primitive interface to retrieve a value that is null, instead a
* user must check [[isNullAt]] before attempting to retrieve a value that might be null.
*/
trait Row extends Seq[Any] with Serializable {
def apply(i: Int): Any
def isNullAt(i: Int): Boolean
def getInt(i: Int): Int
def getLong(i: Int): Long
def getDouble(i: Int): Double
def getFloat(i: Int): Float
def getBoolean(i: Int): Boolean
def getShort(i: Int): Short
def getByte(i: Int): Byte
def getString(i: Int): String
override def toString() =
s"[${this.mkString(",")}]"
def copy(): Row
}
/**
* An extended interface to [[Row]] that allows the values for each column to be updated. Setting
* a value through a primitive function implicitly marks that column as not null.
*/
trait MutableRow extends Row {
def setNullAt(i: Int): Unit
def update(ordinal: Int, value: Any)
def setInt(ordinal: Int, value: Int)
def setLong(ordinal: Int, value: Long)
def setDouble(ordinal: Int, value: Double)
def setBoolean(ordinal: Int, value: Boolean)
def setShort(ordinal: Int, value: Short)
def setByte(ordinal: Int, value: Byte)
def setFloat(ordinal: Int, value: Float)
def setString(ordinal: Int, value: String)
/**
* EXPERIMENTAL
*
* Returns a mutable string builder for the specified column. A given row should return the
* result of any mutations made to the returned buffer next time getString is called for the same
* column.
*/
def getStringBuilder(ordinal: Int): StringBuilder
}
/**
* A row with no data. Calling any methods will result in an error. Can be used as a placeholder.
*/
object EmptyRow extends Row {
def apply(i: Int): Any = throw new UnsupportedOperationException
def iterator = Iterator.empty
def length = 0
def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException
def getInt(i: Int): Int = throw new UnsupportedOperationException
def getLong(i: Int): Long = throw new UnsupportedOperationException
def getDouble(i: Int): Double = throw new UnsupportedOperationException
def getFloat(i: Int): Float = throw new UnsupportedOperationException
def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException
def getShort(i: Int): Short = throw new UnsupportedOperationException
def getByte(i: Int): Byte = throw new UnsupportedOperationException
def getString(i: Int): String = throw new UnsupportedOperationException
def copy() = this
}
/**
* A row implementation that uses an array of objects as the underlying storage. Note that, while
* the array is not copied, and thus could technically be mutated after creation, this is not
* allowed.
*/
class GenericRow(protected[catalyst] val values: Array[Any]) extends Row {
/** No-arg constructor for serialization. */
def this() = this(null)
def this(size: Int) = this(new Array[Any](size))
def iterator = values.iterator
def length = values.length
def apply(i: Int) = values(i)
def isNullAt(i: Int) = values(i) == null
def getInt(i: Int): Int = {
if (values(i) == null) sys.error("Failed to check null bit for primitive int value.")
values(i).asInstanceOf[Int]
}
def getLong(i: Int): Long = {
if (values(i) == null) sys.error("Failed to check null bit for primitive long value.")
values(i).asInstanceOf[Long]
}
def getDouble(i: Int): Double = {
if (values(i) == null) sys.error("Failed to check null bit for primitive double value.")
values(i).asInstanceOf[Double]
}
def getFloat(i: Int): Float = {
if (values(i) == null) sys.error("Failed to check null bit for primitive float value.")
values(i).asInstanceOf[Float]
}
def getBoolean(i: Int): Boolean = {
if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.")
values(i).asInstanceOf[Boolean]
}
def getShort(i: Int): Short = {
if (values(i) == null) sys.error("Failed to check null bit for primitive short value.")
values(i).asInstanceOf[Short]
}
def getByte(i: Int): Byte = {
if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.")
values(i).asInstanceOf[Byte]
}
def getString(i: Int): String = {
if (values(i) == null) sys.error("Failed to check null bit for primitive String value.")
values(i).asInstanceOf[String]
}
def copy() = this
}
class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow {
/** No-arg constructor for serialization. */
def this() = this(0)
def getStringBuilder(ordinal: Int): StringBuilder = ???
override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value }
override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value }
override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value }
override def setFloat(ordinal: Int,value: Float): Unit = { values(ordinal) = value }
override def setInt(ordinal: Int,value: Int): Unit = { values(ordinal) = value }
override def setLong(ordinal: Int,value: Long): Unit = { values(ordinal) = value }
override def setString(ordinal: Int,value: String): Unit = { values(ordinal) = value }
override def setNullAt(i: Int): Unit = { values(i) = null }
override def setShort(ordinal: Int,value: Short): Unit = { values(ordinal) = value }
override def update(ordinal: Int,value: Any): Unit = { values(ordinal) = value }
override def copy() = new GenericRow(values.clone())
}
class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] {
def compare(a: Row, b: Row): Int = {
var i = 0
while (i < ordering.size) {
val order = ordering(i)
val left = order.child.apply(a)
val right = order.child.apply(b)
if (left == null && right == null) {
// Both null, continue looking.
} else if (left == null) {
return if (order.direction == Ascending) -1 else 1
} else if (right == null) {
return if (order.direction == Ascending) 1 else -1
} else {
val comparison = order.dataType match {
case n: NativeType if order.direction == Ascending =>
n.ordering.asInstanceOf[Ordering[Any]].compare(left, right)
case n: NativeType if order.direction == Descending =>
n.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right)
}
if (comparison != 0) return comparison
}
i += 1
}
return 0
}
}

View file

@ -0,0 +1,41 @@
/*
* 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
package catalyst
package expressions
import types._
case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression])
extends Expression {
type EvaluatedType = Any
def references = children.flatMap(_.references).toSet
def nullable = true
override def apply(input: Row): Any = {
children.size match {
case 1 => function.asInstanceOf[(Any) => Any](children(0).apply(input))
case 2 =>
function.asInstanceOf[(Any, Any) => Any](
children(0).apply(input),
children(1).apply(input))
}
}
}

View file

@ -0,0 +1,34 @@
/*
* 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
package catalyst
package expressions
abstract sealed class SortDirection
case object Ascending extends SortDirection
case object Descending extends SortDirection
/**
* An expression that can be used to sort a tuple. This class extends expression primarily so that
* transformations over expression will descend into its child.
*/
case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression {
def dataType = child.dataType
def nullable = child.nullable
override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}"
}

View file

@ -0,0 +1,49 @@
/*
* 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
package catalyst
package expressions
import scala.language.dynamics
import types._
case object DynamicType extends DataType
case class WrapDynamic(children: Seq[Attribute]) extends Expression {
type EvaluatedType = DynamicRow
def nullable = false
def references = children.toSet
def dataType = DynamicType
override def apply(input: Row): DynamicRow = input match {
// Avoid copy for generic rows.
case g: GenericRow => new DynamicRow(children, g.values)
case otherRowType => new DynamicRow(children, otherRowType.toArray)
}
}
class DynamicRow(val schema: Seq[Attribute], values: Array[Any])
extends GenericRow(values) with Dynamic {
def selectDynamic(attributeName: String): String = {
val ordinal = schema.indexWhere(_.name == attributeName)
values(ordinal).toString
}
}

View file

@ -0,0 +1,265 @@
/*
* 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
package catalyst
package expressions
import catalyst.types._
abstract class AggregateExpression extends Expression {
self: Product =>
/**
* Creates a new instance that can be used to compute this aggregate expression for a group
* of input rows/
*/
def newInstance: AggregateFunction
}
/**
* Represents an aggregation that has been rewritten to be performed in two steps.
*
* @param finalEvaluation an aggregate expression that evaluates to same final result as the
* original aggregation.
* @param partialEvaluations A sequence of [[NamedExpression]]s that can be computed on partial
* data sets and are required to compute the `finalEvaluation`.
*/
case class SplitEvaluation(
finalEvaluation: Expression,
partialEvaluations: Seq[NamedExpression])
/**
* An [[AggregateExpression]] that can be partially computed without seeing all relevent tuples.
* These partial evaluations can then be combined to compute the actual answer.
*/
abstract class PartialAggregate extends AggregateExpression {
self: Product =>
/**
* Returns a [[SplitEvaluation]] that computes this aggregation using partial aggregation.
*/
def asPartial: SplitEvaluation
}
/**
* A specific implementation of an aggregate function. Used to wrap a generic
* [[AggregateExpression]] with an algorithm that will be used to compute one specific result.
*/
abstract class AggregateFunction
extends AggregateExpression with Serializable with trees.LeafNode[Expression] {
self: Product =>
type EvaluatedType = Any
/** Base should return the generic aggregate expression that this function is computing */
val base: AggregateExpression
def references = base.references
def nullable = base.nullable
def dataType = base.dataType
def update(input: Row): Unit
override def apply(input: Row): Any
// Do we really need this?
def newInstance = makeCopy(productIterator.map { case a: AnyRef => a }.toArray)
}
case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
def references = child.references
def nullable = false
def dataType = IntegerType
override def toString = s"COUNT($child)"
def asPartial: SplitEvaluation = {
val partialCount = Alias(Count(child), "PartialCount")()
SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil)
}
override def newInstance = new CountFunction(child, this)
}
case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression {
def children = expressions
def references = expressions.flatMap(_.references).toSet
def nullable = false
def dataType = IntegerType
override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})"
override def newInstance = new CountDistinctFunction(expressions, this)
}
case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
def references = child.references
def nullable = false
def dataType = DoubleType
override def toString = s"AVG($child)"
override def asPartial: SplitEvaluation = {
val partialSum = Alias(Sum(child), "PartialSum")()
val partialCount = Alias(Count(child), "PartialCount")()
val castedSum = Cast(Sum(partialSum.toAttribute), dataType)
val castedCount = Cast(Sum(partialCount.toAttribute), dataType)
SplitEvaluation(
Divide(castedSum, castedCount),
partialCount :: partialSum :: Nil)
}
override def newInstance = new AverageFunction(child, this)
}
case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
def references = child.references
def nullable = false
def dataType = child.dataType
override def toString = s"SUM($child)"
override def asPartial: SplitEvaluation = {
val partialSum = Alias(Sum(child), "PartialSum")()
SplitEvaluation(
Sum(partialSum.toAttribute),
partialSum :: Nil)
}
override def newInstance = new SumFunction(child, this)
}
case class SumDistinct(child: Expression)
extends AggregateExpression with trees.UnaryNode[Expression] {
def references = child.references
def nullable = false
def dataType = child.dataType
override def toString = s"SUM(DISTINCT $child)"
override def newInstance = new SumDistinctFunction(child, this)
}
case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
def references = child.references
def nullable = child.nullable
def dataType = child.dataType
override def toString = s"FIRST($child)"
override def asPartial: SplitEvaluation = {
val partialFirst = Alias(First(child), "PartialFirst")()
SplitEvaluation(
First(partialFirst.toAttribute),
partialFirst :: Nil)
}
override def newInstance = new FirstFunction(child, this)
}
case class AverageFunction(expr: Expression, base: AggregateExpression)
extends AggregateFunction {
def this() = this(null, null) // Required for serialization.
private var count: Long = _
private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(EmptyRow))
private val sumAsDouble = Cast(sum, DoubleType)
private val addFunction = Add(sum, expr)
override def apply(input: Row): Any =
sumAsDouble.apply(EmptyRow).asInstanceOf[Double] / count.toDouble
def update(input: Row): Unit = {
count += 1
sum.update(addFunction, input)
}
}
case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
def this() = this(null, null) // Required for serialization.
var count: Int = _
def update(input: Row): Unit = {
val evaluatedExpr = expr.map(_.apply(input))
if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) {
count += 1
}
}
override def apply(input: Row): Any = count
}
case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
def this() = this(null, null) // Required for serialization.
private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null))
private val addFunction = Add(sum, expr)
def update(input: Row): Unit = {
sum.update(addFunction, input)
}
override def apply(input: Row): Any = sum.apply(null)
}
case class SumDistinctFunction(expr: Expression, base: AggregateExpression)
extends AggregateFunction {
def this() = this(null, null) // Required for serialization.
val seen = new scala.collection.mutable.HashSet[Any]()
def update(input: Row): Unit = {
val evaluatedExpr = expr.apply(input)
if (evaluatedExpr != null) {
seen += evaluatedExpr
}
}
override def apply(input: Row): Any =
seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus)
}
case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression)
extends AggregateFunction {
def this() = this(null, null) // Required for serialization.
val seen = new scala.collection.mutable.HashSet[Any]()
def update(input: Row): Unit = {
val evaluatedExpr = expr.map(_.apply(input))
if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) {
seen += evaluatedExpr
}
}
override def apply(input: Row): Any = seen.size
}
case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
def this() = this(null, null) // Required for serialization.
var result: Any = null
def update(input: Row): Unit = {
if (result == null) {
result = expr.apply(input)
}
}
override def apply(input: Row): Any = result
}

View file

@ -0,0 +1,89 @@
/*
* 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
package catalyst
package expressions
import catalyst.analysis.UnresolvedException
import catalyst.types._
case class UnaryMinus(child: Expression) extends UnaryExpression {
type EvaluatedType = Any
def dataType = child.dataType
override def foldable = child.foldable
def nullable = child.nullable
override def toString = s"-$child"
override def apply(input: Row): Any = {
n1(child, input, _.negate(_))
}
}
abstract class BinaryArithmetic extends BinaryExpression {
self: Product =>
type EvaluatedType = Any
def nullable = left.nullable || right.nullable
override lazy val resolved =
left.resolved && right.resolved && left.dataType == right.dataType
def dataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}")
}
left.dataType
}
}
case class Add(left: Expression, right: Expression) extends BinaryArithmetic {
def symbol = "+"
override def apply(input: Row): Any = n2(input, left, right, _.plus(_, _))
}
case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic {
def symbol = "-"
override def apply(input: Row): Any = n2(input, left, right, _.minus(_, _))
}
case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic {
def symbol = "*"
override def apply(input: Row): Any = n2(input, left, right, _.times(_, _))
}
case class Divide(left: Expression, right: Expression) extends BinaryArithmetic {
def symbol = "/"
override def apply(input: Row): Any = dataType match {
case _: FractionalType => f2(input, left, right, _.div(_, _))
case _: IntegralType => i2(input, left , right, _.quot(_, _))
}
}
case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic {
def symbol = "%"
override def apply(input: Row): Any = i2(input, left, right, _.rem(_, _))
}

View file

@ -0,0 +1,96 @@
/*
* 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
package catalyst
package expressions
import types._
/**
* Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`.
*/
case class GetItem(child: Expression, ordinal: Expression) extends Expression {
type EvaluatedType = Any
val children = child :: ordinal :: Nil
/** `Null` is returned for invalid ordinals. */
override def nullable = true
override def references = children.flatMap(_.references).toSet
def dataType = child.dataType match {
case ArrayType(dt) => dt
case MapType(_, vt) => vt
}
override lazy val resolved =
childrenResolved &&
(child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType])
override def toString = s"$child[$ordinal]"
override def apply(input: Row): Any = {
if (child.dataType.isInstanceOf[ArrayType]) {
val baseValue = child.apply(input).asInstanceOf[Seq[_]]
val o = ordinal.apply(input).asInstanceOf[Int]
if (baseValue == null) {
null
} else if (o >= baseValue.size || o < 0) {
null
} else {
baseValue(o)
}
} else {
val baseValue = child.apply(input).asInstanceOf[Map[Any, _]]
val key = ordinal.apply(input)
if (baseValue == null) {
null
} else {
baseValue.get(key).orNull
}
}
}
}
/**
* Returns the value of fields in the Struct `child`.
*/
case class GetField(child: Expression, fieldName: String) extends UnaryExpression {
type EvaluatedType = Any
def dataType = field.dataType
def nullable = field.nullable
protected def structType = child.dataType match {
case s: StructType => s
case otherType => sys.error(s"GetField is not valid on fields of type $otherType")
}
lazy val field =
structType.fields
.find(_.name == fieldName)
.getOrElse(sys.error(s"No such field $fieldName in ${child.dataType}"))
lazy val ordinal = structType.fields.indexOf(field)
override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType]
override def apply(input: Row): Any = {
val baseValue = child.apply(input).asInstanceOf[Row]
if (baseValue == null) null else baseValue(ordinal)
}
override def toString = s"$child.$fieldName"
}

View file

@ -0,0 +1,116 @@
/*
* 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
package catalyst
package expressions
import catalyst.types._
/**
* An expression that produces zero or more rows given a single input row.
*
* Generators produce multiple output rows instead of a single value like other expressions,
* and thus they must have a schema to associate with the rows that are output.
*
* However, unlike row producing relational operators, which are either leaves or determine their
* output schema functionally from their input, generators can contain other expressions that
* might result in their modification by rules. This structure means that they might be copied
* multiple times after first determining their output schema. If a new output schema is created for
* each copy references up the tree might be rendered invalid. As a result generators must
* instead define a function `makeOutput` which is called only once when the schema is first
* requested. The attributes produced by this function will be automatically copied anytime rules
* result in changes to the Generator or its children.
*/
abstract class Generator extends Expression with (Row => TraversableOnce[Row]) {
self: Product =>
type EvaluatedType = TraversableOnce[Row]
lazy val dataType =
ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable))))
def nullable = false
def references = children.flatMap(_.references).toSet
/**
* Should be overridden by specific generators. Called only once for each instance to ensure
* that rule application does not change the output schema of a generator.
*/
protected def makeOutput(): Seq[Attribute]
private var _output: Seq[Attribute] = null
def output: Seq[Attribute] = {
if (_output == null) {
_output = makeOutput()
}
_output
}
/** Should be implemented by child classes to perform specific Generators. */
def apply(input: Row): TraversableOnce[Row]
/** Overridden `makeCopy` also copies the attributes that are produced by this generator. */
override def makeCopy(newArgs: Array[AnyRef]): this.type = {
val copy = super.makeCopy(newArgs)
copy._output = _output
copy
}
}
/**
* Given an input array produces a sequence of rows for each value in the array.
*/
case class Explode(attributeNames: Seq[String], child: Expression)
extends Generator with trees.UnaryNode[Expression] {
override lazy val resolved =
child.resolved &&
(child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType])
lazy val elementTypes = child.dataType match {
case ArrayType(et) => et :: Nil
case MapType(kt,vt) => kt :: vt :: Nil
}
// TODO: Move this pattern into Generator.
protected def makeOutput() =
if (attributeNames.size == elementTypes.size) {
attributeNames.zip(elementTypes).map {
case (n, t) => AttributeReference(n, t, nullable = true)()
}
} else {
elementTypes.zipWithIndex.map {
case (t, i) => AttributeReference(s"c_$i", t, nullable = true)()
}
}
override def apply(input: Row): TraversableOnce[Row] = {
child.dataType match {
case ArrayType(_) =>
val inputArray = child.apply(input).asInstanceOf[Seq[Any]]
if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v)))
case MapType(_, _) =>
val inputMap = child.apply(input).asInstanceOf[Map[Any,Any]]
if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) }
}
}
override def toString() = s"explode($child)"
}

View file

@ -0,0 +1,73 @@
/*
* 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
package catalyst
package expressions
import types._
object Literal {
def apply(v: Any): Literal = v match {
case i: Int => Literal(i, IntegerType)
case l: Long => Literal(l, LongType)
case d: Double => Literal(d, DoubleType)
case f: Float => Literal(f, FloatType)
case b: Byte => Literal(b, ByteType)
case s: Short => Literal(s, ShortType)
case s: String => Literal(s, StringType)
case b: Boolean => Literal(b, BooleanType)
case null => Literal(null, NullType)
}
}
/**
* Extractor for retrieving Int literals.
*/
object IntegerLiteral {
def unapply(a: Any): Option[Int] = a match {
case Literal(a: Int, IntegerType) => Some(a)
case _ => None
}
}
case class Literal(value: Any, dataType: DataType) extends LeafExpression {
override def foldable = true
def nullable = value == null
def references = Set.empty
override def toString = if (value != null) value.toString else "null"
type EvaluatedType = Any
override def apply(input: Row):Any = value
}
// TODO: Specialize
case class MutableLiteral(var value: Any, nullable: Boolean = true) extends LeafExpression {
type EvaluatedType = Any
val dataType = Literal(value).dataType
def references = Set.empty
def update(expression: Expression, input: Row) = {
value = expression.apply(input)
}
override def apply(input: Row) = value
}

View file

@ -0,0 +1,156 @@
/*
* 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
package catalyst
package expressions
import catalyst.analysis.UnresolvedAttribute
import types._
object NamedExpression {
private val curId = new java.util.concurrent.atomic.AtomicLong()
def newExprId = ExprId(curId.getAndIncrement())
}
/**
* A globally (within this JVM) id for a given named expression.
* Used to identify with attribute output by a relation is being
* referenced in a subsuqent computation.
*/
case class ExprId(id: Long)
abstract class NamedExpression extends Expression {
self: Product =>
def name: String
def exprId: ExprId
def qualifiers: Seq[String]
def toAttribute: Attribute
protected def typeSuffix =
if (resolved) {
dataType match {
case LongType => "L"
case _ => ""
}
} else {
""
}
}
abstract class Attribute extends NamedExpression {
self: Product =>
def withQualifiers(newQualifiers: Seq[String]): Attribute
def references = Set(this)
def toAttribute = this
def newInstance: Attribute
}
/**
* Used to assign a new name to a computation.
* For example the SQL expression "1 + 1 AS a" could be represented as follows:
* Alias(Add(Literal(1), Literal(1), "a")()
*
* @param child the computation being performed
* @param name the name to be associated with the result of computing [[child]].
* @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this
* alias. Auto-assigned if left blank.
*/
case class Alias(child: Expression, name: String)
(val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil)
extends NamedExpression with trees.UnaryNode[Expression] {
type EvaluatedType = Any
override def apply(input: Row) = child.apply(input)
def dataType = child.dataType
def nullable = child.nullable
def references = child.references
def toAttribute = {
if (resolved) {
AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers)
} else {
UnresolvedAttribute(name)
}
}
override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix"
override protected final def otherCopyArgs = exprId :: qualifiers :: Nil
}
/**
* A reference to an attribute produced by another operator in the tree.
*
* @param name The name of this attribute, should only be used during analysis or for debugging.
* @param dataType The [[types.DataType DataType]] of this attribute.
* @param nullable True if null is a valid value for this attribute.
* @param exprId A globally unique id used to check if different AttributeReferences refer to the
* same attribute.
* @param qualifiers a list of strings that can be used to referred to this attribute in a fully
* qualified way. Consider the examples tableName.name, subQueryAlias.name.
* tableName and subQueryAlias are possible qualifiers.
*/
case class AttributeReference(name: String, dataType: DataType, nullable: Boolean = true)
(val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil)
extends Attribute with trees.LeafNode[Expression] {
override def equals(other: Any) = other match {
case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType
case _ => false
}
override def hashCode: Int = {
// See http://stackoverflow.com/questions/113511/hash-code-implementation
var h = 17
h = h * 37 + exprId.hashCode()
h = h * 37 + dataType.hashCode()
h
}
def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers)
/**
* Returns a copy of this [[AttributeReference]] with changed nullability.
*/
def withNullability(newNullability: Boolean) = {
if (nullable == newNullability) {
this
} else {
AttributeReference(name, dataType, newNullability)(exprId, qualifiers)
}
}
/**
* Returns a copy of this [[AttributeReference]] with new qualifiers.
*/
def withQualifiers(newQualifiers: Seq[String]) = {
if (newQualifiers == qualifiers) {
this
} else {
AttributeReference(name, dataType, nullable)(exprId, newQualifiers)
}
}
override def toString: String = s"$name#${exprId.id}$typeSuffix"
}

View file

@ -0,0 +1,75 @@
/*
* 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
package catalyst
package expressions
import catalyst.analysis.UnresolvedException
case class Coalesce(children: Seq[Expression]) extends Expression {
type EvaluatedType = Any
/** Coalesce is nullable if all of its children are nullable, or if it has no children. */
def nullable = !children.exists(!_.nullable)
def references = children.flatMap(_.references).toSet
// Coalesce is foldable if all children are foldable.
override def foldable = !children.exists(!_.foldable)
// Only resolved if all the children are of the same type.
override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1)
override def toString = s"Coalesce(${children.mkString(",")})"
def dataType = if (resolved) {
children.head.dataType
} else {
throw new UnresolvedException(this, "Coalesce cannot have children of different types.")
}
override def apply(input: Row): Any = {
var i = 0
var result: Any = null
while(i < children.size && result == null) {
result = children(i).apply(input)
i += 1
}
result
}
}
case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] {
def references = child.references
override def foldable = child.foldable
def nullable = false
override def apply(input: Row): Any = {
child.apply(input) == null
}
}
case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] {
def references = child.references
override def foldable = child.foldable
def nullable = false
override def toString = s"IS NOT NULL $child"
override def apply(input: Row): Any = {
child.apply(input) != null
}
}

View file

@ -0,0 +1,51 @@
/*
* 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
package catalyst
/**
* A set of classes that can be used to represent trees of relational expressions. A key goal of
* the expression library is to hide the details of naming and scoping from developers who want to
* manipulate trees of relational operators. As such, the library defines a special type of
* expression, a [[NamedExpression]] in addition to the standard collection of expressions.
*
* ==Standard Expressions==
* A library of standard expressions (e.g., [[Add]], [[Equals]]), aggregates (e.g., SUM, COUNT),
* and other computations (e.g. UDFs). Each expression type is capable of determining its output
* schema as a function of its children's output schema.
*
* ==Named Expressions==
* Some expression are named and thus can be referenced by later operators in the dataflow graph.
* The two types of named expressions are [[AttributeReference]]s and [[Alias]]es.
* [[AttributeReference]]s refer to attributes of the input tuple for a given operator and form
* the leaves of some expression trees. Aliases assign a name to intermediate computations.
* For example, in the SQL statement `SELECT a+b AS c FROM ...`, the expressions `a` and `b` would
* be represented by `AttributeReferences` and `c` would be represented by an `Alias`.
*
* During [[analysis]], all named expressions are assigned a globally unique expression id, which
* can be used for equality comparisons. While the original names are kept around for debugging
* purposes, they should never be used to check if two attributes refer to the same value, as
* plan transformations can result in the introduction of naming ambiguity. For example, consider
* a plan that contains subqueries, both of which are reading from the same table. If an
* optimization removes the subqueries, scoping information would be destroyed, eliminating the
* ability to reason about which subquery produced a given attribute.
*
* ==Evaluation==
* The result of expressions can be evaluated using the [[Evaluate]] object.
*/
package object expressions

View file

@ -0,0 +1,213 @@
/*
* 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
package catalyst
package expressions
import types._
import catalyst.analysis.UnresolvedException
trait Predicate extends Expression {
self: Product =>
def dataType = BooleanType
type EvaluatedType = Any
}
trait PredicateHelper {
def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match {
case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2)
case other => other :: Nil
}
}
abstract class BinaryPredicate extends BinaryExpression with Predicate {
self: Product =>
def nullable = left.nullable || right.nullable
}
case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] {
def references = child.references
override def foldable = child.foldable
def nullable = child.nullable
override def toString = s"NOT $child"
override def apply(input: Row): Any = {
child.apply(input) match {
case null => null
case b: Boolean => !b
}
}
}
/**
* Evaluates to `true` if `list` contains `value`.
*/
case class In(value: Expression, list: Seq[Expression]) extends Predicate {
def children = value +: list
def references = children.flatMap(_.references).toSet
def nullable = true // TODO: Figure out correct nullability semantics of IN.
override def toString = s"$value IN ${list.mkString("(", ",", ")")}"
override def apply(input: Row): Any = {
val evaluatedValue = value.apply(input)
list.exists(e => e.apply(input) == evaluatedValue)
}
}
case class And(left: Expression, right: Expression) extends BinaryPredicate {
def symbol = "&&"
override def apply(input: Row): Any = {
val l = left.apply(input)
val r = right.apply(input)
if (l == false || r == false) {
false
} else if (l == null || r == null ) {
null
} else {
true
}
}
}
case class Or(left: Expression, right: Expression) extends BinaryPredicate {
def symbol = "||"
override def apply(input: Row): Any = {
val l = left.apply(input)
val r = right.apply(input)
if (l == true || r == true) {
true
} else if (l == null || r == null) {
null
} else {
false
}
}
}
abstract class BinaryComparison extends BinaryPredicate {
self: Product =>
}
case class Equals(left: Expression, right: Expression) extends BinaryComparison {
def symbol = "="
override def apply(input: Row): Any = {
val l = left.apply(input)
val r = right.apply(input)
if (l == null || r == null) null else l == r
}
}
case class LessThan(left: Expression, right: Expression) extends BinaryComparison {
def symbol = "<"
override def apply(input: Row): Any = {
if (left.dataType == StringType && right.dataType == StringType) {
val l = left.apply(input)
val r = right.apply(input)
if(l == null || r == null) {
null
} else {
l.asInstanceOf[String] < r.asInstanceOf[String]
}
} else {
n2(input, left, right, _.lt(_, _))
}
}
}
case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
def symbol = "<="
override def apply(input: Row): Any = {
if (left.dataType == StringType && right.dataType == StringType) {
val l = left.apply(input)
val r = right.apply(input)
if(l == null || r == null) {
null
} else {
l.asInstanceOf[String] <= r.asInstanceOf[String]
}
} else {
n2(input, left, right, _.lteq(_, _))
}
}
}
case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison {
def symbol = ">"
override def apply(input: Row): Any = {
if (left.dataType == StringType && right.dataType == StringType) {
val l = left.apply(input)
val r = right.apply(input)
if(l == null || r == null) {
null
} else {
l.asInstanceOf[String] > r.asInstanceOf[String]
}
} else {
n2(input, left, right, _.gt(_, _))
}
}
}
case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
def symbol = ">="
override def apply(input: Row): Any = {
if (left.dataType == StringType && right.dataType == StringType) {
val l = left.apply(input)
val r = right.apply(input)
if(l == null || r == null) {
null
} else {
l.asInstanceOf[String] >= r.asInstanceOf[String]
}
} else {
n2(input, left, right, _.gteq(_, _))
}
}
}
case class If(predicate: Expression, trueValue: Expression, falseValue: Expression)
extends Expression {
def children = predicate :: trueValue :: falseValue :: Nil
def nullable = trueValue.nullable || falseValue.nullable
def references = children.flatMap(_.references).toSet
override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType
def dataType = {
if (!resolved) {
throw new UnresolvedException(
this,
s"Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}")
}
trueValue.dataType
}
type EvaluatedType = Any
override def apply(input: Row): Any = {
if (predicate(input).asInstanceOf[Boolean]) {
trueValue.apply(input)
} else {
falseValue.apply(input)
}
}
override def toString = s"if ($predicate) $trueValue else $falseValue"
}

View file

@ -0,0 +1,29 @@
/*
* 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
package catalyst
package expressions
import catalyst.types.BooleanType
case class Like(left: Expression, right: Expression) extends BinaryExpression {
def dataType = BooleanType
def nullable = left.nullable // Right cannot be null.
def symbol = "LIKE"
}

View file

@ -0,0 +1,167 @@
/*
* 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
package catalyst
package optimizer
import catalyst.expressions._
import catalyst.plans.logical._
import catalyst.rules._
import catalyst.types.BooleanType
import catalyst.plans.Inner
object Optimizer extends RuleExecutor[LogicalPlan] {
val batches =
Batch("Subqueries", Once,
EliminateSubqueries) ::
Batch("ConstantFolding", Once,
ConstantFolding,
BooleanSimplification,
SimplifyCasts) ::
Batch("Filter Pushdown", Once,
EliminateSubqueries,
CombineFilters,
PushPredicateThroughProject,
PushPredicateThroughInnerJoin) :: Nil
}
/**
* Removes [[catalyst.plans.logical.Subquery Subquery]] operators from the plan. Subqueries are
* only required to provide scoping information for attributes and can be removed once analysis is
* complete.
*/
object EliminateSubqueries extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case Subquery(_, child) => child
}
}
/**
* Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with
* equivalent [[catalyst.expressions.Literal Literal]] values.
*/
object ConstantFolding extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case q: LogicalPlan => q transformExpressionsDown {
// Skip redundant folding of literals.
case l: Literal => l
case e if e.foldable => Literal(e.apply(null), e.dataType)
}
}
}
/**
* Simplifies boolean expressions where the answer can be determined without evaluating both sides.
* Note that this rule can eliminate expressions that might otherwise have been evaluated and thus
* is only safe when evaluations of expressions does not result in side effects.
*/
object BooleanSimplification extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case q: LogicalPlan => q transformExpressionsUp {
case and @ And(left, right) => {
(left, right) match {
case (Literal(true, BooleanType), r) => r
case (l, Literal(true, BooleanType)) => l
case (Literal(false, BooleanType), _) => Literal(false)
case (_, Literal(false, BooleanType)) => Literal(false)
case (_, _) => and
}
}
case or @ Or(left, right) => {
(left, right) match {
case (Literal(true, BooleanType), _) => Literal(true)
case (_, Literal(true, BooleanType)) => Literal(true)
case (Literal(false, BooleanType), r) => r
case (l, Literal(false, BooleanType)) => l
case (_, _) => or
}
}
}
}
}
/**
* Combines two adjacent [[catalyst.plans.logical.Filter Filter]] operators into one, merging the
* conditions into one conjunctive predicate.
*/
object CombineFilters extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case ff@Filter(fc, nf@Filter(nc, grandChild)) => Filter(And(nc, fc), grandChild)
}
}
/**
* Pushes [[catalyst.plans.logical.Filter Filter]] operators through
* [[catalyst.plans.logical.Project Project]] operators, in-lining any
* [[catalyst.expressions.Alias Aliases]] that were defined in the projection.
*
* This heuristic is valid assuming the expression evaluation cost is minimal.
*/
object PushPredicateThroughProject extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case filter@Filter(condition, project@Project(fields, grandChild)) =>
val sourceAliases = fields.collect { case a@Alias(c, _) => a.toAttribute -> c }.toMap
project.copy(child = filter.copy(
replaceAlias(condition, sourceAliases),
grandChild))
}
//
def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = {
condition transform {
case a: AttributeReference => sourceAliases.getOrElse(a, a)
}
}
}
/**
* Pushes down [[catalyst.plans.logical.Filter Filter]] operators where the `condition` can be
* evaluated using only the attributes of the left or right side of an inner join. Other
* [[catalyst.plans.logical.Filter Filter]] conditions are moved into the `condition` of the
* [[catalyst.plans.logical.Join Join]].
*/
object PushPredicateThroughInnerJoin extends Rule[LogicalPlan] with PredicateHelper {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case f @ Filter(filterCondition, Join(left, right, Inner, joinCondition)) =>
val allConditions =
splitConjunctivePredicates(filterCondition) ++
joinCondition.map(splitConjunctivePredicates).getOrElse(Nil)
// Split the predicates into those that can be evaluated on the left, right, and those that
// must be evaluated after the join.
val (rightConditions, leftOrJoinConditions) =
allConditions.partition(_.references subsetOf right.outputSet)
val (leftConditions, joinConditions) =
leftOrJoinConditions.partition(_.references subsetOf left.outputSet)
// Build the new left and right side, optionally with the pushed down filters.
val newLeft = leftConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left)
val newRight = rightConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right)
Join(newLeft, newRight, Inner, joinConditions.reduceLeftOption(And))
}
}
/**
* Removes [[catalyst.expressions.Cast Casts]] that are unnecessary because the input is already
* the correct type.
*/
object SimplifyCasts extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case Cast(e, dataType) if e.dataType == dataType => e
}
}

View file

@ -0,0 +1,64 @@
/*
* 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
package catalyst
package planning
import plans.logical.LogicalPlan
import trees._
/**
* Abstract class for transforming [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans.
* Child classes are responsible for specifying a list of [[Strategy]] objects that each of which
* can return a list of possible physical plan options. If a given strategy is unable to plan all
* of the remaining operators in the tree, it can call [[planLater]], which returns a placeholder
* object that will be filled in using other available strategies.
*
* TODO: RIGHT NOW ONLY ONE PLAN IS RETURNED EVER...
* PLAN SPACE EXPLORATION WILL BE IMPLEMENTED LATER.
*
* @tparam PhysicalPlan The type of physical plan produced by this [[QueryPlanner]]
*/
abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] {
/** A list of execution strategies that can be used by the planner */
def strategies: Seq[Strategy]
/**
* Given a [[plans.logical.LogicalPlan LogicalPlan]], returns a list of `PhysicalPlan`s that can
* be used for execution. If this strategy does not apply to the give logical operation then an
* empty list should be returned.
*/
abstract protected class Strategy extends Logging {
def apply(plan: LogicalPlan): Seq[PhysicalPlan]
}
/**
* Returns a placeholder for a physical plan that executes `plan`. This placeholder will be
* filled in automatically by the QueryPlanner using the other execution strategies that are
* available.
*/
protected def planLater(plan: LogicalPlan) = apply(plan).next()
def apply(plan: LogicalPlan): Iterator[PhysicalPlan] = {
// Obviously a lot to do here still...
val iter = strategies.view.flatMap(_(plan)).toIterator
assert(iter.hasNext, s"No plan for $plan")
iter
}
}

View file

@ -0,0 +1,24 @@
/*
* 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
package catalyst
/**
* Contains classes for enumerating possible physical plans for a given logical query plan.
*/
package object planning

View file

@ -0,0 +1,117 @@
/*
* 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
package catalyst
package planning
import scala.annotation.tailrec
import expressions._
import plans.logical._
/**
* A pattern that matches any number of filter operations on top of another relational operator.
* Adjacent filter operators are collected and their conditions are broken up and returned as a
* sequence of conjunctive predicates.
*
* @return A tuple containing a sequence of conjunctive predicates that should be used to filter the
* output and a relational operator.
*/
object FilteredOperation extends PredicateHelper {
type ReturnType = (Seq[Expression], LogicalPlan)
def unapply(plan: LogicalPlan): Option[ReturnType] = Some(collectFilters(Nil, plan))
@tailrec
private def collectFilters(filters: Seq[Expression], plan: LogicalPlan): ReturnType = plan match {
case Filter(condition, child) =>
collectFilters(filters ++ splitConjunctivePredicates(condition), child)
case other => (filters, other)
}
}
/**
* A pattern that matches any number of project or filter operations on top of another relational
* operator. All filter operators are collected and their conditions are broken up and returned
* together with the top project operator. [[Alias Aliases]] are in-lined/substituted if necessary.
*/
object PhysicalOperation extends PredicateHelper {
type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan)
def unapply(plan: LogicalPlan): Option[ReturnType] = {
val (fields, filters, child, _) = collectProjectsAndFilters(plan)
Some((fields.getOrElse(child.output), filters, child))
}
/**
* Collects projects and filters, in-lining/substituting aliases if necessary. Here are two
* examples for alias in-lining/substitution. Before:
* {{{
* SELECT c1 FROM (SELECT key AS c1 FROM t1) t2 WHERE c1 > 10
* SELECT c1 AS c2 FROM (SELECT key AS c1 FROM t1) t2 WHERE c1 > 10
* }}}
* After:
* {{{
* SELECT key AS c1 FROM t1 WHERE key > 10
* SELECT key AS c2 FROM t1 WHERE key > 10
* }}}
*/
def collectProjectsAndFilters(plan: LogicalPlan):
(Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, Map[Attribute, Expression]) =
plan match {
case Project(fields, child) =>
val (_, filters, other, aliases) = collectProjectsAndFilters(child)
val substitutedFields = fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]]
(Some(substitutedFields), filters, other, collectAliases(substitutedFields))
case Filter(condition, child) =>
val (fields, filters, other, aliases) = collectProjectsAndFilters(child)
val substitutedCondition = substitute(aliases)(condition)
(fields, filters ++ splitConjunctivePredicates(substitutedCondition), other, aliases)
case other =>
(None, Nil, other, Map.empty)
}
def collectAliases(fields: Seq[Expression]) = fields.collect {
case a @ Alias(child, _) => a.toAttribute.asInstanceOf[Attribute] -> child
}.toMap
def substitute(aliases: Map[Attribute, Expression])(expr: Expression) = expr.transform {
case a @ Alias(ref: AttributeReference, name) =>
aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a)
case a: AttributeReference =>
aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a)
}
}
/**
* A pattern that collects all adjacent unions and returns their children as a Seq.
*/
object Unions {
def unapply(plan: LogicalPlan): Option[Seq[LogicalPlan]] = plan match {
case u: Union => Some(collectUnionChildren(u))
case _ => None
}
private def collectUnionChildren(plan: LogicalPlan): Seq[LogicalPlan] = plan match {
case Union(l, r) => collectUnionChildren(l) ++ collectUnionChildren(r)
case other => other :: Nil
}
}

View file

@ -0,0 +1,128 @@
/*
* 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
package catalyst
package plans
import catalyst.expressions.{SortOrder, Attribute, Expression}
import catalyst.trees._
abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] {
self: PlanType with Product =>
def output: Seq[Attribute]
/**
* Returns the set of attributes that are output by this node.
*/
def outputSet: Set[Attribute] = output.toSet
/**
* Runs [[transform]] with `rule` on all expressions present in this query operator.
* Users should not expect a specific directionality. If a specific directionality is needed,
* transformExpressionsDown or transformExpressionsUp should be used.
* @param rule the rule to be applied to every expression in this operator.
*/
def transformExpressions(rule: PartialFunction[Expression, Expression]): this.type = {
transformExpressionsDown(rule)
}
/**
* Runs [[transformDown]] with `rule` on all expressions present in this query operator.
* @param rule the rule to be applied to every expression in this operator.
*/
def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = {
var changed = false
@inline def transformExpressionDown(e: Expression) = {
val newE = e.transformDown(rule)
if (newE.id != e.id && newE != e) {
changed = true
newE
} else {
e
}
}
val newArgs = productIterator.map {
case e: Expression => transformExpressionDown(e)
case Some(e: Expression) => Some(transformExpressionDown(e))
case m: Map[_,_] => m
case seq: Traversable[_] => seq.map {
case e: Expression => transformExpressionDown(e)
case other => other
}
case other: AnyRef => other
}.toArray
if (changed) makeCopy(newArgs) else this
}
/**
* Runs [[transformUp]] with `rule` on all expressions present in this query operator.
* @param rule the rule to be applied to every expression in this operator.
* @return
*/
def transformExpressionsUp(rule: PartialFunction[Expression, Expression]): this.type = {
var changed = false
@inline def transformExpressionUp(e: Expression) = {
val newE = e.transformUp(rule)
if (newE.id != e.id && newE != e) {
changed = true
newE
} else {
e
}
}
val newArgs = productIterator.map {
case e: Expression => transformExpressionUp(e)
case Some(e: Expression) => Some(transformExpressionUp(e))
case m: Map[_,_] => m
case seq: Traversable[_] => seq.map {
case e: Expression => transformExpressionUp(e)
case other => other
}
case other: AnyRef => other
}.toArray
if (changed) makeCopy(newArgs) else this
}
/** Returns the result of running [[transformExpressions]] on this node
* and all its children. */
def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = {
transform {
case q: QueryPlan[_] => q.transformExpressions(rule).asInstanceOf[PlanType]
}.asInstanceOf[this.type]
}
/** Returns all of the expressions present in this query plan operator. */
def expressions: Seq[Expression] = {
productIterator.flatMap {
case e: Expression => e :: Nil
case Some(e: Expression) => e :: Nil
case seq: Traversable[_] => seq.flatMap {
case e: Expression => e :: Nil
case other => Nil
}
case other => Nil
}.toSeq
}
}

View file

@ -0,0 +1,26 @@
/*
* 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
package catalyst
package plans
sealed abstract class JoinType
case object Inner extends JoinType
case object LeftOuter extends JoinType
case object RightOuter extends JoinType
case object FullOuter extends JoinType

View file

@ -0,0 +1,28 @@
/*
* 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
package catalyst
package plans
package logical
abstract class BaseRelation extends LeafNode {
self: Product =>
def tableName: String
def isPartitioned: Boolean = false
}

View file

@ -0,0 +1,132 @@
/*
* 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
package catalyst
package plans
package logical
import catalyst.expressions._
import catalyst.errors._
import catalyst.types.StructType
abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
self: Product =>
/**
* Returns the set of attributes that are referenced by this node
* during evaluation.
*/
def references: Set[Attribute]
/**
* Returns the set of attributes that this node takes as
* input from its children.
*/
lazy val inputSet: Set[Attribute] = children.flatMap(_.output).toSet
/**
* Returns true if this expression and all its children have been resolved to a specific schema
* and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan
* can override this (e.g. [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]] should
* return `false`).
*/
lazy val resolved: Boolean = !expressions.exists(!_.resolved) && childrenResolved
/**
* Returns true if all its children of this query plan have been resolved.
*/
def childrenResolved = !children.exists(!_.resolved)
/**
* Optionally resolves the given string to a
* [[catalyst.expressions.NamedExpression NamedExpression]]. The attribute is expressed as
* as string in the following form: `[scope].AttributeName.[nested].[fields]...`.
*/
def resolve(name: String): Option[NamedExpression] = {
val parts = name.split("\\.")
// Collect all attributes that are output by this nodes children where either the first part
// matches the name or where the first part matches the scope and the second part matches the
// name. Return these matches along with any remaining parts, which represent dotted access to
// struct fields.
val options = children.flatMap(_.output).flatMap { option =>
// If the first part of the desired name matches a qualifier for this possible match, drop it.
val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts
if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil
}
options.distinct match {
case (a, Nil) :: Nil => Some(a) // One match, no nested fields, use it.
// One match, but we also need to extract the requested nested field.
case (a, nestedFields) :: Nil =>
a.dataType match {
case StructType(fields) =>
Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)())
case _ => None // Don't know how to resolve these field references
}
case Nil => None // No matches.
case ambiguousReferences =>
throw new TreeNodeException(
this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}")
}
}
}
/**
* A logical plan node with no children.
*/
abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] {
self: Product =>
// Leaf nodes by definition cannot reference any input attributes.
def references = Set.empty
}
/**
* A logical node that represents a non-query command to be executed by the system. For example,
* commands can be used by parsers to represent DDL operations.
*/
abstract class Command extends LeafNode {
self: Product =>
def output = Seq.empty
}
/**
* Returned for commands supported by a given parser, but not catalyst. In general these are DDL
* commands that are passed directly to another system.
*/
case class NativeCommand(cmd: String) extends Command
/**
* Returned by a parser when the users only wants to see what query plan would be executed, without
* actually performing the execution.
*/
case class ExplainCommand(plan: LogicalPlan) extends Command
/**
* A logical plan node with single child.
*/
abstract class UnaryNode extends LogicalPlan with trees.UnaryNode[LogicalPlan] {
self: Product =>
}
/**
* A logical plan node with a left and right child.
*/
abstract class BinaryNode extends LogicalPlan with trees.BinaryNode[LogicalPlan] {
self: Product =>
}

View file

@ -0,0 +1,38 @@
/*
* 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
package catalyst
package plans
package logical
import expressions._
/**
* Transforms the input by forking and running the specified script.
*
* @param input the set of expression that should be passed to the script.
* @param script the command that should be executed.
* @param output the attributes that are produced by the script.
*/
case class ScriptTransformation(
input: Seq[Expression],
script: String,
output: Seq[Attribute],
child: LogicalPlan) extends UnaryNode {
def references = input.flatMap(_.references).toSet
}

View file

@ -0,0 +1,47 @@
/*
* 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
package catalyst
package plans
package logical
import expressions._
import rules._
object LocalRelation {
def apply(output: Attribute*) =
new LocalRelation(output)
}
case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil)
extends LeafNode with analysis.MultiInstanceRelation {
// TODO: Validate schema compliance.
def loadData(newData: Seq[Product]) = new LocalRelation(output, data ++ newData)
/**
* Returns an identical copy of this relation with new exprIds for all attributes. Different
* attributes are required when a relation is going to be included multiple times in the same
* query.
*/
override final def newInstance: this.type = {
LocalRelation(output.map(_.newInstance), data).asInstanceOf[this.type]
}
override protected def stringArgs = Iterator(output)
}

View file

@ -0,0 +1,158 @@
/*
* 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
package catalyst
package plans
package logical
import expressions._
case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
def output = projectList.map(_.toAttribute)
def references = projectList.flatMap(_.references).toSet
}
/**
* Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the
* output of each into a new stream of rows. This operation is similar to a `flatMap` in functional
* programming with one important additional feature, which allows the input rows to be joined with
* their output.
* @param join when true, each output row is implicitly joined with the input tuple that produced
* it.
* @param outer when true, each input row will be output at least once, even if the output of the
* given `generator` is empty. `outer` has no effect when `join` is false.
* @param alias when set, this string is applied to the schema of the output of the transformation
* as a qualifier.
*/
case class Generate(
generator: Generator,
join: Boolean,
outer: Boolean,
alias: Option[String],
child: LogicalPlan)
extends UnaryNode {
protected def generatorOutput =
alias
.map(a => generator.output.map(_.withQualifiers(a :: Nil)))
.getOrElse(generator.output)
def output =
if (join) child.output ++ generatorOutput else generatorOutput
def references =
if (join) child.outputSet else generator.references
}
case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode {
def output = child.output
def references = condition.references
}
case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode {
// TODO: These aren't really the same attributes as nullability etc might change.
def output = left.output
override lazy val resolved =
childrenResolved &&
!left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType }
def references = Set.empty
}
case class Join(
left: LogicalPlan,
right: LogicalPlan,
joinType: JoinType,
condition: Option[Expression]) extends BinaryNode {
def references = condition.map(_.references).getOrElse(Set.empty)
def output = left.output ++ right.output
}
case class InsertIntoTable(
table: BaseRelation,
partition: Map[String, Option[String]],
child: LogicalPlan,
overwrite: Boolean)
extends LogicalPlan {
// The table being inserted into is a child for the purposes of transformations.
def children = table :: child :: Nil
def references = Set.empty
def output = child.output
override lazy val resolved = childrenResolved && child.output.zip(table.output).forall {
case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType
}
}
case class InsertIntoCreatedTable(
databaseName: Option[String],
tableName: String,
child: LogicalPlan) extends UnaryNode {
def references = Set.empty
def output = child.output
}
case class WriteToFile(
path: String,
child: LogicalPlan) extends UnaryNode {
def references = Set.empty
def output = child.output
}
case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode {
def output = child.output
def references = order.flatMap(_.references).toSet
}
case class Aggregate(
groupingExpressions: Seq[Expression],
aggregateExpressions: Seq[NamedExpression],
child: LogicalPlan)
extends UnaryNode {
def output = aggregateExpressions.map(_.toAttribute)
def references = child.references
}
case class StopAfter(limit: Expression, child: LogicalPlan) extends UnaryNode {
def output = child.output
def references = limit.references
}
case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode {
def output = child.output.map(_.withQualifiers(alias :: Nil))
def references = Set.empty
}
case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: LogicalPlan)
extends UnaryNode {
def output = child.output
def references = Set.empty
}
case class Distinct(child: LogicalPlan) extends UnaryNode {
def output = child.output
def references = child.outputSet
}
case object NoRelation extends LeafNode {
def output = Nil
}

View file

@ -0,0 +1,46 @@
/*
* 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
package catalyst
package plans
package logical
import expressions._
/**
* Performs a physical redistribution of the data. Used when the consumer of the query
* result have expectations about the distribution and ordering of partitioned input data.
*/
abstract class RedistributeData extends UnaryNode {
self: Product =>
def output = child.output
}
case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan)
extends RedistributeData {
def references = sortExpressions.flatMap(_.references).toSet
}
case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan)
extends RedistributeData {
def references = partitionExpressions.flatMap(_.references).toSet
}

View file

@ -0,0 +1,25 @@
/*
* 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
package catalyst
/**
* A a collection of common abstractions for query plans as well as
* a base logical plan representation.
*/
package object plans

View file

@ -0,0 +1,201 @@
/*
* 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
package catalyst
package plans
package physical
import expressions._
import types._
/**
* Specifies how tuples that share common expressions will be distributed when a query is executed
* in parallel on many machines. Distribution can be used to refer to two distinct physical
* properties:
* - Inter-node partitioning of data: In this case the distribution describes how tuples are
* partitioned across physical machines in a cluster. Knowing this property allows some
* operators (e.g., Aggregate) to perform partition local operations instead of global ones.
* - Intra-partition ordering of data: In this case the distribution describes guarantees made
* about how tuples are distributed within a single partition.
*/
sealed trait Distribution
/**
* Represents a distribution where no promises are made about co-location of data.
*/
case object UnspecifiedDistribution extends Distribution
/**
* Represents a distribution that only has a single partition and all tuples of the dataset
* are co-located.
*/
case object AllTuples extends Distribution
/**
* Represents data where tuples that share the same values for the `clustering`
* [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this
* can mean such tuples are either co-located in the same partition or they will be contiguous
* within a single partition.
*/
case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution {
require(
clustering != Nil,
"The clustering expressions of a ClusteredDistribution should not be Nil. " +
"An AllTuples should be used to represent a distribution that only has " +
"a single partition.")
}
/**
* Represents data where tuples have been ordered according to the `ordering`
* [[catalyst.expressions.Expression Expressions]]. This is a strictly stronger guarantee than
* [[ClusteredDistribution]] as an ordering will ensure that tuples that share the same value for
* the ordering expressions are contiguous and will never be split across partitions.
*/
case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution {
require(
ordering != Nil,
"The ordering expressions of a OrderedDistribution should not be Nil. " +
"An AllTuples should be used to represent a distribution that only has " +
"a single partition.")
def clustering = ordering.map(_.child).toSet
}
sealed trait Partitioning {
/** Returns the number of partitions that the data is split across */
val numPartitions: Int
/**
* Returns true iff the guarantees made by this
* [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy
* the partitioning scheme mandated by the `required`
* [[catalyst.plans.physical.Distribution Distribution]], i.e. the current dataset does not
* need to be re-partitioned for the `required` Distribution (it is possible that tuples within
* a partition need to be reorganized).
*/
def satisfies(required: Distribution): Boolean
/**
* Returns true iff all distribution guarantees made by this partitioning can also be made
* for the `other` specified partitioning.
* For example, two [[catalyst.plans.physical.HashPartitioning HashPartitioning]]s are
* only compatible if the `numPartitions` of them is the same.
*/
def compatibleWith(other: Partitioning): Boolean
}
case class UnknownPartitioning(numPartitions: Int) extends Partitioning {
override def satisfies(required: Distribution): Boolean = required match {
case UnspecifiedDistribution => true
case _ => false
}
override def compatibleWith(other: Partitioning): Boolean = other match {
case UnknownPartitioning(_) => true
case _ => false
}
}
case object SinglePartition extends Partitioning {
val numPartitions = 1
override def satisfies(required: Distribution): Boolean = true
override def compatibleWith(other: Partitioning) = other match {
case SinglePartition => true
case _ => false
}
}
case object BroadcastPartitioning extends Partitioning {
val numPartitions = 1
override def satisfies(required: Distribution): Boolean = true
override def compatibleWith(other: Partitioning) = other match {
case SinglePartition => true
case _ => false
}
}
/**
* Represents a partitioning where rows are split up across partitions based on the hash
* of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be
* in the same partition.
*/
case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
extends Expression
with Partitioning {
def children = expressions
def references = expressions.flatMap(_.references).toSet
def nullable = false
def dataType = IntegerType
lazy val clusteringSet = expressions.toSet
override def satisfies(required: Distribution): Boolean = required match {
case UnspecifiedDistribution => true
case ClusteredDistribution(requiredClustering) =>
clusteringSet.subsetOf(requiredClustering.toSet)
case _ => false
}
override def compatibleWith(other: Partitioning) = other match {
case BroadcastPartitioning => true
case h: HashPartitioning if h == this => true
case _ => false
}
}
/**
* Represents a partitioning where rows are split across partitions based on some total ordering of
* the expressions specified in `ordering`. When data is partitioned in this manner the following
* two conditions are guaranteed to hold:
* - All row where the expressions in `ordering` evaluate to the same values will be in the same
* partition.
* - Each partition will have a `min` and `max` row, relative to the given ordering. All rows
* that are in between `min` and `max` in this `ordering` will reside in this partition.
*/
case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int)
extends Expression
with Partitioning {
def children = ordering
def references = ordering.flatMap(_.references).toSet
def nullable = false
def dataType = IntegerType
lazy val clusteringSet = ordering.map(_.child).toSet
override def satisfies(required: Distribution): Boolean = required match {
case UnspecifiedDistribution => true
case OrderedDistribution(requiredOrdering) =>
val minSize = Seq(requiredOrdering.size, ordering.size).min
requiredOrdering.take(minSize) == ordering.take(minSize)
case ClusteredDistribution(requiredClustering) =>
clusteringSet.subsetOf(requiredClustering.toSet)
case _ => false
}
override def compatibleWith(other: Partitioning) = other match {
case BroadcastPartitioning => true
case r: RangePartitioning if r == this => true
case _ => false
}
}

View file

@ -0,0 +1,33 @@
/*
* 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
package catalyst
package rules
import trees._
abstract class Rule[TreeType <: TreeNode[_]] extends Logging {
/** Name for this rule, automatically inferred based on class name. */
val ruleName: String = {
val className = getClass.getName
if (className endsWith "$") className.dropRight(1) else className
}
def apply(plan: TreeType): TreeType
}

View file

@ -0,0 +1,79 @@
/*
* 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
package catalyst
package rules
import trees._
import util._
abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
/**
* An execution strategy for rules that indicates the maximum number of executions. If the
* execution reaches fix point (i.e. converge) before maxIterations, it will stop.
*/
abstract class Strategy { def maxIterations: Int }
/** A strategy that only runs once. */
case object Once extends Strategy { val maxIterations = 1 }
/** A strategy that runs until fix point or maxIterations times, whichever comes first. */
case class FixedPoint(maxIterations: Int) extends Strategy
/** A batch of rules. */
protected case class Batch(name: String, strategy: Strategy, rules: Rule[TreeType]*)
/** Defines a sequence of rule batches, to be overridden by the implementation. */
protected val batches: Seq[Batch]
/**
* Executes the batches of rules defined by the subclass. The batches are executed serially
* using the defined execution strategy. Within each batch, rules are also executed serially.
*/
def apply(plan: TreeType): TreeType = {
var curPlan = plan
batches.foreach { batch =>
var iteration = 1
var lastPlan = curPlan
curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => rule(curPlan) }
// Run until fix point (or the max number of iterations as specified in the strategy.
while (iteration < batch.strategy.maxIterations && !curPlan.fastEquals(lastPlan)) {
lastPlan = curPlan
curPlan = batch.rules.foldLeft(curPlan) {
case (curPlan, rule) =>
val result = rule(curPlan)
if (!result.fastEquals(curPlan)) {
logger.debug(
s"""
|=== Applying Rule ${rule.ruleName} ===
|${sideBySide(curPlan.treeString, result.treeString).mkString("\n")}
""".stripMargin)
}
result
}
iteration += 1
}
}
curPlan
}
}

View file

@ -0,0 +1,24 @@
/*
* 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
package catalyst
/**
* A framework for applying batches rewrite rules to trees, possibly to fixed point.
*/
package object rules

View file

@ -0,0 +1,364 @@
/*
* 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
package catalyst
package trees
import errors._
object TreeNode {
private val currentId = new java.util.concurrent.atomic.AtomicLong
protected def nextId() = currentId.getAndIncrement()
}
/** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */
private class MutableInt(var i: Int)
abstract class TreeNode[BaseType <: TreeNode[BaseType]] {
self: BaseType with Product =>
/** Returns a Seq of the children of this node */
def children: Seq[BaseType]
/**
* A globally unique id for this specific instance. Not preserved across copies.
* Unlike `equals`, `id` can be used to differentiate distinct but structurally
* identical branches of a tree.
*/
val id = TreeNode.nextId()
/**
* Returns true if other is the same [[catalyst.trees.TreeNode TreeNode]] instance. Unlike
* `equals` this function will return false for different instances of structurally identical
* trees.
*/
def sameInstance(other: TreeNode[_]): Boolean = {
this.id == other.id
}
/**
* Faster version of equality which short-circuits when two treeNodes are the same instance.
* We don't just override Object.Equals, as doing so prevents the scala compiler from from
* generating case class `equals` methods
*/
def fastEquals(other: TreeNode[_]): Boolean = {
sameInstance(other) || this == other
}
/**
* Runs the given function on this node and then recursively on [[children]].
* @param f the function to be applied to each node in the tree.
*/
def foreach(f: BaseType => Unit): Unit = {
f(this)
children.foreach(_.foreach(f))
}
/**
* Returns a Seq containing the result of applying the given function to each
* node in this tree in a preorder traversal.
* @param f the function to be applied.
*/
def map[A](f: BaseType => A): Seq[A] = {
val ret = new collection.mutable.ArrayBuffer[A]()
foreach(ret += f(_))
ret
}
/**
* Returns a Seq by applying a function to all nodes in this tree and using the elements of the
* resulting collections.
*/
def flatMap[A](f: BaseType => TraversableOnce[A]): Seq[A] = {
val ret = new collection.mutable.ArrayBuffer[A]()
foreach(ret ++= f(_))
ret
}
/**
* Returns a Seq containing the result of applying a partial function to all elements in this
* tree on which the function is defined.
*/
def collect[B](pf: PartialFunction[BaseType, B]): Seq[B] = {
val ret = new collection.mutable.ArrayBuffer[B]()
val lifted = pf.lift
foreach(node => lifted(node).foreach(ret.+=))
ret
}
/**
* Returns a copy of this node where `f` has been applied to all the nodes children.
*/
def mapChildren(f: BaseType => BaseType): this.type = {
var changed = false
val newArgs = productIterator.map {
case arg: TreeNode[_] if children contains arg =>
val newChild = f(arg.asInstanceOf[BaseType])
if (newChild fastEquals arg) {
arg
} else {
changed = true
newChild
}
case nonChild: AnyRef => nonChild
case null => null
}.toArray
if (changed) makeCopy(newArgs) else this
}
/**
* Returns a copy of this node with the children replaced.
* TODO: Validate somewhere (in debug mode?) that children are ordered correctly.
*/
def withNewChildren(newChildren: Seq[BaseType]): this.type = {
assert(newChildren.size == children.size, "Incorrect number of children")
var changed = false
val remainingNewChildren = newChildren.toBuffer
val remainingOldChildren = children.toBuffer
val newArgs = productIterator.map {
case arg: TreeNode[_] if children contains arg =>
val newChild = remainingNewChildren.remove(0)
val oldChild = remainingOldChildren.remove(0)
if (newChild fastEquals oldChild) {
oldChild
} else {
changed = true
newChild
}
case nonChild: AnyRef => nonChild
case null => null
}.toArray
if (changed) makeCopy(newArgs) else this
}
/**
* Returns a copy of this node where `rule` has been recursively applied to the tree.
* When `rule` does not apply to a given node it is left unchanged.
* Users should not expect a specific directionality. If a specific directionality is needed,
* transformDown or transformUp should be used.
* @param rule the function use to transform this nodes children
*/
def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = {
transformDown(rule)
}
/**
* Returns a copy of this node where `rule` has been recursively applied to it and all of its
* children (pre-order). When `rule` does not apply to a given node it is left unchanged.
* @param rule the function used to transform this nodes children
*/
def transformDown(rule: PartialFunction[BaseType, BaseType]): BaseType = {
val afterRule = rule.applyOrElse(this, identity[BaseType])
// Check if unchanged and then possibly return old copy to avoid gc churn.
if (this fastEquals afterRule) {
transformChildrenDown(rule)
} else {
afterRule.transformChildrenDown(rule)
}
}
/**
* Returns a copy of this node where `rule` has been recursively applied to all the children of
* this node. When `rule` does not apply to a given node it is left unchanged.
* @param rule the function used to transform this nodes children
*/
def transformChildrenDown(rule: PartialFunction[BaseType, BaseType]): this.type = {
var changed = false
val newArgs = productIterator.map {
case arg: TreeNode[_] if children contains arg =>
val newChild = arg.asInstanceOf[BaseType].transformDown(rule)
if (!(newChild fastEquals arg)) {
changed = true
newChild
} else {
arg
}
case m: Map[_,_] => m
case args: Traversable[_] => args.map {
case arg: TreeNode[_] if children contains arg =>
val newChild = arg.asInstanceOf[BaseType].transformDown(rule)
if (!(newChild fastEquals arg)) {
changed = true
newChild
} else {
arg
}
case other => other
}
case nonChild: AnyRef => nonChild
case null => null
}.toArray
if (changed) makeCopy(newArgs) else this
}
/**
* Returns a copy of this node where `rule` has been recursively applied first to all of its
* children and then itself (post-order). When `rule` does not apply to a given node, it is left
* unchanged.
* @param rule the function use to transform this nodes children
*/
def transformUp(rule: PartialFunction[BaseType, BaseType]): BaseType = {
val afterRuleOnChildren = transformChildrenUp(rule);
if (this fastEquals afterRuleOnChildren) {
rule.applyOrElse(this, identity[BaseType])
} else {
rule.applyOrElse(afterRuleOnChildren, identity[BaseType])
}
}
def transformChildrenUp(rule: PartialFunction[BaseType, BaseType]): this.type = {
var changed = false
val newArgs = productIterator.map {
case arg: TreeNode[_] if children contains arg =>
val newChild = arg.asInstanceOf[BaseType].transformUp(rule)
if (!(newChild fastEquals arg)) {
changed = true
newChild
} else {
arg
}
case m: Map[_,_] => m
case args: Traversable[_] => args.map {
case arg: TreeNode[_] if children contains arg =>
val newChild = arg.asInstanceOf[BaseType].transformUp(rule)
if (!(newChild fastEquals arg)) {
changed = true
newChild
} else {
arg
}
case other => other
}
case nonChild: AnyRef => nonChild
case null => null
}.toArray
if (changed) makeCopy(newArgs) else this
}
/**
* Args to the constructor that should be copied, but not transformed.
* These are appended to the transformed args automatically by makeCopy
* @return
*/
protected def otherCopyArgs: Seq[AnyRef] = Nil
/**
* Creates a copy of this type of tree node after a transformation.
* Must be overridden by child classes that have constructor arguments
* that are not present in the productIterator.
* @param newArgs the new product arguments.
*/
def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") {
try {
val defaultCtor = getClass.getConstructors.head
if (otherCopyArgs.isEmpty) {
defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type]
} else {
defaultCtor.newInstance((newArgs ++ otherCopyArgs).toArray: _*).asInstanceOf[this.type]
}
} catch {
case e: java.lang.IllegalArgumentException =>
throw new TreeNodeException(
this, s"Failed to copy node. Is otherCopyArgs specified correctly for $nodeName?")
}
}
/** Returns the name of this type of TreeNode. Defaults to the class name. */
def nodeName = getClass.getSimpleName
/**
* The arguments that should be included in the arg string. Defaults to the `productIterator`.
*/
protected def stringArgs = productIterator
/** Returns a string representing the arguments to this node, minus any children */
def argString: String = productIterator.flatMap {
case tn: TreeNode[_] if children contains tn => Nil
case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil
case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil
case set: Set[_] => set.mkString("{", ",", "}") :: Nil
case other => other :: Nil
}.mkString(", ")
/** String representation of this node without any children */
def simpleString = s"$nodeName $argString"
override def toString: String = treeString
/** Returns a string representation of the nodes in this tree */
def treeString = generateTreeString(0, new StringBuilder).toString
/**
* Returns a string representation of the nodes in this tree, where each operator is numbered.
* The numbers can be used with [[trees.TreeNode.apply apply]] to easily access specific subtrees.
*/
def numberedTreeString =
treeString.split("\n").zipWithIndex.map { case (line, i) => f"$i%02d $line" }.mkString("\n")
/**
* Returns the tree node at the specified number.
* Numbers for each node can be found in the [[numberedTreeString]].
*/
def apply(number: Int): BaseType = getNodeNumbered(new MutableInt(number))
protected def getNodeNumbered(number: MutableInt): BaseType = {
if (number.i < 0) {
null.asInstanceOf[BaseType]
} else if (number.i == 0) {
this
} else {
number.i -= 1
children.map(_.getNodeNumbered(number)).find(_ != null).getOrElse(null.asInstanceOf[BaseType])
}
}
/** Appends the string represent of this node and its children to the given StringBuilder. */
protected def generateTreeString(depth: Int, builder: StringBuilder): StringBuilder = {
builder.append(" " * depth)
builder.append(simpleString)
builder.append("\n")
children.foreach(_.generateTreeString(depth + 1, builder))
builder
}
}
/**
* A [[TreeNode]] that has two children, [[left]] and [[right]].
*/
trait BinaryNode[BaseType <: TreeNode[BaseType]] {
def left: BaseType
def right: BaseType
def children = Seq(left, right)
}
/**
* A [[TreeNode]] with no children.
*/
trait LeafNode[BaseType <: TreeNode[BaseType]] {
def children = Nil
}
/**
* A [[TreeNode]] with a single [[child]].
*/
trait UnaryNode[BaseType <: TreeNode[BaseType]] {
def child: BaseType
def children = child :: Nil
}

View file

@ -0,0 +1,38 @@
/*
* 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
package catalyst
/**
* A library for easily manipulating trees of operators. Operators that extend TreeNode are
* granted the following interface:
* <ul>
* <li>Scala collection like methods (foreach, map, flatMap, collect, etc)</li>
* <li>
* transform - accepts a partial function that is used to generate a new tree. When the
* partial function can be applied to a given tree segment, that segment is replaced with the
* result. After attempting to apply the partial function to a given node, the transform
* function recursively attempts to apply the function to that node's children.
* </li>
* <li>debugging support - pretty printing, easy splicing of trees, etc.</li>
* </ul>
*/
package object trees {
// Since we want tree nodes to be lightweight, we create one logger for all treenode instances.
protected val logger = Logger("catalyst.trees")
}

View file

@ -0,0 +1,137 @@
/*
* 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
package catalyst
package types
import expressions.Expression
abstract class DataType {
/** Matches any expression that evaluates to this DataType */
def unapply(a: Expression): Boolean = a match {
case e: Expression if e.dataType == this => true
case _ => false
}
}
case object NullType extends DataType
abstract class NativeType extends DataType {
type JvmType
val ordering: Ordering[JvmType]
}
case object StringType extends NativeType {
type JvmType = String
val ordering = implicitly[Ordering[JvmType]]
}
case object BinaryType extends DataType {
type JvmType = Array[Byte]
}
case object BooleanType extends NativeType {
type JvmType = Boolean
val ordering = implicitly[Ordering[JvmType]]
}
abstract class NumericType extends NativeType {
// Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for
// implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a
// type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets
// desugared by the compiler into an argument to the objects constructor. This means there is no
// longer an no argument constructor and thus the JVM cannot serialize the object anymore.
val numeric: Numeric[JvmType]
}
/** Matcher for any expressions that evaluate to [[IntegralType]]s */
object IntegralType {
def unapply(a: Expression): Boolean = a match {
case e: Expression if e.dataType.isInstanceOf[IntegralType] => true
case _ => false
}
}
abstract class IntegralType extends NumericType {
val integral: Integral[JvmType]
}
case object LongType extends IntegralType {
type JvmType = Long
val numeric = implicitly[Numeric[Long]]
val integral = implicitly[Integral[Long]]
val ordering = implicitly[Ordering[JvmType]]
}
case object IntegerType extends IntegralType {
type JvmType = Int
val numeric = implicitly[Numeric[Int]]
val integral = implicitly[Integral[Int]]
val ordering = implicitly[Ordering[JvmType]]
}
case object ShortType extends IntegralType {
type JvmType = Short
val numeric = implicitly[Numeric[Short]]
val integral = implicitly[Integral[Short]]
val ordering = implicitly[Ordering[JvmType]]
}
case object ByteType extends IntegralType {
type JvmType = Byte
val numeric = implicitly[Numeric[Byte]]
val integral = implicitly[Integral[Byte]]
val ordering = implicitly[Ordering[JvmType]]
}
/** Matcher for any expressions that evaluate to [[FractionalType]]s */
object FractionalType {
def unapply(a: Expression): Boolean = a match {
case e: Expression if e.dataType.isInstanceOf[FractionalType] => true
case _ => false
}
}
abstract class FractionalType extends NumericType {
val fractional: Fractional[JvmType]
}
case object DecimalType extends FractionalType {
type JvmType = BigDecimal
val numeric = implicitly[Numeric[BigDecimal]]
val fractional = implicitly[Fractional[BigDecimal]]
val ordering = implicitly[Ordering[JvmType]]
}
case object DoubleType extends FractionalType {
type JvmType = Double
val numeric = implicitly[Numeric[Double]]
val fractional = implicitly[Fractional[Double]]
val ordering = implicitly[Ordering[JvmType]]
}
case object FloatType extends FractionalType {
type JvmType = Float
val numeric = implicitly[Numeric[Float]]
val fractional = implicitly[Fractional[Float]]
val ordering = implicitly[Ordering[JvmType]]
}
case class ArrayType(elementType: DataType) extends DataType
case class StructField(name: String, dataType: DataType, nullable: Boolean)
case class StructType(fields: Seq[StructField]) extends DataType
case class MapType(keyType: DataType, valueType: DataType) extends DataType

View file

@ -0,0 +1,24 @@
/*
* 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
package catalyst
/**
* Contains a type system for attributes produced by relations, including complex types like
* structs, arrays and maps.
*/
package object types

View file

@ -0,0 +1,122 @@
/*
* 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
package catalyst
import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File}
package object util {
/**
* Returns a path to a temporary file that probably does not exist.
* Note, there is always the race condition that someone created this
* file since the last time we checked. Thus, this shouldn't be used
* for anything security conscious.
*/
def getTempFilePath(prefix: String, suffix: String = ""): File = {
val tempFile = File.createTempFile(prefix, suffix)
tempFile.delete()
tempFile
}
def fileToString(file: File, encoding: String = "UTF-8") = {
val inStream = new FileInputStream(file)
val outStream = new ByteArrayOutputStream
try {
var reading = true
while ( reading ) {
inStream.read() match {
case -1 => reading = false
case c => outStream.write(c)
}
}
outStream.flush()
}
finally {
inStream.close()
}
new String(outStream.toByteArray, encoding)
}
def resourceToString(
resource:String,
encoding: String = "UTF-8",
classLoader: ClassLoader = this.getClass.getClassLoader) = {
val inStream = classLoader.getResourceAsStream(resource)
val outStream = new ByteArrayOutputStream
try {
var reading = true
while ( reading ) {
inStream.read() match {
case -1 => reading = false
case c => outStream.write(c)
}
}
outStream.flush()
}
finally {
inStream.close()
}
new String(outStream.toByteArray, encoding)
}
def stringToFile(file: File, str: String): File = {
val out = new PrintWriter(file)
out.write(str)
out.close()
file
}
def sideBySide(left: String, right: String): Seq[String] = {
sideBySide(left.split("\n"), right.split("\n"))
}
def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = {
val maxLeftSize = left.map(_.size).max
val leftPadded = left ++ Seq.fill(math.max(right.size - left.size, 0))("")
val rightPadded = right ++ Seq.fill(math.max(left.size - right.size, 0))("")
leftPadded.zip(rightPadded).map {
case (l, r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r
}
}
def stackTraceToString(t: Throwable): String = {
val out = new java.io.ByteArrayOutputStream
val writer = new PrintWriter(out)
t.printStackTrace(writer)
writer.flush()
new String(out.toByteArray)
}
def stringOrNull(a: AnyRef) = if (a == null) null else a.toString
def benchmark[A](f: => A): A = {
val startTime = System.nanoTime()
val ret = f
val endTime = System.nanoTime()
println(s"${(endTime - startTime).toDouble / 1000000}ms")
ret
}
/* FIX ME
implicit class debugLogging(a: AnyRef) {
def debugLogging() {
org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG)
}
} */
}

View file

@ -0,0 +1,49 @@
/*
* 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
/**
* Allows the execution of relational queries, including those expressed in SQL using Spark.
*
* Note that this package is located in catalyst instead of in core so that all subprojects can
* inherit the settings from this package object.
*/
package object sql {
protected[sql] def Logger(name: String) =
com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name))
protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging
type Row = catalyst.expressions.Row
object Row {
/**
* This method can be used to extract fields from a [[Row]] object in a pattern match. Example:
* {{{
* import org.apache.spark.sql._
*
* val pairs = sql("SELECT key, value FROM src").rdd.map {
* case Row(key: Int, value: String) =>
* key -> value
* }
* }}}
*/
def unapplySeq(row: Row): Some[Seq[Any]] = Some(row)
}
}

View file

@ -0,0 +1,41 @@
/*
* 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
package catalyst
package analysis
import org.scalatest.FunSuite
import analysis._
import expressions._
import plans.logical._
import types._
import dsl._
import dsl.expressions._
class AnalysisSuite extends FunSuite {
val analyze = SimpleAnalyzer
val testRelation = LocalRelation('a.int)
test("analyze project") {
assert(analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation))
}
}

View file

@ -0,0 +1,175 @@
/*
* 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
package catalyst
import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.plans.physical._
/* Implicit conversions */
import org.apache.spark.sql.catalyst.dsl.expressions._
class DistributionSuite extends FunSuite {
protected def checkSatisfied(
inputPartitioning: Partitioning,
requiredDistribution: Distribution,
satisfied: Boolean) {
if (inputPartitioning.satisfies(requiredDistribution) != satisfied)
fail(
s"""
|== Input Partitioning ==
|$inputPartitioning
|== Required Distribution ==
|$requiredDistribution
|== Does input partitioning satisfy required distribution? ==
|Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)}
""".stripMargin)
}
test("HashPartitioning is the output partitioning") {
// Cases which do not need an exchange between two data properties.
checkSatisfied(
HashPartitioning(Seq('a, 'b, 'c), 10),
UnspecifiedDistribution,
true)
checkSatisfied(
HashPartitioning(Seq('a, 'b, 'c), 10),
ClusteredDistribution(Seq('a, 'b, 'c)),
true)
checkSatisfied(
HashPartitioning(Seq('b, 'c), 10),
ClusteredDistribution(Seq('a, 'b, 'c)),
true)
checkSatisfied(
SinglePartition,
ClusteredDistribution(Seq('a, 'b, 'c)),
true)
checkSatisfied(
SinglePartition,
OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)),
true)
// Cases which need an exchange between two data properties.
checkSatisfied(
HashPartitioning(Seq('a, 'b, 'c), 10),
ClusteredDistribution(Seq('b, 'c)),
false)
checkSatisfied(
HashPartitioning(Seq('a, 'b, 'c), 10),
ClusteredDistribution(Seq('d, 'e)),
false)
checkSatisfied(
HashPartitioning(Seq('a, 'b, 'c), 10),
AllTuples,
false)
checkSatisfied(
HashPartitioning(Seq('a, 'b, 'c), 10),
OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)),
false)
checkSatisfied(
HashPartitioning(Seq('b, 'c), 10),
OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)),
false)
// TODO: We should check functional dependencies
/*
checkSatisfied(
ClusteredDistribution(Seq('b)),
ClusteredDistribution(Seq('b + 1)),
true)
*/
}
test("RangePartitioning is the output partitioning") {
// Cases which do not need an exchange between two data properties.
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
UnspecifiedDistribution,
true)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)),
true)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
OrderedDistribution(Seq('a.asc, 'b.asc)),
true)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)),
true)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
ClusteredDistribution(Seq('a, 'b, 'c)),
true)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
ClusteredDistribution(Seq('c, 'b, 'a)),
true)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
ClusteredDistribution(Seq('b, 'c, 'a, 'd)),
true)
// Cases which need an exchange between two data properties.
// TODO: We can have an optimization to first sort the dataset
// by a.asc and then sort b, and c in a partition. This optimization
// should tradeoff the benefit of a less number of Exchange operators
// and the parallelism.
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)),
false)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
OrderedDistribution(Seq('b.asc, 'a.asc)),
false)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
ClusteredDistribution(Seq('a, 'b)),
false)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
ClusteredDistribution(Seq('c, 'd)),
false)
checkSatisfied(
RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10),
AllTuples,
false)
}
}

View file

@ -0,0 +1,115 @@
/*
* 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
package catalyst
package expressions
import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types._
/* Implict conversions */
import org.apache.spark.sql.catalyst.dsl.expressions._
class ExpressionEvaluationSuite extends FunSuite {
test("literals") {
assert((Literal(1) + Literal(1)).apply(null) === 2)
}
/**
* Checks for three-valued-logic. Based on:
* http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29
*
* p q p OR q p AND q p = q
* True True True True True
* True False True False False
* True Unknown True Unknown Unknown
* False True True False False
* False False False False True
* False Unknown Unknown False Unknown
* Unknown True True Unknown Unknown
* Unknown False Unknown False Unknown
* Unknown Unknown Unknown Unknown Unknown
*
* p NOT p
* True False
* False True
* Unknown Unknown
*/
val notTrueTable =
(true, false) ::
(false, true) ::
(null, null) :: Nil
test("3VL Not") {
notTrueTable.foreach {
case (v, answer) =>
val expr = Not(Literal(v, BooleanType))
val result = expr.apply(null)
if (result != answer)
fail(s"$expr should not evaluate to $result, expected: $answer") }
}
booleanLogicTest("AND", _ && _,
(true, true, true) ::
(true, false, false) ::
(true, null, null) ::
(false, true, false) ::
(false, false, false) ::
(false, null, false) ::
(null, true, null) ::
(null, false, false) ::
(null, null, null) :: Nil)
booleanLogicTest("OR", _ || _,
(true, true, true) ::
(true, false, true) ::
(true, null, true) ::
(false, true, true) ::
(false, false, false) ::
(false, null, null) ::
(null, true, true) ::
(null, false, null) ::
(null, null, null) :: Nil)
booleanLogicTest("=", _ === _,
(true, true, true) ::
(true, false, false) ::
(true, null, null) ::
(false, true, false) ::
(false, false, true) ::
(false, null, null) ::
(null, true, null) ::
(null, false, null) ::
(null, null, null) :: Nil)
def booleanLogicTest(name: String, op: (Expression, Expression) => Expression, truthTable: Seq[(Any, Any, Any)]) {
test(s"3VL $name") {
truthTable.foreach {
case (l,r,answer) =>
val expr = op(Literal(l, BooleanType), Literal(r, BooleanType))
val result = expr.apply(null)
if (result != answer)
fail(s"$expr should not evaluate to $result, expected: $answer")
}
}
}
}

View file

@ -0,0 +1,74 @@
/*
* 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
package catalyst
package analysis
import org.scalatest.FunSuite
import catalyst.types._
class HiveTypeCoercionSuite extends FunSuite {
val rules = new HiveTypeCoercion { }
import rules._
test("tightest common bound for numeric and boolean types") {
def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) {
var found = WidenTypes.findTightestCommonType(t1, t2)
assert(found == tightestCommon,
s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found")
// Test both directions to make sure the widening is symmetric.
found = WidenTypes.findTightestCommonType(t2, t1)
assert(found == tightestCommon,
s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found")
}
// Boolean
widenTest(NullType, BooleanType, Some(BooleanType))
widenTest(BooleanType, BooleanType, Some(BooleanType))
widenTest(IntegerType, BooleanType, None)
widenTest(LongType, BooleanType, None)
// Integral
widenTest(NullType, ByteType, Some(ByteType))
widenTest(NullType, IntegerType, Some(IntegerType))
widenTest(NullType, LongType, Some(LongType))
widenTest(ShortType, IntegerType, Some(IntegerType))
widenTest(ShortType, LongType, Some(LongType))
widenTest(IntegerType, LongType, Some(LongType))
widenTest(LongType, LongType, Some(LongType))
// Floating point
widenTest(NullType, FloatType, Some(FloatType))
widenTest(NullType, DoubleType, Some(DoubleType))
widenTest(FloatType, DoubleType, Some(DoubleType))
widenTest(FloatType, FloatType, Some(FloatType))
widenTest(DoubleType, DoubleType, Some(DoubleType))
// Integral mixed with floating point.
widenTest(NullType, FloatType, Some(FloatType))
widenTest(NullType, DoubleType, Some(DoubleType))
widenTest(IntegerType, FloatType, Some(FloatType))
widenTest(IntegerType, DoubleType, Some(DoubleType))
widenTest(IntegerType, DoubleType, Some(DoubleType))
widenTest(LongType, FloatType, Some(FloatType))
widenTest(LongType, DoubleType, Some(DoubleType))
}
}

View file

@ -0,0 +1,57 @@
/*
* 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
package catalyst
package trees
import org.scalatest.FunSuite
import expressions._
import rules._
class RuleExecutorSuite extends FunSuite {
object DecrementLiterals extends Rule[Expression] {
def apply(e: Expression): Expression = e transform {
case IntegerLiteral(i) if i > 0 => Literal(i - 1)
}
}
test("only once") {
object ApplyOnce extends RuleExecutor[Expression] {
val batches = Batch("once", Once, DecrementLiterals) :: Nil
}
assert(ApplyOnce(Literal(10)) === Literal(9))
}
test("to fixed point") {
object ToFixedPoint extends RuleExecutor[Expression] {
val batches = Batch("fixedPoint", FixedPoint(100), DecrementLiterals) :: Nil
}
assert(ToFixedPoint(Literal(10)) === Literal(0))
}
test("to maxIterations") {
object ToFixedPoint extends RuleExecutor[Expression] {
val batches = Batch("fixedPoint", FixedPoint(10), DecrementLiterals) :: Nil
}
assert(ToFixedPoint(Literal(100)) === Literal(90))
}
}

View file

@ -0,0 +1,81 @@
/*
* 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
package catalyst
package trees
import scala.collection.mutable.ArrayBuffer
import expressions._
import org.scalatest.{FunSuite}
class TreeNodeSuite extends FunSuite {
test("top node changed") {
val after = Literal(1) transform { case Literal(1, _) => Literal(2) }
assert(after === Literal(2))
}
test("one child changed") {
val before = Add(Literal(1), Literal(2))
val after = before transform { case Literal(2, _) => Literal(1) }
assert(after === Add(Literal(1), Literal(1)))
}
test("no change") {
val before = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4))))
val after = before transform { case Literal(5, _) => Literal(1)}
assert(before === after)
assert(before.map(_.id) === after.map(_.id))
}
test("collect") {
val tree = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4))))
val literals = tree collect {case l: Literal => l}
assert(literals.size === 4)
(1 to 4).foreach(i => assert(literals contains Literal(i)))
}
test("pre-order transform") {
val actual = new ArrayBuffer[String]()
val expected = Seq("+", "1", "*", "2", "-", "3", "4")
val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))
expression transformDown {
case b: BinaryExpression => {actual.append(b.symbol); b}
case l: Literal => {actual.append(l.toString); l}
}
assert(expected === actual)
}
test("post-order transform") {
val actual = new ArrayBuffer[String]()
val expected = Seq("1", "2", "3", "4", "-", "*", "+")
val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))
expression transformUp {
case b: BinaryExpression => {actual.append(b.symbol); b}
case l: Literal => {actual.append(l.toString); l}
}
assert(expected === actual)
}
}

View file

@ -0,0 +1,176 @@
/*
* 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
package catalyst
package optimizer
import types.IntegerType
import util._
import plans.logical.{LogicalPlan, LocalRelation}
import rules._
import expressions._
import dsl.plans._
import dsl.expressions._
class ConstantFoldingSuite extends OptimizerTest {
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
Batch("Subqueries", Once,
EliminateSubqueries) ::
Batch("ConstantFolding", Once,
ConstantFolding,
BooleanSimplification) :: Nil
}
val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
test("eliminate subqueries") {
val originalQuery =
testRelation
.subquery('y)
.select('a)
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.select('a.attr)
.analyze
comparePlans(optimized, correctAnswer)
}
/**
* Unit tests for constant folding in expressions.
*/
test("Constant folding test: expressions only have literals") {
val originalQuery =
testRelation
.select(
Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"),
Literal(2) * Literal(3) + Literal(4) as Symbol("2*3+4"),
Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)"))
.where(
Literal(1) === Literal(1) &&
Literal(2) > Literal(3) ||
Literal(3) > Literal(2) )
.groupBy(
Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2))
)(Literal(9) / Literal(3) as Symbol("9/3"))
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.select(
Literal(9) as Symbol("2+3+4"),
Literal(10) as Symbol("2*3+4"),
Literal(14) as Symbol("2*(3+4)"))
.where(Literal(true))
.groupBy(Literal(3))(Literal(3) as Symbol("9/3"))
.analyze
comparePlans(optimized, correctAnswer)
}
test("Constant folding test: expressions have attribute references and literals in " +
"arithmetic operations") {
val originalQuery =
testRelation
.select(
Literal(2) + Literal(3) + 'a as Symbol("c1"),
'a + Literal(2) + Literal(3) as Symbol("c2"),
Literal(2) * 'a + Literal(4) as Symbol("c3"),
'a * (Literal(3) + Literal(4)) as Symbol("c4"))
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.select(
Literal(5) + 'a as Symbol("c1"),
'a + Literal(2) + Literal(3) as Symbol("c2"),
Literal(2) * 'a + Literal(4) as Symbol("c3"),
'a * (Literal(7)) as Symbol("c4"))
.analyze
comparePlans(optimized, correctAnswer)
}
test("Constant folding test: expressions have attribute references and literals in " +
"predicates") {
val originalQuery =
testRelation
.where(
(('a > 1 && Literal(1) === Literal(1)) ||
('a < 10 && Literal(1) === Literal(2)) ||
(Literal(1) === Literal(1) && 'b > 1) ||
(Literal(1) === Literal(2) && 'b < 10)) &&
(('a > 1 || Literal(1) === Literal(1)) &&
('a < 10 || Literal(1) === Literal(2)) &&
(Literal(1) === Literal(1) || 'b > 1) &&
(Literal(1) === Literal(2) || 'b < 10)))
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.where(('a > 1 || 'b > 1) && ('a < 10 && 'b < 10))
.analyze
comparePlans(optimized, correctAnswer)
}
test("Constant folding test: expressions have foldable functions") {
val originalQuery =
testRelation
.select(
Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"),
Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2"))
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.select(
Literal(5) + 'a as Symbol("c1"),
Literal(3) as Symbol("c2"))
.analyze
comparePlans(optimized, correctAnswer)
}
test("Constant folding test: expressions have nonfoldable functions") {
val originalQuery =
testRelation
.select(
Rand + Literal(1) as Symbol("c1"),
Sum('a) as Symbol("c2"))
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.select(
Rand + Literal(1.0) as Symbol("c1"),
Sum('a) as Symbol("c2"))
.analyze
comparePlans(optimized, correctAnswer)
}
}

View file

@ -0,0 +1,222 @@
package org.apache.spark.sql
package catalyst
package optimizer
import expressions._
import plans.logical._
import rules._
import util._
import dsl.plans._
import dsl.expressions._
class FilterPushdownSuite extends OptimizerTest {
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
Batch("Subqueries", Once,
EliminateSubqueries) ::
Batch("Filter Pushdown", Once,
EliminateSubqueries,
CombineFilters,
PushPredicateThroughProject,
PushPredicateThroughInnerJoin) :: Nil
}
val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
// This test already passes.
test("eliminate subqueries") {
val originalQuery =
testRelation
.subquery('y)
.select('a)
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.select('a.attr)
.analyze
comparePlans(optimized, correctAnswer)
}
// After this line is unimplemented.
test("simple push down") {
val originalQuery =
testRelation
.select('a)
.where('a === 1)
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.where('a === 1)
.select('a)
.analyze
comparePlans(optimized, correctAnswer)
}
test("can't push without rewrite") {
val originalQuery =
testRelation
.select('a + 'b as 'e)
.where('e === 1)
.analyze
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.where('a + 'b === 1)
.select('a + 'b as 'e)
.analyze
comparePlans(optimized, correctAnswer)
}
test("filters: combines filters") {
val originalQuery = testRelation
.select('a)
.where('a === 1)
.where('a === 2)
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
.where('a === 1 && 'a === 2)
.select('a).analyze
comparePlans(optimized, correctAnswer)
}
test("joins: push to either side") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val originalQuery = {
x.join(y)
.where("x.b".attr === 1)
.where("y.b".attr === 2)
}
val optimized = Optimize(originalQuery.analyze)
val left = testRelation.where('b === 1)
val right = testRelation.where('b === 2)
val correctAnswer =
left.join(right).analyze
comparePlans(optimized, correctAnswer)
}
test("joins: push to one side") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val originalQuery = {
x.join(y)
.where("x.b".attr === 1)
}
val optimized = Optimize(originalQuery.analyze)
val left = testRelation.where('b === 1)
val right = testRelation
val correctAnswer =
left.join(right).analyze
comparePlans(optimized, correctAnswer)
}
test("joins: rewrite filter to push to either side") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val originalQuery = {
x.join(y)
.where("x.b".attr === 1 && "y.b".attr === 2)
}
val optimized = Optimize(originalQuery.analyze)
val left = testRelation.where('b === 1)
val right = testRelation.where('b === 2)
val correctAnswer =
left.join(right).analyze
comparePlans(optimized, correctAnswer)
}
test("joins: can't push down") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val originalQuery = {
x.join(y, condition = Some("x.b".attr === "y.b".attr))
}
val optimized = Optimize(originalQuery.analyze)
comparePlans(optimizer.EliminateSubqueries(originalQuery.analyze), optimized)
}
test("joins: conjunctive predicates") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val originalQuery = {
x.join(y)
.where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("y.a".attr === 1))
}
val optimized = Optimize(originalQuery.analyze)
val left = testRelation.where('a === 1).subquery('x)
val right = testRelation.where('a === 1).subquery('y)
val correctAnswer =
left.join(right, condition = Some("x.b".attr === "y.b".attr))
.analyze
comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer))
}
test("joins: conjunctive predicates #2") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val originalQuery = {
x.join(y)
.where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1))
}
val optimized = Optimize(originalQuery.analyze)
val left = testRelation.where('a === 1).subquery('x)
val right = testRelation.subquery('y)
val correctAnswer =
left.join(right, condition = Some("x.b".attr === "y.b".attr))
.analyze
comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer))
}
test("joins: conjunctive predicates #3") {
val x = testRelation.subquery('x)
val y = testRelation.subquery('y)
val z = testRelation.subquery('z)
val originalQuery = {
z.join(x.join(y))
.where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr))
}
val optimized = Optimize(originalQuery.analyze)
val lleft = testRelation.where('a >= 3).subquery('z)
val left = testRelation.where('a === 1).subquery('x)
val right = testRelation.subquery('y)
val correctAnswer =
lleft.join(
left.join(right, condition = Some("x.b".attr === "y.b".attr)),
condition = Some("z.a".attr === "x.b".attr))
.analyze
comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer))
}
}

View file

@ -0,0 +1,44 @@
package org.apache.spark.sql
package catalyst
package optimizer
import org.scalatest.FunSuite
import types.IntegerType
import util._
import plans.logical.{LogicalPlan, LocalRelation}
import expressions._
import dsl._
/* Implicit conversions for creating query plans */
/**
* Provides helper methods for comparing plans produced by optimization rules with the expected
* result
*/
class OptimizerTest extends FunSuite {
/**
* Since attribute references are given globally unique ids during analysis,
* we must normalize them to check if two different queries are identical.
*/
protected def normalizeExprIds(plan: LogicalPlan) = {
val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min
plan transformAllExpressions {
case a: AttributeReference =>
AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId))
}
}
/** Fails the test if the two plans do not match */
protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) {
val normalized1 = normalizeExprIds(plan1)
val normalized2 = normalizeExprIds(plan2)
if (normalized1 != normalized2)
fail(
s"""
|== FAIL: Plans do not match ===
|${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")}
""".stripMargin)
}
}

76
sql/core/pom.xml Normal file
View file

@ -0,0 +1,76 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
<version>1.0.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project SQL</name>
<url>http://spark.apache.org/</url>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-catalyst_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-column</artifactId>
<version>${parquet.version}</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>${parquet.version}</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
<artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>

View file

@ -0,0 +1,99 @@
/*
* 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.rdd
import scala.language.implicitConversions
import scala.reflect._
import scala.collection.mutable.ArrayBuffer
import org.apache.spark._
import org.apache.spark.Aggregator
import org.apache.spark.SparkContext._
import org.apache.spark.util.collection.AppendOnlyMap
/**
* Extra functions on RDDs that perform only local operations. These can be used when data has
* already been partitioned correctly.
*/
private[spark] class PartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
extends Logging
with Serializable {
/**
* Cogroup corresponding partitions of `this` and `other`. These two RDDs should have
* the same number of partitions. Partitions of these two RDDs are cogrouped
* according to the indexes of partitions. If we have two RDDs and
* each of them has n partitions, we will cogroup the partition i from `this`
* with the partition i from `other`.
* This function will not introduce a shuffling operation.
*/
def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = {
val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => {
val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]]
val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => {
if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any])
}
val getSeq = (k: K) => {
map.changeValue(k, update)
}
iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 }
iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 }
map.iterator
}).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])}
cg
}
/**
* Group the values for each key within a partition of the RDD into a single sequence.
* This function will not introduce a shuffling operation.
*/
def groupByKeyLocally(): RDD[(K, Seq[V])] = {
def createCombiner(v: V) = ArrayBuffer(v)
def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v
val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner, mergeValue, _ ++ _)
val bufs = self.mapPartitionsWithContext((context, iter) => {
new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context))
}, preservesPartitioning = true)
bufs.asInstanceOf[RDD[(K, Seq[V])]]
}
/**
* Join corresponding partitions of `this` and `other`.
* If we have two RDDs and each of them has n partitions,
* we will join the partition i from `this` with the partition i from `other`.
* This function will not introduce a shuffling operation.
*/
def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = {
cogroupLocally(other).flatMapValues {
case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
}
}
}
private[spark] object PartitionLocalRDDFunctions {
implicit def rddToPartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
new PartitionLocalRDDFunctions(rdd)
}

View file

@ -0,0 +1,174 @@
/*
* 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
import scala.language.implicitConversions
import scala.reflect.runtime.universe.TypeTag
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.dsl
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.optimizer.Optimizer
import org.apache.spark.sql.catalyst.planning.QueryPlanner
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand, WriteToFile}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution._
/**
* <span class="badge" style="float: right; background-color: darkblue;">ALPHA COMPONENT</span>
*
* The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]]
* objects and the execution of SQL queries.
*
* @groupname userf Spark SQL Functions
* @groupname Ungrouped Support functions for language integrated queries.
*/
class SQLContext(@transient val sparkContext: SparkContext)
extends Logging
with dsl.ExpressionConversions
with Serializable {
self =>
@transient
protected[sql] lazy val catalog: Catalog = new SimpleCatalog
@transient
protected[sql] lazy val analyzer: Analyzer =
new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true)
@transient
protected[sql] val optimizer = Optimizer
@transient
protected[sql] val parser = new catalyst.SqlParser
protected[sql] def parseSql(sql: String): LogicalPlan = parser(sql)
protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql))
protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution =
new this.QueryExecution { val logical = plan }
/**
* <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span>
*
* Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan
* interface is considered internal, and thus not guranteed to be stable. As a result, using
* them directly is not reccomended.
*/
implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan)
/**
* Creates a SchemaRDD from an RDD of case classes.
*
* @group userf
*/
implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) =
new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd)))
/**
* Loads a parequet file, returning the result as a [[SchemaRDD]].
*
* @group userf
*/
def parquetFile(path: String): SchemaRDD =
new SchemaRDD(this, parquet.ParquetRelation("ParquetFile", path))
/**
* Registers the given RDD as a temporary table in the catalog. Temporary tables exist only
* during the lifetime of this instance of SQLContext.
*
* @group userf
*/
def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = {
catalog.registerTable(None, tableName, rdd.logicalPlan)
}
/**
* Executes a SQL query using Spark, returning the result as a SchemaRDD.
*
* @group userf
*/
def sql(sqlText: String): SchemaRDD = {
val result = new SchemaRDD(this, parseSql(sqlText))
// We force query optimization to happen right away instead of letting it happen lazily like
// when using the query DSL. This is so DDL commands behave as expected. This is only
// generates the RDD lineage for DML queries, but do not perform any execution.
result.queryExecution.toRdd
result
}
protected[sql] class SparkPlanner extends SparkStrategies {
val sparkContext = self.sparkContext
val strategies: Seq[Strategy] =
TopK ::
PartialAggregation ::
SparkEquiInnerJoin ::
BasicOperators ::
CartesianProduct ::
BroadcastNestedLoopJoin :: Nil
}
@transient
protected[sql] val planner = new SparkPlanner
/**
* Prepares a planned SparkPlan for execution by binding references to specific ordinals, and
* inserting shuffle operations as needed.
*/
@transient
protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] {
val batches =
Batch("Add exchange", Once, AddExchange) ::
Batch("Prepare Expressions", Once, new BindReferences[SparkPlan]) :: Nil
}
/**
* The primary workflow for executing relational queries using Spark. Designed to allow easy
* access to the intermediate phases of query execution for developers.
*/
protected abstract class QueryExecution {
def logical: LogicalPlan
lazy val analyzed = analyzer(logical)
lazy val optimizedPlan = optimizer(analyzed)
// TODO: Don't just pick the first one...
lazy val sparkPlan = planner(optimizedPlan).next()
lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan)
/** Internal version of the RDD. Avoids copies and has no schema */
lazy val toRdd: RDD[Row] = executedPlan.execute()
protected def stringOrError[A](f: => A): String =
try f.toString catch { case e: Throwable => e.toString }
override def toString: String =
s"""== Logical Plan ==
|${stringOrError(analyzed)}
|== Optimized Logical Plan
|${stringOrError(optimizedPlan)}
|== Physical Plan ==
|${stringOrError(executedPlan)}
""".stripMargin.trim
/**
* Runs the query after interposing operators that print the result of each intermediate step.
*/
def debugExec() = DebugQuery(executedPlan).execute().collect()
}
}

View file

@ -0,0 +1,342 @@
/*
* 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
import org.apache.spark.{OneToOneDependency, Dependency, Partition, TaskContext}
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.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.types.BooleanType
/**
* <span class="badge" style="float: right; background-color: darkblue;">ALPHA COMPONENT</span>
*
* An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions,
* SchemaRDDs can be used in relational queries, as shown in the examples below.
*
* Importing a SQLContext brings an implicit into scope that automatically converts a standard RDD
* whose elements are scala case classes into a SchemaRDD. This conversion can also be done
* explicitly using the `createSchemaRDD` function on a [[SQLContext]].
*
* A `SchemaRDD` can also be created by loading data in from external sources, for example,
* by using the `parquetFile` method on [[SQLContext]].
*
* == SQL Queries ==
* A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it. Once
* an RDD has been registered as a table, it can be used in the FROM clause of SQL statements.
*
* {{{
* // One method for defining the schema of an RDD is to make a case class with the desired column
* // names and types.
* case class Record(key: Int, value: String)
*
* val sc: SparkContext // An existing spark context.
* val sqlContext = new SQLContext(sc)
*
* // Importing the SQL context gives access to all the SQL functions and implicit conversions.
* import sqlContext._
*
* val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_\$i")))
* // Any RDD containing case classes can be registered as a table. The schema of the table is
* // automatically inferred using scala reflection.
* rdd.registerAsTable("records")
*
* val results: SchemaRDD = sql("SELECT * FROM records")
* }}}
*
* == Language Integrated Queries ==
*
* {{{
*
* case class Record(key: Int, value: String)
*
* val sc: SparkContext // An existing spark context.
* val sqlContext = new SQLContext(sc)
*
* // Importing the SQL context gives access to all the SQL functions and implicit conversions.
* import sqlContext._
*
* val rdd = sc.parallelize((1 to 100).map(i => Record(i, "val_" + i)))
*
* // Example of language integrated queries.
* rdd.where('key === 1).orderBy('value.asc).select('key).collect()
* }}}
*
* @todo There is currently no support for creating SchemaRDDs from either Java or Python RDDs.
*
* @groupname Query Language Integrated Queries
* @groupdesc Query Functions that create new queries from SchemaRDDs. The
* result of all query functions is also a SchemaRDD, allowing multiple operations to be
* chained using a builder pattern.
* @groupprio Query -2
* @groupname schema SchemaRDD Functions
* @groupprio schema -1
* @groupname Ungrouped Base RDD Functions
*/
class SchemaRDD(
@transient val sqlContext: SQLContext,
@transient val logicalPlan: LogicalPlan)
extends RDD[Row](sqlContext.sparkContext, Nil) {
/**
* A lazily computed query execution workflow. All other RDD operations are passed
* through to the RDD that is produced by this workflow.
*
* We want this to be lazy because invoking the whole query optimization pipeline can be
* expensive.
*/
@transient
protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan)
override def toString =
s"""${super.toString}
|== Query Plan ==
|${queryExecution.executedPlan}""".stripMargin.trim
// =========================================================================================
// RDD functions: Copy the interal row representation so we present immutable data to users.
// =========================================================================================
override def compute(split: Partition, context: TaskContext): Iterator[Row] =
firstParent[Row].compute(split, context).map(_.copy())
override def getPartitions: Array[Partition] = firstParent[Row].partitions
override protected def getDependencies: Seq[Dependency[_]] =
List(new OneToOneDependency(queryExecution.toRdd))
// =======================================================================
// Query DSL
// =======================================================================
/**
* Changes the output of this relation to the given expressions, similar to the `SELECT` clause
* in SQL.
*
* {{{
* schemaRDD.select('a, 'b + 'c, 'd as 'aliasedName)
* }}}
*
* @param exprs a set of logical expression that will be evaluated for each input row.
*
* @group Query
*/
def select(exprs: NamedExpression*): SchemaRDD =
new SchemaRDD(sqlContext, Project(exprs, logicalPlan))
/**
* Filters the ouput, only returning those rows where `condition` evaluates to true.
*
* {{{
* schemaRDD.where('a === 'b)
* schemaRDD.where('a === 1)
* schemaRDD.where('a + 'b > 10)
* }}}
*
* @group Query
*/
def where(condition: Expression): SchemaRDD =
new SchemaRDD(sqlContext, Filter(condition, logicalPlan))
/**
* Performs a relational join on two SchemaRDDs
*
* @param otherPlan the [[SchemaRDD]] that should be joined with this one.
* @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.`
* @param condition An optional condition for the join operation. This is equivilent to the `ON`
* clause in standard SQL. In the case of `Inner` joins, specifying a
* `condition` is equivilent to adding `where` clauses after the `join`.
*
* @group Query
*/
def join(
otherPlan: SchemaRDD,
joinType: JoinType = Inner,
condition: Option[Expression] = None): SchemaRDD =
new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, condition))
/**
* Sorts the results by the given expressions.
* {{{
* schemaRDD.orderBy('a)
* schemaRDD.orderBy('a, 'b)
* schemaRDD.orderBy('a.asc, 'b.desc)
* }}}
*
* @group Query
*/
def orderBy(sortExprs: SortOrder*): SchemaRDD =
new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan))
/**
* Performs a grouping followed by an aggregation.
*
* {{{
* schemaRDD.groupBy('year)(Sum('sales) as 'totalSales)
* }}}
*
* @group Query
*/
def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*): SchemaRDD = {
val aliasedExprs = aggregateExprs.map {
case ne: NamedExpression => ne
case e => Alias(e, e.toString)()
}
new SchemaRDD(sqlContext, Aggregate(groupingExprs, aliasedExprs, logicalPlan))
}
/**
* Applies a qualifier to the attributes of this relation. Can be used to disambiguate attributes
* with the same name, for example, when peforming self-joins.
*
* {{{
* val x = schemaRDD.where('a === 1).subquery('x)
* val y = schemaRDD.where('a === 2).subquery('y)
* x.join(y).where("x.a".attr === "y.a".attr),
* }}}
*
* @group Query
*/
def subquery(alias: Symbol) =
new SchemaRDD(sqlContext, Subquery(alias.name, logicalPlan))
/**
* Combines the tuples of two RDDs with the same schema, keeping duplicates.
*
* @group Query
*/
def unionAll(otherPlan: SchemaRDD) =
new SchemaRDD(sqlContext, Union(logicalPlan, otherPlan.logicalPlan))
/**
* Filters tuples using a function over the value of the specified column.
*
* {{{
* schemaRDD.sfilter('a)((a: Int) => ...)
* }}}
*
* @group Query
*/
def where[T1](arg1: Symbol)(udf: (T1) => Boolean) =
new SchemaRDD(
sqlContext,
Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan))
/**
* <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span>
*
* Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use
* scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of
* the column is not known at compile time, all attributes are converted to strings before
* being passed to the function.
*
* {{{
* schemaRDD.where(r => r.firstName == "Bob" && r.lastName == "Smith")
* }}}
*
* @group Query
*/
def where(dynamicUdf: (DynamicRow) => Boolean) =
new SchemaRDD(
sqlContext,
Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan))
/**
* <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span>
*
* Returns a sampled version of the underlying dataset.
*
* @group Query
*/
def sample(
fraction: Double,
withReplacement: Boolean = true,
seed: Int = (math.random * 1000).toInt) =
new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan))
/**
* <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span>
*
* Applies the given Generator, or table generating function, to this relation.
*
* @param generator A table generating function. The API for such functions is likely to change
* in future releases
* @param join when set to true, each output row of the generator is joined with the input row
* that produced it.
* @param outer when set to true, at least one row will be produced for each input row, similar to
* an `OUTER JOIN` in SQL. When no output rows are produced by the generator for a
* given row, a single row will be output, with `NULL` values for each of the
* generated columns.
* @param alias an optional alias that can be used as qualif for the attributes that are produced
* by this generate operation.
*
* @group Query
*/
def generate(
generator: Generator,
join: Boolean = false,
outer: Boolean = false,
alias: Option[String] = None) =
new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan))
/**
* <span class="badge badge-red" style="float: right;">EXPERIMENTAL</span>
*
* Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is
* no notion of persistent tables, and thus queries that contain this operator will fail to
* optimize. When working with an extension of a SQLContext that has a persistent catalog, such
* as a `HiveContext`, this operation will result in insertions to the table specified.
*
* @group schema
*/
def insertInto(tableName: String, overwrite: Boolean = false) =
new SchemaRDD(
sqlContext,
InsertIntoTable(UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite))
/**
* Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that
* are written out using this method can be read back in as a SchemaRDD using the ``function
*
* @group schema
*/
def saveAsParquetFile(path: String): Unit = {
sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
}
/**
* Registers this RDD as a temporary table using the given name. The lifetime of this temporary
* table is tied to the [[SQLContext]] that was used to create this SchemaRDD.
*
* @group schema
*/
def registerAsTable(tableName: String): Unit = {
sqlContext.registerRDDAsTable(this, tableName)
}
/**
* Returns this RDD as a SchemaRDD.
* @group schema
*/
def toSchemaRDD = this
def analyze = sqlContext.analyzer(logicalPlan)
}

View file

@ -0,0 +1,170 @@
/*
* 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
package execution
import java.nio.ByteBuffer
import com.esotericsoftware.kryo.{Kryo, Serializer}
import com.esotericsoftware.kryo.io.{Output, Input}
import org.apache.spark.{SparkConf, RangePartitioner, HashPartitioner}
import org.apache.spark.rdd.ShuffledRDD
import org.apache.spark.serializer.KryoSerializer
import org.apache.spark.util.MutablePair
import catalyst.rules.Rule
import catalyst.errors._
import catalyst.expressions._
import catalyst.plans.physical._
private class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
override def newKryo(): Kryo = {
val kryo = new Kryo
kryo.setRegistrationRequired(true)
kryo.register(classOf[MutablePair[_,_]])
kryo.register(classOf[Array[Any]])
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow])
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow])
kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]])
kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer)
kryo.setReferences(false)
kryo.setClassLoader(this.getClass.getClassLoader)
kryo
}
}
private class BigDecimalSerializer extends Serializer[BigDecimal] {
def write(kryo: Kryo, output: Output, bd: math.BigDecimal) {
// TODO: There are probably more efficient representations than strings...
output.writeString(bd.toString)
}
def read(kryo: Kryo, input: Input, tpe: Class[BigDecimal]): BigDecimal = {
BigDecimal(input.readString())
}
}
case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode {
override def outputPartitioning = newPartitioning
def output = child.output
def execute() = attachTree(this , "execute") {
newPartitioning match {
case HashPartitioning(expressions, numPartitions) => {
// TODO: Eliminate redundant expressions in grouping key and value.
val rdd = child.execute().mapPartitions { iter =>
val hashExpressions = new MutableProjection(expressions)
val mutablePair = new MutablePair[Row, Row]()
iter.map(r => mutablePair.update(hashExpressions(r), r))
}
val part = new HashPartitioner(numPartitions)
val shuffled = new ShuffledRDD[Row, Row, MutablePair[Row, Row]](rdd, part)
shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false)))
shuffled.map(_._2)
}
case RangePartitioning(sortingExpressions, numPartitions) => {
// TODO: RangePartitioner should take an Ordering.
implicit val ordering = new RowOrdering(sortingExpressions)
val rdd = child.execute().mapPartitions { iter =>
val mutablePair = new MutablePair[Row, Null](null, null)
iter.map(row => mutablePair.update(row, null))
}
val part = new RangePartitioner(numPartitions, rdd, ascending = true)
val shuffled = new ShuffledRDD[Row, Null, MutablePair[Row, Null]](rdd, part)
shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false)))
shuffled.map(_._1)
}
case SinglePartition =>
child.execute().coalesce(1, true)
case _ => sys.error(s"Exchange not implemented for $newPartitioning")
// TODO: Handle BroadcastPartitioning.
}
}
}
/**
* Ensures that the [[catalyst.plans.physical.Partitioning Partitioning]] of input data meets the
* [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting
* [[Exchange]] Operators where required.
*/
object AddExchange extends Rule[SparkPlan] {
// TODO: Determine the number of partitions.
val numPartitions = 8
def apply(plan: SparkPlan): SparkPlan = plan.transformUp {
case operator: SparkPlan =>
// Check if every child's outputPartitioning satisfies the corresponding
// required data distribution.
def meetsRequirements =
!operator.requiredChildDistribution.zip(operator.children).map {
case (required, child) =>
val valid = child.outputPartitioning.satisfies(required)
logger.debug(
s"${if (valid) "Valid" else "Invalid"} distribution," +
s"required: $required current: ${child.outputPartitioning}")
valid
}.exists(!_)
// Check if outputPartitionings of children are compatible with each other.
// It is possible that every child satisfies its required data distribution
// but two children have incompatible outputPartitionings. For example,
// A dataset is range partitioned by "a.asc" (RangePartitioning) and another
// dataset is hash partitioned by "a" (HashPartitioning). Tuples in these two
// datasets are both clustered by "a", but these two outputPartitionings are not
// compatible.
// TODO: ASSUMES TRANSITIVITY?
def compatible =
!operator.children
.map(_.outputPartitioning)
.sliding(2)
.map {
case Seq(a) => true
case Seq(a,b) => a compatibleWith b
}.exists(!_)
// Check if the partitioning we want to ensure is the same as the child's output
// partitioning. If so, we do not need to add the Exchange operator.
def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan) =
if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child
if (meetsRequirements && compatible) {
operator
} else {
// At least one child does not satisfies its required data distribution or
// at least one child's outputPartitioning is not compatible with another child's
// outputPartitioning. In this case, we need to add Exchange operators.
val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map {
case (AllTuples, child) =>
addExchangeIfNecessary(SinglePartition, child)
case (ClusteredDistribution(clustering), child) =>
addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child)
case (OrderedDistribution(ordering), child) =>
addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child)
case (UnspecifiedDistribution, child) => child
case (dist, _) => sys.error(s"Don't know how to ensure $dist")
}
operator.withNewChildren(repartitionedChildren)
}
}
}

View file

@ -0,0 +1,69 @@
/*
* 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
package execution
import catalyst.expressions._
import catalyst.types._
/**
* Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the
* output of each into a new stream of rows. This operation is similar to a `flatMap` in functional
* programming with one important additional feature, which allows the input rows to be joined with
* their output.
* @param join when true, each output row is implicitly joined with the input tuple that produced
* it.
* @param outer when true, each input row will be output at least once, even if the output of the
* given `generator` is empty. `outer` has no effect when `join` is false.
*/
case class Generate(
generator: Generator,
join: Boolean,
outer: Boolean,
child: SparkPlan)
extends UnaryNode {
def output =
if (join) child.output ++ generator.output else generator.output
def execute() = {
if (join) {
child.execute().mapPartitions { iter =>
val nullValues = Seq.fill(generator.output.size)(Literal(null))
// Used to produce rows with no matches when outer = true.
val outerProjection =
new Projection(child.output ++ nullValues, child.output)
val joinProjection =
new Projection(child.output ++ generator.output, child.output ++ generator.output)
val joinedRow = new JoinedRow
iter.flatMap {row =>
val outputRows = generator(row)
if (outer && outputRows.isEmpty) {
outerProjection(row) :: Nil
} else {
outputRows.map(or => joinProjection(joinedRow(row, or)))
}
}
}
} else {
child.execute().mapPartitions(iter => iter.flatMap(generator))
}
}
}

View file

@ -0,0 +1,21 @@
/*
* 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
package execution
class QueryExecutionException(message: String) extends Exception(message)

View file

@ -0,0 +1,89 @@
/*
* 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
package execution
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.trees
abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
self: Product =>
// TODO: Move to `DistributedPlan`
/** Specifies how data is partitioned across different nodes in the cluster. */
def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH!
/** Specifies any partition requirements on the input data for this operator. */
def requiredChildDistribution: Seq[Distribution] =
Seq.fill(children.size)(UnspecifiedDistribution)
/**
* Runs this query returning the result as an RDD.
*/
def execute(): RDD[Row]
/**
* Runs this query returning the result as an array.
*/
def executeCollect(): Array[Row] = execute().collect()
protected def buildRow(values: Seq[Any]): Row =
new catalyst.expressions.GenericRow(values.toArray)
}
/**
* Allows already planned SparkQueries to be linked into logical query plans.
*
* Note that in general it is not valid to use this class to link multiple copies of the same
* physical operator into the same query plan as this violates the uniqueness of expression ids.
* Special handling exists for ExistingRdd as these are already leaf operators and thus we can just
* replace the output attributes with new copies of themselves without breaking any attribute
* linking.
*/
case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
extends logical.LogicalPlan with MultiInstanceRelation {
def output = alreadyPlanned.output
def references = Set.empty
def children = Nil
override final def newInstance: this.type = {
SparkLogicalPlan(
alreadyPlanned match {
case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd)
case _ => sys.error("Multiple instance of the same relation detected.")
}).asInstanceOf[this.type]
}
}
trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] {
self: Product =>
}
trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] {
self: Product =>
override def outputPartitioning: Partitioning = child.outputPartitioning
}
trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] {
self: Product =>
}

View file

@ -0,0 +1,229 @@
/*
* 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
package execution
import org.apache.spark.SparkContext
import catalyst.expressions._
import catalyst.planning._
import catalyst.plans._
import catalyst.plans.logical.LogicalPlan
import catalyst.plans.physical._
import parquet.ParquetRelation
import parquet.InsertIntoParquetTable
abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
val sparkContext: SparkContext
object SparkEquiInnerJoin extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) =>
logger.debug(s"Considering join: ${predicates ++ condition}")
// Find equi-join predicates that can be evaluated before the join, and thus can be used
// as join keys. Note we can only mix in the conditions with other predicates because the
// match above ensures that this is and Inner join.
val (joinPredicates, otherPredicates) = (predicates ++ condition).partition {
case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) ||
(canEvaluate(l, right) && canEvaluate(r, left)) => true
case _ => false
}
val joinKeys = joinPredicates.map {
case Equals(l,r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r)
case Equals(l,r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l)
}
// Do not consider this strategy if there are no join keys.
if (joinKeys.nonEmpty) {
val leftKeys = joinKeys.map(_._1)
val rightKeys = joinKeys.map(_._2)
val joinOp = execution.SparkEquiInnerJoin(
leftKeys, rightKeys, planLater(left), planLater(right))
// Make sure other conditions are met if present.
if (otherPredicates.nonEmpty) {
execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil
} else {
joinOp :: Nil
}
} else {
logger.debug(s"Avoiding spark join with no join keys.")
Nil
}
case _ => Nil
}
private def combineConjunctivePredicates(predicates: Seq[Expression]) =
predicates.reduceLeft(And)
/** Returns true if `expr` can be evaluated using only the output of `plan`. */
protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean =
expr.references subsetOf plan.outputSet
}
object PartialAggregation extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.Aggregate(groupingExpressions, aggregateExpressions, child) =>
// Collect all aggregate expressions.
val allAggregates =
aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a})
// Collect all aggregate expressions that can be computed partially.
val partialAggregates =
aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p})
// Only do partial aggregation if supported by all aggregate expressions.
if (allAggregates.size == partialAggregates.size) {
// Create a map of expressions to their partial evaluations for all aggregate expressions.
val partialEvaluations: Map[Long, SplitEvaluation] =
partialAggregates.map(a => (a.id, a.asPartial)).toMap
// We need to pass all grouping expressions though so the grouping can happen a second
// time. However some of them might be unnamed so we alias them allowing them to be
// referenced in the second aggregation.
val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map {
case n: NamedExpression => (n, n)
case other => (other, Alias(other, "PartialGroup")())
}.toMap
// Replace aggregations with a new expression that computes the result from the already
// computed partial evaluations and grouping values.
val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp {
case e: Expression if partialEvaluations.contains(e.id) =>
partialEvaluations(e.id).finalEvaluation
case e: Expression if namedGroupingExpressions.contains(e) =>
namedGroupingExpressions(e).toAttribute
}).asInstanceOf[Seq[NamedExpression]]
val partialComputation =
(namedGroupingExpressions.values ++
partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq
// Construct two phased aggregation.
execution.Aggregate(
partial = false,
namedGroupingExpressions.values.map(_.toAttribute).toSeq,
rewrittenAggregateExpressions,
execution.Aggregate(
partial = true,
groupingExpressions,
partialComputation,
planLater(child))(sparkContext))(sparkContext) :: Nil
} else {
Nil
}
case _ => Nil
}
}
object BroadcastNestedLoopJoin extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.Join(left, right, joinType, condition) =>
execution.BroadcastNestedLoopJoin(
planLater(left), planLater(right), joinType, condition)(sparkContext) :: Nil
case _ => Nil
}
}
object CartesianProduct extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.Join(left, right, _, None) =>
execution.CartesianProduct(planLater(left), planLater(right)) :: Nil
case logical.Join(left, right, Inner, Some(condition)) =>
execution.Filter(condition,
execution.CartesianProduct(planLater(left), planLater(right))) :: Nil
case _ => Nil
}
}
protected lazy val singleRowRdd =
sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1)
def convertToCatalyst(a: Any): Any = a match {
case s: Seq[Any] => s.map(convertToCatalyst)
case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray)
case other => other
}
object TopK extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.StopAfter(IntegerLiteral(limit), logical.Sort(order, child)) =>
execution.TopK(limit, order, planLater(child))(sparkContext) :: Nil
case _ => Nil
}
}
// Can we automate these 'pass through' operations?
object BasicOperators extends Strategy {
// TOOD: Set
val numPartitions = 200
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.Distinct(child) =>
execution.Aggregate(
partial = false, child.output, child.output, planLater(child))(sparkContext) :: Nil
case logical.Sort(sortExprs, child) =>
// This sort is a global sort. Its requiredDistribution will be an OrderedDistribution.
execution.Sort(sortExprs, global = true, planLater(child)):: Nil
case logical.SortPartitions(sortExprs, child) =>
// This sort only sorts tuples within a partition. Its requiredDistribution will be
// an UnspecifiedDistribution.
execution.Sort(sortExprs, global = false, planLater(child)) :: Nil
case logical.Project(projectList, r: ParquetRelation)
if projectList.forall(_.isInstanceOf[Attribute]) =>
// simple projection of data loaded from Parquet file
parquet.ParquetTableScan(
projectList.asInstanceOf[Seq[Attribute]],
r,
None)(sparkContext) :: Nil
case logical.Project(projectList, child) =>
execution.Project(projectList, planLater(child)) :: Nil
case logical.Filter(condition, child) =>
execution.Filter(condition, planLater(child)) :: Nil
case logical.Aggregate(group, agg, child) =>
execution.Aggregate(partial = false, group, agg, planLater(child))(sparkContext) :: Nil
case logical.Sample(fraction, withReplacement, seed, child) =>
execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil
case logical.LocalRelation(output, data) =>
val dataAsRdd =
sparkContext.parallelize(data.map(r =>
new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row))
execution.ExistingRdd(output, dataAsRdd) :: Nil
case logical.StopAfter(IntegerLiteral(limit), child) =>
execution.StopAfter(limit, planLater(child))(sparkContext) :: Nil
case Unions(unionChildren) =>
execution.Union(unionChildren.map(planLater))(sparkContext) :: Nil
case logical.Generate(generator, join, outer, _, child) =>
execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil
case logical.NoRelation =>
execution.ExistingRdd(Nil, singleRowRdd) :: Nil
case logical.Repartition(expressions, child) =>
execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil
case logical.WriteToFile(path, child) =>
val relation =
ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None)
InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil
case p: parquet.ParquetRelation =>
parquet.ParquetTableScan(p.output, p, None)(sparkContext) :: Nil
case SparkLogicalPlan(existingPlan) => existingPlan :: Nil
case _ => Nil
}
}
}

View file

@ -0,0 +1,117 @@
/*
* 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
package execution
import org.apache.spark.SparkContext
import catalyst.errors._
import catalyst.expressions._
import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples}
import catalyst.types._
import org.apache.spark.rdd.PartitionLocalRDDFunctions._
/**
* Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each
* group.
*
* @param partial if true then aggregation is done partially on local data without shuffling to
* ensure all values where `groupingExpressions` are equal are present.
* @param groupingExpressions expressions that are evaluated to determine grouping.
* @param aggregateExpressions expressions that are computed for each group.
* @param child the input data source.
*/
case class Aggregate(
partial: Boolean,
groupingExpressions: Seq[Expression],
aggregateExpressions: Seq[NamedExpression],
child: SparkPlan)(@transient sc: SparkContext)
extends UnaryNode {
override def requiredChildDistribution =
if (partial) {
UnspecifiedDistribution :: Nil
} else {
if (groupingExpressions == Nil) {
AllTuples :: Nil
} else {
ClusteredDistribution(groupingExpressions) :: Nil
}
}
override def otherCopyArgs = sc :: Nil
def output = aggregateExpressions.map(_.toAttribute)
/* Replace all aggregate expressions with spark functions that will compute the result. */
def createAggregateImplementations() = aggregateExpressions.map { agg =>
val impl = agg transform {
case a: AggregateExpression => a.newInstance
}
val remainingAttributes = impl.collect { case a: Attribute => a }
// If any references exist that are not inside agg functions then the must be grouping exprs
// in this case we must rebind them to the grouping tuple.
if (remainingAttributes.nonEmpty) {
val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c }
// An exact match with a grouping expression
val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match {
case -1 => None
case ordinal => Some(BoundReference(ordinal, Alias(impl, "AGGEXPR")().toAttribute))
}
exactGroupingExpr.getOrElse(
sys.error(s"$agg is not in grouping expressions: $groupingExpressions"))
} else {
impl
}
}
def execute() = attachTree(this, "execute") {
// TODO: If the child of it is an [[catalyst.execution.Exchange]],
// do not evaluate the groupingExpressions again since we have evaluated it
// in the [[catalyst.execution.Exchange]].
val grouped = child.execute().mapPartitions { iter =>
val buildGrouping = new Projection(groupingExpressions)
iter.map(row => (buildGrouping(row), row.copy()))
}.groupByKeyLocally()
val result = grouped.map { case (group, rows) =>
val aggImplementations = createAggregateImplementations()
// Pull out all the functions so we can feed each row into them.
val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f })
rows.foreach { row =>
aggFunctions.foreach(_.update(row))
}
buildRow(aggImplementations.map(_.apply(group)))
}
// TODO: THIS BREAKS PIPELINING, DOUBLE COMPUTES THE ANSWER, AND USES TOO MUCH MEMORY...
if (groupingExpressions.isEmpty && result.count == 0) {
// When there there is no output to the Aggregate operator, we still output an empty row.
val aggImplementations = createAggregateImplementations()
sc.makeRDD(buildRow(aggImplementations.map(_.apply(null))) :: Nil)
} else {
result
}
}
}

View file

@ -0,0 +1,137 @@
/*
* 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
package execution
import scala.reflect.runtime.universe.TypeTag
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext
import catalyst.errors._
import catalyst.expressions._
import catalyst.plans.physical.{UnspecifiedDistribution, OrderedDistribution}
import catalyst.plans.logical.LogicalPlan
import catalyst.ScalaReflection
case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {
def output = projectList.map(_.toAttribute)
def execute() = child.execute().mapPartitions { iter =>
@transient val resuableProjection = new MutableProjection(projectList)
iter.map(resuableProjection)
}
}
case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
def output = child.output
def execute() = child.execute().mapPartitions { iter =>
iter.filter(condition.apply(_).asInstanceOf[Boolean])
}
}
case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan)
extends UnaryNode {
def output = child.output
// TODO: How to pick seed?
def execute() = child.execute().sample(withReplacement, fraction, seed)
}
case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan {
// TODO: attributes output by union should be distinct for nullability purposes
def output = children.head.output
def execute() = sc.union(children.map(_.execute()))
override def otherCopyArgs = sc :: Nil
}
case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode {
override def otherCopyArgs = sc :: Nil
def output = child.output
override def executeCollect() = child.execute().map(_.copy()).take(limit)
// TODO: Terminal split should be implemented differently from non-terminal split.
// TODO: Pick num splits based on |limit|.
def execute() = sc.makeRDD(executeCollect(), 1)
}
case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
(@transient sc: SparkContext) extends UnaryNode {
override def otherCopyArgs = sc :: Nil
def output = child.output
@transient
lazy val ordering = new RowOrdering(sortOrder)
override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ordering)
// TODO: Terminal split should be implemented differently from non-terminal split.
// TODO: Pick num splits based on |limit|.
def execute() = sc.makeRDD(executeCollect(), 1)
}
case class Sort(
sortOrder: Seq[SortOrder],
global: Boolean,
child: SparkPlan)
extends UnaryNode {
override def requiredChildDistribution =
if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
@transient
lazy val ordering = new RowOrdering(sortOrder)
def execute() = attachTree(this, "sort") {
// TODO: Optimize sorting operation?
child.execute()
.mapPartitions(
iterator => iterator.map(_.copy()).toArray.sorted(ordering).iterator,
preservesPartitioning = true)
}
def output = child.output
}
object ExistingRdd {
def convertToCatalyst(a: Any): Any = a match {
case s: Seq[Any] => s.map(convertToCatalyst)
case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray)
case other => other
}
def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = {
// TODO: Reuse the row, don't use map on the product iterator. Maybe code gen?
data.map(r => new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)
}
def fromProductRdd[A <: Product : TypeTag](productRdd: RDD[A]) = {
ExistingRdd(ScalaReflection.attributesFor[A], productToRowRdd(productRdd))
}
}
case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode {
def execute() = rdd
}

View file

@ -0,0 +1,46 @@
/*
* 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
package execution
object DebugQuery {
def apply(plan: SparkPlan): SparkPlan = {
val visited = new collection.mutable.HashSet[Long]()
plan transform {
case s: SparkPlan if !visited.contains(s.id) =>
visited += s.id
DebugNode(s)
}
}
}
case class DebugNode(child: SparkPlan) extends UnaryNode {
def references = Set.empty
def output = child.output
def execute() = {
val childRdd = child.execute()
println(
s"""
|=========================
|${child.simpleString}
|=========================
""".stripMargin)
childRdd.foreach(println(_))
childRdd
}
}

View file

@ -0,0 +1,158 @@
/*
* 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
package execution
import scala.collection.mutable
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext
import catalyst.errors._
import catalyst.expressions._
import catalyst.plans._
import catalyst.plans.physical.{ClusteredDistribution, Partitioning}
import org.apache.spark.rdd.PartitionLocalRDDFunctions._
case class SparkEquiInnerJoin(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
left: SparkPlan,
right: SparkPlan) extends BinaryNode {
override def outputPartitioning: Partitioning = left.outputPartitioning
override def requiredChildDistribution =
ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
def output = left.output ++ right.output
def execute() = attachTree(this, "execute") {
val leftWithKeys = left.execute().mapPartitions { iter =>
val generateLeftKeys = new Projection(leftKeys, left.output)
iter.map(row => (generateLeftKeys(row), row.copy()))
}
val rightWithKeys = right.execute().mapPartitions { iter =>
val generateRightKeys = new Projection(rightKeys, right.output)
iter.map(row => (generateRightKeys(row), row.copy()))
}
// Do the join.
val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
// Drop join keys and merge input tuples.
joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) }
}
/**
* Filters any rows where the any of the join keys is null, ensuring three-valued
* logic for the equi-join conditions.
*/
protected def filterNulls(rdd: RDD[(Row, Row)]) =
rdd.filter {
case (key: Seq[_], _) => !key.exists(_ == null)
}
}
case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode {
def output = left.output ++ right.output
def execute() = left.execute().map(_.copy()).cartesian(right.execute().map(_.copy())).map {
case (l: Row, r: Row) => buildRow(l ++ r)
}
}
case class BroadcastNestedLoopJoin(
streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression])
(@transient sc: SparkContext)
extends BinaryNode {
// TODO: Override requiredChildDistribution.
override def outputPartitioning: Partitioning = streamed.outputPartitioning
override def otherCopyArgs = sc :: Nil
def output = left.output ++ right.output
/** The Streamed Relation */
def left = streamed
/** The Broadcast relation */
def right = broadcast
@transient lazy val boundCondition =
condition
.map(c => BindReferences.bindReference(c, left.output ++ right.output))
.getOrElse(Literal(true))
def execute() = {
val broadcastedRelation = sc.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq)
val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter =>
val matchedRows = new mutable.ArrayBuffer[Row]
val includedBroadcastTuples = new mutable.BitSet(broadcastedRelation.value.size)
val joinedRow = new JoinedRow
streamedIter.foreach { streamedRow =>
var i = 0
var matched = false
while (i < broadcastedRelation.value.size) {
// TODO: One bitset per partition instead of per row.
val broadcastedRow = broadcastedRelation.value(i)
if (boundCondition(joinedRow(streamedRow, broadcastedRow)).asInstanceOf[Boolean]) {
matchedRows += buildRow(streamedRow ++ broadcastedRow)
matched = true
includedBroadcastTuples += i
}
i += 1
}
if (!matched && (joinType == LeftOuter || joinType == FullOuter)) {
matchedRows += buildRow(streamedRow ++ Array.fill(right.output.size)(null))
}
}
Iterator((matchedRows, includedBroadcastTuples))
}
val includedBroadcastTuples = streamedPlusMatches.map(_._2)
val allIncludedBroadcastTuples =
if (includedBroadcastTuples.count == 0) {
new scala.collection.mutable.BitSet(broadcastedRelation.value.size)
} else {
streamedPlusMatches.map(_._2).reduce(_ ++ _)
}
val rightOuterMatches: Seq[Row] =
if (joinType == RightOuter || joinType == FullOuter) {
broadcastedRelation.value.zipWithIndex.filter {
case (row, i) => !allIncludedBroadcastTuples.contains(i)
}.map {
// TODO: Use projection.
case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row)
}
} else {
Vector()
}
// TODO: Breaks lineage.
sc.union(
streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches))
}
}

View file

@ -0,0 +1,29 @@
/*
* 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
/**
* An execution engine for relational query plans that runs on top Spark and returns RDDs.
*
* Note that the operators in this package are created automatically by a query planner using a
* [[SQLContext]] and are not intended to be used directly by end users of Spark SQL. They are
* documented here in order to make it easier for others to understand the performance
* characteristics of query plans that are generated by Spark SQL.
*/
package object execution {
}

View file

@ -0,0 +1,276 @@
/*
* 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.parquet
import java.io.{IOException, FileNotFoundException}
import org.apache.hadoop.fs.{Path, FileSystem}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.Job
import org.apache.hadoop.fs.permission.FsAction
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, BaseRelation}
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.types.ArrayType
import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference, Attribute}
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import parquet.schema.{MessageTypeParser, MessageType}
import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName}
import parquet.schema.{PrimitiveType => ParquetPrimitiveType}
import parquet.schema.{Type => ParquetType}
import parquet.schema.Type.Repetition
import parquet.io.api.{Binary, RecordConsumer}
import parquet.hadoop.{Footer, ParquetFileWriter, ParquetFileReader}
import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata}
import parquet.hadoop.util.ContextUtil
import scala.collection.JavaConversions._
/**
* Relation that consists of data stored in a Parquet columnar format.
*
* Users should interact with parquet files though a SchemaRDD, created by a [[SQLContext]] instead
* of using this class directly.
*
* {{{
* val parquetRDD = sqlContext.parquetFile("path/to/parequet.file")
* }}}
*
* @param tableName The name of the relation that can be used in queries.
* @param path The path to the Parquet file.
*/
case class ParquetRelation(val tableName: String, val path: String) extends BaseRelation {
/** Schema derived from ParquetFile **/
def parquetSchema: MessageType =
ParquetTypesConverter
.readMetaData(new Path(path))
.getFileMetaData
.getSchema
/** Attributes **/
val attributes =
ParquetTypesConverter
.convertToAttributes(parquetSchema)
/** Output **/
override val output = attributes
// Parquet files have no concepts of keys, therefore no Partitioner
// Note: we could allow Block level access; needs to be thought through
override def isPartitioned = false
}
object ParquetRelation {
// The element type for the RDDs that this relation maps to.
type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow
/**
* Creates a new ParquetRelation and underlying Parquetfile for the given
* LogicalPlan. Note that this is used inside [[SparkStrategies]] to
* create a resolved relation as a data sink for writing to a Parquetfile.
* The relation is empty but is initialized with ParquetMetadata and
* can be inserted into.
*
* @param pathString The directory the Parquetfile will be stored in.
* @param child The child node that will be used for extracting the schema.
* @param conf A configuration configuration to be used.
* @param tableName The name of the resulting relation.
* @return An empty ParquetRelation inferred metadata.
*/
def create(pathString: String,
child: LogicalPlan,
conf: Configuration,
tableName: Option[String]): ParquetRelation = {
if (!child.resolved) {
throw new UnresolvedException[LogicalPlan](
child,
"Attempt to create Parquet table from unresolved child (when schema is not available)")
}
val name = s"${tableName.getOrElse(child.nodeName)}_parquet"
val path = checkPath(pathString, conf)
ParquetTypesConverter.writeMetaData(child.output, path, conf)
new ParquetRelation(name, path.toString)
}
private def checkPath(pathStr: String, conf: Configuration): Path = {
if (pathStr == null) {
throw new IllegalArgumentException("Unable to create ParquetRelation: path is null")
}
val origPath = new Path(pathStr)
val fs = origPath.getFileSystem(conf)
if (fs == null) {
throw new IllegalArgumentException(
s"Unable to create ParquetRelation: incorrectly formatted path $pathStr")
}
val path = origPath.makeQualified(fs)
if (fs.exists(path) &&
!fs.getFileStatus(path)
.getPermission
.getUserAction
.implies(FsAction.READ_WRITE)) {
throw new IOException(
s"Unable to create ParquetRelation: path $path not read-writable")
}
path
}
}
object ParquetTypesConverter {
def toDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match {
// for now map binary to string type
// TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema
case ParquetPrimitiveTypeName.BINARY => StringType
case ParquetPrimitiveTypeName.BOOLEAN => BooleanType
case ParquetPrimitiveTypeName.DOUBLE => DoubleType
case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType)
case ParquetPrimitiveTypeName.FLOAT => FloatType
case ParquetPrimitiveTypeName.INT32 => IntegerType
case ParquetPrimitiveTypeName.INT64 => LongType
case ParquetPrimitiveTypeName.INT96 => {
// TODO: add BigInteger type? TODO(andre) use DecimalType instead????
sys.error("Warning: potential loss of precision: converting INT96 to long")
LongType
}
case _ => sys.error(
s"Unsupported parquet datatype $parquetType")
}
def fromDataType(ctype: DataType): ParquetPrimitiveTypeName = ctype match {
case StringType => ParquetPrimitiveTypeName.BINARY
case BooleanType => ParquetPrimitiveTypeName.BOOLEAN
case DoubleType => ParquetPrimitiveTypeName.DOUBLE
case ArrayType(ByteType) => ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
case FloatType => ParquetPrimitiveTypeName.FLOAT
case IntegerType => ParquetPrimitiveTypeName.INT32
case LongType => ParquetPrimitiveTypeName.INT64
case _ => sys.error(s"Unsupported datatype $ctype")
}
def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = {
ctype match {
case StringType => consumer.addBinary(
Binary.fromByteArray(
record(index).asInstanceOf[String].getBytes("utf-8")
)
)
case IntegerType => consumer.addInteger(record.getInt(index))
case LongType => consumer.addLong(record.getLong(index))
case DoubleType => consumer.addDouble(record.getDouble(index))
case FloatType => consumer.addFloat(record.getFloat(index))
case BooleanType => consumer.addBoolean(record.getBoolean(index))
case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer")
}
}
def getSchema(schemaString : String) : MessageType =
MessageTypeParser.parseMessageType(schemaString)
def convertToAttributes(parquetSchema: MessageType) : Seq[Attribute] = {
parquetSchema.getColumns.map {
case (desc) => {
val ctype = toDataType(desc.getType)
val name: String = desc.getPath.mkString(".")
new AttributeReference(name, ctype, false)()
}
}
}
// TODO: allow nesting?
def convertFromAttributes(attributes: Seq[Attribute]): MessageType = {
val fields: Seq[ParquetType] = attributes.map {
a => new ParquetPrimitiveType(Repetition.OPTIONAL, fromDataType(a.dataType), a.name)
}
new MessageType("root", fields)
}
def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) {
if (origPath == null) {
throw new IllegalArgumentException("Unable to write Parquet metadata: path is null")
}
val fs = origPath.getFileSystem(conf)
if (fs == null) {
throw new IllegalArgumentException(
s"Unable to write Parquet metadata: path $origPath is incorrectly formatted")
}
val path = origPath.makeQualified(fs)
if (fs.exists(path) && !fs.getFileStatus(path).isDir) {
throw new IllegalArgumentException(s"Expected to write to directory $path but found file")
}
val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE)
if (fs.exists(metadataPath)) {
try {
fs.delete(metadataPath, true)
} catch {
case e: IOException =>
throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE at $metadataPath")
}
}
val extraMetadata = new java.util.HashMap[String, String]()
extraMetadata.put("path", path.toString)
// TODO: add extra data, e.g., table name, date, etc.?
val parquetSchema: MessageType =
ParquetTypesConverter.convertFromAttributes(attributes)
val metaData: FileMetaData = new FileMetaData(
parquetSchema,
extraMetadata,
"Spark")
ParquetFileWriter.writeMetadataFile(
conf,
path,
new Footer(path, new ParquetMetadata(metaData, Nil)) :: Nil)
}
/**
* Try to read Parquet metadata at the given Path. We first see if there is a summary file
* in the parent directory. If so, this is used. Else we read the actual footer at the given
* location.
* @param path The path at which we expect one (or more) Parquet files.
* @return The `ParquetMetadata` containing among other things the schema.
*/
def readMetaData(origPath: Path): ParquetMetadata = {
if (origPath == null) {
throw new IllegalArgumentException("Unable to read Parquet metadata: path is null")
}
val job = new Job()
// TODO: since this is called from ParquetRelation (LogicalPlan) we don't have access
// to SparkContext's hadoopConfig; in principle the default FileSystem may be different(?!)
val conf = ContextUtil.getConfiguration(job)
val fs: FileSystem = origPath.getFileSystem(conf)
if (fs == null) {
throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath")
}
val path = origPath.makeQualified(fs)
val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE)
if (fs.exists(metadataPath) && fs.isFile(metadataPath)) {
// TODO: improve exception handling, etc.
ParquetFileReader.readFooter(conf, metadataPath)
} else {
if (!fs.exists(path) || !fs.isFile(path)) {
throw new FileNotFoundException(
s"Could not find file ${path.toString} when trying to read metadata")
}
ParquetFileReader.readFooter(conf, path)
}
}
}

View file

@ -0,0 +1,212 @@
/*
* 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.parquet
import parquet.io.InvalidRecordException
import parquet.schema.MessageType
import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat}
import parquet.hadoop.util.ContextUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.{TaskContext, SerializableWritable, SparkContext}
import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression}
import org.apache.spark.sql.execution.{SparkPlan, UnaryNode, LeafNode}
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import java.io.IOException
import java.text.SimpleDateFormat
import java.util.Date
/**
* Parquet table scan operator. Imports the file that backs the given
* [[ParquetRelation]] as a RDD[Row].
*/
case class ParquetTableScan(
@transient output: Seq[Attribute],
@transient relation: ParquetRelation,
@transient columnPruningPred: Option[Expression])(
@transient val sc: SparkContext)
extends LeafNode {
override def execute(): RDD[Row] = {
val job = new Job(sc.hadoopConfiguration)
ParquetInputFormat.setReadSupportClass(
job,
classOf[org.apache.spark.sql.parquet.RowReadSupport])
val conf: Configuration = ContextUtil.getConfiguration(job)
conf.set(
RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA,
ParquetTypesConverter.convertFromAttributes(output).toString)
// TODO: think about adding record filters
/* Comments regarding record filters: it would be nice to push down as much filtering
to Parquet as possible. However, currently it seems we cannot pass enough information
to materialize an (arbitrary) Catalyst [[Predicate]] inside Parquet's
``FilteredRecordReader`` (via Configuration, for example). Simple
filter-rows-by-column-values however should be supported.
*/
sc.newAPIHadoopFile(
relation.path,
classOf[ParquetInputFormat[Row]],
classOf[Void], classOf[Row],
conf)
.map(_._2)
}
/**
* Applies a (candidate) projection.
*
* @param prunedAttributes The list of attributes to be used in the projection.
* @return Pruned TableScan.
*/
def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = {
val success = validateProjection(prunedAttributes)
if (success) {
ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc)
} else {
sys.error("Warning: Could not validate Parquet schema projection in pruneColumns")
this
}
}
/**
* Evaluates a candidate projection by checking whether the candidate is a subtype
* of the original type.
*
* @param projection The candidate projection.
* @return True if the projection is valid, false otherwise.
*/
private def validateProjection(projection: Seq[Attribute]): Boolean = {
val original: MessageType = relation.parquetSchema
val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection)
try {
original.checkContains(candidate)
true
} catch {
case e: InvalidRecordException => {
false
}
}
}
}
case class InsertIntoParquetTable(
@transient relation: ParquetRelation,
@transient child: SparkPlan)(
@transient val sc: SparkContext)
extends UnaryNode with SparkHadoopMapReduceUtil {
/**
* Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since
* Parquet files are write-once.
*/
override def execute() = {
// TODO: currently we do not check whether the "schema"s are compatible
// That means if one first creates a table and then INSERTs data with
// and incompatible schema the execution will fail. It would be nice
// to catch this early one, maybe having the planner validate the schema
// before calling execute().
val childRdd = child.execute()
assert(childRdd != null)
val job = new Job(sc.hadoopConfiguration)
ParquetOutputFormat.setWriteSupportClass(
job,
classOf[org.apache.spark.sql.parquet.RowWriteSupport])
// TODO: move that to function in object
val conf = job.getConfiguration
conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString)
val fspath = new Path(relation.path)
val fs = fspath.getFileSystem(conf)
try {
fs.delete(fspath, true)
} catch {
case e: IOException =>
throw new IOException(
s"Unable to clear output directory ${fspath.toString} prior"
+ s" to InsertIntoParquetTable:\n${e.toString}")
}
saveAsHadoopFile(childRdd, relation.path.toString, conf)
// We return the child RDD to allow chaining (alternatively, one could return nothing).
childRdd
}
override def output = child.output
// based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]]
// TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2?
// .. then we could use the default one and could use [[MutablePair]]
// instead of ``Tuple2``
private def saveAsHadoopFile(
rdd: RDD[Row],
path: String,
conf: Configuration) {
val job = new Job(conf)
val keyType = classOf[Void]
val outputFormatType = classOf[parquet.hadoop.ParquetOutputFormat[Row]]
job.setOutputKeyClass(keyType)
job.setOutputValueClass(classOf[Row])
val wrappedConf = new SerializableWritable(job.getConfiguration)
NewFileOutputFormat.setOutputPath(job, new Path(path))
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
val jobtrackerID = formatter.format(new Date())
val stageId = sc.newRddId()
def writeShard(context: TaskContext, iter: Iterator[Row]): Int = {
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
// around by taking a mod. We expect that no task will be attempted 2 billion times.
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
/* "reduce task" <split #> <attempt # = spark task #> */
val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
attemptNumber)
val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
val format = outputFormatType.newInstance
val committer = format.getOutputCommitter(hadoopContext)
committer.setupTask(hadoopContext)
val writer = format.getRecordWriter(hadoopContext)
while (iter.hasNext) {
val row = iter.next()
writer.write(null, row)
}
writer.close(hadoopContext)
committer.commitTask(hadoopContext)
return 1
}
val jobFormat = outputFormatType.newInstance
/* apparently we need a TaskAttemptID to construct an OutputCommitter;
* however we're only going to use this local OutputCommitter for
* setupJob/commitJob, so we just use a dummy "map" task.
*/
val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0)
val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId)
val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext)
jobCommitter.setupJob(jobTaskContext)
sc.runJob(rdd, writeShard _)
jobCommitter.commitJob(jobTaskContext)
}
}

View file

@ -0,0 +1,220 @@
/*
* 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.parquet
import org.apache.hadoop.conf.Configuration
import org.apache.spark.Logging
import parquet.io.api._
import parquet.schema.{MessageTypeParser, MessageType}
import parquet.hadoop.api.{WriteSupport, ReadSupport}
import parquet.hadoop.api.ReadSupport.ReadContext
import parquet.hadoop.ParquetOutputFormat
import parquet.column.ParquetProperties
import org.apache.spark.sql.catalyst.expressions.{Row, Attribute}
import org.apache.spark.sql.catalyst.types._
/**
* A `parquet.io.api.RecordMaterializer` for Rows.
*
*@param root The root group converter for the record.
*/
class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] {
def this(parquetSchema: MessageType) =
this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema)))
override def getCurrentRecord: Row = root.getCurrentRecord
override def getRootConverter: GroupConverter = root
}
/**
* A `parquet.hadoop.api.ReadSupport` for Row objects.
*/
class RowReadSupport extends ReadSupport[Row] with Logging {
override def prepareForRead(
conf: Configuration,
stringMap: java.util.Map[String, String],
fileSchema: MessageType,
readContext: ReadContext): RecordMaterializer[Row] = {
log.debug(s"preparing for read with schema ${fileSchema.toString}")
new RowRecordMaterializer(readContext.getRequestedSchema)
}
override def init(
configuration: Configuration,
keyValueMetaData: java.util.Map[String, String],
fileSchema: MessageType): ReadContext = {
val requested_schema_string =
configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString)
val requested_schema =
MessageTypeParser.parseMessageType(requested_schema_string)
log.debug(s"read support initialized for original schema ${requested_schema.toString}")
new ReadContext(requested_schema, keyValueMetaData)
}
}
object RowReadSupport {
val PARQUET_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema"
}
/**
* A `parquet.hadoop.api.WriteSupport` for Row ojects.
*/
class RowWriteSupport extends WriteSupport[Row] with Logging {
def setSchema(schema: MessageType, configuration: Configuration) {
// for testing
this.schema = schema
// TODO: could use Attributes themselves instead of Parquet schema?
configuration.set(
RowWriteSupport.PARQUET_ROW_SCHEMA,
schema.toString)
configuration.set(
ParquetOutputFormat.WRITER_VERSION,
ParquetProperties.WriterVersion.PARQUET_1_0.toString)
}
def getSchema(configuration: Configuration): MessageType = {
return MessageTypeParser.parseMessageType(
configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA))
}
private var schema: MessageType = null
private var writer: RecordConsumer = null
private var attributes: Seq[Attribute] = null
override def init(configuration: Configuration): WriteSupport.WriteContext = {
schema = if (schema == null) getSchema(configuration) else schema
attributes = ParquetTypesConverter.convertToAttributes(schema)
new WriteSupport.WriteContext(
schema,
new java.util.HashMap[java.lang.String, java.lang.String]());
}
override def prepareForWrite(recordConsumer: RecordConsumer): Unit = {
writer = recordConsumer
}
// TODO: add groups (nested fields)
override def write(record: Row): Unit = {
var index = 0
writer.startMessage()
while(index < attributes.size) {
// null values indicate optional fields but we do not check currently
if (record(index) != null && record(index) != Nil) {
writer.startField(attributes(index).name, index)
ParquetTypesConverter.consumeType(writer, attributes(index).dataType, record, index)
writer.endField(attributes(index).name, index)
}
index = index + 1
}
writer.endMessage()
}
}
object RowWriteSupport {
val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema"
}
/**
* A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
* to a [[org.apache.spark.sql.catalyst.expressions.Row]] object.
*
* @param schema The corresponding Catalyst schema in the form of a list of attributes.
*/
class CatalystGroupConverter(
schema: Seq[Attribute],
protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter {
def this(schema: Seq[Attribute]) = this(schema, new ParquetRelation.RowType(schema.length))
val converters: Array[Converter] = schema.map {
a => a.dataType match {
case ctype: NativeType =>
// note: for some reason matching for StringType fails so use this ugly if instead
if (ctype == StringType) new CatalystPrimitiveStringConverter(this, schema.indexOf(a))
else new CatalystPrimitiveConverter(this, schema.indexOf(a))
case _ => throw new RuntimeException(
s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter")
}
}.toArray
override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
private[parquet] def getCurrentRecord: ParquetRelation.RowType = current
override def start(): Unit = {
var i = 0
while (i < schema.length) {
current.setNullAt(i)
i = i + 1
}
}
override def end(): Unit = {}
}
/**
* A `parquet.io.api.PrimitiveConverter` that converts Parquet types to Catalyst types.
*
* @param parent The parent group converter.
* @param fieldIndex The index inside the record.
*/
class CatalystPrimitiveConverter(
parent: CatalystGroupConverter,
fieldIndex: Int) extends PrimitiveConverter {
// TODO: consider refactoring these together with ParquetTypesConverter
override def addBinary(value: Binary): Unit =
// TODO: fix this once a setBinary will become available in MutableRow
parent.getCurrentRecord.setByte(fieldIndex, value.getBytes.apply(0))
override def addBoolean(value: Boolean): Unit =
parent.getCurrentRecord.setBoolean(fieldIndex, value)
override def addDouble(value: Double): Unit =
parent.getCurrentRecord.setDouble(fieldIndex, value)
override def addFloat(value: Float): Unit =
parent.getCurrentRecord.setFloat(fieldIndex, value)
override def addInt(value: Int): Unit =
parent.getCurrentRecord.setInt(fieldIndex, value)
override def addLong(value: Long): Unit =
parent.getCurrentRecord.setLong(fieldIndex, value)
}
/**
* A `parquet.io.api.PrimitiveConverter` that converts Parquet strings (fixed-length byte arrays)
* into Catalyst Strings.
*
* @param parent The parent group converter.
* @param fieldIndex The index inside the record.
*/
class CatalystPrimitiveStringConverter(
parent: CatalystGroupConverter,
fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) {
override def addBinary(value: Binary): Unit =
parent.getCurrentRecord.setString(fieldIndex, value.toStringUsingUTF8)
}

View file

@ -0,0 +1,103 @@
/*
* 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.parquet
import org.apache.hadoop.fs.Path
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.Job
import parquet.schema.{MessageTypeParser, MessageType}
import parquet.hadoop.util.ContextUtil
import parquet.hadoop.ParquetWriter
import org.apache.spark.sql.catalyst.util.getTempFilePath
import org.apache.spark.sql.catalyst.expressions.GenericRow
import java.nio.charset.Charset
object ParquetTestData {
val testSchema =
"""message myrecord {
|optional boolean myboolean;
|optional int32 myint;
|optional binary mystring;
|optional int64 mylong;
|optional float myfloat;
|optional double mydouble;
|}""".stripMargin
// field names for test assertion error messages
val testSchemaFieldNames = Seq(
"myboolean:Boolean",
"mtint:Int",
"mystring:String",
"mylong:Long",
"myfloat:Float",
"mydouble:Double"
)
val subTestSchema =
"""
|message myrecord {
|optional boolean myboolean;
|optional int64 mylong;
|}
""".stripMargin
// field names for test assertion error messages
val subTestSchemaFieldNames = Seq(
"myboolean:Boolean",
"mylong:Long"
)
val testFile = getTempFilePath("testParquetFile").getCanonicalFile
lazy val testData = new ParquetRelation("testData", testFile.toURI.toString)
def writeFile = {
testFile.delete
val path: Path = new Path(testFile.toURI)
val job = new Job()
val configuration: Configuration = ContextUtil.getConfiguration(job)
val schema: MessageType = MessageTypeParser.parseMessageType(testSchema)
val writeSupport = new RowWriteSupport()
writeSupport.setSchema(schema, configuration)
val writer = new ParquetWriter(path, writeSupport)
for(i <- 0 until 15) {
val data = new Array[Any](6)
if (i % 3 == 0) {
data.update(0, true)
} else {
data.update(0, false)
}
if (i % 5 == 0) {
data.update(1, 5)
} else {
data.update(1, null) // optional
}
data.update(2, "abc")
data.update(3, i.toLong << 33)
data.update(4, 2.5F)
data.update(5, 4.5D)
writer.write(new GenericRow(data.toArray))
}
writer.close()
}
}

View file

@ -0,0 +1,24 @@
/*
* 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
package sql
package test
/** A SQLContext that can be used for local testing. */
object TestSQLContext
extends SQLContext(new SparkContext("local", "TestSQLContext", new SparkConf()))

View file

@ -0,0 +1,52 @@
#
# 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.
#
# Set everything to be logged to the file core/target/unit-tests.log
log4j.rootLogger=DEBUG, CA, FA
#Console Appender
log4j.appender.CA=org.apache.log4j.ConsoleAppender
log4j.appender.CA.layout=org.apache.log4j.PatternLayout
log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n
log4j.appender.CA.Threshold = WARN
#File Appender
log4j.appender.FA=org.apache.log4j.FileAppender
log4j.appender.FA.append=false
log4j.appender.FA.file=target/unit-tests.log
log4j.appender.FA.layout=org.apache.log4j.PatternLayout
log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n
# Set the logger level of File Appender to WARN
log4j.appender.FA.Threshold = INFO
# Some packages are noisy for no good reason.
log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false
log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF
log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false
log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF
log4j.additivity.hive.ql.metadata.Hive=false
log4j.logger.hive.ql.metadata.Hive=OFF
# Parquet logging
parquet.hadoop.InternalParquetRecordReader=WARN
log4j.logger.parquet.hadoop.InternalParquetRecordReader=WARN
parquet.hadoop.ParquetInputFormat=WARN
log4j.logger.parquet.hadoop.ParquetInputFormat=WARN

View file

@ -0,0 +1,201 @@
/*
* 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
import org.scalatest.{BeforeAndAfterAll, FunSuite}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.test._
/* Implicits */
import TestSQLContext._
class DslQuerySuite extends QueryTest {
import TestData._
test("table scan") {
checkAnswer(
testData,
testData.collect().toSeq)
}
test("agg") {
checkAnswer(
testData2.groupBy('a)('a, Sum('b)),
Seq((1,3),(2,3),(3,3))
)
}
test("select *") {
checkAnswer(
testData.select(Star(None)),
testData.collect().toSeq)
}
test("simple select") {
checkAnswer(
testData.where('key === 1).select('value),
Seq(Seq("1")))
}
test("sorting") {
checkAnswer(
testData2.orderBy('a.asc, 'b.asc),
Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2)))
checkAnswer(
testData2.orderBy('a.asc, 'b.desc),
Seq((1,2), (1,1), (2,2), (2,1), (3,2), (3,1)))
checkAnswer(
testData2.orderBy('a.desc, 'b.desc),
Seq((3,2), (3,1), (2,2), (2,1), (1,2), (1,1)))
checkAnswer(
testData2.orderBy('a.desc, 'b.asc),
Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2)))
}
test("average") {
checkAnswer(
testData2.groupBy()(Average('a)),
2.0)
}
test("count") {
checkAnswer(
testData2.groupBy()(Count(1)),
testData2.count()
)
}
test("null count") {
checkAnswer(
testData3.groupBy('a)('a, Count('b)),
Seq((1,0), (2, 1))
)
checkAnswer(
testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)),
(2, 1, 2, 2, 1) :: Nil
)
}
test("inner join where, one match per row") {
checkAnswer(
upperCaseData.join(lowerCaseData, Inner).where('n === 'N),
Seq(
(1, "A", 1, "a"),
(2, "B", 2, "b"),
(3, "C", 3, "c"),
(4, "D", 4, "d")
))
}
test("inner join ON, one match per row") {
checkAnswer(
upperCaseData.join(lowerCaseData, Inner, Some('n === 'N)),
Seq(
(1, "A", 1, "a"),
(2, "B", 2, "b"),
(3, "C", 3, "c"),
(4, "D", 4, "d")
))
}
test("inner join, where, multiple matches") {
val x = testData2.where('a === 1).subquery('x)
val y = testData2.where('a === 1).subquery('y)
checkAnswer(
x.join(y).where("x.a".attr === "y.a".attr),
(1,1,1,1) ::
(1,1,1,2) ::
(1,2,1,1) ::
(1,2,1,2) :: Nil
)
}
test("inner join, no matches") {
val x = testData2.where('a === 1).subquery('x)
val y = testData2.where('a === 2).subquery('y)
checkAnswer(
x.join(y).where("x.a".attr === "y.a".attr),
Nil)
}
test("big inner join, 4 matches per row") {
val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData)
val bigDataX = bigData.subquery('x)
val bigDataY = bigData.subquery('y)
checkAnswer(
bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr),
testData.flatMap(
row => Seq.fill(16)((row ++ row).toSeq)).collect().toSeq)
}
test("cartisian product join") {
checkAnswer(
testData3.join(testData3),
(1, null, 1, null) ::
(1, null, 2, 2) ::
(2, 2, 1, null) ::
(2, 2, 2, 2) :: Nil)
}
test("left outer join") {
checkAnswer(
upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N)),
(1, "A", 1, "a") ::
(2, "B", 2, "b") ::
(3, "C", 3, "c") ::
(4, "D", 4, "d") ::
(5, "E", null, null) ::
(6, "F", null, null) :: Nil)
}
test("right outer join") {
checkAnswer(
lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N)),
(1, "a", 1, "A") ::
(2, "b", 2, "B") ::
(3, "c", 3, "C") ::
(4, "d", 4, "D") ::
(null, null, 5, "E") ::
(null, null, 6, "F") :: Nil)
}
test("full outer join") {
val left = upperCaseData.where('N <= 4).subquery('left)
val right = upperCaseData.where('N >= 3).subquery('right)
checkAnswer(
left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)),
(1, "A", null, null) ::
(2, "B", null, null) ::
(3, "C", 3, "C") ::
(4, "D", 4, "D") ::
(null, null, 5, "E") ::
(null, null, 6, "F") :: Nil)
}
}

Some files were not shown because too many files have changed in this diff Show more