If I'm running this locally and my path points to S3, this would currently error out because of incorrect FS.
I tested this in a scenario that previously didn't work, this change seemed to fix the issue.
Author: Volodymyr Lyubinets <vlyubin@gmail.com>
Closes#5020 from vlyubin/parquertbug and squashes the following commits:
a645ad5 [Volodymyr Lyubinets] Fix filesystem bug in newParquet relation
Still, we keep only a single HiveContext within ThriftServer, and we also create a object called `SQLSession` for isolating the different user states.
Developers can obtain/release a new user session via `openSession` and `closeSession`, and `SQLContext` and `HiveContext` will also provide a default session if no `openSession` called, for backward-compatibility.
Author: Cheng Hao <hao.cheng@intel.com>
Closes#4885 from chenghao-intel/multisessions_singlecontext and squashes the following commits:
1c47b2a [Cheng Hao] rename the tss => tlSession
815b27a [Cheng Hao] code style issue
57e3fa0 [Cheng Hao] openSession is not compatible between Hive0.12 & 0.13.1
4665b0d [Cheng Hao] thriftservice with single context
This PR adds a specialized in-memory column type for fixed-precision decimals.
For all other column types, a single integer column type ID is enough to determine which column type to use. However, this doesn't apply to fixed-precision decimal types with different precision and scale parameters. Moreover, according to the previous design, there seems no trivial way to encode precision and scale information into the columnar byte buffer. On the other hand, considering we always know the data type of the column to be built / scanned ahead of time. This PR no longer use column type ID to construct `ColumnBuilder`s and `ColumnAccessor`s, but resorts to the actual column data type. In this way, we can pass precision / scale information along the way.
The column type ID is now not used anymore and can be removed in a future PR.
### Micro benchmark result
The following micro benchmark builds a simple table with 2 million decimals (precision = 10, scale = 0), cache it in memory, then count all the rows. Code (simply paste it into Spark shell):
```scala
import sc._
import sqlContext._
import sqlContext.implicits._
import org.apache.spark.sql.types._
import com.google.common.base.Stopwatch
def benchmark(n: Int)(f: => Long) {
val stopwatch = new Stopwatch()
def run() = {
stopwatch.reset()
stopwatch.start()
f
stopwatch.stop()
stopwatch.elapsedMillis()
}
val records = (0 until n).map(_ => run())
(0 until n).foreach(i => println(s"Round $i: ${records(i)} ms"))
println(s"Average: ${records.sum / n.toDouble} ms")
}
// Explicit casting is required because ScalaReflection can't inspect decimal precision
parallelize(1 to 2000000)
.map(i => Tuple1(Decimal(i, 10, 0)))
.toDF("dec")
.select($"dec" cast DecimalType(10, 0))
.registerTempTable("dec")
sql("CACHE TABLE dec")
val df = table("dec")
// Warm up
df.count()
df.count()
benchmark(5) {
df.count()
}
```
With `FIXED_DECIMAL` column type:
- Round 0: 75 ms
- Round 1: 97 ms
- Round 2: 75 ms
- Round 3: 70 ms
- Round 4: 72 ms
- Average: 77.8 ms
Without `FIXED_DECIMAL` column type:
- Round 0: 1233 ms
- Round 1: 1170 ms
- Round 2: 1171 ms
- Round 3: 1141 ms
- Round 4: 1141 ms
- Average: 1171.2 ms
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4938)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4938 from liancheng/decimal-column-type and squashes the following commits:
fef5338 [Cheng Lian] Updates fixed decimal column type related test cases
e08ab5b [Cheng Lian] Only resorts to FIXED_DECIMAL when the value can be held in a long
4db713d [Cheng Lian] Adds in-memory column type for fixed-precision decimals
use prettyString instead of toString() (which include id of expression) as column name in agg()
Author: Davies Liu <davies@databricks.com>
Closes#5006 from davies/prettystring and squashes the following commits:
cb1fdcf [Davies Liu] use prettyString as column name in agg()
All the contents in this file are not referenced anywhere and should have been removed in #4116 when I tried to get rid of the old Parquet test suites.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/5010)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#5010 from liancheng/spark-6285 and squashes the following commits:
06ed057 [Cheng Lian] Removes unused ParquetTestData and duplicated TestGroupWriteSupport
Author: Volodymyr Lyubinets <vlyubin@gmail.com>
Closes#4988 from vlyubin/columncomp and squashes the following commits:
92d7c8f [Volodymyr Lyubinets] Added equals to Column
Avoid `UnsupportedOperationException` from JsonRDD.inferSchema on empty RDD.
Not sure if this is supposed to be an error (but a better one), but it seems like this case can come up if the input is down-sampled so much that nothing is sampled.
Now stuff like this:
```
sqlContext.jsonRDD(sc.parallelize(List[String]()))
```
just results in
```
org.apache.spark.sql.DataFrame = []
```
Author: Sean Owen <sowen@cloudera.com>
Closes#4971 from srowen/SPARK-6245 and squashes the following commits:
3699964 [Sean Owen] Set() -> Set.empty
3c619e1 [Sean Owen] Avoid UnsupportedOperationException from JsonRDD.inferSchema on empty RDD
Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.
Author: Sean Owen <sowen@cloudera.com>
Closes#4950 from srowen/SPARK-6225 and squashes the following commits:
3080972 [Sean Owen] Ordered imports: Java, Scala, 3rd party, Spark
c67985b [Sean Owen] Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.
Author: Michael Armbrust <michael@databricks.com>
Closes#4920 from marmbrus/openStrategies and squashes the following commits:
cbc35c0 [Michael Armbrust] [SQL] Make Strategies a public developer API
jira: https://issues.apache.org/jira/browse/SPARK-6163
Author: Yin Huai <yhuai@databricks.com>
Closes#4896 from yhuai/SPARK-6163 and squashes the following commits:
45e023e [Yin Huai] Address @chenghao-intel's comment.
2e8734e [Yin Huai] Use JSON data source for jsonFile.
92a4a33 [Yin Huai] Test.
Based on #4904 with style errors fixed.
`LogicalPlan#resolve` will not only produce `Attribute`, but also "`GetField` chain".
So in `ResolveSortReferences`, after resolve the ordering expressions, we should not just collect the `Attribute` results, but also `Attribute` at the bottom of "`GetField` chain".
Author: Wenchen Fan <cloud0fan@outlook.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#4918 from marmbrus/pr/4904 and squashes the following commits:
997f84e [Michael Armbrust] fix style
3eedbfc [Wenchen Fan] fix 6145
Option 1 of 2: Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11
Author: Sean Owen <sowen@cloudera.com>
Closes#4912 from srowen/SPARK-6182.1 and squashes the following commits:
eff60de [Sean Owen] Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11
Integration test suites in the JDBC data source (`MySQLIntegration` and `PostgresIntegration`) depend on docker-client 2.7.5, which transitively depends on Guava 17.0. Unfortunately, Guava 17.0 is causing test runtime binary compatibility issues when Spark is compiled against Hive 0.12.0, or Hadoop 2.4.
Considering `MySQLIntegration` and `PostgresIntegration` are ignored right now, I'd suggest moving them from the Spark project to the [Spark integration tests] [1] project. This PR removes both the JDBC data source integration tests and the docker-client test dependency.
[1]: |https://github.com/databricks/spark-integration-tests
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4872)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4872 from liancheng/remove-docker-client and squashes the following commits:
1f4169e [Cheng Lian] Removes DockerHacks
159b24a [Cheng Lian] Removed JDBC integration tests which depends on docker-client
- Various Fixes to docs
- Make data source traits actually interfaces
Based on #4862 but with fixed conflicts.
Author: Reynold Xin <rxin@databricks.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#4868 from marmbrus/pr/4862 and squashes the following commits:
fe091ea [Michael Armbrust] Merge remote-tracking branch 'origin/master' into pr/4862
0208497 [Reynold Xin] Test fixes.
34e0a28 [Reynold Xin] [SPARK-5310][SQL] Various fixes to Spark SQL docs.
This PR contains the following changes:
1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However, the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values).
2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types.
3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust.
5. Update the equality check of JSON relation. Since JSON does not really cares nullability, `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables.
JIRA: https://issues.apache.org/jira/browse/SPARK-5950
Thanks viirya for the initial work in #4729.
cc marmbrus liancheng
Author: Yin Huai <yhuai@databricks.com>
Closes#4826 from yhuai/insertNullabilityCheck and squashes the following commits:
3b61a04 [Yin Huai] Revert change on equals.
80e487e [Yin Huai] asNullable in UDT.
587d88b [Yin Huai] Make methods private.
0cb7ea2 [Yin Huai] marmbrus's comments.
3cec464 [Yin Huai] Cheng's comments.
486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
d3747d1 [Yin Huai] Remove unnecessary change.
8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check.
0eb5578 [Yin Huai] Fix tests.
f6ed813 [Yin Huai] Update old parquet path.
e4f397c [Yin Huai] Unit tests.
b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check.
8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data.
bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data.
0a703e7 [Yin Huai] Test failed again since we cannot read correct content.
9a26611 [Yin Huai] Make InsertIntoTable happy.
8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability
4ec17fd [Yin Huai] Failed test.
Constructs like Hive `TRANSFORM` may generate malformed rows (via badly authored external scripts for example). I'm a bit hesitant to have this feature, since it introduces per-tuple cost when caching tables. However, considering caching tables is usually a one-time cost, this is probably worth having.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4842)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4842 from liancheng/spark-6082 and squashes the following commits:
b05dbff [Cheng Lian] Provides better error message for malformed rows when caching tables
The API signatire for join requires the JoinType to be the third parameter. The code examples provided for join show JoinType being provided as the 2nd parater resuling in errors (i.e. "df1.join(df2, "outer", $"df1Key" === $"df2Key") ). The correct sample code is df1.join(df2, $"df1Key" === $"df2Key", "outer")
Author: Paul Power <paul.power@peerside.com>
Closes#4847 from peerside/master and squashes the following commits:
ebc1efa [Paul Power] Merge pull request #1 from peerside/peerside-patch-1
e353340 [Paul Power] Updated comments use correct sample code for Dataframe joins
Always set `containsNull = true` when infer the schema of JSON datasets. If we set `containsNull` based on records we scanned, we may miss arrays with null values when we do sampling. Also, because future data can have arrays with null values, if we convert JSON data to parquet, always setting `containsNull = true` is a more robust way to go.
JIRA: https://issues.apache.org/jira/browse/SPARK-6052
Author: Yin Huai <yhuai@databricks.com>
Closes#4806 from yhuai/jsonArrayContainsNull and squashes the following commits:
05eab9d [Yin Huai] Change containsNull to true.
This is needed for the SQL bindings to work on Yarn.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#4822 from vanzin/SPARK-6074 and squashes the following commits:
fb52001 [Marcelo Vanzin] [SPARK-6074] [sql] Package pyspark sql bindings.
JIRA: https://issues.apache.org/jira/browse/SPARK-6024
Author: Yin Huai <yhuai@databricks.com>
Closes#4795 from yhuai/wideSchema and squashes the following commits:
4882e6f [Yin Huai] Address comments.
73e71b4 [Yin Huai] Address comments.
143927a [Yin Huai] Simplify code.
cc1d472 [Yin Huai] Make the schema wider.
12bacae [Yin Huai] If the JSON string of a schema is too large, split it before storing it in metastore.
e9b4f70 [Yin Huai] Failed test.
`FilteringParquetRowInputFormat` manually merges Parquet schemas before computing splits. However, it is duplicate because the schemas are already merged in `ParquetRelation2`. We don't need to re-merge them at `InputFormat`.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#4786 from viirya/dup_parquet_schemas_merge and squashes the following commits:
ef78a5a [Liang-Chi Hsieh] Avoiding duplicate Parquet schema merging.
It is useful to let the user decide the number of rows to show in DataFrame.show
Author: Jacky Li <jacky.likun@huawei.com>
Closes#4767 from jackylk/show and squashes the following commits:
a0e0f4b [Jacky Li] fix testcase
7cdbe91 [Jacky Li] modify according to comment
bb54537 [Jacky Li] for Java compatibility
d7acc18 [Jacky Li] modify according to comments
981be52 [Jacky Li] add numRows param in DataFrame.show()
Please see JIRA (https://issues.apache.org/jira/browse/SPARK-6016) for details of the bug.
Author: Yin Huai <yhuai@databricks.com>
Closes#4775 from yhuai/parquetFooterCache and squashes the following commits:
78787b1 [Yin Huai] Remove footerCache in FilteringParquetRowInputFormat.
dff6fba [Yin Huai] Failed unit test.
DataFrame.explain return wrong result when the query is DDL command.
For example, the following two queries should print out the same execution plan, but it not.
sql("create table tb as select * from src where key > 490").explain(true)
sql("explain extended create table tb as select * from src where key > 490")
This is because DataFrame.explain leverage logicalPlan which had been forced executed, we should use the unexecuted plan queryExecution.logical.
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#4707 from yanboliang/spark-5926 and squashes the following commits:
fa6db63 [Yanbo Liang] logicalPlan is not lazy
0e40a1b [Yanbo Liang] make DataFrame.explain leverage queryExecution.logical
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#4760 from viirya/dup_literal and squashes the following commits:
06e7516 [Liang-Chi Hsieh] Remove duplicate Literal matching block.
`ReadContext.init` calls `InitContext.getMergedKeyValueMetadata`, which doesn't know how to merge conflicting user defined key-value metadata and throws exception. In our case, when dealing with different but compatible schemas, we have different Spark SQL schema JSON strings in different Parquet part-files, thus causes this problem. Reading similar Parquet files generated by Hive doesn't suffer from this issue.
In this PR, we manually merge the schemas before passing it to `ReadContext` to avoid the exception.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4768)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4768 from liancheng/spark-6010 and squashes the following commits:
9002f0a [Cheng Lian] Fixes SPARK-6010
Author: Michael Armbrust <michael@databricks.com>
Closes#4757 from marmbrus/udtConversions and squashes the following commits:
3714aad [Michael Armbrust] [SPARK-5996][SQL] Fix specialized outbound conversions
Also added desc/asc function for constructing sorting expressions more conveniently. And added a small fix to lift alias out of cast expression.
Author: Reynold Xin <rxin@databricks.com>
Closes#4752 from rxin/SPARK-5985 and squashes the following commits:
aeda5ae [Reynold Xin] Added Experimental flag to ColumnName.
047ad03 [Reynold Xin] Lift alias out of cast.
c9cf17c [Reynold Xin] [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.
Added a new test suite to make sure Java DF programs can use varargs properly.
Also moved all suites into test.org.apache.spark package to make sure the suites also test for method visibility.
Author: Reynold Xin <rxin@databricks.com>
Closes#4751 from rxin/df-tests and squashes the following commits:
1e8b8e4 [Reynold Xin] Fixed imports and renamed JavaAPISuite.
a6ca53b [Reynold Xin] [SPARK-5904][SQL] DataFrame Java API test suites.
Author: Michael Armbrust <michael@databricks.com>
Closes#4738 from marmbrus/udtRepart and squashes the following commits:
c06d7b5 [Michael Armbrust] fix compilation
91c8829 [Michael Armbrust] [SQL][SPARK-5532] Repartition should not use external rdd representation
Author: Michael Armbrust <michael@databricks.com>
Closes#4736 from marmbrus/asExprs and squashes the following commits:
5ba97e4 [Michael Armbrust] [SPARK-5910][SQL] Support for as in selectExpr
1. Column is no longer a DataFrame to simplify class hierarchy.
2. Don't use varargs on abstract methods (see Scala compiler bug SI-9013).
Author: Reynold Xin <rxin@databricks.com>
Closes#4686 from rxin/SPARK-5904 and squashes the following commits:
fd9b199 [Reynold Xin] Fixed Python tests.
df25cef [Reynold Xin] Non final.
5221530 [Reynold Xin] [SPARK-5904][SQL] DataFrame API fixes.
The `int` is 64-bit on 64-bit machine (very common now), we should infer it as LongType for it in Spark SQL.
Also, LongType in SQL will come back as `int`.
Author: Davies Liu <davies@databricks.com>
Closes#4666 from davies/long and squashes the following commits:
6bc6cc4 [Davies Liu] infer int as LongType
Also added test cases for checking the serializability of HiveContext and SQLContext.
Author: Reynold Xin <rxin@databricks.com>
Closes#4628 from rxin/SPARK-5840 and squashes the following commits:
ecb3bcd [Reynold Xin] test cases and reviews.
55eb822 [Reynold Xin] [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction.
This pull request replaces calls to deprecated methods from `java.util.Date` with near-equivalents in `java.util.Calendar`.
Author: Tor Myklebust <tmyklebu@gmail.com>
Closes#4668 from tmyklebu/master and squashes the following commits:
66215b1 [Tor Myklebust] Use GregorianCalendar instead of Timestamp get methods.
Although we've migrated to the DataFrame API, lots of code still uses `rdd` or `srdd` as local variable names. This PR tries to address these naming inconsistencies and some other minor DataFrame related style issues.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4670)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4670 from liancheng/df-cleanup and squashes the following commits:
3e14448 [Cheng Lian] Cleans up DataFrame variable names and toDF() calls
The problem is that after we create an empty hive metastore parquet table (e.g. `CREATE TABLE test (a int) STORED AS PARQUET`), Hive will create an empty dir for us, which cause our data source `ParquetRelation2` fail to get the schema of the table. See JIRA for the case to reproduce the bug and the exception.
This PR is based on #4562 from chenghao-intel.
JIRA: https://issues.apache.org/jira/browse/SPARK-5852
Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Hao <hao.cheng@intel.com>
Closes#4655 from yhuai/CTASParquet and squashes the following commits:
b8b3450 [Yin Huai] Update tests.
2ac94f7 [Yin Huai] Update tests.
3db3d20 [Yin Huai] Minor update.
d7e2308 [Yin Huai] Revert changes in HiveMetastoreCatalog.scala.
36978d1 [Cheng Hao] Update the code as feedback
a04930b [Cheng Hao] fix bug of scan an empty parquet based table
442ffe0 [Cheng Hao] passdown the schema for Parquet File in HiveContext
Author: Michael Armbrust <michael@databricks.com>
Closes#4657 from marmbrus/pythonUdfs and squashes the following commits:
a7823a8 [Michael Armbrust] [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#4649 from viirya/use_checkpath and squashes the following commits:
0f9a1a1 [Liang-Chi Hsieh] Use same function to check path parameter.
Author: Reynold Xin <rxin@databricks.com>
Closes#4640 from rxin/SPARK-5853 and squashes the following commits:
9c6f569 [Reynold Xin] [SPARK-5853][SQL] Schema support in Row.
Added a bunch of tags.
Also changed parquetFile to take varargs rather than a string followed by varargs.
Author: Reynold Xin <rxin@databricks.com>
Closes#4636 from rxin/df-doc and squashes the following commits:
651f80c [Reynold Xin] Fixed parquetFile in PySpark.
8dc3024 [Reynold Xin] [SQL] Various DataFrame doc changes.
This PR adds a `ShowTablesCommand` to support `SHOW TABLES [IN databaseName]` SQL command. The result of `SHOW TABLE` has two columns, `tableName` and `isTemporary`. For temporary tables, the value of `isTemporary` column will be `false`.
JIRA: https://issues.apache.org/jira/browse/SPARK-4865
Author: Yin Huai <yhuai@databricks.com>
Closes#4618 from yhuai/showTablesCommand and squashes the following commits:
0c09791 [Yin Huai] Use ShowTablesCommand.
85ee76d [Yin Huai] Since SHOW TABLES is not a Hive native command any more and we will not see "OK" (originally generated by Hive's driver), use SHOW DATABASES in the test.
94bacac [Yin Huai] Add SHOW TABLES to the list of noExplainCommands.
d71ed09 [Yin Huai] Fix test.
a4a6ec3 [Yin Huai] Add SHOW TABLE command.
Existing implementation of arithmetic operators and BinaryComparison operators have redundant type checking codes, e.g.:
Expression.n2 is used by Add/Subtract/Multiply.
(1) n2 always checks left.dataType == right.dataType. However, this checking should be done once when we resolve expression types;
(2) n2 requires dataType is a NumericType. This can be done once.
This PR optimizes arithmetic and predicate operators by removing such redundant type-checking codes.
Some preliminary benchmarking on 10G TPC-H data over 5 r3.2xlarge EC2 machines shows that this PR can reduce the query time by 5.5% to 11%.
The benchmark queries follow the template below, where OP is plus/minus/times/divide/remainder/bitwise and/bitwise or/bitwise xor.
SELECT l_returnflag, l_linestatus, SUM(l_quantity OP cnt1), SUM(l_quantity OP cnt2), ...., SUM(l_quantity OP cnt700)
FROM (
SELECT l_returnflag, l_linestatus, l_quantity, 1 AS cnt1, 2 AS cnt2, ..., 700 AS cnt700
FROM lineitem
WHERE l_shipdate <= '1998-09-01'
)
GROUP BY l_returnflag, l_linestatus;
Author: kai <kaizeng@eecs.berkeley.edu>
Closes#4472 from kai-zeng/arithmetic-optimize and squashes the following commits:
fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into arithmetic-optimize
4b3a1bb [kai] chmod a-x
5a41e49 [kai] chmod a-x Expression.scala
cb37c94 [kai] rebase onto spark master
7f6e968 [kai] chmod 100755 -> 100644
6cddb46 [kai] format
7490dbc [kai] fix unresolved-expression exception for EqualTo
9c40bc0 [kai] fix bitwisenot
3cbd363 [kai] clean up test code
ca47801 [kai] override evalInternal for bitwise ops
8fa84a1 [kai] add bitwise or and xor
6892fc4 [kai] revert override evalInternal
f8eba24 [kai] override evalInternal
31ccdd4 [kai] rewrite all bitwise op and remove evalInternal
86297e2 [kai] generalized
cb92ae1 [kai] bitwise-and: override eval
97a7d6c [kai] bitwise-and: override evalInternal using and func
0906c39 [kai] add bitwise test
62abbbc [kai] clean up predicate and arithmetic
b34d58d [kai] add caching and benmark option
12c5b32 [kai] override eval
1cd7571 [kai] fix sqrt and maxof
03fd0c3 [kai] fix predicate
16fd84c [kai] optimize + - * / % -(unary) abs < > <= >=
fd95823 [kai] remove unnecessary type checking
24d062f [kai] test suite