Commit graph

14411 commits

Author SHA1 Message Date
Takeshi YAMAMURO 73862a1eb9 [SPARK-11394][SQL] Throw IllegalArgumentException for unsupported types in postgresql
If DataFrame has BYTE types, throws an exception:
org.postgresql.util.PSQLException: ERROR: type "byte" does not exist

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #9350 from maropu/FixBugInPostgreJdbc.
2015-12-28 21:28:32 -08:00
Reynold Xin 1a91be8078 [SPARK-12547][SQL] Tighten scala style checker enforcement for UDF registration
We use scalastyle:off to turn off style checks in certain places where it is not possible to follow the style guide. This is usually ok. However, in udf registration, we disable the checker for a large amount of code simply because some of them exceed 100 char line limit. It is better to just disable the line limit check rather than everything.

In this pull request, I only disabled line length check, and fixed a problem (lack explicit types for public methods).

Author: Reynold Xin <rxin@databricks.com>

Closes #10501 from rxin/SPARK-12547.
2015-12-28 20:43:06 -08:00
gatorsmile 043135819c [SPARK-12522][SQL][MINOR] Add the missing document strings for the SQL configuration
Fixing the missing the document for the configuration. We can see the missing messages "TODO" when issuing the command "SET -V".
```
spark.sql.columnNameOfCorruptRecord
spark.sql.hive.verifyPartitionPath
spark.sql.sources.parallelPartitionDiscovery.threshold
spark.sql.hive.convertMetastoreParquet.mergeSchema
spark.sql.hive.convertCTAS
spark.sql.hive.thriftServer.async
```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10471 from gatorsmile/commandDesc.
2015-12-28 17:22:18 -08:00
Josh Rosen 124a3a5e4e [SPARK-12490] Don't use Javascript for web UI's paginated table controls
The web UI's paginated table uses Javascript to implement certain navigation controls, such as table sorting and the "go to page" form. This is unnecessary and should be simplified to use plain HTML form controls and links.

/cc zsxwing, who wrote this original code, and yhuai.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10441 from JoshRosen/simplify-paginated-table-sorting.
2015-12-28 16:42:11 -08:00
Shixiong Zhu 710b411729 [SPARK-12489][CORE][SQL][MLIB] Fix minor issues found by FindBugs
Include the following changes:

1. Close `java.sql.Statement`
2. Fix incorrect `asInstanceOf`.
3. Remove unnecessary `synchronized` and `ReentrantLock`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10440 from zsxwing/findbugs.
2015-12-28 15:01:51 -08:00
Josh Rosen fb572c6e4b [SPARK-12525] Fix fatal compiler warnings in Kinesis ASL due to @transient annotations
The Scala 2.11 SBT build currently fails for Spark 1.6.0 and master due to warnings about the `transient` annotation:

```
[error] [warn] /Users/joshrosen/Documents/spark/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala:73: no valid targets for annotation on value sc - it is discarded unused. You may specify targets with meta-annotations, e.g. (transient param)
[error] [warn]     transient sc: SparkContext,
```

This fix implemented here is the same as what we did in #8433: remove the `transient` annotations when they are not necessary and replace use  `transient private val` in the remaining cases.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10479 from JoshRosen/fix-sbt-2.11.
2015-12-28 14:51:22 -08:00
Daoyuan Wang a6d385322e [SPARK-12222][CORE] Deserialize RoaringBitmap using Kryo serializer throw Buffer underflow exception
Since we only need to implement `def skipBytes(n: Int)`,
code in #10213 could be simplified.
davies scwf

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

Closes #10253 from adrian-wang/kryo.
2015-12-29 07:02:30 +09:00
gatorsmile 01ba95d8bf [SPARK-12441][SQL] Fixing missingInput in Generate/MapPartitions/AppendColumns/MapGroups/CoGroup
When explain any plan with Generate, we will see an exclamation mark in the plan. Normally, when we see this mark, it means the plan has an error. This PR is to correct the `missingInput` in `Generate`.

For example,
```scala
val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters")
val df2 =
  df.explode('letters) {
    case Row(letters: String) => letters.split(" ").map(Tuple1(_)).toSeq
  }

df2.explain(true)
```
Before the fix, the plan is like
```
== Parsed Logical Plan ==
'Generate UserDefinedGenerator('letters), true, false, None
+- Project [_1#0 AS number#2,_2#1 AS letters#3]
   +- LocalRelation [_1#0,_2#1], [[1,a b c],[2,a b],[3,a]]

== Analyzed Logical Plan ==
number: int, letters: string, _1: string
Generate UserDefinedGenerator(letters#3), true, false, None, [_1#8]
+- Project [_1#0 AS number#2,_2#1 AS letters#3]
   +- LocalRelation [_1#0,_2#1], [[1,a b c],[2,a b],[3,a]]

== Optimized Logical Plan ==
Generate UserDefinedGenerator(letters#3), true, false, None, [_1#8]
+- LocalRelation [number#2,letters#3], [[1,a b c],[2,a b],[3,a]]

== Physical Plan ==
!Generate UserDefinedGenerator(letters#3), true, false, [number#2,letters#3,_1#8]
+- LocalTableScan [number#2,letters#3], [[1,a b c],[2,a b],[3,a]]
```

**Updates**: The same issues are also found in the other four Dataset operators: `MapPartitions`/`AppendColumns`/`MapGroups`/`CoGroup`. Fixed all these four.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10393 from gatorsmile/generateExplain.
2015-12-28 12:48:30 -08:00
Stephan Kessler a6a4812434 [SPARK-7727][SQL] Avoid inner classes in RuleExecutor
Moved (case) classes Strategy, Once, FixedPoint and Batch to the companion object. This is necessary if we want to have the Optimizer easily extendable in the following sense: Usually a user wants to add additional rules, and just take the ones that are already there. However, inner classes made that impossible since the code did not compile

This allows easy extension of existing Optimizers see the DefaultOptimizerExtendableSuite for a corresponding test case.

Author: Stephan Kessler <stephan.kessler@sap.com>

Closes #10174 from stephankessler/SPARK-7727.
2015-12-28 12:46:20 -08:00
Kousuke Saruta 07165ca06f [SPARK-12424][ML] The implementation of ParamMap#filter is wrong.
ParamMap#filter uses `mutable.Map#filterKeys`. The return type of `filterKey` is collection.Map, not mutable.Map but the result is casted to mutable.Map using `asInstanceOf` so we get `ClassCastException`.
Also, the return type of Map#filterKeys is not Serializable. It's the issue of Scala (https://issues.scala-lang.org/browse/SI-6654).

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #10381 from sarutak/SPARK-12424.
2015-12-29 05:33:19 +09:00
gatorsmile e01c6c8664 [SPARK-12287][SQL] Support UnsafeRow in MapPartitions/MapGroups/CoGroup
Support Unsafe Row in MapPartitions/MapGroups/CoGroup.

Added a test case for MapPartitions. Since MapGroups and CoGroup are built on AppendColumns, all the related dataset test cases already can verify the correctness when MapGroups and CoGroup processing unsafe rows.

davies cloud-fan Not sure if my understanding is right, please correct me. Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10398 from gatorsmile/unsafeRowMapGroup.
2015-12-28 12:23:28 -08:00
Yaron Weinsberg 73b70f076d [SPARK-12517] add default RDD name for one created via sc.textFile
The feature was first added at commit: 7b877b2705 but was later removed (probably by mistake) at commit: fc8b58195a.
This change sets the default path of RDDs created via sc.textFile(...) to the path argument.

Here is the symptom:

* Using spark-1.5.2-bin-hadoop2.6:

scala> sc.textFile("/home/root/.bashrc").name
res5: String = null

scala> sc.binaryFiles("/home/root/.bashrc").name
res6: String = /home/root/.bashrc

* while using Spark 1.3.1:

scala> sc.textFile("/home/root/.bashrc").name
res0: String = /home/root/.bashrc

scala> sc.binaryFiles("/home/root/.bashrc").name
res1: String = /home/root/.bashrc

Author: Yaron Weinsberg <wyaron@gmail.com>
Author: yaron <yaron@il.ibm.com>

Closes #10456 from wyaron/master.
2015-12-29 05:19:11 +09:00
Kevin Yu fd50df413f [SPARK-12231][SQL] create a combineFilters' projection when we call buildPartitionedTableScan
Hello Michael & All:

We have some issues to submit the new codes in the other PR(#10299), so we closed that PR and open this one with the fix.

The reason for the previous failure is that the projection for the scan when there is a filter that is not pushed down (the "left-over" filter) could be different, in elements or ordering, from the original projection.

With this new codes, the approach to solve this problem is:

Insert a new Project if the "left-over" filter is nonempty and (the original projection is not empty and the projection for the scan has more than one elements which could otherwise cause different ordering in projection).

We create 3 test cases to cover the otherwise failure cases.

Author: Kevin Yu <qyu@us.ibm.com>

Closes #10388 from kevinyu98/spark-12231.
2015-12-28 11:58:33 -08:00
Wenchen Fan 8543997f2d [HOT-FIX] bypass hive test when parse logical plan to json
https://github.com/apache/spark/pull/10311 introduces some rare, non-deterministic flakiness for hive udf tests, see https://github.com/apache/spark/pull/10311#issuecomment-166548851

I can't reproduce it locally, and may need more time to investigate, a quick solution is: bypass hive tests for json serialization.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10430 from cloud-fan/hot-fix.
2015-12-28 11:45:44 -08:00
Josh Rosen ab6bedd85d [SPARK-12508][PROJECT-INFRA] Fix minor bugs in dev/tests/pr_public_classes.sh script
This patch fixes a handful of minor bugs in the `dev/tests/pr_public_classes.sh` script, which is used by the `run_tests_jenkins` script to detect the addition of new public classes:

- Account for differences between BSD and GNU `sed` in order to allow the script to run on OS X.
- Diff `$ghprbActualCommit^...$ghprbActualCommit ` instead of `master...$ghprbActualCommit`: since `ghprbActualCommit` is a merge commit which results from merging the PR into the target branch, this will give us the desired diff and will avoid certain race-conditions which could lead to false-positives.
- Use `echo -e` instead of `echo` so that newline characters are handled correctly in output. This should fix a formatting glitch which caused the output to appear on a single line in the GitHub comment (see [the SC2028 page](https://github.com/koalaman/shellcheck/wiki/SC2028) on the Shellcheck wiki for more details).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10455 from JoshRosen/fix-pr-public-classes-test.
2015-12-28 10:40:03 -08:00
Cheng Lian 8e23d8db7f [SPARK-12218] Fixes ORC conjunction predicate push down
This PR is a follow-up of PR #10362.

Two major changes:

1.  The fix introduced in #10362 is OK for Parquet, but may disable ORC PPD in many cases

    PR #10362 stops converting an `AND` predicate if any branch is inconvertible.  On the other hand, `OrcFilters` combines all filters into a single big conjunction first and then tries to convert it into ORC `SearchArgument`.  This means, if any filter is inconvertible, no filters can be pushed down.  This PR fixes this issue by finding out all convertible filters first before doing the actual conversion.

    The reason behind the current implementation is mostly due to the limitation of ORC `SearchArgument` builder, which is documented in this PR in detail.

1.  Copied the `AND` predicate fix for ORC from #10362 to avoid merge conflict.

Same as #10362, this PR targets master (2.0.0-SNAPSHOT), branch-1.6, and branch-1.5.

Author: Cheng Lian <lian@databricks.com>

Closes #10377 from liancheng/spark-12218.fix-orc-conjunction-ppd.
2015-12-28 08:48:44 -08:00
jerryshao 8d49400921 [SPARK-12353][STREAMING][PYSPARK] Fix countByValue inconsistent output in Python API
The semantics of Python countByValue is different from Scala API, it is more like countDistinctValue, so here change to make it consistent with Scala/Java API.

Author: jerryshao <sshao@hortonworks.com>

Closes #10350 from jerryshao/SPARK-12353.
2015-12-28 10:43:23 +00:00
felixcheung 5aa2710c1e [SPARK-12515][SQL][DOC] minor doc update for read.jdbc
Author: felixcheung <felixcheung_m@hotmail.com>

Closes #10465 from felixcheung/dfreaderjdbcdoc.
2015-12-28 10:22:45 +00:00
gatorsmile 9ab296ecdc [SPARK-12520] [PYSPARK] Correct Descriptions and Add Use Cases in Equi-Join
After reading the JIRA https://issues.apache.org/jira/browse/SPARK-12520, I double checked the code.

For example, users can do the Equi-Join like
  ```df.join(df2, 'name', 'outer').select('name', 'height').collect()```
- There exists a bug in 1.5 and 1.4. The code just ignores the third parameter (join type) users pass. However, the join type we called is `Inner`, even if the user-specified type is the other type (e.g., `Outer`).
- After a PR: https://github.com/apache/spark/pull/8600, the 1.6 does not have such an issue, but the description has not been updated.

Plan to submit another PR to fix 1.5 and issue an error message if users specify a non-inner join type when using Equi-Join.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10477 from gatorsmile/pyOuterJoin.
2015-12-27 23:18:48 -08:00
echo2mei 1e97813951 [SPARK-12396][CORE] Modify the function scheduleAtFixedRate to schedule.
Instead of just cancel the registrationRetryTimer to avoid driver retry connect to master, change the function to schedule.
It is no need to register to master iteratively.

Author: echo2mei <534384876@qq.com>

Closes #10447 from echoTomei/master.
2015-12-25 17:42:24 -08:00
pierre-borckmans ea4aab7e87 [SPARK-12440][CORE] Avoid setCheckpoint warning when directory is not local
In SparkContext method `setCheckpointDir`, a warning is issued when spark master is not local and the passed directory for the checkpoint dir appears to be local.

In practice, when relying on HDFS configuration file and using a relative path for the checkpoint directory (using an incomplete URI without HDFS scheme, ...), this warning should not be issued and might be confusing.
In fact, in this case, the checkpoint directory is successfully created, and the checkpointing mechanism works as expected.

This PR uses the `FileSystem` instance created with the given directory, and checks whether it is local or not.
(The rationale is that since this same `FileSystem` instance is used to create the checkpoint dir anyway and can therefore be reliably used to determine if it is local or not).

The warning is only issued if the directory is not local, on top of the existing conditions.

Author: pierre-borckmans <pierre.borckmans@realimpactanalytics.com>

Closes #10392 from pierre-borckmans/SPARK-12440_CheckpointDir_Warning_NonLocal.
2015-12-24 13:48:21 +00:00
CK50 502476e45c [SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT syntax
In the past Spark JDBC write only worked with technologies which support the following INSERT statement syntax (JdbcUtils.scala: insertStatement()):

INSERT INTO $table VALUES ( ?, ?, ..., ? )

But some technologies require a list of column names:

INSERT INTO $table ( $colNameList ) VALUES ( ?, ?, ..., ? )

This was blocking the use of e.g. the Progress JDBC Driver for Cassandra.

Another limitation is that syntax 1 relies no the dataframe field ordering match that of the target table. This works fine, as long as the target table has been created by writer.jdbc().

If the target table contains more columns (not created by writer.jdbc()), then the insert fails due mismatch of number of columns or their data types.

This PR switches to the recommended second INSERT syntax. Column names are taken from datafram field names.

Author: CK50 <christian.kurz@oracle.com>

Closes #10380 from CK50/master-SPARK-12010-2.
2015-12-24 13:39:11 +00:00
Kazuaki Ishizaki 3920466118 [SPARK-12311][CORE] Restore previous value of "os.arch" property in test suites after forcing to set specific value to "os.arch" property
Restore the original value of os.arch property after each test

Since some of tests forced to set the specific value to os.arch property, we need to set the original value.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #10289 from kiszk/SPARK-12311.
2015-12-24 13:37:28 +00:00
Kazuaki Ishizaki 9e85bb71ad [SPARK-12502][BUILD][PYTHON] Script /dev/run-tests fails when IBM Java is used
fix an exception with IBM JDK by removing update field from a JavaVersion tuple. This is because IBM JDK does not have information on update '_xx'

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #10463 from kiszk/SPARK-12502.
2015-12-24 21:27:55 +09:00
Adrian Bridgett ead6abf7e7 [SPARK-12499][BUILD] don't force MAVEN_OPTS
allow the user to override MAVEN_OPTS (2GB wasn't sufficient for me)

Author: Adrian Bridgett <adrian@smop.co.uk>

Closes #10448 from abridgett/feature/do_not_force_maven_opts.
2015-12-23 16:00:03 -08:00
Sean Owen ae1f54aa0e [SPARK-12500][CORE] Fix Tachyon deprecations; pull Tachyon dependency into one class
Fix Tachyon deprecations; pull Tachyon dependency into `TachyonBlockManager` only

CC calvinjia as I probably need a double-check that the usage of the new API is correct.

Author: Sean Owen <sowen@cloudera.com>

Closes #10449 from srowen/SPARK-12500.
2015-12-23 13:24:06 -08:00
pierre-borckmans 43b2a63900 [SPARK-12477][SQL] - Tungsten projection fails for null values in array fields
Accessing null elements in an array field fails when tungsten is enabled.
It works in Spark 1.3.1, and in Spark > 1.5 with Tungsten disabled.

This PR solves this by checking if the accessed element in the array field is null, in the generated code.

Example:
```
// Array of String
case class AS( as: Seq[String] )
val dfAS = sc.parallelize( Seq( AS ( Seq("a",null,"b") ) ) ).toDF
dfAS.registerTempTable("T_AS")
for (i <- 0 to 2) { println(i + " = " + sqlContext.sql(s"select as[$i] from T_AS").collect.mkString(","))}
```

With Tungsten disabled:
```
0 = [a]
1 = [null]
2 = [b]
```

With Tungsten enabled:
```
0 = [a]
15/12/22 09:32:50 ERROR Executor: Exception in task 7.0 in stage 1.0 (TID 15)
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.UnsafeRowWriters$UTF8StringWriter.getSize(UnsafeRowWriters.java:90)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.TungstenProject$$anonfun$3$$anonfun$apply$3.apply(basicOperators.scala:90)
	at org.apache.spark.sql.execution.TungstenProject$$anonfun$3$$anonfun$apply$3.apply(basicOperators.scala:88)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
```

Author: pierre-borckmans <pierre.borckmans@realimpactanalytics.com>

Closes #10429 from pierre-borckmans/SPARK-12477_Tungsten-Projection-Null-Element-In-Array.
2015-12-22 23:00:42 -08:00
Liang-Chi Hsieh 50301c0a28 [SPARK-11164][SQL] Add InSet pushdown filter back for Parquet
When the filter is ```"b in ('1', '2')"```, the filter is not pushed down to Parquet. Thanks!

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10278 from gatorsmile/parquetFilterNot.
2015-12-23 14:08:29 +08:00
Cheng Lian 86761e10e1 [SPARK-12478][SQL] Bugfix: Dataset fields of product types can't be null
When creating extractors for product types (i.e. case classes and tuples), a null check is missing, thus we always assume input product values are non-null.

This PR adds a null check in the extractor expression for product types. The null check is stripped off for top level product fields, which are mapped to the outermost `Row`s, since they can't be null.

Thanks cloud-fan for helping investigating this issue!

Author: Cheng Lian <lian@databricks.com>

Closes #10431 from liancheng/spark-12478.top-level-null-field.
2015-12-23 10:21:00 +08:00
Shixiong Zhu 20591afd79 [SPARK-12429][STREAMING][DOC] Add Accumulator and Broadcast example for Streaming
This PR adds Scala, Java and Python examples to show how to use Accumulator and Broadcast in Spark Streaming to support checkpointing.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10385 from zsxwing/accumulator-broadcast-example.
2015-12-22 16:39:10 -08:00
Shixiong Zhu 93db50d1c2 [SPARK-12487][STREAMING][DOCUMENT] Add docs for Kafka message handler
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10439 from zsxwing/kafka-message-handler-doc.
2015-12-22 15:33:30 -08:00
Dilip Biswal b374a25831 [SPARK-12102][SQL] Cast a non-nullable struct field to a nullable field during analysis
Compare both left and right side of the case expression ignoring nullablity when checking for type equality.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #10156 from dilipbiswal/spark-12102.
2015-12-22 15:21:49 -08:00
Nong Li 575a132797 [SPARK-12471][CORE] Spark daemons will log their pid on start up.
Author: Nong Li <nong@databricks.com>

Closes #10422 from nongli/12471-pids.
2015-12-22 13:27:28 -08:00
Jacek Laskowski 7c970f9093 Minor corrections, i.e. typo fixes and follow deprecated
Author: Jacek Laskowski <jacek@japila.pl>

Closes #10432 from jaceklaskowski/minor-corrections.
2015-12-22 10:47:10 -08:00
Xiu Guo b5ce84a1bb [SPARK-12456][SQL] Add ExpressionDescription to misc functions
First try, not sure how much information we need to provide in the usage part.

Author: Xiu Guo <xguo27@gmail.com>

Closes #10423 from xguo27/SPARK-12456.
2015-12-22 10:44:01 -08:00
Josh Rosen bc0f30d0f5 [SPARK-12475][BUILD] Upgrade Zinc from 0.3.5.3 to 0.3.9
We should update to the latest version of Zinc in order to match our SBT version.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10426 from JoshRosen/update-zinc.
2015-12-22 10:23:24 -08:00
hyukjinkwon 364d244a50 [SPARK-11677][SQL][FOLLOW-UP] Add tests for checking the ORC filter creation against pushed down filters.
https://issues.apache.org/jira/browse/SPARK-11677
Although it checks correctly the filters by the number of results if ORC filter-push-down is enabled, the filters themselves are not being tested.
So, this PR includes the test similarly with `ParquetFilterSuite`.
Since the results are checked by `OrcQuerySuite`, this `OrcFilterSuite` only checks if the appropriate filters are created.

One thing different with `ParquetFilterSuite` here is, it does not check the results because that is checked in `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10341 from HyukjinKwon/SPARK-11677-followup.
2015-12-23 00:39:49 +08:00
Cheng Lian 42bfde2983 [SPARK-12371][SQL] Runtime nullability check for NewInstance
This PR adds a new expression `AssertNotNull` to ensure non-nullable fields of products and case classes don't receive null values at runtime.

Author: Cheng Lian <lian@databricks.com>

Closes #10331 from liancheng/dataset-nullability-check.
2015-12-22 19:41:44 +08:00
Takeshi YAMAMURO 8c1b867cee [SPARK-12446][SQL] Add unit tests for JDBCRDD internal functions
No tests done for JDBCRDD#compileFilter.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #10409 from maropu/AddTestsInJdbcRdd.
2015-12-22 00:50:05 -08:00
Holden Karau 969d5665bb [SPARK-12296][PYSPARK][MLLIB] Feature parity for pyspark mllib standard scaler model
Some methods are missing, such as ways to access the std, mean, etc. This PR is for feature parity for pyspark.mllib.feature.StandardScaler & StandardScalerModel.

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

Closes #10298 from holdenk/SPARK-12296-feature-parity-pyspark-mllib-StandardScalerModel.
2015-12-22 09:14:12 +02:00
Josh Rosen 2235cd4440 [SPARK-11823][SQL] Fix flaky JDBC cancellation test in HiveThriftBinaryServerSuite
This patch fixes a flaky "test jdbc cancel" test in HiveThriftBinaryServerSuite. This test is prone to a race-condition which causes it to block indefinitely with while waiting for an extremely slow query to complete, which caused many Jenkins builds to time out.

For more background, see my comments on #6207 (the PR which introduced this test).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10425 from JoshRosen/SPARK-11823.
2015-12-21 23:12:05 -08:00
Shixiong Zhu 93da8565fe [MINOR] Fix typos in JavaStreamingContext
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10424 from zsxwing/typo.
2015-12-21 22:28:18 -08:00
Reynold Xin 0a38637d05 [SPARK-11807] Remove support for Hadoop < 2.2
i.e. Hadoop 1 and Hadoop 2.0

Author: Reynold Xin <rxin@databricks.com>

Closes #10404 from rxin/SPARK-11807.
2015-12-21 22:15:52 -08:00
Davies Liu 29cecd4a42 [SPARK-12388] change default compression to lz4
According the benchmark [1], LZ4-java could be 80% (or 30%) faster than Snappy.

After changing the compressor to LZ4, I saw 20% improvement on end-to-end time for a TPCDS query (Q4).

[1] https://github.com/ning/jvm-compressor-benchmark/wiki

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #10342 from davies/lz4.
2015-12-21 14:21:43 -08:00
Andrew Or d655d37ddf [SPARK-12466] Fix harmless NPE in tests
```
[info] ReplayListenerSuite:
[info] - Simple replay (58 milliseconds)
java.lang.NullPointerException
	at org.apache.spark.deploy.master.Master$$anonfun$asyncRebuildSparkUI$1.applyOrElse(Master.scala:982)
	at org.apache.spark.deploy.master.Master$$anonfun$asyncRebuildSparkUI$1.applyOrElse(Master.scala:980)
```
https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Test/job/Spark-Master-SBT/4316/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.2,label=spark-test/consoleFull

This was introduced in #10284. It's harmless because the NPE is caused by a race that occurs mainly in `local-cluster` tests (but don't actually fail the tests).

Tested locally to verify that the NPE is gone.

Author: Andrew Or <andrew@databricks.com>

Closes #10417 from andrewor14/fix-harmless-npe.
2015-12-21 14:09:04 -08:00
Reynold Xin a820ca19de [SPARK-2331] SparkContext.emptyRDD should return RDD[T] not EmptyRDD[T]
Author: Reynold Xin <rxin@databricks.com>

Closes #10394 from rxin/SPARK-2331.
2015-12-21 14:07:48 -08:00
Alex Bozarth b0849b8aea [SPARK-12339][SPARK-11206][WEBUI] Added a null check that was removed in
Updates made in SPARK-11206 missed an edge case which cause's a NullPointerException when a task is killed. In some cases when a task ends in failure taskMetrics is initialized as null (see JobProgressListener.onTaskEnd()). To address this a null check was added. Before the changes in SPARK-11206 this null check was called at the start of the updateTaskAccumulatorValues() function.

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

Closes #10405 from ajbozarth/spark12339.
2015-12-21 14:06:36 -08:00
pshearer fc6dbcc703 Doc typo: ltrim = trim from left end, not right
Author: pshearer <pshearer@massmutual.com>

Closes #10414 from pshearer/patch-1.
2015-12-21 14:04:59 -08:00
Takeshi YAMAMURO 1eb90bc9ca [SPARK-5882][GRAPHX] Add a test for GraphLoader.edgeListFile
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #4674 from maropu/AddGraphLoaderSuite.
2015-12-21 14:04:23 -08:00
Takeshi YAMAMURO 935f466306 [SPARK-12392][CORE] Optimize a location order of broadcast blocks by considering preferred local hosts
When multiple workers exist in a host, we can bypass unnecessary remote access for broadcasts; block managers fetch broadcast blocks from the same host instead of remote hosts.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #10346 from maropu/OptimizeBlockLocationOrder.
2015-12-21 14:03:23 -08:00