Commit graph

9310 commits

Author SHA1 Message Date
Yuhao Yang 76389c5b99 [SPARK-5234][ml]examples for ml don't have sparkContext.stop
JIRA issue: https://issues.apache.org/jira/browse/SPARK-5234

simply add the call.

Author: Yuhao Yang <yuhao@yuhaodevbox.sh.intel.com>

Closes #4044 from hhbyyh/addscStop and squashes the following commits:

c1f75ac [Yuhao Yang] add SparkContext.stop to 3 ml examples
2015-01-14 11:53:43 -08:00
Alex Baretta 2fd7f72b6b [SPARK-5235] Make SQLConf Serializable
Declare SQLConf to be serializable to fix "Task not serializable" exceptions in SparkSQL

Author: Alex Baretta <alexbaretta@gmail.com>

Closes #4031 from alexbaretta/SPARK-5235-SQLConf and squashes the following commits:

c2103f5 [Alex Baretta] [SPARK-5235] Make SQLConf Serializable
2015-01-14 11:51:55 -08:00
Josh Rosen 259936be71 [SPARK-4014] Add TaskContext.attemptNumber and deprecate TaskContext.attemptId
`TaskContext.attemptId` is misleadingly-named, since it currently returns a taskId, which uniquely identifies a particular task attempt within a particular SparkContext, instead of an attempt number, which conveys how many times a task has been attempted.

This patch deprecates `TaskContext.attemptId` and add `TaskContext.taskId` and `TaskContext.attemptNumber` fields.  Prior to this change, it was impossible to determine whether a task was being re-attempted (or was a speculative copy), which made it difficult to write unit tests for tasks that fail on early attempts or speculative tasks that complete faster than original tasks.

Earlier versions of the TaskContext docs suggest that `attemptId` behaves like `attemptNumber`, so there's an argument to be made in favor of changing this method's implementation.  Since we've decided against making that change in maintenance branches, I think it's simpler to add better-named methods and retain the old behavior for `attemptId`; if `attemptId` behaved differently in different branches, then this would cause confusing build-breaks when backporting regression tests that rely on the new `attemptId` behavior.

Most of this patch is fairly straightforward, but there is a bit of trickiness related to Mesos tasks: since there's no field in MesosTaskInfo to encode the attemptId, I packed it into the `data` field alongside the task binary.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3849 from JoshRosen/SPARK-4014 and squashes the following commits:

89d03e0 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
5cfff05 [Josh Rosen] Introduce wrapper for serializing Mesos task launch data.
38574d4 [Josh Rosen] attemptId -> taskAttemptId in PairRDDFunctions
a180b88 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
1d43aa6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
eee6a45 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
0b10526 [Josh Rosen] Use putInt instead of putLong (silly mistake)
8c387ce [Josh Rosen] Use local with maxRetries instead of local-cluster.
cbe4d76 [Josh Rosen] Preserve attemptId behavior and deprecate it:
b2dffa3 [Josh Rosen] Address some of Reynold's minor comments
9d8d4d1 [Josh Rosen] Doc typo
1e7a933 [Josh Rosen] [SPARK-4014] Change TaskContext.attemptId to return attempt number instead of task ID.
fd515a5 [Josh Rosen] Add failing test for SPARK-4014
2015-01-14 11:45:40 -08:00
Kousuke Saruta 9d4449c4b3 [SPARK-5228][WebUI] Hide tables for "Active Jobs/Completed Jobs/Failed Jobs" when they are empty
In current WebUI, tables for Active Stages, Completed Stages, Skipped Stages and Failed Stages are hidden when they are empty while tables for Active Jobs, Completed Jobs and Failed Jobs are not hidden though they are empty.

This is before my patch is applied.

![2015-01-13 14 13 03](https://cloud.githubusercontent.com/assets/4736016/5730793/2b73d6f4-9b32-11e4-9a24-1784d758c644.png)

And this is after my patch is applied.

![2015-01-13 14 38 13](https://cloud.githubusercontent.com/assets/4736016/5730797/359ea2da-9b32-11e4-97b0-544739ddbf4c.png)

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

Closes #4028 from sarutak/SPARK-5228 and squashes the following commits:

b1e6e8b [Kousuke Saruta] Fixed a small typo
daab563 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5228
9493a1d [Kousuke Saruta] Modified AllJobPage.scala so that hide Active Jobs/Completed Jobs/Failed Jobs when they are empty
2015-01-14 11:10:29 -08:00
MechCoder 5840f5464b [SPARK-2909] [MLlib] [PySpark] SparseVector in pyspark now supports indexing
Slightly different than the scala code which converts the sparsevector into a densevector and then checks the index.

I also hope I've added tests in the right place.

Author: MechCoder <manojkumarsivaraj334@gmail.com>

Closes #4025 from MechCoder/spark-2909 and squashes the following commits:

07d0f26 [MechCoder] STY: Rename item to index
f02148b [MechCoder] [SPARK-2909] [Mlib] SparseVector in pyspark now supports indexing
2015-01-14 11:03:11 -08:00
Daoyuan Wang 38bdc992a1 [SQL] some comments fix for GROUPING SETS
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4000 from adrian-wang/comment and squashes the following commits:

9c24fc4 [Daoyuan Wang] some comments
2015-01-14 09:50:01 -08:00
Yin Huai 81f72a0df2 [SPARK-5211][SQL]Restore HiveMetastoreTypes.toDataType
jira: https://issues.apache.org/jira/browse/SPARK-5211

Author: Yin Huai <yhuai@databricks.com>

Closes #4026 from yhuai/SPARK-5211 and squashes the following commits:

15ee32b [Yin Huai] Remove extra line.
c6c1651 [Yin Huai] Get back HiveMetastoreTypes.toDataType.
2015-01-14 09:47:30 -08:00
Daoyuan Wang a3f7421b42 [SPARK-5248] [SQL] move sql.types.decimal.Decimal to sql.types.Decimal
rxin follow up of #3732

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

Closes #4041 from adrian-wang/decimal and squashes the following commits:

aa3d738 [Daoyuan Wang] fix auto refactor
7777a58 [Daoyuan Wang] move sql.types.decimal.Decimal to sql.types.Decimal
2015-01-14 09:36:59 -08:00
Reynold Xin d5eeb35167 [SPARK-5167][SQL] Move Row into sql package and make it usable for Java.
Mostly just moving stuff around. This should still be source compatible since we type aliased Row previously in org.apache.spark.sql.Row.

Added the following APIs to Row:
```scala
def getMap[K, V](i: Int): scala.collection.Map[K, V]
def getJavaMap[K, V](i: Int): java.util.Map[K, V]
def getSeq[T](i: Int): Seq[T]
def getList[T](i: Int): java.util.List[T]
def getStruct(i: Int): StructType
```

Author: Reynold Xin <rxin@databricks.com>

Closes #4030 from rxin/sql-row and squashes the following commits:

6c85c29 [Reynold Xin] Fixed style violation by adding a new line to Row.scala.
82b064a [Reynold Xin] [SPARK-5167][SQL] Move Row into sql package and make it usable for Java.
2015-01-14 00:38:55 -08:00
Reynold Xin f9969098c8 [SPARK-5123][SQL] Reconcile Java/Scala API for data types.
Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box.

As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code.

This subsumes https://github.com/apache/spark/pull/3925

Author: Reynold Xin <rxin@databricks.com>

Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits:

66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API).
2015-01-13 17:16:41 -08:00
Reynold Xin 14e3f114ef [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext
This change should be binary and source backward compatible since we didn't change any user facing APIs.

Author: Reynold Xin <rxin@databricks.com>

Closes #3965 from rxin/SPARK-5168-sqlconf and squashes the following commits:

42eec09 [Reynold Xin] Fix default conf value.
0ef86cc [Reynold Xin] Fix constructor ordering.
4d7f910 [Reynold Xin] Properly override config.
ccc8e6a [Reynold Xin] [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext
2015-01-13 13:30:35 -08:00
Yin Huai 6463e0b9e8 [SPARK-4912][SQL] Persistent tables for the Spark SQL data sources api
With changes in this PR, users can persist metadata of tables created based on the data source API in metastore through DDLs.

Author: Yin Huai <yhuai@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3960 from yhuai/persistantTablesWithSchema2 and squashes the following commits:

069c235 [Yin Huai] Make exception messages user friendly.
c07cbc6 [Yin Huai] Get the location of test file in a correct way.
4456e98 [Yin Huai] Test data.
5315dfc [Yin Huai] rxin's comments.
7fc4b56 [Yin Huai] Add DDLStrategy and HiveDDLStrategy to plan DDLs based on the data source API.
aeaf4b3 [Yin Huai] Add comments.
06f9b0c [Yin Huai] Revert unnecessary changes.
feb88aa [Yin Huai] Merge remote-tracking branch 'apache/master' into persistantTablesWithSchema2
172db80 [Yin Huai] Fix unit test.
49bf1ac [Yin Huai] Unit tests.
8f8f1a1 [Yin Huai] [SPARK-4574][SQL] Adding support for defining schema in foreign DDL commands. #3431
f47fda1 [Yin Huai] Unit tests.
2b59723 [Michael Armbrust] Set external when creating tables
c00bb1b [Michael Armbrust] Don't use reflection to read options
1ea6e7b [Michael Armbrust] Don't fail when trying to uncache a table that doesn't exist
6edc710 [Michael Armbrust] Add tests.
d7da491 [Michael Armbrust] First draft of persistent tables.
2015-01-13 13:01:27 -08:00
Davies Liu 8ead999fd6 [SPARK-5223] [MLlib] [PySpark] fix MapConverter and ListConverter in MLlib
It will introduce problems if the object in dict/list/tuple can not support by py4j, such as Vector.
Also, pickle may have better performance for larger object (less RPC).

In some cases that the object in dict/list can not be pickled (such as JavaObject), we should still use MapConvert/ListConvert.

This PR should be ported into branch-1.2

Author: Davies Liu <davies@databricks.com>

Closes #4023 from davies/listconvert and squashes the following commits:

55d4ab2 [Davies Liu] fix MapConverter and ListConverter in MLlib
2015-01-13 12:50:31 -08:00
uncleGen 39e333ec43 [SPARK-5131][Streaming][DOC]: There is a discrepancy in WAL implementation and configuration doc.
There is a discrepancy in WAL implementation and configuration doc.

Author: uncleGen <hustyugm@gmail.com>

Closes #3930 from uncleGen/master-clean-doc and squashes the following commits:

3a4245f [uncleGen] doc typo
8e407d3 [uncleGen] doc typo
2015-01-13 10:07:19 -08:00
WangTaoTheTonic 9dea64e53a [SPARK-4697][YARN]System properties should override environment variables
I found some arguments in yarn module take environment variables before system properties while the latter override the former in core module.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3557 from WangTaoTheTonic/SPARK4697 and squashes the following commits:

836b9ef [WangTaoTheTonic] fix type mismatch
e3e486a [WangTaoTheTonic] remove the comma
1262d57 [WangTaoTheTonic] handle spark.app.name and SPARK_YARN_APP_NAME in SparkSubmitArguments
bee9447 [WangTaoTheTonic] wrong brace
81833bb [WangTaoTheTonic] rebase
40934b4 [WangTaoTheTonic] just switch blocks
5f43f45 [WangTao] System property can override environment variable
2015-01-13 09:50:14 -08:00
WangTaoTheTonic f7741a9a72 [SPARK-5006][Deploy]spark.port.maxRetries doesn't work
https://issues.apache.org/jira/browse/SPARK-5006

I think the issue is produced in https://github.com/apache/spark/pull/1777.

Not digging mesos's backend yet. Maybe should add same logic either.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits:

8cdf96d [WangTao] indent thing
2d86d65 [WangTaoTheTonic] fix line length
7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor
61a370d [WangTaoTheTonic] some minor fixes
bc6e1ec [WangTaoTheTonic] rebase
67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments
f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg
29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable
396c226 [WangTaoTheTonic] make the grammar more like scala
191face [WangTaoTheTonic] invalid value name
62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work
2015-01-13 09:29:25 -08:00
Gabe Mulley 1e42e96ece [SPARK-5138][SQL] Ensure schema can be inferred from a namedtuple
When attempting to infer the schema of an RDD that contains namedtuples, pyspark fails to identify the records as namedtuples, resulting in it raising an error.

Example:

```python
from pyspark import SparkContext
from pyspark.sql import SQLContext
from collections import namedtuple
import os

sc = SparkContext()
rdd = sc.textFile(os.path.join(os.getenv('SPARK_HOME'), 'README.md'))
TextLine = namedtuple('TextLine', 'line length')
tuple_rdd = rdd.map(lambda l: TextLine(line=l, length=len(l)))
tuple_rdd.take(5)  # This works

sqlc = SQLContext(sc)

# The following line raises an error
schema_rdd = sqlc.inferSchema(tuple_rdd)
```

The error raised is:
```
  File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/worker.py", line 107, in main
    process()
  File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/worker.py", line 98, in process
    serializer.dump_stream(func(split_index, iterator), outfile)
  File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/serializers.py", line 227, in dump_stream
    vs = list(itertools.islice(iterator, batch))
  File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/rdd.py", line 1107, in takeUpToNumLeft
    yield next(iterator)
  File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/sql.py", line 816, in convert_struct
    raise ValueError("unexpected tuple: %s" % obj)
TypeError: not all arguments converted during string formatting
```

Author: Gabe Mulley <gabe@edx.org>

Closes #3978 from mulby/inferschema-namedtuple and squashes the following commits:

98c61cc [Gabe Mulley] Ensure exception message is populated correctly
375d96b [Gabe Mulley] Ensure schema can be inferred from a namedtuple
2015-01-12 21:44:51 -08:00
Michael Armbrust 5d9fa55082 [SPARK-5049][SQL] Fix ordering of partition columns in ParquetTableScan
Followup to #3870.  Props to rahulaggarwalguavus for identifying the issue.

Author: Michael Armbrust <michael@databricks.com>

Closes #3990 from marmbrus/SPARK-5049 and squashes the following commits:

dd03e4e [Michael Armbrust] Fill in the partition values of parquet scans instead of using JoinedRow
2015-01-12 15:19:09 -08:00
jerryshao 3aed3051c0 [SPARK-4999][Streaming] Change storeInBlockManager to false by default
Currently WAL-backed block is read out from HDFS and put into BlockManger with storage level MEMORY_ONLY_SER by default, since WAL-backed block is already materialized in HDFS with fault-tolerance, no need to put into BlockManger again by default.

Author: jerryshao <saisai.shao@intel.com>

Closes #3906 from jerryshao/SPARK-4999 and squashes the following commits:

b95f95e [jerryshao] Change storeInBlockManager to false by default
2015-01-12 13:14:44 -08:00
Sean Owen aff49a3ee1 SPARK-5172 [BUILD] spark-examples-***.jar shades a wrong Hadoop distribution
In addition to the `hadoop-2.x` profiles in the parent POM, there is actually another set of profiles in `examples` that has to be activated differently to get the right Hadoop 1 vs 2 flavor of HBase. This wasn't actually used in making Hadoop 2 distributions, hence the problem.

To reduce complexity, I suggest merging them with the parent POM profiles, which is possible now.

You'll see this changes appears to update the HBase version, but actually, the default 0.94 version was not being used. HBase is only used in examples, and the examples POM always chose one profile or the other that updated the version to 0.98.x anyway.

Author: Sean Owen <sowen@cloudera.com>

Closes #3992 from srowen/SPARK-5172 and squashes the following commits:

17830d9 [Sean Owen] Control hbase hadoop1/2 flavor in the parent POM with existing hadoop-2.x profiles
2015-01-12 12:15:34 -08:00
Michael Armbrust a3978f3e15 [SPARK-5078] Optionally read from SPARK_LOCAL_HOSTNAME
Current spark lets you set the ip address using SPARK_LOCAL_IP, but then this is given to akka after doing a reverse DNS lookup. This makes it difficult to run spark in Docker. You can already change the hostname that is used programmatically, but it would be nice to be able to do this with an environment variable as well.

Author: Michael Armbrust <michael@databricks.com>

Closes #3893 from marmbrus/localHostnameEnv and squashes the following commits:

85045b6 [Michael Armbrust] Optionally read from SPARK_LOCAL_HOSTNAME
2015-01-12 11:57:59 -08:00
Sean Owen 13e610b88e SPARK-4159 [BUILD] Addendum: improve running of single test after enabling Java tests
https://issues.apache.org/jira/browse/SPARK-4159 was resolved but as Sandy points out, the guidance in https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools under "Running Individual Tests" no longer quite works, not optimally.

This minor change is not really the important change, which is an update to the wiki text. The correct way to run one Scala test suite in Maven is now:

```
mvn test -DwildcardSuites=org.apache.spark.io.CompressionCodecSuite -Dtests=none
```

The correct way to run one Java test is

```
mvn test -DwildcardSuites=none -Dtests=org.apache.spark.streaming.JavaAPISuite
```

Basically, you have to set two properties in order to suppress all of one type of test (with a non-existent test name like 'none') and all but one test of the other type.

The change in the PR just prevents Surefire from barfing when it finds no "none" test.

Author: Sean Owen <sowen@cloudera.com>

Closes #3993 from srowen/SPARK-4159 and squashes the following commits:

83106d7 [Sean Owen] Default failIfNoTests to false to enable the -DwildcardSuites=... -Dtests=... syntax for running one test to work
2015-01-12 11:01:00 -08:00
lianhuiwang ef9224e080 [SPARK-5102][Core]subclass of MapStatus needs to be registered with Kryo
CompressedMapStatus and HighlyCompressedMapStatus needs to be registered with Kryo, because they are subclass of MapStatus.

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #4007 from lianhuiwang/SPARK-5102 and squashes the following commits:

9d2238a [lianhuiwang] remove register of MapStatus
05a285d [lianhuiwang] subclass of MapStatus needs to be registered with Kryo
2015-01-12 10:57:12 -08:00
Josh Rosen 82fd38dcdc [SPARK-5200] Disable web UI in Hive ThriftServer tests
Disables the Spark web UI in HiveThriftServer2Suite in order to prevent Jenkins test failures due to port contention.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3998 from JoshRosen/SPARK-5200 and squashes the following commits:

a384416 [Josh Rosen] [SPARK-5200] Disable web UI in Hive Thriftserver tests.
2015-01-12 10:47:12 -08:00
Travis Galoppo 2130de9d8f SPARK-5018 [MLlib] [WIP] Make MultivariateGaussian public
Moving MutlivariateGaussian from private[mllib] to public.  The class uses Breeze vectors internally, so this involves creating a public interface using MLlib vectors and matrices.

This initial commit provides public construction, accessors for mean/covariance, density and log-density.

Other potential methods include entropy and sample generation.

Author: Travis Galoppo <tjg2107@columbia.edu>

Closes #3923 from tgaloppo/spark-5018 and squashes the following commits:

2b15587 [Travis Galoppo] Style correction
b4121b4 [Travis Galoppo] Merge remote-tracking branch 'upstream/master' into spark-5018
e30a100 [Travis Galoppo] Made mu, sigma private[mllib] members of MultivariateGaussian Moved MultivariateGaussian (and test suite) from stat.impl to stat.distribution (required updates in GaussianMixture{EM,Model}.scala) Marked MultivariateGaussian as @DeveloperApi Fixed style error
9fa3bb7 [Travis Galoppo] Style improvements
91a5fae [Travis Galoppo] Rearranged equation for part of density function
8c35381 [Travis Galoppo] Fixed accessor methods to match member variable names. Modified calculations to avoid log(pow(x,y)) calculations
0943dc4 [Travis Galoppo] SPARK-5018
4dee9e1 [Travis Galoppo] SPARK-5018
2015-01-11 21:31:16 -08:00
huangzhaowei f38ef6586c [SPARK-4033][Examples]Input of the SparkPi too big causes the emption exception
If input of the SparkPi args is larger than the 25000, the integer 'n' inside the code will be overflow, and may be a negative number.
And it causes  the (0 until n) Seq as an empty seq, then doing the action 'reduce'  will throw the UnsupportedOperationException("empty collection").

The max size of the input of sc.parallelize is Int.MaxValue - 1, not the Int.MaxValue.

Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #2874 from SaintBacchus/SparkPi and squashes the following commits:

62d7cd7 [huangzhaowei] Add a commit to explain the modify
4cdc388 [huangzhaowei] Update SparkPi.scala
9a2fb7b [huangzhaowei] Input of the SparkPi is too big
2015-01-11 16:32:47 -08:00
zsxwing 6942b974ad [SPARK-4951][Core] Fix the issue that a busy executor may be killed
A few changes to fix this issue:

1. Handle the case that receiving `SparkListenerTaskStart` before `SparkListenerBlockManagerAdded`.
2. Don't add `executorId` to `removeTimes` when the executor is busy.
3. Use `HashMap.retain` to safely traverse the HashMap and remove items.
4. Use the same lock in ExecutorAllocationManager and ExecutorAllocationListener to fix the race condition in `totalPendingTasks`.
5. Move the blocking codes out of the message processing code in YarnSchedulerActor.

Author: zsxwing <zsxwing@gmail.com>

Closes #3783 from zsxwing/SPARK-4951 and squashes the following commits:

d51fa0d [zsxwing] Add comments
2e365ce [zsxwing] Remove expired executors from 'removeTimes' and add idle executors back when a new executor joins
49f61a9 [zsxwing] Eliminate duplicate executor registered warnings
d4c4e9a [zsxwing] Minor fixes for the code style
05f6238 [zsxwing] Move the blocking codes out of the message processing code
105ba3a [zsxwing] Fix the race condition in totalPendingTasks
d5c615d [zsxwing] Fix the issue that a busy executor may be killed
2015-01-11 16:23:28 -08:00
lewuathe 1656aae2b4 [SPARK-5073] spark.storage.memoryMapThreshold have two default value
Because major OS page sizes is about 4KB, the default value of spark.storage.memoryMapThreshold is integrated to 2 * 4096

Author: lewuathe <lewuathe@me.com>

Closes #3900 from Lewuathe/integrate-memoryMapThreshold and squashes the following commits:

e417acd [lewuathe] [SPARK-5073] Update docs/configuration
834aba4 [lewuathe] [SPARK-5073] Fix style
adcea33 [lewuathe] [SPARK-5073] Integrate memory map threshold to 2MB
fcce2e5 [lewuathe] [SPARK-5073] spark.storage.memoryMapThreshold have two default value
2015-01-11 13:50:42 -08:00
Joseph K. Bradley 3313260909 [SPARK-5032] [graphx] Remove GraphX MIMA exclude for 1.3
Since GraphX is no longer alpha as of 1.2, MimaExcludes should not exclude GraphX for 1.3

Here are the individual excludes I had to add + the associated commits:

```
            // SPARK-4444
            ProblemFilters.exclude[IncompatibleResultTypeProblem](
              "org.apache.spark.graphx.EdgeRDD.fromEdges"),
            ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.EdgeRDD.filter"),
            ProblemFilters.exclude[IncompatibleResultTypeProblem](
              "org.apache.spark.graphx.impl.EdgeRDDImpl.filter"),
```
[9ac2bb18ed]

```
            // SPARK-3623
            ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.checkpoint")
```
[e895e0cbec]

```
            // SPARK-4620
            ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.unpersist"),
```
[8817fc7fe8]

CC: rxin

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #3856 from jkbradley/graphx-mima and squashes the following commits:

1eea2f6 [Joseph K. Bradley] moved cleanup to run-tests
527ccd9 [Joseph K. Bradley] fixed jenkins script to remove ivy2 cache
802e252 [Joseph K. Bradley] Removed GraphX MIMA excludes and added line to clear spark from .m2 dir before Jenkins tests.  This may not work yet...
30f8bb4 [Joseph K. Bradley] added individual mima excludes for graphx
a3fea42 [Joseph K. Bradley] removed graphx mima exclude for 1.3
2015-01-10 17:25:39 -08:00
scwf d22a31f5e8 [SPARK-5029][SQL] Enable from follow multiple brackets
Enable from follow multiple brackets:
```
select key from ((select * from testData limit 1) union all (select * from testData limit 1)) x limit 1
```

Author: scwf <wangfei1@huawei.com>

Closes #3853 from scwf/from and squashes the following commits:

14f110a [scwf] enable from follow multiple brackets
2015-01-10 17:07:34 -08:00
wangfei 92d9a704ce [SPARK-4871][SQL] Show sql statement in spark ui when run sql with spark-sql
Author: wangfei <wangfei1@huawei.com>

Closes #3718 from scwf/sparksqlui and squashes the following commits:

e0d6b5d [wangfei] format fix
383b505 [wangfei] fix conflicts
4d2038a [wangfei] using setJobDescription
df79837 [wangfei] fix compile error
92ce834 [wangfei] show sql statement in spark ui when run sql use spark-sql
2015-01-10 17:04:56 -08:00
GuoQiang Li 8a29dc716e [Minor]Resolve sbt warnings during build (MQTTStreamSuite.scala).
cc andrewor14

Author: GuoQiang Li <witgo@qq.com>

Closes #3989 from witgo/MQTTStreamSuite and squashes the following commits:

a6e967e [GuoQiang Li] Resolve sbt warnings during build (MQTTStreamSuite.scala).
2015-01-10 15:38:43 -08:00
CodingCat f0d558b6e6 [SPARK-5181] do not print writing WAL log when WAL is disabled
https://issues.apache.org/jira/browse/SPARK-5181

Currently, even the logManager is not created, we still see the log entry
s"Writing to log $record"

a simple fix to make log more accurate

Author: CodingCat <zhunansjtu@gmail.com>

Closes #3985 from CodingCat/SPARK-5181 and squashes the following commits:

0e27dc5 [CodingCat] do not print writing WAL log when WAL is disabled
2015-01-10 15:35:41 -08:00
YanTangZhai 0ca51cc31d [SPARK-4692] [SQL] Support ! boolean logic operator like NOT
Support ! boolean logic operator like NOT in sql as follows
select * from for_test where !(col1 > col2)

Author: YanTangZhai <hakeemzhai@tencent.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3555 from YanTangZhai/SPARK-4692 and squashes the following commits:

1a9f605 [YanTangZhai] Update HiveQuerySuite.scala
7c03c68 [YanTangZhai] Merge pull request #23 from apache/master
992046e [YanTangZhai] Update HiveQuerySuite.scala
ea618f4 [YanTangZhai] Update HiveQuerySuite.scala
192411d [YanTangZhai] Merge pull request #17 from YanTangZhai/master
e4c2c0a [YanTangZhai] Merge pull request #15 from apache/master
1e1ebb4 [YanTangZhai] Update HiveQuerySuite.scala
efc4210 [YanTangZhai] Update HiveQuerySuite.scala
bd2c444 [YanTangZhai] Update HiveQuerySuite.scala
1893956 [YanTangZhai] Merge pull request #14 from marmbrus/pr/3555
59e4de9 [Michael Armbrust] make hive test
718afeb [YanTangZhai] Merge pull request #12 from apache/master
950b21e [YanTangZhai] Update HiveQuerySuite.scala
74175b4 [YanTangZhai] Update HiveQuerySuite.scala
92242c7 [YanTangZhai] Update HiveQl.scala
6e643f8 [YanTangZhai] Merge pull request #11 from apache/master
e249846 [YanTangZhai] Merge pull request #10 from apache/master
d26d982 [YanTangZhai] Merge pull request #9 from apache/master
76d4027 [YanTangZhai] Merge pull request #8 from apache/master
03b62b0 [YanTangZhai] Merge pull request #7 from apache/master
8a00106 [YanTangZhai] Merge pull request #6 from apache/master
cbcba66 [YanTangZhai] Merge pull request #3 from apache/master
cdef539 [YanTangZhai] Merge pull request #1 from apache/master
2015-01-10 15:05:23 -08:00
Michael Armbrust 3684fd21e1 [SPARK-5187][SQL] Fix caching of tables with HiveUDFs in the WHERE clause
Author: Michael Armbrust <michael@databricks.com>

Closes #3987 from marmbrus/hiveUdfCaching and squashes the following commits:

8bca2fa [Michael Armbrust] [SPARK-5187][SQL] Fix caching of tables with HiveUDFs in the WHERE clause
2015-01-10 14:25:45 -08:00
Yanbo Liang 77106df691 SPARK-4963 [SQL] Add copy to SQL's Sample operator
https://issues.apache.org/jira/browse/SPARK-4963
SchemaRDD.sample() return wrong results due to GapSamplingIterator operating on mutable row.
HiveTableScan make RDD with SpecificMutableRow and SchemaRDD.sample() will return GapSamplingIterator for iterating.

override def next(): T = {
    val r = data.next()
    advance
    r
  }

GapSamplingIterator.next() return the current underlying element and assigned it to r.
However if the underlying iterator is mutable row just like what HiveTableScan returned, underlying iterator and r will point to the same object.
After advance operation, we drop some underlying elments and it also changed r which is not expected. Then we return the wrong value different from initial r.

To fix this issue, the most direct way is to make HiveTableScan return mutable row with copy just like the initial commit that I have made. This solution will make HiveTableScan can not get the full advantage of reusable MutableRow, but it can make sample operation return correct result.
Further more, we need to investigate  GapSamplingIterator.next() and make it can implement copy operation inside it. To achieve this, we should define every elements that RDD can store implement the function like cloneable and it will make huge change.

Author: Yanbo Liang <yanbohappy@gmail.com>

Closes #3827 from yanbohappy/spark-4963 and squashes the following commits:

0912ca0 [Yanbo Liang] code format keep
65c4e7c [Yanbo Liang] import file and clear annotation
55c7c56 [Yanbo Liang] better output of test case
cea7e2e [Yanbo Liang] SchemaRDD add copy operation before Sample operator
e840829 [Yanbo Liang] HiveTableScan return mutable row with copy
2015-01-10 14:19:32 -08:00
scwf b3e86dc624 [SPARK-4861][SQL] Refactory command in spark sql
Follow up for #3712.
This PR finally remove ```CommandStrategy``` and make all commands follow ```RunnableCommand``` so they can go with ```case r: RunnableCommand => ExecutedCommand(r) :: Nil```.

One exception is the ```DescribeCommand``` of hive, which is a special case and need to distinguish hive table and temporary table, so still keep ```HiveCommandStrategy``` here.

Author: scwf <wangfei1@huawei.com>

Closes #3948 from scwf/followup-SPARK-4861 and squashes the following commits:

6b48e64 [scwf] minor style fix
2c62e9d [scwf] fix for hive module
5a7a819 [scwf] Refactory command in spark sql
2015-01-10 14:08:04 -08:00
scwf 693a323a70 [SPARK-4574][SQL] Adding support for defining schema in foreign DDL commands.
Adding support for defining schema in foreign DDL commands. Now foreign DDL support commands like:
```
CREATE TEMPORARY TABLE avroTable
USING org.apache.spark.sql.avro
OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")
```
With this PR user can define schema instead of infer from file, so  support ddl command as follows:
```
CREATE TEMPORARY TABLE avroTable(a int, b string)
USING org.apache.spark.sql.avro
OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")
```

Author: scwf <wangfei1@huawei.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Fei Wang <wangfei1@huawei.com>
Author: wangfei <wangfei1@huawei.com>

Closes #3431 from scwf/ddl and squashes the following commits:

7e79ce5 [Fei Wang] Merge pull request #22 from yhuai/pr3431yin
38f634e [Yin Huai] Remove Option from createRelation.
65e9c73 [Yin Huai] Revert all changes since applying a given schema has not been testd.
a852b10 [scwf] remove cleanIdentifier
f336a16 [Fei Wang] Merge pull request #21 from yhuai/pr3431yin
baf79b5 [Yin Huai] Test special characters quoted by backticks.
50a03b0 [Yin Huai] Use JsonRDD.nullTypeToStringType to convert NullType to StringType.
1eeb769 [Fei Wang] Merge pull request #20 from yhuai/pr3431yin
f5c22b0 [Yin Huai] Refactor code and update test cases.
f1cffe4 [Yin Huai] Revert "minor refactory"
b621c8f [scwf] minor refactory
d02547f [scwf] fix HiveCompatibilitySuite test failure
8dfbf7a [scwf] more tests for complex data type
ddab984 [Fei Wang] Merge pull request #19 from yhuai/pr3431yin
91ad91b [Yin Huai] Parse data types in DDLParser.
cf982d2 [scwf] fixed test failure
445b57b [scwf] address comments
02a662c [scwf] style issue
44eb70c [scwf] fix decimal parser issue
83b6fc3 [scwf] minor fix
9bf12f8 [wangfei] adding test case
7787ec7 [wangfei] added SchemaRelationProvider
0ba70df [wangfei] draft version
2015-01-10 13:53:21 -08:00
Alex Liu 4b39fd1e63 [SPARK-4943][SQL] Allow table name having dot for db/catalog
The pull only fixes the parsing error and changes API to use tableIdentifier. Joining different catalog datasource related change is not done in this pull.

Author: Alex Liu <alex_liu68@yahoo.com>

Closes #3941 from alexliu68/SPARK-SQL-4943-3 and squashes the following commits:

343ae27 [Alex Liu] [SPARK-4943][SQL] refactoring according to review
29e5e55 [Alex Liu] [SPARK-4943][SQL] fix failed Hive CTAS tests
6ae77ce [Alex Liu] [SPARK-4943][SQL] fix TestHive matching error
3652997 [Alex Liu] [SPARK-4943][SQL] Allow table name having dot to support db/catalog ...
2015-01-10 13:23:09 -08:00
Alex Liu 1e56eba5d9 [SPARK-4925][SQL] Publish Spark SQL hive-thriftserver maven artifact
Author: Alex Liu <alex_liu68@yahoo.com>

Closes #3766 from alexliu68/SPARK-SQL-4925 and squashes the following commits:

3137b51 [Alex Liu] [SPARK-4925][SQL] Remove sql/hive-thriftserver module from pom.xml
15f2e38 [Alex Liu] [SPARK-4925][SQL] Publish Spark SQL hive-thriftserver maven artifact
2015-01-10 13:19:12 -08:00
luogankun 545dfcb92f [SPARK-5141][SQL]CaseInsensitiveMap throws java.io.NotSerializableException
CaseInsensitiveMap throws java.io.NotSerializableException.

Author: luogankun <luogankun@gmail.com>

Closes #3944 from luogankun/SPARK-5141 and squashes the following commits:

b6d63d5 [luogankun] [SPARK-5141]CaseInsensitiveMap throws java.io.NotSerializableException
2015-01-09 20:38:41 -08:00
MechCoder 4554529dce [SPARK-4406] [MLib] FIX: Validate k in SVD
Raise exception when k is non-positive in SVD

Author: MechCoder <manojkumarsivaraj334@gmail.com>

Closes #3945 from MechCoder/spark-4406 and squashes the following commits:

64e6d2d [MechCoder] TST: Add better test errors and messages
12dae73 [MechCoder] [SPARK-4406] FIX: Validate k in SVD
2015-01-09 17:45:18 -08:00
WangTaoTheTonic 8782eb992f [SPARK-4990][Deploy]to find default properties file, search SPARK_CONF_DIR first
https://issues.apache.org/jira/browse/SPARK-4990

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3823 from WangTaoTheTonic/SPARK-4990 and squashes the following commits:

133c43e [WangTao] Update spark-submit2.cmd
b1ab402 [WangTao] Update spark-submit
4cc7f34 [WangTaoTheTonic] rebase
55300bc [WangTaoTheTonic] use export to make it global
d8d3cb7 [WangTaoTheTonic] remove blank line
07b9ebf [WangTaoTheTonic] check SPARK_CONF_DIR instead of checking properties file
c5a85eb [WangTaoTheTonic] to find default properties file, search SPARK_CONF_DIR first
2015-01-09 17:10:02 -08:00
bilna 4e1f12d997 [Minor] Fix import order and other coding style
fixed import order and other coding style

Author: bilna <bilnap@am.amrita.edu>
Author: Bilna P <bilna.p@gmail.com>

Closes #3966 from Bilna/master and squashes the following commits:

5e76f04 [bilna] fix import order and other coding style
5718d66 [bilna] Merge remote-tracking branch 'upstream/master'
ae56514 [bilna] Merge remote-tracking branch 'upstream/master'
acea3a3 [bilna] Adding dependency with scope test
28681fa [bilna] Merge remote-tracking branch 'upstream/master'
fac3904 [bilna] Correction in Indentation and coding style
ed9db4c [bilna] Merge remote-tracking branch 'upstream/master'
4b34ee7 [Bilna P] Update MQTTStreamSuite.scala
04503cf [bilna] Added embedded broker service for mqtt test
89d804e [bilna] Merge remote-tracking branch 'upstream/master'
fc8eb28 [bilna] Merge remote-tracking branch 'upstream/master'
4b58094 [Bilna P] Update MQTTStreamSuite.scala
b1ac4ad [bilna] Added BeforeAndAfter
5f6bfd2 [bilna] Added BeforeAndAfter
e8b6623 [Bilna P] Update MQTTStreamSuite.scala
5ca6691 [Bilna P] Update MQTTStreamSuite.scala
8616495 [bilna] [SPARK-4631] unit test for MQTT
2015-01-09 14:45:28 -08:00
Kousuke Saruta ae628725ab [DOC] Fixed Mesos version in doc from 0.18.1 to 0.21.0
#3934 upgraded Mesos version so we should also fix docs right?

This issue is really minor so I don't file in JIRA.

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

Closes #3982 from sarutak/fix-mesos-version and squashes the following commits:

9a86ee3 [Kousuke Saruta] Fixed mesos version from 0.18.1 to 0.21.0
2015-01-09 14:40:45 -08:00
mcheah e0f28e010c [SPARK-4737] Task set manager properly handles serialization errors
Dealing with [SPARK-4737], the handling of serialization errors should not be the DAGScheduler's responsibility. The task set manager now catches the error and aborts the stage.

If the TaskSetManager throws a TaskNotSerializableException, the TaskSchedulerImpl will return an empty list of task descriptions, because no tasks were started. The scheduler should abort the stage gracefully.

Note that I'm not too familiar with this part of the codebase and its place in the overall architecture of the Spark stack. If implementing it this way will have any averse side effects please voice that loudly.

Author: mcheah <mcheah@palantir.com>

Closes #3638 from mccheah/task-set-manager-properly-handle-ser-err and squashes the following commits:

1545984 [mcheah] Some more style fixes from Andrew Or.
5267929 [mcheah] Fixing style suggestions from Andrew Or.
dfa145b [mcheah] Fixing style from Josh Rosen's feedback
b2a430d [mcheah] Not returning empty seq when a task set cannot be serialized.
94844d7 [mcheah] Fixing compilation error, one brace too many
5f486f4 [mcheah] Adding license header for fake task class
bf5e706 [mcheah] Fixing indentation.
097e7a2 [mcheah] [SPARK-4737] Catching task serialization exception in TaskSetManager
2015-01-09 14:16:20 -08:00
WangTaoTheTonic e966452060 [SPARK-1953][YARN]yarn client mode Application Master memory size is same as driver memory...
... size

Ways to set Application Master's memory on yarn-client mode:
1.  `spark.yarn.am.memory` in SparkConf or System Properties
2.  default value 512m

Note: this arguments is only available in yarn-client mode.

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3607 from WangTaoTheTonic/SPARK4181 and squashes the following commits:

d5ceb1b [WangTaoTheTonic] spark.driver.memeory is used in both modes
6c1b264 [WangTaoTheTonic] rebase
b8410c0 [WangTaoTheTonic] minor optiminzation
ddcd592 [WangTaoTheTonic] fix the bug produced in rebase and some improvements
3bf70cc [WangTaoTheTonic] rebase and give proper hint
987b99d [WangTaoTheTonic] disable --driver-memory in client mode
2b27928 [WangTaoTheTonic] inaccurate description
b7acbb2 [WangTaoTheTonic] incorrect method invoked
2557c5e [WangTaoTheTonic] missing a single blank
42075b0 [WangTaoTheTonic] arrange the args and warn logging
69c7dba [WangTaoTheTonic] rebase
1960d16 [WangTaoTheTonic] fix wrong comment
7fa9e2e [WangTaoTheTonic] log a warning
f6bee0e [WangTaoTheTonic] docs issue
d619996 [WangTaoTheTonic] Merge branch 'master' into SPARK4181
b09c309 [WangTaoTheTonic] use code format
ab16bb5 [WangTaoTheTonic] fix bug and add comments
44e48c2 [WangTaoTheTonic] minor fix
6fd13e1 [WangTaoTheTonic] add overhead mem and remove some configs
0566bb8 [WangTaoTheTonic] yarn client mode Application Master memory size is same as driver memory size
2015-01-09 13:23:13 -08:00
Joseph K. Bradley 7e8e62aec1 [SPARK-5015] [mllib] Random seed for GMM + make test suite deterministic
Issues:
* From JIRA: GaussianMixtureEM uses randomness but does not take a random seed. It should take one as a parameter.
* This also makes the test suite flaky since initialization can fail due to stochasticity.

Fix:
* Add random seed
* Use it in test suite

CC: mengxr  tgaloppo

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #3981 from jkbradley/gmm-seed and squashes the following commits:

f0df4fd [Joseph K. Bradley] Added seed parameter to GMM.  Updated test suite to use seed to prevent flakiness
2015-01-09 13:00:15 -08:00
Jongyoul Lee 454fe129ee [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688
- update version from 0.18.1 to 0.21.0
- I'm doing some tests in order to verify some spark jobs work fine on mesos 0.21.0 environment.

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #3934 from jongyoul/SPARK-3619 and squashes the following commits:

ab994fa [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - update version from 0.18.1 to 0.21.0
2015-01-09 10:47:08 -08:00
Liang-Chi Hsieh e9ca16ec94 [SPARK-5145][Mllib] Add BLAS.dsyr and use it in GaussianMixtureEM
This pr uses BLAS.dsyr to replace few implementations in GaussianMixtureEM.

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

Closes #3949 from viirya/blas_dsyr and squashes the following commits:

4e4d6cf [Liang-Chi Hsieh] Add unit test. Rename function name, modify doc and style.
3f57fd2 [Liang-Chi Hsieh] Add BLAS.dsyr and use it in GaussianMixtureEM.
2015-01-09 10:27:33 -08:00