aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-13234] [SQL] remove duplicated SQL metricsDavies Liu2016-02-1024-208/+80
| | | | | | | | | | | | | | | | For lots of SQL operators, we have metrics for both of input and output, the number of input rows should be exactly the number of output rows of child, we could only have metrics for output rows. After we improved the performance using whole stage codegen, the overhead of SQL metrics are not trivial anymore, we should avoid that if it's not necessary. This PR remove all the SQL metrics for number of input rows, add SQL metric of number of output rows for all LeafNode. All remove the SQL metrics from those operators that have the same number of rows from input and output (for example, Projection, we may don't need that). The new SQL UI will looks like: ![metrics](https://cloud.githubusercontent.com/assets/40902/12965227/63614e5e-d009-11e5-88b3-84fea04f9c20.png) Author: Davies Liu <davies@databricks.com> Closes #11163 from davies/remove_metrics.
* [SPARK-12706] [SQL] grouping() and grouping_id()Davies Liu2016-02-1017-63/+309
| | | | | | | | | | | | Grouping() returns a column is aggregated or not, grouping_id() returns the aggregation levels. grouping()/grouping_id() could be used with window function, but does not work in having/sort clause, will be fixed by another PR. The GROUPING__ID/grouping_id() in Hive is wrong (according to docs), we also did it wrongly, this PR change that to match the behavior in most databases (also the docs of Hive). Author: Davies Liu <davies@databricks.com> Closes #10677 from davies/grouping.
* [SPARK-13205][SQL] SQL Generation Support for Self Joingatorsmile2016-02-113-2/+22
| | | | | | | | | | | | This PR addresses two issues: - Self join does not work in SQL Generation - When creating new instances for `LogicalRelation`, `metastoreTableIdentifier` is lost. liancheng Could you please review the code changes? Thank you! Author: gatorsmile <gatorsmile@gmail.com> Closes #11084 from gatorsmile/selfJoinInSQLGen.
* [SPARK-12725][SQL] Resolving Name Conflicts in SQL Generation and Name ↵gatorsmile2016-02-119-36/+78
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Ambiguity Caused by Internally Generated Expressions Some analysis rules generate aliases or auxiliary attribute references with the same name but different expression IDs. For example, `ResolveAggregateFunctions` introduces `havingCondition` and `aggOrder`, and `DistinctAggregationRewriter` introduces `gid`. This is OK for normal query execution since these attribute references get expression IDs. However, it's troublesome when converting resolved query plans back to SQL query strings since expression IDs are erased. Here's an example Spark 1.6.0 snippet for illustration: ```scala sqlContext.range(10).select('id as 'a, 'id as 'b).registerTempTable("t") sqlContext.sql("SELECT SUM(a) FROM t GROUP BY a, b ORDER BY COUNT(a), COUNT(b)").explain(true) ``` The above code produces the following resolved plan: ``` == Analyzed Logical Plan == _c0: bigint Project [_c0#101L] +- Sort [aggOrder#102L ASC,aggOrder#103L ASC], true +- Aggregate [a#47L,b#48L], [(sum(a#47L),mode=Complete,isDistinct=false) AS _c0#101L,(count(a#47L),mode=Complete,isDistinct=false) AS aggOrder#102L,(count(b#48L),mode=Complete,isDistinct=false) AS aggOrder#103L] +- Subquery t +- Project [id#46L AS a#47L,id#46L AS b#48L] +- LogicalRDD [id#46L], MapPartitionsRDD[44] at range at <console>:26 ``` Here we can see that both aggregate expressions in `ORDER BY` are extracted into an `Aggregate` operator, and both of them are named `aggOrder` with different expression IDs. The solution is to automatically add the expression IDs into the attribute name for the Alias and AttributeReferences that are generated by Analyzer in SQL Generation. In this PR, it also resolves another issue. Users could use the same name as the internally generated names. The duplicate names should not cause name ambiguity. When resolving the column, Catalyst should not pick the column that is internally generated. Could you review the solution? marmbrus liancheng I did not set the newly added flag for all the alias and attribute reference generated by Analyzers. Please let me know if I should do it? Thank you! Author: gatorsmile <gatorsmile@gmail.com> Closes #11050 from gatorsmile/namingConflicts.
* [SPARK-13274] Fix Aggregator Links on GroupedDataset Scala APIraela2016-02-101-4/+8
| | | | | | | | Update Aggregator links to point to #org.apache.spark.sql.expressions.Aggregator Author: raela <raela@databricks.com> Closes #11158 from raelawang/master.
* [SPARK-13146][SQL] Management API for continuous queriesTathagata Das2016-02-1017-109/+1680
| | | | | | | | | | | | | | | | | | | | | | | | | | ### Management API for Continuous Queries **API for getting status of each query** - Whether active or not - Unique name of each query - Status of the sources and sinks - Exceptions **API for managing each query** - Immediately stop an active query - Waiting for a query to be terminated, correctly or with error **API for managing multiple queries** - Listing all active queries - Getting an active query by name - Waiting for any one of the active queries to be terminated **API for listening to query life cycle events** - ContinuousQueryListener API for query start, progress and termination events. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #11030 from tdas/streaming-df-management-api.
* [SPARK-12414][CORE] Remove closure serializerSean Owen2016-02-104-14/+3
| | | | | | | | | | Remove spark.closure.serializer option and use JavaSerializer always CC andrewor14 rxin I see there's a discussion in the JIRA but just thought I'd offer this for a look at what the change would be. Author: Sean Owen <sowen@cloudera.com> Closes #11150 from srowen/SPARK-12414.
* [SPARK-13057][SQL] Add benchmark codes and the performance results for ↵Takeshi YAMAMURO2016-02-101-0/+240
| | | | | | | | | | implemented compression schemes for InMemoryRelation This pr adds benchmark codes for in-memory cache compression to make future developments and discussions more smooth. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #10965 from maropu/ImproveColumnarCache.
* [HOTFIX] Fix Scala 2.10 build break in TakeOrderedAndProjectSuite.Josh Rosen2016-02-101-2/+2
|
* [SPARK-13126] fix the right margin of history page.zhuol2016-02-101-1/+1
| | | | | | | | The right margin of the history page is little bit off. A simple fix for that issue. Author: zhuol <zhuol@yahoo-inc.com> Closes #11029 from zhuoliu/13126.
* [SPARK-13163][WEB UI] Column width on new History Server DataTables not ↵Alex Bozarth2016-02-101-0/+1
| | | | | | | | | | getting set correctly The column width for the new DataTables now adjusts for the current page rather than being hard-coded for the entire table's data. Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #11057 from ajbozarth/spark13163.
* [SPARK-13254][SQL] Fix planning of TakeOrderedAndProject operatorJosh Rosen2016-02-106-44/+159
| | | | | | | | | | | | | | The patch for SPARK-8964 ("use Exchange to perform shuffle in Limit" / #7334) inadvertently broke the planning of the TakeOrderedAndProject operator: because ReturnAnswer was the new root of the query plan, the TakeOrderedAndProject rule was unable to match before BasicOperators. This patch fixes this by moving the `TakeOrderedAndCollect` and `CollectLimit` rules into the same strategy. In addition, I made changes to the TakeOrderedAndProject operator in order to make its `doExecute()` method lazy and added a new TakeOrderedAndProjectSuite which tests the new code path. /cc davies and marmbrus for review. Author: Josh Rosen <joshrosen@databricks.com> Closes #11145 from JoshRosen/take-ordered-and-project-fix.
* [SPARK-5095][MESOS] Support launching multiple mesos executors in coarse ↵Michael Gummelt2016-02-109-275/+521
| | | | | | | | | | | | | | | | | | | | grained mesos mode. This is the next iteration of tnachen's previous PR: https://github.com/apache/spark/pull/4027 In that PR, we resolved with andrewor14 and pwendell to implement the Mesos scheduler's support of `spark.executor.cores` to be consistent with YARN and Standalone. This PR implements that resolution. This PR implements two high-level features. These two features are co-dependent, so they're implemented both here: - Mesos support for spark.executor.cores - Multiple executors per slave We at Mesosphere have been working with Typesafe on a Spark/Mesos integration test suite: https://github.com/typesafehub/mesos-spark-integration-tests, which passes for this PR. The contribution is my original work and I license the work to the project under the project's open source license. Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #10993 from mgummelt/executor_sizing.
* [SPARK-9307][CORE][SPARK] Logging: Make it either stable or privateSean Owen2016-02-101-6/+2
| | | | | | | | Make Logging private[spark]. Pretty much all there is to it. Author: Sean Owen <sowen@cloudera.com> Closes #11103 from srowen/SPARK-9307.
* [SPARK-13203] Add scalastyle rule banning use of mutable.SynchronizedBuffertedyu2016-02-101-0/+12
| | | | | | | | | andrewor14 Please take a look Author: tedyu <yuzhihong@gmail.com> Closes #11134 from tedyu/master.
* [SPARK-11518][DEPLOY, WINDOWS] Handle spaces in Windows command scriptsJon Maurer2016-02-1014-30/+27
| | | | | | | Author: Jon Maurer <tritab@gmail.com> Author: Jonathan Maurer <jmaurer@Jonathans-MacBook-Pro.local> Closes #10789 from tritab/cmd_updates.
* [SPARK-11565] Replace deprecated DigestUtils.shaHex callGábor Lipták2016-02-102-2/+6
| | | | | | Author: Gábor Lipták <gliptak@gmail.com> Closes #9532 from gliptak/SPARK-11565.
* [SPARK-13149][SQL] Add FileStreamSourceShixiong Zhu2016-02-096-7/+710
| | | | | | | | | | `FileStreamSource` is an implementation of `org.apache.spark.sql.execution.streaming.Source`. It takes advantage of the existing `HadoopFsRelationProvider` to support various file formats. It remembers files in each batch and stores it into the metadata files so as to recover them when restarting. The metadata files are stored in the file system. There will be a further PR to clean up the metadata files periodically. This is based on the initial work from marmbrus. Author: Shixiong Zhu <shixiong@databricks.com> Closes #11034 from zsxwing/stream-df-file-source.
* [SPARK-12476][SQL] Implement JdbcRelation#unhandledFilters for removing ↵Takeshi YAMAMURO2016-02-103-21/+56
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | unnecessary Spark Filter Input: SELECT * FROM jdbcTable WHERE col0 = 'xxx' Current plan: ``` == Optimized Logical Plan == Project [col0#0,col1#1] +- Filter (col0#0 = xxx) +- Relation[col0#0,col1#1] JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver}) == Physical Plan == +- Filter (col0#0 = xxx) +- Scan JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver})[col0#0,col1#1] PushedFilters: [EqualTo(col0,xxx)] ``` This patch enables a plan below; ``` == Optimized Logical Plan == Project [col0#0,col1#1] +- Filter (col0#0 = xxx) +- Relation[col0#0,col1#1] JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver}) == Physical Plan == Scan JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver})[col0#0,col1#1] PushedFilters: [EqualTo(col0,xxx)] ``` Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #10427 from maropu/RemoveFilterInJdbcScan.
* [SPARK-10524][ML] Use the soft prediction to order categories' binsLiang-Chi Hsieh2016-02-094-133/+194
| | | | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-10524 Currently we use the hard prediction (`ImpurityCalculator.predict`) to order categories' bins. But we should use the soft prediction. Author: Liang-Chi Hsieh <viirya@gmail.com> Author: Liang-Chi Hsieh <viirya@appier.com> Author: Joseph K. Bradley <joseph@databricks.com> Closes #8734 from viirya/dt-soft-centroids.
* [SPARK-12950] [SQL] Improve lookup of BytesToBytesMap in aggregateDavies Liu2016-02-0910-127/+182
| | | | | | | | | | | | This PR improve the lookup of BytesToBytesMap by: 1. Generate code for calculate the hash code of grouping keys. 2. Do not use MemoryLocation, fetch the baseObject and offset for key and value directly (remove the indirection). Author: Davies Liu <davies@databricks.com> Closes #11010 from davies/gen_map.
* [SPARK-13245][CORE] Call shuffleMetrics methods only in one thread for ↵Shixiong Zhu2016-02-091-11/+27
| | | | | | | | | | | | ShuffleBlockFetcherIterator Call shuffleMetrics's incRemoteBytesRead and incRemoteBlocksFetched when polling FetchResult from `results` so as to always use shuffleMetrics in one thread. Also fix a race condition that could cause memory leak. Author: Shixiong Zhu <shixiong@databricks.com> Closes #11138 from zsxwing/SPARK-13245.
* [SPARK-12888] [SQL] [FOLLOW-UP] benchmark the new hash expressionWenchen Fan2016-02-092-9/+35
| | | | | | | | | | | | | | | | Adds the benchmark results as comments. The codegen version is slower than the interpreted version for `simple` case becasue of 3 reasons: 1. codegen version use a more complex hash algorithm than interpreted version, i.e. `Murmur3_x86_32.hashInt` vs [simple multiplication and addition](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala#L153). 2. codegen version will write the hash value to a row first and then read it out. I tried to create a `GenerateHasher` that can generate code to return hash value directly and got about 60% speed up for the `simple` case, does it worth? 3. the row in `simple` case only has one int field, so the runtime reflection may be removed because of branch prediction, which makes the interpreted version faster. The `array` case is also slow for similar reasons, e.g. array elements are of same type, so interpreted version can probably get rid of runtime reflection by branch prediction. Author: Wenchen Fan <wenchen@databricks.com> Closes #10917 from cloud-fan/hash-benchmark.
* [SPARK-13189] Cleanup build references to Scala 2.10Luciano Resende2016-02-096-19/+20
| | | | | | Author: Luciano Resende <lresende@apache.org> Closes #11092 from lresende/SPARK-13189.
* [SPARK-12807][YARN] Spark External Shuffle not working in Hadoop clusters ↵Steve Loughran2016-02-091-1/+48
| | | | | | | | | | | | | | | with Jackson 2.2.3 Patch to 1. Shade jackson 2.x in spark-yarn-shuffle JAR: core, databind, annotation 2. Use maven antrun to verify the JAR has the renamed classes Being Maven-based, I don't know if the verification phase kicks in on an SBT/jenkins build. It will on a `mvn install` Author: Steve Loughran <stevel@hortonworks.com> Closes #10780 from steveloughran/stevel/patches/SPARK-12807-master-shuffle.
* [SPARK-13170][STREAMING] Investigate replacing SynchronizedQueue as it is ↵Sean Owen2016-02-094-23/+39
| | | | | | | | | | deprecated Replace SynchronizeQueue with synchronized access to a Queue Author: Sean Owen <sowen@cloudera.com> Closes #11111 from srowen/SPARK-13170.
* [SPARK-13086][SHELL] Use the Scala REPL settings, to enable things like `-i ↵Iulian Dragos2016-02-091-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | file`. Now: ``` $ bin/spark-shell -i test.scala NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). 16/01/29 17:37:38 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 16/01/29 17:37:39 INFO Main: Created spark context.. Spark context available as sc (master = local[*], app id = local-1454085459000). 16/01/29 17:37:39 INFO Main: Created sql context.. SQL context available as sqlContext. Loading test.scala... hello Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.0.0-SNAPSHOT /_/ Using Scala version 2.11.7 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_45) Type in expressions to have them evaluated. Type :help for more information. ``` Author: Iulian Dragos <jaguarul@gmail.com> Closes #10984 from dragos/issue/repl-eval-file.
* [SPARK-13177][EXAMPLES] Update ActorWordCount example to not directly use ↵sachin aggarwal2016-02-091-4/+4
| | | | | | | | low level linked list as it is deprecated. Author: sachin aggarwal <different.sachin@gmail.com> Closes #11113 from agsachin/master.
* [SPARK-13040][DOCS] Update JDBC deprecated SPARK_CLASSPATH documentationSebastián Ramírez2016-02-091-1/+1
| | | | | | | | | | | | Update JDBC documentation based on http://stackoverflow.com/a/30947090/219530 as SPARK_CLASSPATH is deprecated. Also, that's how it worked, it didn't work with the SPARK_CLASSPATH or the --jars alone. This would solve issue: https://issues.apache.org/jira/browse/SPARK-13040 Author: Sebastián Ramírez <tiangolo@gmail.com> Closes #10948 from tiangolo/patch-docs-jdbc.
* [SPARK-13201][SPARK-13200] Deprecation warning cleanups: KMeans & ↵Holden Karau2016-02-093-5/+13
| | | | | | | | | | | | | | | | MFDataGenerator KMeans: Make a private non-deprecated version of setRuns API so that we can call it from the PythonAPI without deprecation warnings in our own build. Also use it internally when being called from train. Add a logWarning for non-1 values MFDataGenerator: Apparently we are calling round on an integer which now in Scala 2.11 results in a warning (it didn't make any sense before either). Figure out if this is a mistake we can just remove or if we got the types wrong somewhere. I put these two together since they are both deprecation fixes in MLlib and pretty small, but I can split them up if we would prefer it that way. Author: Holden Karau <holden@us.ibm.com> Closes #11112 from holdenk/SPARK-13201-non-deprecated-setRuns-SPARK-mathround-integer.
* [SPARK-13165][STREAMING] Replace deprecated synchronizedBuffer in streamingHolden Karau2016-02-0919-215/+226
| | | | | | | | | | | | Building with Scala 2.11 results in the warning trait SynchronizedBuffer in package mutable is deprecated: Synchronization via traits is deprecated as it is inherently unreliable. Consider java.util.concurrent.ConcurrentLinkedQueue as an alternative - we already use ConcurrentLinkedQueue elsewhere so lets replace it. Some notes about how behaviour is different for reviewers: The Seq from a SynchronizedBuffer that was implicitly converted would continue to receive updates - however when we do the same conversion explicitly on the ConcurrentLinkedQueue this isn't the case. Hence changing some of the (internal & test) APIs to pass an Iterable. toSeq is safe to use if there are no more updates. Author: Holden Karau <holden@us.ibm.com> Author: tedyu <yuzhihong@gmail.com> Closes #11067 from holdenk/SPARK-13165-replace-deprecated-synchronizedBuffer-in-streaming.
* [SPARK-13176][CORE] Use native file linking instead of external process lnJakob Odersky2016-02-091-19/+8
| | | | | | | | Since Spark requires at least JRE 1.7, it is safe to use built-in java.nio.Files. Author: Jakob Odersky <jakob@odersky.com> Closes #11098 from jodersky/SPARK-13176.
* [SPARK-12992] [SQL] Support vectorized decoding in UnsafeRowParquetRecordReader.Nong Li2016-02-0816-90/+549
| | | | | | | | | | | | | | | | | WIP: running tests. Code needs a bit of clean up. This patch completes the vectorized decoding with the goal of passing the existing tests. There is still more patches to support the rest of the format spec, even just for flat schemas. This patch adds a new flag to enable the vectorized decoding. Tests were updated to try with both modes where applicable. Once this is working well, we can remove the previous code path. Author: Nong Li <nong@databricks.com> Closes #11055 from nongli/spark-12992-2.
* [SPARK-10620][SPARK-13054] Minor addendum to #10835Andrew Or2016-02-0817-49/+66
| | | | | | | | Additional changes to #10835, mainly related to style and visibility. This patch also adds back a few deprecated methods for backward compatibility. Author: Andrew Or <andrew@databricks.com> Closes #10958 from andrewor14/task-metrics-to-accums-followups.
* [SPARK-13095] [SQL] improve performance for broadcast join with dimension tableDavies Liu2016-02-088-69/+438
| | | | | | | | | | | | | | | | This PR improve the performance for Broadcast join with dimension tables, which is common in data warehouse. If the join key can fit in a long, we will use a special api `get(Long)` to get the rows from HashedRelation. If the HashedRelation only have unique keys, we will use a special api `getValue(Long)` or `getValue(InternalRow)`. If the keys can fit within a long, also the keys are dense, we will use a array of UnsafeRow, instead a hash map. TODO: will do cleanup Author: Davies Liu <davies@databricks.com> Closes #11065 from davies/gen_dim.
* [SPARK-13210][SQL] catch OOM when allocate memory and expand arrayDavies Liu2016-02-087-21/+35
| | | | | | | | | | | | There is a bug when we try to grow the buffer, OOM is ignore wrongly (the assert also skipped by JVM), then we try grow the array again, this one will trigger spilling free the current page, the current record we inserted will be invalid. The root cause is that JVM has less free memory than MemoryManager thought, it will OOM when allocate a page without trigger spilling. We should catch the OOM, and acquire memory again to trigger spilling. And also, we could not grow the array in `insertRecord` of `InMemorySorter` (it was there just for easy testing). Author: Davies Liu <davies@databricks.com> Closes #11095 from davies/fix_expand.
* [SPARK-13101][SQL] nullability of array type element should not fail ↵Wenchen Fan2016-02-087-104/+64
| | | | | | | | | | analysis of encoder nullability should only be considered as an optimization rather than part of the type system, so instead of failing analysis for mismatch nullability, we should pass analysis and add runtime null check. Author: Wenchen Fan <wenchen@databricks.com> Closes #11035 from cloud-fan/ignore-nullability.
* [SPARK-8964] [SQL] Use Exchange to perform shuffle in LimitJosh Rosen2016-02-088-160/+223
| | | | | | | | | | | | | | This patch changes the implementation of the physical `Limit` operator so that it relies on the `Exchange` operator to perform data movement rather than directly using `ShuffledRDD`. In addition to improving efficiency, this lays the necessary groundwork for further optimization of limit, such as limit pushdown or whole-stage codegen. At a high-level, this replaces the old physical `Limit` operator with two new operators, `LocalLimit` and `GlobalLimit`. `LocalLimit` performs per-partition limits, while `GlobalLimit` applies the final limit to a single partition; `GlobalLimit`'s declares that its `requiredInputDistribution` is `SinglePartition`, which will cause the planner to use an `Exchange` to perform the appropriate shuffles. Thus, a logical `Limit` appearing in the middle of a query plan will be expanded into `LocalLimit -> Exchange to one partition -> GlobalLimit`. In the old code, calling `someDataFrame.limit(100).collect()` or `someDataFrame.take(100)` would actually skip the shuffle and use a fast-path which used `executeTake()` in order to avoid computing all partitions in case only a small number of rows were requested. This patch preserves this optimization by treating logical `Limit` operators specially when they appear as the terminal operator in a query plan: if a `Limit` is the final operator, then we will plan a special `CollectLimit` physical operator which implements the old `take()`-based logic. In order to be able to match on operators only at the root of the query plan, this patch introduces a special `ReturnAnswer` logical operator which functions similar to `BroadcastHint`: this dummy operator is inserted at the root of the optimized logical plan before invoking the physical planner, allowing the planner to pattern-match on it. Author: Josh Rosen <joshrosen@databricks.com> Closes #7334 from JoshRosen/remove-copy-in-limit.
* [SPARK-12986][DOC] Fix pydoc warnings in mllib/regression.pyNam Pham2016-02-081-13/+21
| | | | | | | | I have fixed the warnings by running "make html" under "python/docs/". They are caused by not having blank lines around indented paragraphs. Author: Nam Pham <phamducnam@gmail.com> Closes #11025 from nampham2/SPARK-12986.
* [SPARK-10963][STREAMING][KAFKA] make KafkaCluster publiccody koeninger2016-02-071-9/+10
| | | | | | Author: cody koeninger <cody@koeninger.org> Closes #9007 from koeninger/SPARK-10963.
* [SPARK-13132][MLLIB] cache standardization param value in LogisticRegressionGary King2016-02-072-2/+5
| | | | | | | | | | | | cache the value of the standardization Param in LogisticRegression, rather than re-fetching it from the ParamMap for every index and every optimization step in the quasi-newton optimizer also, fix Param#toString to cache the stringified representation, rather than re-interpolating it on every call, so any other implementations that have similar repeated access patterns will see a benefit. this change improves training times for one of my test sets from ~7m30s to ~4m30s Author: Gary King <gary@idibon.com> Closes #11027 from idigary/spark-13132-optimize-logistic-regression.
* [SPARK-5865][API DOC] Add doc warnings for methods that return local data ↵Tommy YU2016-02-067-0/+72
| | | | | | | | | | | | | structures rxin srowen I work out note message for rdd.take function, please help to review. If it's fine, I can apply to all other function later. Author: Tommy YU <tummyyu@163.com> Closes #10874 from Wenpei/spark-5865-add-warning-for-localdatastructure.
* [HOTFIX] fix float part of avgRateDavies Liu2016-02-051-1/+1
|
* [SPARK-13171][CORE] Replace future calls with FutureJakob Odersky2016-02-058-23/+23
| | | | | | | | | Trivial search-and-replace to eliminate deprecation warnings in Scala 2.11. Also works with 2.10 Author: Jakob Odersky <jakob@odersky.com> Closes #11085 from jodersky/SPARK-13171.
* [SPARK-13215] [SQL] remove fallback in codegenDavies Liu2016-02-053-66/+8
| | | | | | | | Since we remove the configuration for codegen, we are heavily reply on codegen (also TungstenAggregate require the generated MutableProjection to update UnsafeRow), should remove the fallback, which could make user confusing, see the discussion in SPARK-13116. Author: Davies Liu <davies@databricks.com> Closes #11097 from davies/remove_fallback.
* [SPARK-13002][MESOS] Send initial request of executors for dyn allocationLuc Bourlier2016-02-052-15/+19
| | | | | | | | | | | | | | | | | Fix for [SPARK-13002](https://issues.apache.org/jira/browse/SPARK-13002) about the initial number of executors when running with dynamic allocation on Mesos. Instead of fixing it just for the Mesos case, made the change in `ExecutorAllocationManager`. It is already driving the number of executors running on Mesos, only no the initial value. The `None` and `Some(0)` are internal details on the computation of resources to reserved, in the Mesos backend scheduler. `executorLimitOption` has to be initialized correctly, otherwise the Mesos backend scheduler will, either, create to many executors at launch, or not create any executors and not be able to recover from this state. Removed the 'special case' description in the doc. It was not totally accurate, and is not needed anymore. This doesn't fix the same problem visible with Spark standalone. There is no straightforward way to send the initial value in standalone mode. Somebody knowing this part of the yarn support should review this change. Author: Luc Bourlier <luc.bourlier@typesafe.com> Closes #11047 from skyluc/issue/initial-dyn-alloc-2.
* [SPARK-13214][DOCS] update dynamicAllocation documentationBill Chambers2016-02-051-2/+2
| | | | | | Author: Bill Chambers <bill@databricks.com> Closes #11094 from anabranch/dynamic-docs.
* [SPARK-12939][SQL] migrate encoder resolution logic to AnalyzerWenchen Fan2016-02-0512-104/+230
| | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-12939 Now we will catch `ObjectOperator` in `Analyzer` and resolve the `fromRowExpression/deserializer` inside it. Also update the `MapGroups` and `CoGroup` to pass in `dataAttributes`, so that we can correctly resolve value deserializer(the `child.output` contains both groupking key and values, which may mess things up if they have same-name attribtues). End-to-end tests are added. follow-ups: * remove encoders from typed aggregate expression. * completely remove resolve/bind in `ExpressionEncoder` Author: Wenchen Fan <wenchen@databricks.com> Closes #10852 from cloud-fan/bug.
* [SPARK-13166][SQL] Rename DataStreamReaderWriterSuite to ↵Shixiong Zhu2016-02-051-1/+1
| | | | | | | | | | DataFrameReaderWriterSuite A follow up PR for #11062 because it didn't rename the test suite. Author: Shixiong Zhu <shixiong@databricks.com> Closes #11096 from zsxwing/rename.
* [SPARK-13187][SQL] Add boolean/long/double options in DataFrameReader/WriterReynold Xin2016-02-043-0/+67
| | | | | | | | | | | | | | | | | | | This patch adds option function for boolean, long, and double types. This makes it slightly easier for Spark users to specify options without turning them into strings. Using the JSON data source as an example. Before this patch: ```scala sqlContext.read.option("primitivesAsString", "true").json("/path/to/json") ``` After this patch: Before this patch: ```scala sqlContext.read.option("primitivesAsString", true).json("/path/to/json") ``` Author: Reynold Xin <rxin@databricks.com> Closes #11072 from rxin/SPARK-13187.