Commit graph

12585 commits

Author SHA1 Message Date
Yin Huai d5a9af3230 [SPARK-9664] [SQL] Remove UDAFRegistration and add apply to UserDefinedAggregateFunction.
https://issues.apache.org/jira/browse/SPARK-9664

Author: Yin Huai <yhuai@databricks.com>

Closes #7982 from yhuai/udafRegister and squashes the following commits:

0cc2287 [Yin Huai] Remove UDAFRegistration and add apply to UserDefinedAggregateFunction.
2015-08-05 21:50:35 -07:00
Reynold Xin 9270bd06fd [SPARK-9674][SQL] Remove GeneratedAggregate.
The new aggregate replaces the old GeneratedAggregate.

Author: Reynold Xin <rxin@databricks.com>

Closes #7983 from rxin/remove-generated-agg and squashes the following commits:

8334aae [Reynold Xin] [SPARK-9674][SQL] Remove GeneratedAggregate.
2015-08-05 21:50:14 -07:00
Cheng Hao 119b590538 [SPARK-6923] [SPARK-7550] [SQL] Persists data source relations in Hive compatible format when possible
This PR is a fork of PR #5733 authored by chenghao-intel.  For committers who's going to merge this PR, please set the author to "Cheng Hao <hao.chengintel.com>".

----

When a data source relation meets the following requirements, we persist it in Hive compatible format, so that other systems like Hive can access it:

1. It's a `HadoopFsRelation`
2. It has only one input path
3. It's non-partitioned
4. It's data source provider can be naturally mapped to a Hive builtin SerDe (e.g. ORC and Parquet)

Author: Cheng Lian <lian@databricks.com>
Author: Cheng Hao <hao.cheng@intel.com>

Closes #7967 from liancheng/spark-6923/refactoring-pr-5733 and squashes the following commits:

5175ee6 [Cheng Lian] Fixes an oudated comment
3870166 [Cheng Lian] Fixes build error and comments
864acee [Cheng Lian] Refactors PR #5733
3490cdc [Cheng Hao] update the scaladoc
6f57669 [Cheng Hao] write schema info to hivemetastore for data source
2015-08-06 11:13:44 +08:00
Yin Huai 4581badbc8 [SPARK-9611] [SQL] Fixes a few corner cases when we spill a UnsafeFixedWidthAggregationMap
This PR has the following three small fixes.

1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an UnsafeInMemorySorter if its BytesToBytesMap is empty.
2. We will not not spill a InMemorySorter if it is empty.
3. We will not add a SpillReader to a SpillMerger if this SpillReader is empty.

JIRA: https://issues.apache.org/jira/browse/SPARK-9611

Author: Yin Huai <yhuai@databricks.com>

Closes #7948 from yhuai/unsafeEmptyMap and squashes the following commits:

9727abe [Yin Huai] Address Josh's comments.
34b6f76 [Yin Huai] 1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an UnsafeInMemorySorter if its BytesToBytesMap is empty. 2. Do not spill a InMemorySorter if it is empty. 3. Do not add spill to SpillMerger if this spill is empty.
2015-08-05 19:19:09 -07:00
Marcelo Vanzin 4399b7b090 [SPARK-9651] Fix UnsafeExternalSorterSuite.
First, it's probably a bad idea to call generated Scala methods
from Java. In this case, the method being called wasn't actually
"Utils.createTempDir()", but actually the method that returns the
first default argument to the actual createTempDir method, which
is just the location of java.io.tmpdir; meaning that all tests in
the class were using the same temp dir, and thus affecting each
other.

Second, spillingOccursInResponseToMemoryPressure was not writing
enough records to actually cause a spill.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #7970 from vanzin/SPARK-9651 and squashes the following commits:

74d357f [Marcelo Vanzin] Clean up temp dir on test tear down.
a64f36a [Marcelo Vanzin] [SPARK-9651] Fix UnsafeExternalSorterSuite.
2015-08-05 17:58:36 -07:00
Yijie Shen 8c320e45b5 [SPARK-6591] [SQL] Python data source load options should auto convert common types into strings
JIRA: https://issues.apache.org/jira/browse/SPARK-6591

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7926 from yjshen/py_dsload_opt and squashes the following commits:

b207832 [Yijie Shen] fix style
efdf834 [Yijie Shen] resolve comment
7a8f6a2 [Yijie Shen] lowercase
822e769 [Yijie Shen] convert load opts to string
2015-08-05 17:28:23 -07:00
Xusen Yin a018b85716 [SPARK-5895] [ML] Add VectorSlicer - updated
Add VectorSlicer transformer to spark.ml, with features specified as either indices or names.  Transfers feature attributes for selected features.

Updated version of [https://github.com/apache/spark/pull/5731]

CC: yinxusen This updates your PR.  You'll still be the primary author of this PR.

CC: mengxr

Author: Xusen Yin <yinxusen@gmail.com>
Author: Joseph K. Bradley <joseph@databricks.com>

Closes #7972 from jkbradley/yinxusen-SPARK-5895 and squashes the following commits:

b16e86e [Joseph K. Bradley] fixed scala style
71c65d2 [Joseph K. Bradley] fix import order
86e9739 [Joseph K. Bradley] cleanups per code review
9d8d6f1 [Joseph K. Bradley] style fix
83bc2e9 [Joseph K. Bradley] Updated VectorSlicer
98c6939 [Xusen Yin] fix style error
ecbf2d3 [Xusen Yin] change interfaces and params
f6be302 [Xusen Yin] Merge branch 'master' into SPARK-5895
e4781f2 [Xusen Yin] fix commit error
fd154d7 [Xusen Yin] add test suite of vector slicer
17171f8 [Xusen Yin] fix slicer
9ab9747 [Xusen Yin] add vector slicer
aa5a0bf [Xusen Yin] add vector slicer
2015-08-05 17:07:55 -07:00
Josh Rosen 9c878923db [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in SMJ
This patches renames `RowOrdering` to `InterpretedOrdering` and updates SortMergeJoin to use the `SparkPlan` methods for constructing its ordering so that it may benefit from codegen.

This is an updated version of #7408.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7973 from JoshRosen/SPARK-9054 and squashes the following commits:

e610655 [Josh Rosen] Add comment RE: Ascending ordering
34b8e0c [Josh Rosen] Import ordering
be19a0f [Josh Rosen] [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in more places.
2015-08-05 16:33:42 -07:00
Feynman Liang dac090d1e9 [SPARK-9657] Fix return type of getMaxPatternLength
mengxr

Author: Feynman Liang <fliang@databricks.com>

Closes #7974 from feynmanliang/SPARK-9657 and squashes the following commits:

7ca533f [Feynman Liang] Fix return type of getMaxPatternLength
2015-08-05 15:42:18 -07:00
Reynold Xin f9c2a2af1e Closes #7474 since it's marked as won't fix. 2015-08-05 14:15:57 -07:00
Andrew Or 5f0fb6466f [SPARK-9649] Fix flaky test MasterSuite - randomize ports
```
Error Message

Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries!
Stacktrace

      java.net.BindException: Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries!
      at org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272)
      at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393)
      at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389)
      at scala.util.Success$$anonfun$map$1.apply(Try.scala:206)
      at scala.util.Try$.apply(Try.scala:161)
```

Author: Andrew Or <andrew@databricks.com>

Closes #7968 from andrewor14/fix-master-flaky-test and squashes the following commits:

fcc42ef [Andrew Or] Randomize port
2015-08-05 14:12:22 -07:00
Reynold Xin eb5b8f4a60 Closes #7778 since it is done as #7893. 2015-08-05 12:51:12 -07:00
Liang-Chi Hsieh e1e05873fc [SPARK-9403] [SQL] Add codegen support in In and InSet
This continues tarekauel's work in #7778.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #7893 from viirya/codegen_in and squashes the following commits:

81ff97b [Liang-Chi Hsieh] For comments.
47761c6 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
cf4bf41 [Liang-Chi Hsieh] For comments.
f532b3c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
446bbcd [Liang-Chi Hsieh] Fix bug.
b3d0ab4 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
4610eff [Liang-Chi Hsieh] Relax the types of references and update optimizer test.
224f18e [Liang-Chi Hsieh] Beef up the test cases for In and InSet to include all primitive data types.
86dc8aa [Liang-Chi Hsieh] Only convert In to InSet when the number of items in set is more than the threshold.
b7ded7e [Tarek Auel] [SPARK-9403][SQL] codeGen in / inSet
2015-08-05 11:38:56 -07:00
Yin Huai 1f8c364b9c [SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920
This is a follow-up of https://github.com/apache/spark/pull/7920 to fix comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #7964 from yhuai/SPARK-9141-follow-up and squashes the following commits:

4d0ee80 [Yin Huai] Fix comments.
2015-08-05 11:03:02 -07:00
linweizhong 7a969a6967 [SPARK-9519] [YARN] Confirm stop sc successfully when application was killed
Currently, when we kill application on Yarn, then will call sc.stop() at Yarn application state monitor thread, then in YarnClientSchedulerBackend.stop() will call interrupt this will cause SparkContext not stop fully as we will wait executor to exit.

Author: linweizhong <linweizhong@huawei.com>

Closes #7846 from Sephiroth-Lin/SPARK-9519 and squashes the following commits:

1ae736d [linweizhong] Update comments
2e8e365 [linweizhong] Add comment explaining the code
ad0e23b [linweizhong] Update
243d2c7 [linweizhong] Confirm stop sc successfully when application was killed
2015-08-05 10:16:12 -07:00
Michael Armbrust 23d982204b [SPARK-9141] [SQL] Remove project collapsing from DataFrame API
Currently we collapse successive projections that are added by `withColumn`.  However, this optimization violates the constraint that adding nodes to a plan will never change its analyzed form and thus breaks caching.  Instead of doing early optimization, in this PR I just fix some low-hanging slowness in the analyzer.  In particular, I add a mechanism for skipping already analyzed subplans, `resolveOperators` and `resolveExpression`.  Since trees are generally immutable after construction, it's safe to annotate a plan as already analyzed as any transformation will create a new tree with this bit no longer set.

Together these result in a faster analyzer than before, even with added timing instrumentation.

```
Original Code
[info] 3430ms
[info] 2205ms
[info] 1973ms
[info] 1982ms
[info] 1916ms

Without Project Collapsing in DataFrame
[info] 44610ms
[info] 45977ms
[info] 46423ms
[info] 46306ms
[info] 54723ms

With analyzer optimizations
[info] 6394ms
[info] 4630ms
[info] 4388ms
[info] 4093ms
[info] 4113ms

With resolveOperators
[info] 2495ms
[info] 1380ms
[info] 1685ms
[info] 1414ms
[info] 1240ms
```

Author: Michael Armbrust <michael@databricks.com>

Closes #7920 from marmbrus/withColumnCache and squashes the following commits:

2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from DataFrame API
38b1c83 [Michael Armbrust] WIP
2015-08-05 09:01:45 -07:00
Mike Dusenberry 34dcf10104 [SPARK-6486] [MLLIB] [PYTHON] Add BlockMatrix to PySpark.
mengxr This adds the `BlockMatrix` to PySpark.  I have the conversions to `IndexedRowMatrix` and `CoordinateMatrix` ready as well, so once PR #7554 is completed (which relies on PR #7746), this PR can be finished.

Author: Mike Dusenberry <mwdusenb@us.ibm.com>

Closes #7761 from dusenberrymw/SPARK-6486_Add_BlockMatrix_to_PySpark and squashes the following commits:

27195c2 [Mike Dusenberry] Adding one more check to _convert_to_matrix_block_tuple, and a few minor documentation changes.
ae50883 [Mike Dusenberry] Minor update: BlockMatrix should inherit from DistributedMatrix.
b8acc1c [Mike Dusenberry] Moving BlockMatrix to pyspark.mllib.linalg.distributed, updating the logic to match that of the other distributed matrices, adding conversions, and adding documentation.
c014002 [Mike Dusenberry] Using properties for better documentation.
3bda6ab [Mike Dusenberry] Adding documentation.
8fb3095 [Mike Dusenberry] Small cleanup.
e17af2e [Mike Dusenberry] Adding BlockMatrix to PySpark.
2015-08-05 07:40:50 -07:00
Cheng Hao 519cf6d3f7 [SPARK-9381] [SQL] Migrate JSON data source to the new partitioning data source
Support partitioning for the JSON data source.

Still 2 open issues for the `HadoopFsRelation`
- `refresh()` will invoke the `discoveryPartition()`, which will auto infer the data type for the partition columns, and maybe conflict with the given partition columns. (TODO enable `HadoopFsRelationSuite.Partition column type casting"
- When insert data into a cached HadoopFsRelation based table, we need to invalidate the cache after the insertion (TODO enable `InsertSuite.Caching`)

Author: Cheng Hao <hao.cheng@intel.com>

Closes #7696 from chenghao-intel/json and squashes the following commits:

d90b104 [Cheng Hao] revert the change for JacksonGenerator.apply
307111d [Cheng Hao] fix bug in the unit test
8738c8a [Cheng Hao] fix bug in unit testing
35f2cde [Cheng Hao] support partition for json format
2015-08-05 22:35:55 +08:00
Nathan Howell eb8bfa3eaa [SPARK-9618] [SQL] Use the specified schema when reading Parquet files
The user specified schema is currently ignored when loading Parquet files.

One workaround is to use the `format` and `load` methods instead of `parquet`, e.g.:

```
val schema = ???

// schema is ignored
sqlContext.read.schema(schema).parquet("hdfs:///test")

// schema is retained
sqlContext.read.schema(schema).format("parquet").load("hdfs:///test")
```

The fix is simple, but I wonder if the `parquet` method should instead be written in a similar fashion to `orc`:

```
def parquet(path: String): DataFrame = format("parquet").load(path)
```

Author: Nathan Howell <nhowell@godaddy.com>

Closes #7947 from NathanHowell/SPARK-9618 and squashes the following commits:

d1ea62c [Nathan Howell] [SPARK-9618] [SQL] Use the specified schema when reading Parquet files
2015-08-05 22:16:56 +08:00
Cheng Lian 70112ff22b [SPARK-9593] [SQL] Fixes Hadoop shims loading
This PR is used to workaround CDH Hadoop versions like 2.0.0-mr1-cdh4.1.1.

Internally, Hive `ShimLoader` tries to load different versions of Hadoop shims by checking version information gathered from Hadoop jar files.  If the major version number is 1, `Hadoop20SShims` will be loaded.  Otherwise, if the major version number is 2, `Hadoop23Shims` will be chosen.  However, CDH Hadoop versions like 2.0.0-mr1-cdh4.1.1 have 2 as major version number, but contain Hadoop 1 code.  This confuses Hive `ShimLoader` and loads wrong version of shims.

In this PR we check for existence of the `Path.getPathWithoutSchemeAndAuthority` method, which doesn't exist in Hadoop 1 (it's also the method that reveals this shims loading issue), and load `Hadoop20SShims` when it doesn't exist.

Author: Cheng Lian <lian@databricks.com>

Closes #7929 from liancheng/spark-9593/fix-hadoop-shims-loading and squashes the following commits:

c99b497 [Cheng Lian] Narrows down the fix to handle "2.0.0-*cdh4*" Hadoop versions only
b17e955 [Cheng Lian] Updates comments
490d8f2 [Cheng Lian] Fixes Scala style issue
9c6c12d [Cheng Lian] Fixes Hadoop shims loading
2015-08-05 20:03:54 +08:00
Ryan Williams e27a8c4cb3 [SPARK-9607] [SPARK-9608] fix zinc-port handling in build/mvn
- pass `$ZINC_PORT` to zinc status/shutdown commands
- fix path check that sets `$ZINC_INSTALL_FLAG`, which was incorrectly
  causing zinc to be shutdown and restarted every time (with mismatched
  ports on those commands to boot)
- pass `-DzincPort=${ZINC_PORT}` to maven, to use the correct zinc port
  when building

Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #7944 from ryan-williams/zinc-status and squashes the following commits:

619c520 [Ryan Williams] fix zinc status/shutdown commands
2015-08-05 11:10:47 +01:00
Josh Rosen 26b06f1c46 [HOTFIX] Add static import to fix build break from #7676. 2015-08-05 02:40:50 -07:00
Yijie Shen 84ca3183b6 [SPARK-9628][SQL]Rename int to SQLDate, long to SQLTimestamp for better readability
JIRA: https://issues.apache.org/jira/browse/SPARK-9628

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7953 from yjshen/datetime_alias and squashes the following commits:

3cac3cc [Yijie Shen] rename int to SQLDate, long to SQLTimestamp for better readability
2015-08-05 02:04:28 -07:00
zsxwing 1b0317f64c [SPARK-8861][SPARK-8862][SQL] Add basic instrumentation to each SparkPlan operator and add a new SQL tab
This PR includes the following changes:

### SPARK-8862: Add basic instrumentation to each SparkPlan operator

A SparkPlan can override `def accumulators: Map[String, Accumulator[_]]` to expose its metrics that can be displayed in UI. The UI will use them to track the updates and show them in the web page in real-time.

### SparkSQLExecution and SQLSparkListener

`SparkSQLExecution.withNewExecutionId` will set `spark.sql.execution.id` to the local properties so that we can use it to track all jobs that belong to the same query.

SQLSparkListener is a listener to track all accumulator updates of all tasks for a query. It receives them from heartbeats can the UI can query them in real-time.

When running a query, `SQLSparkListener.onExecutionStart` will be called. When a query is finished,  `SQLSparkListener.onExecutionEnd` will be called. And the Spark jobs with the same execution id will be tracked and stored with this query.

`SQLSparkListener` has to store all accumulator updates for tasks separately. When a task fails and starts to retry, we need to drop the old accumulator updates. Because we can not revert our changes to an accumulator, we have to maintain these accumulator updates by ourselves so as to drop accumulator updates for a failed task.

### SPARK-8862: A new SQL tab
Includes two pages:
#### A page for all DataFrame/SQL queries
It will show the running, completed and failed queries in 3 tables. It also displays the jobs and their links for a query in each row.
#### A detail page for a DataFrame/SQL query
In this page, it also shows the SparkPlan metrics in real-time. Run a long-running query, such as
```
val testData = sc.parallelize((1 to 1000000).map(i => (i, i.toString))).toDF()
testData.select($"_1").filter($"_1" < 1000).foreach(_ => Thread.sleep(60))
```
and you will see the metrics keep updating in real-time.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7774)
<!-- Reviewable:end -->

Author: zsxwing <zsxwing@gmail.com>

Closes #7774 from zsxwing/sql-ui and squashes the following commits:

5a2bc99 [zsxwing] Remove UISeleniumSuite and its dependency
57d4cd2 [zsxwing] Use VisibleForTesting annotation
cc1c736 [zsxwing] Add SparkPlan.trackNumOfRowsEnabled to make subclasses easy to track the number of rows; fix the issue that the "save" action cannot collect metrics
3771ab0 [zsxwing] Register SQL metrics accmulators
3a101c0 [zsxwing] Change prepareCalled's type to AtomicBoolean for thread-safety
b8d5605 [zsxwing] Make prepare idempotent; call children's prepare in SparkPlan.prepare; change doPrepare to def
4ed11a1 [zsxwing] var -> val
332639c [zsxwing] Ignore UISeleniumSuite and SQLListenerSuite."no memory leak" because of SPARK-9580
bb52359 [zsxwing] Address other commens in SQLListener
c4d0f5d [zsxwing] Move newPredicate out of the iterator loop
957473c [zsxwing] Move STATIC_RESOURCE_DIR to object SQLTab
7ab4816 [zsxwing] Make SparkPlan accumulator API private[sql]
dae195e [zsxwing] Fix the code style and comments
3a66207 [zsxwing] Ignore irrelevant accumulators
b8484a1 [zsxwing] Merge branch 'master' into sql-ui
9406592 [zsxwing] Implement the SparkPlan viz
4ebce68 [zsxwing] Add SparkPlan.prepare to support BroadcastHashJoin to run background work in parallel
ca1811f [zsxwing] Merge branch 'master' into sql-ui
fef6fc6 [zsxwing] Fix a corner case
25f335c [zsxwing] Fix the code style
6eae828 [zsxwing] SQLSparkListener -> SQLListener; SparkSQLExecutionUIData -> SQLExecutionUIData; SparkSQLExecution -> SQLExecution
822af75 [zsxwing] Add SQLSparkListenerSuite and fix the issue about onExecutionEnd and onJobEnd
6be626f [zsxwing] Add UISeleniumSuite to test UI
d02a24d [zsxwing] Make ExecutionPage private
23abf73 [zsxwing] [SPARK-8862][SPARK-8862][SQL] Add basic instrumentation to each SparkPlan operator and add a new SQL tab
2015-08-05 01:51:22 -07:00
Namit Katariya 1bf608b5ef [SPARK-9601] [DOCS] Fix JavaPairDStream signature for stream-stream and windowed join in streaming guide doc
Author: Namit Katariya <katariya.namit@gmail.com>

Closes #7935 from namitk/SPARK-9601 and squashes the following commits:

03b5784 [Namit Katariya] [SPARK-9601] Fix signature of JavaPairDStream for stream-stream and windowed join in streaming guide doc
2015-08-05 01:07:33 -07:00
Takeshi YAMAMURO 6d8a6e4161 [SPARK-9360] [SQL] Support BinaryType in PrefixComparators for UnsafeExternalSort
The current implementation of UnsafeExternalSort uses NoOpPrefixComparator for binary-typed data.
So, we need to add BinaryPrefixComparator in PrefixComparators.

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

Closes #7676 from maropu/BinaryTypePrefixComparator and squashes the following commits:

fe6f31b [Takeshi YAMAMURO] Apply comments
d943c04 [Takeshi YAMAMURO] Add a codegen'd entry for BinaryType in SortPrefix
ecf3ac5 [Takeshi YAMAMURO] Support BinaryType in PrefixComparator
2015-08-05 00:56:35 -07:00
Reynold Xin d8ef538e56 Closes #7917 2015-08-05 00:44:31 -07:00
Emiliano Leporati 1d1a76c8c5 [SPARK-9581][SQL] Add unit test for JSON UDT
This brings #7416 up-to-date by drubbo.

Author: Emiliano Leporati <emiliano.leporati@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #7917 from rxin/udt-json-test and squashes the following commits:

93e3954 [Reynold Xin] Fix test.
7035308 [Reynold Xin] Merge pull request #7416 from drubbo/master
b5bcd94 [Emiliano Leporati] removed unneded case in MyDenseVector::equals
508a399 [Emiliano Leporati] Merge remote branch 'upstream/master'
7569e42 [Emiliano Leporati] using checkAnswer
62daccd [Emiliano Leporati] added coverage for UDTs in JSON RDDs
2015-08-05 00:42:08 -07:00
Tathagata Das c2a71f0714 [SPARK-9217] [STREAMING] Make the kinesis receiver reliable by recording sequence numbers
This PR is the second one in the larger issue of making the Kinesis integration reliable and provide WAL-free at-least once guarantee. It is based on the design doc - https://docs.google.com/document/d/1k0dl270EnK7uExrsCE7jYw7PYx0YC935uBcxn3p0f58/edit

In this PR, I have updated the Kinesis Receiver to do the following.
- Control the block generation, by creating its own BlockGenerator with own callback methods and using it to keep track of the ranges of sequence numbers that go into each block.
- More specifically, as the KinesisRecordProcessor provides small batches of records, the records are atomically inserted into the block (that is, either the whole batch is in the block, or not). Accordingly the sequence number range of the batch is recorded. Since there may be many batches added to a block, the receiver tracks all the range of sequence numbers that is added to a block.
- When the block is ready to be pushed, the block is pushed and the ranges are reported as metadata of the block. In addition, the ranges are used to find out the latest sequence number for each shard that can be checkpointed through the DynamoDB.
- Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence number for it own shard.
- The array of ranges in the block metadata is used to create KinesisBackedBlockRDDs. The ReceiverInputDStream has been slightly refactored to allow the creation of KinesisBackedBlockRDDs instead of the WALBackedBlockRDDs.

Things to be done
- [x] Add new test to verify that the sequence numbers are recovered.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #7825 from tdas/kinesis-receiver and squashes the following commits:

2159be9 [Tathagata Das] Fixed bug
569be83 [Tathagata Das] Fix scala style issue
bf31e22 [Tathagata Das] Added more documentation to make the kinesis test endpoint more configurable
3ad8361 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into kinesis-receiver
c693a63 [Tathagata Das] Removed unnecessary constructor params from KinesisTestUtils
e1f1d0a [Tathagata Das] Addressed PR comments
b9fa6bf [Tathagata Das] Fix serialization issues
f8b7680 [Tathagata Das] Updated doc
33fe43a [Tathagata Das] Added more tests
7997138 [Tathagata Das] Fix style errors
a806710 [Tathagata Das] Fixed unit test and use KinesisInputDStream
40a1709 [Tathagata Das] Fixed KinesisReceiverSuite tests
7e44df6 [Tathagata Das] Added documentation and fixed checkpointing
096383f [Tathagata Das] Added test, and addressed some of the comments.
84a7892 [Tathagata Das] fixed scala style issue
e19e37d [Tathagata Das] Added license
1cd7b66 [Tathagata Das] Updated kinesis receiver
2015-08-05 00:20:26 -07:00
Davies Liu 781c8d71a0 [SPARK-9119] [SPARK-8359] [SQL] match Decimal.precision/scale with DecimalType
Let Decimal carry the correct precision and scale with DecimalType.

cc rxin yhuai

Author: Davies Liu <davies@databricks.com>

Closes #7925 from davies/decimal_scale and squashes the following commits:

e19701a [Davies Liu] some tweaks
57d78d2 [Davies Liu] fix tests
5d5bc69 [Davies Liu] match precision and scale with DecimalType
2015-08-04 23:12:49 -07:00
Pedro Rodriguez d34548587a [SPARK-8231] [SQL] Add array_contains
This PR is based on #7580 , thanks to EntilZha

PR for work on https://issues.apache.org/jira/browse/SPARK-8231

Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128

Main points are:
1. If the array is empty, null, or the value is null, return false
2. If there is a type mismatch, throw error
3. If comparison is not supported, throw error

Closes #7580

Author: Pedro Rodriguez <prodriguez@trulia.com>
Author: Pedro Rodriguez <ski.rodriguez@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #7949 from davies/array_contains and squashes the following commits:

d3c08bc [Davies Liu] use foreach() to avoid copy
bc3d1fe [Davies Liu] fix array_contains
719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains
e352cf9 [Pedro Rodriguez] fixed diff from master
4d5b0ff [Pedro Rodriguez] added docs and another type check
ffc0591 [Pedro Rodriguez] fixed unit test
7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3
b5ffae8 [Pedro Rodriguez] fixed pyspark test
4e7dce3 [Pedro Rodriguez] added more docs
3082399 [Pedro Rodriguez] fixed unit test
46f9789 [Pedro Rodriguez] reverted change
d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this
8528027 [Pedro Rodriguez] added more tests
686e029 [Pedro Rodriguez] fix scala style
d262e9d [Pedro Rodriguez] reworked type checking code and added more tests
2517a58 [Pedro Rodriguez] removed unused import
28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests
12f8795 [Pedro Rodriguez] fix scala style checks
e8a20a9 [Pedro Rodriguez] added python df (broken atm)
65b562c [Pedro Rodriguez] made array_contains nullable false
33b45aa [Pedro Rodriguez] reordered test
9623c64 [Pedro Rodriguez] fixed test
4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs
72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs
69c46fb [Pedro Rodriguez] added tests and codegen
9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-04 22:34:02 -07:00
Xiangrui Meng a02bcf20c4 [SPARK-9540] [MLLIB] optimize PrefixSpan implementation
This is a major refactoring of the PrefixSpan implementation. It contains the following changes:

1. Expand prefix with one item at a time. The existing implementation generates all subsets for each itemset, which might have scalability issue when the itemset is large.
2. Use a new internal format. `<(12)(31)>` is represented by `[0, 1, 2, 0, 1, 3, 0]` internally. We use `0` because negative numbers are used to indicates partial prefix items, e.g., `_2` is represented by `-2`.
3. Remember the start indices of all partial projections in the projected postfix to help next projection.
4. Reuse the original sequence array for projected postfixes.
5. Use `Prefix` IDs in aggregation rather than its content.
6. Use `ArrayBuilder` for building primitive arrays.
7. Expose `maxLocalProjDBSize`.
8. Tests are not changed except using `0` instead of `-1` as the delimiter.

`Postfix`'s API doc should be a good place to start.

Closes #7594

feynmanliang zhangjiajin

Author: Xiangrui Meng <meng@databricks.com>

Closes #7937 from mengxr/SPARK-9540 and squashes the following commits:

2d0ec31 [Xiangrui Meng] address more comments
48f450c [Xiangrui Meng] address comments from Feynman; fixed a bug in project and added a test
65f90e8 [Xiangrui Meng] naming and documentation
8afc86a [Xiangrui Meng] refactor impl
2015-08-04 22:28:49 -07:00
Reynold Xin f7abd6bec9 Update docs/README.md to put all prereqs together.
This pull request groups all the prereq requirements into a single section.

cc srowen shivaram

Author: Reynold Xin <rxin@databricks.com>

Closes #7951 from rxin/readme-docs and squashes the following commits:

ab7ded0 [Reynold Xin] Updated docs/README.md to put all prereqs together.
2015-08-04 22:17:14 -07:00
zsxwing d34bac0e15 [SPARK-9504] [STREAMING] [TESTS] Fix o.a.s.streaming.StreamingContextSuite.stop gracefully again
The test failure is here: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3150/AMPLAB_JENKINS_BUILD_PROFILE=hadoop1.0,label=centos/testReport/junit/org.apache.spark.streaming/StreamingContextSuite/stop_gracefully/

There is a race condition in TestReceiver that it may add 1 record and increase `TestReceiver.counter` after stopping `BlockGenerator`. This PR just adds `join` to wait the pushing thread.

Author: zsxwing <zsxwing@gmail.com>

Closes #7934 from zsxwing/SPARK-9504-2 and squashes the following commits:

cfd7973 [zsxwing] Wait for the thread to make sure we won't change TestReceiver.counter after stopping BlockGenerator
2015-08-04 20:09:15 -07:00
Davies Liu 2b67fdb60b [SPARK-9513] [SQL] [PySpark] Add python API for DataFrame functions
This adds Python API for those DataFrame functions that is introduced in 1.5.

There is issue with serialize byte_array in Python 3, so some of functions (for BinaryType) does not have tests.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7922 from davies/python_functions and squashes the following commits:

8ad942f [Davies Liu] fix test
5fb6ec3 [Davies Liu] fix bugs
3495ed3 [Davies Liu] fix issues
ea5f7bb [Davies Liu] Add python API for DataFrame functions
2015-08-04 19:25:24 -07:00
zhichao.li 6f8f0e265a [SPARK-7119] [SQL] Give script a default serde with the user specific types
This is to address this issue that there would be not compatible type exception when running this:
`from (from src select transform(key, value) using 'cat' as (thing1 int, thing2 string)) t select thing1 + 2;`

15/04/24 00:58:55 ERROR CliDriver: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ClassCastException: org.apache.spark.sql.types.UTF8String cannot be cast to java.lang.Integer
	at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:106)
	at scala.math.Numeric$IntIsIntegral$.plus(Numeric.scala:57)
	at org.apache.spark.sql.catalyst.expressions.Add.eval(arithmetic.scala:127)
	at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:118)
	at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(Projection.scala:68)
	at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(Projection.scala:52)
	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)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:819)
	at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:819)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1618)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1618)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63)
	at org.apache.spark.scheduler.Task.run(Task.scala:64)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:209)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
	at java.lang.Thread.run(Thread.java:722)

chenghao-intel marmbrus

Author: zhichao.li <zhichao.li@intel.com>

Closes #6638 from zhichao-li/transDataType2 and squashes the following commits:

a36cc7c [zhichao.li] style
b9252a8 [zhichao.li] delete cacheRow
f6968a4 [zhichao.li] give script a default serde
2015-08-04 18:26:05 -07:00
Burak Yavuz c9a4c36d05 [SPARK-8313] R Spark packages support
shivaram cafreeman Could you please help me in testing this out? Exposing and running `rPackageBuilder` from inside the shell works, but for some reason, I can't get it to work during Spark Submit. It just starts relaunching Spark Submit.

For testing, you may use the R branch with [sbt-spark-package](https://github.com/databricks/sbt-spark-package). You can call spPackage, and then pass the jar using `--jars`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #7139 from brkyvz/r-submit and squashes the following commits:

0de384f [Burak Yavuz] remove unused imports 2
d253708 [Burak Yavuz] removed unused imports
6603d0d [Burak Yavuz] addressed comments
4258ffe [Burak Yavuz] merged master
ddfcc06 [Burak Yavuz] added zipping test
3a1be7d [Burak Yavuz] don't zip
77995df [Burak Yavuz] fix URI
ac45527 [Burak Yavuz] added zipping of all libs
e6bf7b0 [Burak Yavuz] add println ignores
1bc5554 [Burak Yavuz] add assumes for tests
9778e03 [Burak Yavuz] addressed comments
b42b300 [Burak Yavuz] merged master
ffd134e [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into r-submit
d867756 [Burak Yavuz] add apache header
eff5ba1 [Burak Yavuz] ready for review
8838edb [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into r-submit
e5b5a06 [Burak Yavuz] added doc
bb751ce [Burak Yavuz] fix null bug
0226768 [Burak Yavuz] fixed issues
8810beb [Burak Yavuz] R packages support
2015-08-04 18:20:12 -07:00
Yijie Shen a7fe48f687 [SPARK-9432][SQL] Audit expression unit tests to make sure we pass the proper numeric ranges
JIRA: https://issues.apache.org/jira/browse/SPARK-9432

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7933 from yjshen/numeric_ranges and squashes the following commits:

e719f78 [Yijie Shen] proper integral range check
2015-08-04 18:19:26 -07:00
Holden Karau d92fa14179 [SPARK-8601] [ML] Add an option to disable standardization for linear regression
All compressed sensing applications, and some of the regression use-cases will have better result by turning the feature scaling off. However, if we implement this naively by training the dataset without doing any standardization, the rate of convergency will not be good. This can be implemented by still standardizing the training dataset but we penalize each component differently to get effectively the same objective function but a better numerical problem. As a result, for those columns with high variances, they will be penalized less, and vice versa. Without this, since all the features are standardized, so they will be penalized the same.

In R, there is an option for this.
standardize

Logical flag for x variable standardization, prior to fitting the model sequence. The coefficients are always returned on the original scale. Default is standardize=TRUE. If variables are in the same units already, you might not wish to standardize. See details below for y standardization with family="gaussian".

Note that the primary author for this PR is holdenk

Author: Holden Karau <holden@pigscanfly.ca>
Author: DB Tsai <dbt@netflix.com>

Closes #7875 from dbtsai/SPARK-8522 and squashes the following commits:

e856036 [DB Tsai] scala doc
596e96c [DB Tsai] minor
bbff347 [DB Tsai] naming
baa0805 [DB Tsai] touch up
d6234ba [DB Tsai] Merge branch 'master' into SPARK-8522-Disable-Linear_featureScaling-Spark-8601-in-Linear_regression
6b1dc09 [Holden Karau] Merge branch 'master' into SPARK-8522-Disable-Linear_featureScaling-Spark-8601-in-Linear_regression
332f140 [Holden Karau] Merge in master
eebe10a [Holden Karau] Use same comparision operator throughout the test
3f92935 [Holden Karau] merge
b83a41e [Holden Karau] Expand the tests and make them similar to the other PR also providing an option to disable standardization (but for LoR).
0c334a2 [Holden Karau] Remove extra line
99ce053 [Holden Karau] merge in master
e54a8a9 [Holden Karau] Fix long line
e47c574 [Holden Karau] Add support for L2 without standardization.
55d3a66 [Holden Karau] Add standardization param for linear regression
00a1dc5 [Holden Karau] Add the param to the linearregression impl
2015-08-04 18:15:26 -07:00
Feynman Liang 629e26f7ee [SPARK-9609] [MLLIB] Fix spelling of Strategy.defaultStrategy
jkbradley

Author: Feynman Liang <fliang@databricks.com>

Closes #7941 from feynmanliang/SPARK-9609-stategy-spelling and squashes the following commits:

d2aafb1 [Feynman Liang] Add deprecated backwards compatibility
aa090a8 [Feynman Liang] Fix spelling
2015-08-04 18:13:18 -07:00
Wenchen Fan 7c8fc1f7cb [SPARK-9598][SQL] do not expose generic getter in internal row
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7932 from cloud-fan/generic-getter and squashes the following commits:

c60de4c [Wenchen Fan] do not expose generic getter in internal row
2015-08-04 17:05:19 -07:00
Joseph K. Bradley b77d3b9688 [SPARK-9586] [ML] Update BinaryClassificationEvaluator to use setRawPredictionCol
Update BinaryClassificationEvaluator to use setRawPredictionCol, rather than setScoreCol. Deprecated setScoreCol.

I don't think setScoreCol was actually used anywhere (based on search).

CC: mengxr

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

Closes #7921 from jkbradley/binary-eval-rawpred and squashes the following commits:

e5d7dfa [Joseph K. Bradley] Update BinaryClassificationEvaluator to use setRawPredictionCol
2015-08-04 16:52:43 -07:00
Mike Dusenberry 571d5b5363 [SPARK-6485] [MLLIB] [PYTHON] Add CoordinateMatrix/RowMatrix/IndexedRowMatrix to PySpark.
This PR adds the RowMatrix, IndexedRowMatrix, and CoordinateMatrix distributed matrices to PySpark.  Each distributed matrix class acts as a wrapper around the Scala/Java counterpart by maintaining a reference to the Java object.  New distributed matrices can be created using factory methods added to DistributedMatrices, which creates the Java distributed matrix and then wraps it with the corresponding PySpark class.  This design allows for simple conversion between the various distributed matrices, and lets us re-use the Scala code.  Serialization between Python and Java is implemented using DataFrames as needed for IndexedRowMatrix and CoordinateMatrix for simplicity.  Associated documentation and unit-tests have also been added.  To facilitate code review, this PR implements access to the rows/entries as RDDs, the number of rows & columns, and conversions between the various distributed matrices (not including BlockMatrix), and does not implement the other linear algebra functions of the matrices, although this will be very simple to add now.

Author: Mike Dusenberry <mwdusenb@us.ibm.com>

Closes #7554 from dusenberrymw/SPARK-6485_Add_CoordinateMatrix_RowMatrix_IndexedMatrix_to_PySpark and squashes the following commits:

bb039cb [Mike Dusenberry] Minor documentation update.
b887c18 [Mike Dusenberry] Updating the matrix conversion logic again to make it even cleaner.  Now, we allow the 'rows' parameter in the constructors to be either an RDD or the Java matrix object. If 'rows' is an RDD, we create a Java matrix object, wrap it, and then store that.  If 'rows' is a Java matrix object of the correct type, we just wrap and store that directly.  This is only for internal usage, and publicly, we still require 'rows' to be an RDD.  We no longer store the 'rows' RDD, and instead just compute it from the Java object when needed.  The point of this is that when we do matrix conversions, we do the conversion on the Scala/Java side, which returns a Java object, so we should use that directly, but exposing 'java_matrix' parameter in the public API is not ideal. This non-public feature of allowing 'rows' to be a Java matrix object is documented in the '__init__' constructor docstrings, which are not part of the generated public API, and doctests are also included.
7f0dcb6 [Mike Dusenberry] Updating module docstring.
cfc1be5 [Mike Dusenberry] Use 'new SQLContext(matrix.rows.sparkContext)' rather than 'SQLContext.getOrCreate', as the later doesn't guarantee that the SparkContext will be the same as for the matrix.rows data.
687e345 [Mike Dusenberry] Improving conversion performance.  This adds an optional 'java_matrix' parameter to the constructors, and pulls the conversion logic out into a '_create_from_java' function. Now, if the constructors are given a valid Java distributed matrix object as 'java_matrix', they will store those internally, rather than create a new one on the Scala/Java side.
3e50b6e [Mike Dusenberry] Moving the distributed matrices to pyspark.mllib.linalg.distributed.
308f197 [Mike Dusenberry] Using properties for better documentation.
1633f86 [Mike Dusenberry] Minor documentation cleanup.
f0c13a7 [Mike Dusenberry] CoordinateMatrix should inherit from DistributedMatrix.
ffdd724 [Mike Dusenberry] Updating doctests to make documentation cleaner.
3fd4016 [Mike Dusenberry] Updating docstrings.
27cd5f6 [Mike Dusenberry] Simplifying input conversions in the constructors for each distributed matrix.
a409cf5 [Mike Dusenberry] Updating doctests to be less verbose by using lists instead of DenseVectors explicitly.
d19b0ba [Mike Dusenberry] Updating code and documentation to note that a vector-like object (numpy array, list, etc.) can be used in place of explicit Vector object, and adding conversions when necessary to RowMatrix construction.
4bd756d [Mike Dusenberry] Adding param documentation to IndexedRow and MatrixEntry.
c6bded5 [Mike Dusenberry] Move conversion logic from tuples to IndexedRow or MatrixEntry types from within the IndexedRowMatrix and CoordinateMatrix constructors to separate _convert_to_indexed_row and _convert_to_matrix_entry functions.
329638b [Mike Dusenberry] Moving the Experimental tag to the top of each docstring.
0be6826 [Mike Dusenberry] Simplifying doctests by removing duplicated rows/entries RDDs within the various tests.
c0900df [Mike Dusenberry] Adding the colons that were accidentally not inserted.
4ad6819 [Mike Dusenberry] Documenting the  and  parameters.
3b854b9 [Mike Dusenberry] Minor updates to documentation.
10046e8 [Mike Dusenberry] Updating documentation to use class constructors instead of the removed DistributedMatrices factory methods.
119018d [Mike Dusenberry] Adding static  methods to each of the distributed matrix classes to consolidate conversion logic.
4d7af86 [Mike Dusenberry] Adding type checks to the constructors.  Although it is slightly verbose, it is better for the user to have a good error message than a cryptic stacktrace.
93b6a3d [Mike Dusenberry] Pulling the DistributedMatrices Python class out of this pull request.
f6f3c68 [Mike Dusenberry] Pulling the DistributedMatrices Scala class out of this pull request.
6a3ecb7 [Mike Dusenberry] Updating pattern matching.
08f287b [Mike Dusenberry] Slight reformatting of the documentation.
a245dc0 [Mike Dusenberry] Updating Python doctests for compatability between Python 2 & 3. Since Python 3 removed the idea of a separate 'long' type, all values that would have been outputted as a 'long' (ex: '4L') will now be treated as an 'int' and outputed as one (ex: '4').  The doctests now explicitly convert to ints so that both Python 2 and 3 will have the same output.  This is fine since the values are all small, and thus can be easily represented as ints.
4d3a37e [Mike Dusenberry] Reformatting a few long Python doctest lines.
7e3ca16 [Mike Dusenberry] Fixing long lines.
f721ead [Mike Dusenberry] Updating documentation for each of the distributed matrices.
ab0e8b6 [Mike Dusenberry] Updating unit test to be more useful.
dda2f89 [Mike Dusenberry] Added wrappers for the conversions between the various distributed matrices.  Added logic to be able to access the rows/entries of the distributed matrices, which requires serialization through DataFrames for IndexedRowMatrix and CoordinateMatrix types. Added unit tests.
0cd7166 [Mike Dusenberry] Implemented the CoordinateMatrix API in PySpark, following the idea of the IndexedRowMatrix API, including using DataFrames for serialization.
3c369cb [Mike Dusenberry] Updating the architecture a bit to make conversions between the various distributed matrix types easier.  The different distributed matrix classes are now only wrappers around the Java objects, and take the Java object as an argument during construction.  This way, we can call  for example on an , which returns a reference to a Java RowMatrix object, and then construct a PySpark RowMatrix object wrapped around the Java object.  This is analogous to the behavior of PySpark RDDs and DataFrames.  We now delegate creation of the various distributed matrices from scratch in PySpark to the factory methods on .
4bdd09b [Mike Dusenberry] Implemented the IndexedRowMatrix API in PySpark, following the idea of the RowMatrix API.  Note that for the IndexedRowMatrix, we use DataFrames to serialize the data between Python and Scala/Java, so we accept PySpark RDDs, then convert to a DataFrame, then convert back to RDDs on the Scala/Java side before constructing the IndexedRowMatrix.
23bf1ec [Mike Dusenberry] Updating documentation to add PySpark RowMatrix. Inserting newline above doctest so that it renders properly in API docs.
b194623 [Mike Dusenberry] Updating design to have a PySpark RowMatrix simply create and keep a reference to a wrapper over a Java RowMatrix.  Updating DistributedMatrices factory methods to accept numRows and numCols with default values.  Updating PySpark DistributedMatrices factory method to simply create a PySpark RowMatrix. Adding additional doctests for numRows and numCols parameters.
bc2d220 [Mike Dusenberry] Adding unit tests for RowMatrix methods.
d7e316f [Mike Dusenberry] Implemented the RowMatrix API in PySpark by doing the following: Added a DistributedMatrices class to contain factory methods for creating the various distributed matrices.  Added a factory method for creating a RowMatrix from an RDD of Vectors.  Added a createRowMatrix function to the PythonMLlibAPI to interface with the factory method.  Added DistributedMatrix, DistributedMatrices, and RowMatrix classes to the pyspark.mllib.linalg api.
2015-08-04 16:30:03 -07:00
Joseph K. Bradley 1833d9c08f [SPARK-9582] [ML] LDA cleanups
Small cleanups to recent LDA additions and docs.

CC: feynmanliang

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

Closes #7916 from jkbradley/lda-cleanups and squashes the following commits:

f7021d9 [Joseph K. Bradley] broadcasting large matrices for LDA in local model and online learning
97947aa [Joseph K. Bradley] a few more cleanups
5b03f88 [Joseph K. Bradley] reverted split of lda log likelihood
c566915 [Joseph K. Bradley] small edit to make review easier
63f6c7d [Joseph K. Bradley] clarified log likelihood for lda models
2015-08-04 15:43:13 -07:00
Joseph K. Bradley e375456063 [SPARK-9447] [ML] [PYTHON] Added HasRawPredictionCol, HasProbabilityCol to RandomForestClassifier
Added HasRawPredictionCol, HasProbabilityCol to RandomForestClassifier, plus doc tests for those columns.

CC: holdenk yanboliang

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

Closes #7903 from jkbradley/rf-prob-python and squashes the following commits:

c62a83f [Joseph K. Bradley] made unit test more robust
14eeba2 [Joseph K. Bradley] added HasRawPredictionCol, HasProbabilityCol to RandomForestClassifier in PySpark
2015-08-04 14:54:26 -07:00
CodingCat 9d668b7368 [SPARK-9602] remove "Akka/Actor" words from comments
https://issues.apache.org/jira/browse/SPARK-9602

Although we have hidden Akka behind RPC interface, I found that the Akka/Actor-related comments are still spreading everywhere. To make it consistent, we shall remove "actor"/"akka" words from the comments...

Author: CodingCat <zhunansjtu@gmail.com>

Closes #7936 from CodingCat/SPARK-9602 and squashes the following commits:

e8296a3 [CodingCat] remove actor words from comments
2015-08-04 14:54:11 -07:00
Josh Rosen ab8ee1a3b9 [SPARK-9452] [SQL] Support records larger than page size in UnsafeExternalSorter
This patch extends UnsafeExternalSorter to support records larger than the page size. The basic strategy is the same as in #7762: store large records in their own overflow pages.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7891 from JoshRosen/large-records-in-sql-sorter and squashes the following commits:

967580b [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
948c344 [Josh Rosen] Add large records tests for KV sorter.
3c17288 [Josh Rosen] Combine memory and disk cleanup into general cleanupResources() method
380f217 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
27eafa0 [Josh Rosen] Fix page size in PackedRecordPointerSuite
a49baef [Josh Rosen] Address initial round of review comments
3edb931 [Josh Rosen] Remove accidentally-committed debug statements.
2b164e2 [Josh Rosen] Support large records in UnsafeExternalSorter.
2015-08-04 14:42:11 -07:00
Wenchen Fan f4b1ac08a1 [SPARK-9553][SQL] remove the no-longer-necessary createCode and createStructCode, and replace the usage
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7890 from cloud-fan/minor and squashes the following commits:

c3b1be3 [Wenchen Fan] fix style
b0cbe2e [Wenchen Fan] remove the createCode and createStructCode, and replace the usage of them by createStructCode
2015-08-04 14:40:46 -07:00
Michael Armbrust a0cc01759b [SPARK-9606] [SQL] Ignore flaky thrift server tests
Author: Michael Armbrust <michael@databricks.com>

Closes #7939 from marmbrus/turnOffThriftTests and squashes the following commits:

80d618e [Michael Armbrust] [SPARK-9606][SQL] Ignore flaky thrift server tests
2015-08-04 12:19:52 -07:00
Holden Karau 5a23213c14 [SPARK-8069] [ML] Add multiclass thresholds for ProbabilisticClassifier
This PR replaces the old "threshold" with a generalized "thresholds" Param.  We keep getThreshold,setThreshold for backwards compatibility for binary classification.

Note that the primary author of this PR is holdenk

Author: Holden Karau <holden@pigscanfly.ca>
Author: Joseph K. Bradley <joseph@databricks.com>

Closes #7909 from jkbradley/holdenk-SPARK-8069-add-cutoff-aka-threshold-to-random-forest and squashes the following commits:

3952977 [Joseph K. Bradley] fixed pyspark doc test
85febc8 [Joseph K. Bradley] made python unit tests a little more robust
7eb1d86 [Joseph K. Bradley] small cleanups
6cc2ed8 [Joseph K. Bradley] Fixed remaining merge issues.
0255e44 [Joseph K. Bradley] Many cleanups for thresholds, some more tests
7565a60 [Holden Karau] fix pep8 style checks, add a getThreshold method similar to our LogisticRegression.scala one for API compat
be87f26 [Holden Karau] Convert threshold to thresholds in the python code, add specialized support for Array[Double] to shared parems codegen, etc.
6747dad [Holden Karau] Override raw2prediction for ProbabilisticClassifier, fix some tests
25df168 [Holden Karau] Fix handling of thresholds in LogisticRegression
c02d6c0 [Holden Karau] No default for thresholds
5e43628 [Holden Karau] CR feedback and fixed the renamed test
f3fbbd1 [Holden Karau] revert the changes to random forest :(
51f581c [Holden Karau] Add explicit types to public methods, fix long line
f7032eb [Holden Karau] Fix a java test bug, remove some unecessary changes
adf15b4 [Holden Karau] rename the classifier suite test to ProbabilisticClassifierSuite now that we only have it in Probabilistic
398078a [Holden Karau] move the thresholding around a bunch based on the design doc
4893bdc [Holden Karau] Use numtrees of 3 since previous result was tied (one tree for each) and the switch from different max methods picked a different element (since they were equal I think this is ok)
638854c [Holden Karau] Add a scala RandomForestClassifierSuite test based on corresponding python test
e09919c [Holden Karau] Fix return type, I need more coffee....
8d92cac [Holden Karau] Use ClassifierParams as the head
3456ed3 [Holden Karau] Add explicit return types even though just test
a0f3b0c [Holden Karau] scala style fixes
6f14314 [Holden Karau] Since hasthreshold/hasthresholds is in root classifier now
ffc8dab [Holden Karau] Update the sharedParams
0420290 [Holden Karau] Allow us to override the get methods selectively
978e77a [Holden Karau] Move HasThreshold into classifier params and start defining the overloaded getThreshold/getThresholds functions
1433e52 [Holden Karau] Revert "try and hide threshold but chainges the API so no dice there"
1f09a2e [Holden Karau] try and hide threshold but chainges the API so no dice there
efb9084 [Holden Karau] move setThresholds only to where its used
6b34809 [Holden Karau] Add a test with thresholding for the RFCS
74f54c3 [Holden Karau] Fix creation of vote array
1986fa8 [Holden Karau] Setting the thresholds only makes sense if the underlying class hasn't overridden predict, so lets push it down.
2f44b18 [Holden Karau] Add a global default of null for thresholds param
f338cfc [Holden Karau] Wait that wasn't a good idea, Revert "Some progress towards unifying threshold and thresholds"
634b06f [Holden Karau] Some progress towards unifying threshold and thresholds
85c9e01 [Holden Karau] Test passes again... little fnur
099c0f3 [Holden Karau] Move thresholds around some more (set on model not trainer)
0f46836 [Holden Karau] Start adding a classifiersuite
f70eb5e [Holden Karau] Fix test compile issues
a7d59c8 [Holden Karau] Move thresholding into Classifier trait
5d999d2 [Holden Karau] Some more progress, start adding a test (maybe try and see if we can find a better thing to use for the base of the test)
1fed644 [Holden Karau] Use thresholds to scale scores in random forest classifcation
31d6bf2 [Holden Karau] Start threading the threshold info through
0ef228c [Holden Karau] Add hasthresholds
2015-08-04 10:12:22 -07:00