aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-11889][SQL] Fix type inference for GroupedDataset.agg in REPLMichael Armbrust2015-11-203-29/+30
| | | | | | | | | | | | | | In this PR I delete a method that breaks type inference for aggregators (only in the REPL) The error when this method is present is: ``` <console>:38: error: missing parameter type for expanded function ((x$2) => x$2._2) ds.groupBy(_._1).agg(sum(_._2), sum(_._3)).collect() ``` Author: Michael Armbrust <michael@databricks.com> Closes #9870 from marmbrus/dataset-repl-agg.
* [SPARK-11787][SPARK-11883][SQL][FOLLOW-UP] Cleanup for this patch.Nong Li2015-11-2010-80/+175
| | | | | | | | | | This mainly moves SqlNewHadoopRDD to the sql package. There is some state that is shared between core and I've left that in core. This allows some other associated minor cleanup. Author: Nong Li <nong@databricks.com> Closes #9845 from nongli/spark-11787.
* [SPARK-11549][DOCS] Replace example code in mllib-evaluation-metrics.md ↵Vikas Nelamangala2015-11-2016-925/+1319
| | | | | | | | using include_example Author: Vikas Nelamangala <vikasnelamangala@Vikass-MacBook-Pro.local> Closes #9689 from vikasnp/master.
* [SPARK-11636][SQL] Support classes defined in the REPL with EncodersMichael Armbrust2015-11-204-7/+43
| | | | | | | | | #theScaryParts (i.e. changes to the repl, executor classloaders and codegen)... Author: Michael Armbrust <michael@databricks.com> Author: Yin Huai <yhuai@databricks.com> Closes #9825 from marmbrus/dataset-replClasses2.
* [SPARK-11756][SPARKR] Fix use of aliases - SparkR can not output help ↵felixcheung2015-11-204-84/+37
| | | | | | | | | | | | | | | | | information for SparkR:::summary correctly Fix use of aliases and changes uses of rdname and seealso `aliases` is the hint for `?` - it should not be linked to some other name - those should be seealso https://cran.r-project.org/web/packages/roxygen2/vignettes/rd.html Clean up usage on family, as multiple use of family with the same rdname is causing duplicated See Also html blocks (like http://spark.apache.org/docs/latest/api/R/count.html) Also changing some rdname for dplyr-like variant for better R user visibility in R doc, eg. rbind, summary, mutate, summarize shivaram yanboliang Author: felixcheung <felixcheung_m@hotmail.com> Closes #9750 from felixcheung/rdocaliases.
* [SPARK-11716][SQL] UDFRegistration just drops the input type when ↵Jean-Baptiste Onofré2015-11-202-24/+39
| | | | | | | | | | | | | | | | | re-creating the UserDefinedFunction https://issues.apache.org/jira/browse/SPARK-11716 This is one is #9739 and a regression test. When commit it, please make sure the author is jbonofre. You can find the original PR at https://github.com/apache/spark/pull/9739 closes #9739 Author: Jean-Baptiste Onofré <jbonofre@apache.org> Author: Yin Huai <yhuai@databricks.com> Closes #9868 from yhuai/SPARK-11716.
* [SPARK-11887] Close PersistenceEngine at the end of PersistenceEngineSuite testsJosh Rosen2015-11-201-48/+52
| | | | | | | | | | | | | | | | | | | | In PersistenceEngineSuite, we do not call `close()` on the PersistenceEngine at the end of the test. For the ZooKeeperPersistenceEngine, this causes us to leak a ZooKeeper client, causing the logs of unrelated tests to be periodically spammed with connection error messages from that client: ``` 15/11/20 05:13:35.789 pool-1-thread-1-ScalaTest-running-PersistenceEngineSuite-SendThread(localhost:15741) INFO ClientCnxn: Opening socket connection to server localhost/127.0.0.1:15741. Will not attempt to authenticate using SASL (unknown error) 15/11/20 05:13:35.790 pool-1-thread-1-ScalaTest-running-PersistenceEngineSuite-SendThread(localhost:15741) WARN ClientCnxn: Session 0x15124ff48dd0000 for server null, unexpected error, closing socket connection and attempting reconnect java.net.ConnectException: Connection refused at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method) at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:739) at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:350) at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1068) ``` This patch fixes this by using a `finally` block. Author: Josh Rosen <joshrosen@databricks.com> Closes #9864 from JoshRosen/close-zookeeper-client-in-tests.
* [SPARK-11870][STREAMING][PYSPARK] Rethrow the exceptions in ↵Shixiong Zhu2015-11-202-0/+19
| | | | | | | | | | TransformFunction and TransformFunctionSerializer TransformFunction and TransformFunctionSerializer don't rethrow the exception, so when any exception happens, it just return None. This will cause some weird NPE and confuse people. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9847 from zsxwing/pyspark-streaming-exception.
* [SPARK-11724][SQL] Change casting between int and timestamp to consistently ↵Nong Li2015-11-208-25/+39
| | | | | | | | | | | | treat int in seconds. Hive has since changed this behavior as well. https://issues.apache.org/jira/browse/HIVE-3454 Author: Nong Li <nong@databricks.com> Author: Nong Li <nongli@gmail.com> Author: Yin Huai <yhuai@databricks.com> Closes #9685 from nongli/spark-11724.
* [SPARK-11650] Reduce RPC timeouts to speed up slow AkkaUtilsSuite testJosh Rosen2015-11-201-1/+2
| | | | | | | | This patch reduces some RPC timeouts in order to speed up the slow "AkkaUtilsSuite.remote fetch ssl on - untrusted server", which used to take two minutes to run. Author: Josh Rosen <joshrosen@databricks.com> Closes #9869 from JoshRosen/SPARK-11650.
* [SPARK-11819][SQL] nice error message for missing encoderWenchen Fan2015-11-202-23/+129
| | | | | | | | | | | | | | | | before this PR, when users try to get an encoder for an un-supported class, they will only get a very simple error message like `Encoder for type xxx is not supported`. After this PR, the error message become more friendly, for example: ``` No Encoder found for abc.xyz.NonEncodable - array element class: "abc.xyz.NonEncodable" - field (class: "scala.Array", name: "arrayField") - root class: "abc.xyz.AnotherClass" ``` Author: Wenchen Fan <wenchen@databricks.com> Closes #9810 from cloud-fan/error-message.
* [SPARK-11817][SQL] Truncating the fractional seconds to prevent inserting a NULLLiang-Chi Hsieh2015-11-202-0/+13
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-11817 Instead of return None, we should truncate the fractional seconds to prevent inserting NULL. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #9834 from viirya/truncate-fractional-sec.
* [SPARK-11876][SQL] Support printSchema in DataSet APIgatorsmile2015-11-202-9/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | DataSet APIs look great! However, I am lost when doing multiple level joins. For example, ``` val ds1 = Seq(("a", 1), ("b", 2)).toDS().as("a") val ds2 = Seq(("a", 1), ("b", 2)).toDS().as("b") val ds3 = Seq(("a", 1), ("b", 2)).toDS().as("c") ds1.joinWith(ds2, $"a._2" === $"b._2").as("ab").joinWith(ds3, $"ab._1._2" === $"c._2").printSchema() ``` The printed schema is like ``` root |-- _1: struct (nullable = true) | |-- _1: struct (nullable = true) | | |-- _1: string (nullable = true) | | |-- _2: integer (nullable = true) | |-- _2: struct (nullable = true) | | |-- _1: string (nullable = true) | | |-- _2: integer (nullable = true) |-- _2: struct (nullable = true) | |-- _1: string (nullable = true) | |-- _2: integer (nullable = true) ``` Personally, I think we need the printSchema function. Sometimes, I do not know how to specify the column, especially when their data types are mixed. For example, if I want to write the following select for the above multi-level join, I have to know the schema: ``` newDS.select(expr("_1._2._2 + 1").as[Int]).collect() ``` marmbrus rxin cloud-fan Do you have the same feeling? Author: gatorsmile <gatorsmile@gmail.com> Closes #9855 from gatorsmile/printSchemaDataSet.
* [SPARK-11689][ML] Add user guide and example code for LDA under spark.mlYuhao Yang2015-11-205-1/+204
| | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-11689 Add simple user guide for LDA under spark.ml and example code under examples/. Use include_example to include example code in the user guide markdown. Check SPARK-11606 for instructions. Author: Yuhao Yang <hhbyyh@gmail.com> Closes #9722 from hhbyyh/ldaMLExample.
* [SPARK-11852][ML] StandardScaler minor refactorYanbo Liang2015-11-202-39/+32
| | | | | | | | ```withStd``` and ```withMean``` should be params of ```StandardScaler``` and ```StandardScalerModel```. Author: Yanbo Liang <ybliang8@gmail.com> Closes #9839 from yanboliang/standardScaler-refactor.
* [SPARK-11877] Prevent agg. fallback conf. from leaking across test suitesJosh Rosen2015-11-201-23/+21
| | | | | | | | | | This patch fixes an issue where the `spark.sql.TungstenAggregate.testFallbackStartsAt` SQLConf setting was not properly reset / cleared at the end of `TungstenAggregationQueryWithControlledFallbackSuite`. This ended up causing test failures in HiveCompatibilitySuite in Maven builds by causing spilling to occur way too frequently. This configuration leak was inadvertently introduced during test cleanup in #9618. Author: Josh Rosen <joshrosen@databricks.com> Closes #9857 from JoshRosen/clear-fallback-prop-in-test-teardown.
* [SPARK-11867] Add save/load for kmeans and naive bayesXusen Yin2015-11-194-28/+195
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11867 Author: Xusen Yin <yinxusen@gmail.com> Closes #9849 from yinxusen/SPARK-11867.
* [SPARK-11869][ML] Clean up TempDirectory properly in ML testsJoseph K. Bradley2015-11-191-1/+1
| | | | | | | | | | | | Need to remove parent directory (```className```) rather than just tempDir (```className/random_name```) I tested this with IDFSuite, which has 2 read/write tests, and it fixes the problem. CC: mengxr Can you confirm this is fine? I believe it is since the same ```random_name``` is used for all tests in a suite; we basically have an extra unneeded level of nesting. Author: Joseph K. Bradley <joseph@databricks.com> Closes #9851 from jkbradley/tempdir-cleanup.
* [SPARK-11875][ML][PYSPARK] Update doc for PySpark HasCheckpointIntervalYanbo Liang2015-11-192-9/+11
| | | | | | | | | * Update doc for PySpark ```HasCheckpointInterval``` that users can understand how to disable checkpoint. * Update doc for PySpark ```cacheNodeIds``` of ```DecisionTreeParams``` to notify the relationship between ```cacheNodeIds``` and ```checkpointInterval```. Author: Yanbo Liang <ybliang8@gmail.com> Closes #9856 from yanboliang/spark-11875.
* [SPARK-11829][ML] Add read/write to estimators under ml.feature (II)Yanbo Liang2015-11-199-33/+338
| | | | | | | | | | | | Add read/write support to the following estimators under spark.ml: * ChiSqSelector * PCA * VectorIndexer * Word2Vec Author: Yanbo Liang <ybliang8@gmail.com> Closes #9838 from yanboliang/spark-11829.
* [SPARK-11846] Add save/load for AFTSurvivalRegression and IsotonicRegressionXusen Yin2015-11-194-22/+210
| | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-11846 mengxr Author: Xusen Yin <yinxusen@gmail.com> Closes #9836 from yinxusen/SPARK-11846.
* [SPARK-11544][SQL][TEST-HADOOP1.0] sqlContext doesn't use PathFilterDilip Biswal2015-11-192-7/+59
| | | | | | | | Apply the user supplied pathfilter while retrieving the files from fs. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #9830 from dilipbiswal/spark-11544.
* [SPARK-11864][SQL] Improve performance of max/minDavies Liu2015-11-195-25/+45
| | | | | | | | | | | | | | | | This PR has the following optimization: 1) The greatest/least already does the null-check, so the `If` and `IsNull` are not necessary. 2) In greatest/least, it should initialize the result using the first child (removing one block). 3) For primitive types, the generated greater expression is too complicated (`a > b ? 1 : (a < b) ? -1 : 0) > 0`), should be as simple as `a > b` Combine these optimization, this could improve the performance of `ss_max` query by 30%. Author: Davies Liu <davies@databricks.com> Closes #9846 from davies/improve_max.
* [SPARK-11845][STREAMING][TEST] Added unit test to verify TrackStateRDD is ↵Tathagata Das2015-11-192-204/+267
| | | | | | | | | | correctly checkpointed To make sure that all lineage is correctly truncated for TrackStateRDD when checkpointed. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9831 from tdas/SPARK-11845.
* [SPARK-4134][CORE] Lower severity of some executor loss logs.Marcelo Vanzin2015-11-195-24/+45
| | | | | | | | | Don't log ERROR messages when executors are explicitly killed or when the exit reason is not yet known. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9780 from vanzin/SPARK-11789.
* [SPARK-11275][SQL] Incorrect results when using rollup/cubeAndrew Ray2015-11-193-34/+90
| | | | | | | | | | | | | | | | | Fixes bug with grouping sets (including cube/rollup) where aggregates that included grouping expressions would return the wrong (null) result. Also simplifies the analyzer rule a bit and leaves column pruning to the optimizer. Added multiple unit tests to DataFrameAggregateSuite and verified it passes hive compatibility suite: ``` build/sbt -Phive -Dspark.hive.whitelist='groupby.*_grouping.*' 'test-only org.apache.spark.sql.hive.execution.HiveCompatibilitySuite' ``` This is an alternative to pr https://github.com/apache/spark/pull/9419 but I think its better as it simplifies the analyzer rule instead of adding another special case to it. Author: Andrew Ray <ray.andrew@gmail.com> Closes #9815 from aray/groupingset-agg-fix.
* [SPARK-11746][CORE] Use cache-aware method dependencieshushan2015-11-191-1/+1
| | | | | | | | a small change Author: hushan <hushan@xiaomi.com> Closes #9691 from suyanNone/unify-getDependency.
* [SPARK-11828][CORE] Register DAGScheduler metrics source after app id is known.Marcelo Vanzin2015-11-192-3/+2
| | | | | | Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9820 from vanzin/SPARK-11828.
* [SPARK-11799][CORE] Make it explicit in executor logs that uncaught e…Srinivasa Reddy Vundela2015-11-191-1/+5
| | | | | | | | | | …xceptions are thrown during executor shutdown This commit will make sure that when uncaught exceptions are prepended with [Container in shutdown] when JVM is shutting down. Author: Srinivasa Reddy Vundela <vsr@cloudera.com> Closes #9809 from vundela/master_11799.
* [SPARK-11831][CORE][TESTS] Use port 0 to avoid port conflicts in testsShixiong Zhu2015-11-192-14/+14
| | | | | | | | Use port 0 to fix port-contention-related flakiness Author: Shixiong Zhu <shixiong@databricks.com> Closes #9841 from zsxwing/SPARK-11831.
* [SPARK-11858][SQL] Move sql.columnar into sql.execution.Reynold Xin2015-11-1930-147/+155
| | | | | | | | In addition, tightened visibility of a lot of classes in the columnar package from private[sql] to private[columnar]. Author: Reynold Xin <rxin@databricks.com> Closes #9842 from rxin/SPARK-11858.
* [SPARK-11812][PYSPARK] invFunc=None works properly with python's ↵David Tolpin2015-11-192-3/+14
| | | | | | | | | | | | | reduceByKeyAndWindow invFunc is optional and can be None. Instead of invFunc (the parameter) invReduceFunc (a local function) was checked for trueness (that is, not None, in this context). A local function is never None, thus the case of invFunc=None (a common one when inverse reduction is not defined) was treated incorrectly, resulting in loss of data. In addition, the docstring used wrong parameter names, also fixed. Author: David Tolpin <david.tolpin@gmail.com> Closes #9775 from dtolpin/master.
* [SPARK-11778][SQL] parse table name before it is passed to lookupRelationHuaxin Gao2015-11-192-1/+12
| | | | | | | | | Fix a bug in DataFrameReader.table (table with schema name such as "db_name.table" doesn't work) Use SqlParser.parseTableIdentifier to parse the table name before lookupRelation. Author: Huaxin Gao <huaxing@oc0558782468.ibm.com> Closes #9773 from huaxingao/spark-11778.
* [SPARK-11750][SQL] revert SPARK-11727 and code clean upWenchen Fan2015-11-1914-1118/+364
| | | | | | | | After some experiment, I found it's not convenient to have separate encoder builders: `FlatEncoder` and `ProductEncoder`. For example, when create encoders for `ScalaUDF`, we have no idea if the type `T` is flat or not. So I revert the splitting change in https://github.com/apache/spark/pull/9693, while still keeping the bug fixes and tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #9726 from cloud-fan/follow.
* [SPARK-11848][SQL] Support EXPLAIN in DataSet APIsgatorsmile2015-11-192-22/+22
| | | | | | | | | | When debugging DataSet API, I always need to print the logical and physical plans. I am wondering if we should provide a simple API for EXPLAIN? Author: gatorsmile <gatorsmile@gmail.com> Closes #9832 from gatorsmile/explainDS.
* [SPARK-11633][SQL] LogicalRDD throws TreeNode Exception : Failed to Copy Nodegatorsmile2015-11-192-0/+18
| | | | | | | | | | | | | When handling self joins, the implementation did not consider the case insensitivity of HiveContext. It could cause an exception as shown in the JIRA: ``` TreeNodeException: Failed to copy node. ``` The fix is low risk. It avoids unnecessary attribute replacement. It should not affect the existing behavior of self joins. Also added the test case to cover this case. Author: gatorsmile <gatorsmile@gmail.com> Closes #9762 from gatorsmile/joinMakeCopy.
* [SPARK-11830][CORE] Make NettyRpcEnv bind to the specified hostzsxwing2015-11-194-10/+28
| | | | | | | | | | | | | This PR includes the following change: 1. Bind NettyRpcEnv to the specified host 2. Fix the port information in the log for NettyRpcEnv. 3. Fix the service name of NettyRpcEnv. Author: zsxwing <zsxwing@gmail.com> Author: Shixiong Zhu <shixiong@databricks.com> Closes #9821 from zsxwing/SPARK-11830.
* [SPARK-11840][SQL] Restore the 1.5's behavior of planning a single distinct ↵Yin Huai2015-11-191-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | aggregation. The impact of this change is for a query that has a single distinct column and does not have any grouping expression like `SELECT COUNT(DISTINCT a) FROM table` The plan will be changed from ``` AGG-2 (count distinct) Shuffle to a single reducer Partial-AGG-2 (count distinct) AGG-1 (grouping on a) Shuffle by a Partial-AGG-1 (grouping on 1) ``` to the following one (1.5 uses this) ``` AGG-2 AGG-1 (grouping on a) Shuffle to a single reducer Partial-AGG-1(grouping on a) ``` The first plan is more robust. However, to better benchmark the impact of this change, we should use 1.5's plan and use the conf of `spark.sql.specializeSingleDistinctAggPlanning` to control the plan. Author: Yin Huai <yhuai@databricks.com> Closes #9828 from yhuai/distinctRewriter.
* [SPARK-11849][SQL] Analyzer should replace current_date and ↵Reynold Xin2015-11-192-5/+60
| | | | | | | | | | | | current_timestamp with literals We currently rely on the optimizer's constant folding to replace current_timestamp and current_date. However, this can still result in different values for different instances of current_timestamp/current_date if the optimizer is not running fast enough. A better solution is to replace these functions in the analyzer in one shot. Author: Reynold Xin <rxin@databricks.com> Closes #9833 from rxin/SPARK-11849.
* [SPARK-11339][SPARKR] Document the list of functions in R base package that ↵felixcheung2015-11-186-6/+77
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | are masked by functions with same name in SparkR Added tests for function that are reported as masked, to make sure the base:: or stats:: function can be called. For those we can't call, added them to SparkR programming guide. It would seem to me `table, sample, subset, filter, cov` not working are not actually expected - I investigated/experimented with them but couldn't get them to work. It looks like as they are defined in base or stats they are missing the S3 generic, eg. ``` > methods("transform") [1] transform,ANY-method transform.data.frame [3] transform,DataFrame-method transform.default see '?methods' for accessing help and source code > methods("subset") [1] subset.data.frame subset,DataFrame-method subset.default [4] subset.matrix see '?methods' for accessing help and source code Warning message: In .S3methods(generic.function, class, parent.frame()) : function 'subset' appears not to be S3 generic; found functions that look like S3 methods ``` Any idea? More information on masking: http://www.ats.ucla.edu/stat/r/faq/referencing_objects.htm http://www.sfu.ca/~sweldon/howTo/guide4.pdf This is what the output doc looks like (minus css): ![image](https://cloud.githubusercontent.com/assets/8969467/11229714/2946e5de-8d4d-11e5-94b0-dda9696b6fdd.png) Author: felixcheung <felixcheung_m@hotmail.com> Closes #9785 from felixcheung/rmasked.
* [SPARK-11842][ML] Small cleanups to existing Readers and WritersJoseph K. Bradley2015-11-1810-25/+38
| | | | | | | | | | | | | | | | | Updates: * Add repartition(1) to save() methods' saving of data for LogisticRegressionModel, LinearRegressionModel. * Strengthen privacy to class and companion object for Writers and Readers * Change LogisticRegressionSuite read/write test to fit intercept * Add Since versions for read/write methods in Pipeline, LogisticRegression * Switch from hand-written class names in Readers to using getClass CC: mengxr CC: yanboliang Would you mind taking a look at this PR? mengxr might not be able to soon. Thank you! Author: Joseph K. Bradley <joseph@databricks.com> Closes #9829 from jkbradley/ml-io-cleanups.
* [SPARK-11614][SQL] serde parameters should be set only when all params are readynavis.ryu2015-11-181-1/+2
| | | | | | | | | | see HIVE-7975 and HIVE-12373 With changed semantic of setters in thrift objects in hive, setter should be called only after all parameters are set. It's not problem of current state but will be a problem in some day. Author: navis.ryu <navis@apache.org> Closes #9580 from navis/SPARK-11614.
* [SPARK-11816][ML] fix some style issue in ML/MLlib examplesYuhao Yang2015-11-185-3/+5
| | | | | | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-11816 Currently I only fixed some obvious comments issue like // scalastyle:off println on the bottom. Yet the style in examples is not quite consistent, like only half of the examples are with // Example usage: ./bin/run-example mllib.FPGrowthExample \, Author: Yuhao Yang <hhbyyh@gmail.com> Closes #9808 from hhbyyh/exampleStyle.
* Revert "[SPARK-11544][SQL] sqlContext doesn't use PathFilter"Yin Huai2015-11-182-54/+7
| | | | This reverts commit 54db79702513e11335c33bcf3a03c59e965e6f16.
* [SPARK-11787][SQL] Improve Parquet scan performance when using flat schemas.Nong Li2015-11-188-43/+944
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch adds an alternate to the Parquet RecordReader from the parquet-mr project that is much faster for flat schemas. Instead of using the general converter mechanism from parquet-mr, this directly uses the lower level APIs from parquet-columnar and a customer RecordReader that directly assembles into UnsafeRows. This is optionally disabled and only used for supported schemas. Using the tpcds store sales table and doing a sum of increasingly more columns, the results are: For 1 Column: Before: 11.3M rows/second After: 18.2M rows/second For 2 Columns: Before: 7.2M rows/second After: 11.2M rows/second For 5 Columns: Before: 2.9M rows/second After: 4.5M rows/second Author: Nong Li <nong@databricks.com> Closes #9774 from nongli/parquet.
* [SPARK-11833][SQL] Add Java tests for Kryo/Java Dataset encodersReynold Xin2015-11-184-40/+166
| | | | | | | | Also added some nicer error messages for incompatible types (private types and primitive types) for Kryo/Java encoder. Author: Reynold Xin <rxin@databricks.com> Closes #9823 from rxin/SPARK-11833.
* [SPARK-11839][ML] refactor save/write traitsXiangrui Meng2015-11-1827-321/+190
| | | | | | | | | | | | * add "ML" prefix to reader/writer/readable/writable to avoid name collision with java.util.* * define `DefaultParamsReadable/Writable` and use them to save some code * use `super.load` instead so people can jump directly to the doc of `Readable.load`, which documents the Java compatibility issues jkbradley Author: Xiangrui Meng <meng@databricks.com> Closes #9827 from mengxr/SPARK-11839.
* [SPARK-11636][SQL] Support classes defined in the REPL with EncodersMichael Armbrust2015-11-1817-82/+193
| | | | | | | | | | | | Before this PR there were two things that would blow up if you called `df.as[MyClass]` if `MyClass` was defined in the REPL: - [x] Because `classForName` doesn't work on the munged names returned by `tpe.erasure.typeSymbol.asClass.fullName` - [x] Because we don't have anything to pass into the constructor for the `$outer` pointer. Note that this PR is just adding the infrastructure for working with inner classes in encoder and is not yet sufficient to make them work in the REPL. Currently, the implementation show in https://github.com/marmbrus/spark/commit/95cec7d413b930b36420724fafd829bef8c732ab is causing a bug that breaks code gen due to some interaction between janino and the `ExecutorClassLoader`. This will be addressed in a follow-up PR. Author: Michael Armbrust <michael@databricks.com> Closes #9602 from marmbrus/dataset-replClasses.
* [SPARK-11791] Fix flaky test in BatchedWriteAheadLogSuiteBurak Yavuz2015-11-181-4/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | stack trace of failure: ``` org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 62 times over 1.006322071 seconds. Last failure message: Argument(s) are different! Wanted: writeAheadLog.write( java.nio.HeapByteBuffer[pos=0 lim=124 cap=124], 10 ); -> at org.apache.spark.streaming.util.BatchedWriteAheadLogSuite$$anonfun$23$$anonfun$apply$mcV$sp$15.apply(WriteAheadLogSuite.scala:518) Actual invocation has different arguments: writeAheadLog.write( java.nio.HeapByteBuffer[pos=0 lim=124 cap=124], 10 ); -> at org.apache.spark.streaming.util.WriteAheadLogSuite$BlockingWriteAheadLog.write(WriteAheadLogSuite.scala:756) ``` I believe the issue was that due to a race condition, the ordering of the events could be messed up in the final ByteBuffer, therefore the comparison fails. By adding eventually between the requests, we make sure the ordering is preserved. Note that in real life situations, the ordering across threads will not matter. Another solution would be to implement a custom mockito matcher that sorts and then compares the results, but that kind of sounds like overkill to me. Let me know what you think tdas zsxwing Author: Burak Yavuz <brkyvz@gmail.com> Closes #9790 from brkyvz/fix-flaky-2.
* [SPARK-11814][STREAMING] Add better default checkpoint durationTathagata Das2015-11-182-1/+56
| | | | | | | | | DStream checkpoint interval is by default set at max(10 second, batch interval). That's bad for large batch intervals where the checkpoint interval = batch interval, and RDDs get checkpointed every batch. This PR is to set the checkpoint interval of trackStateByKey to 10 * batch duration. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9805 from tdas/SPARK-11814.