aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-11480][CORE][WEBUI] Wrong callsite is displayed when using ↵Kousuke Saruta2015-11-161-0/+2
| | | | | | | | | | | | | | | | | | | | AsyncRDDActions#takeAsync When we call AsyncRDDActions#takeAsync, actually another DAGScheduler#runJob is called from another thread so we cannot get proper callsite infomation. Following screenshots are before this patch applied and after. Before: <img width="1268" alt="2015-11-04 1 26 40" src="https://cloud.githubusercontent.com/assets/4736016/10914069/0ffc1306-8294-11e5-8e89-c4fadf58dd12.png"> <img width="1258" alt="2015-11-04 1 26 52" src="https://cloud.githubusercontent.com/assets/4736016/10914070/0ffe84ce-8294-11e5-8b2a-69d36276bedb.png"> After: <img width="1268" alt="2015-11-04 0 48 07" src="https://cloud.githubusercontent.com/assets/4736016/10914080/1d8cfb7a-8294-11e5-9e09-ede25c2563e8.png"> <img width="1269" alt="2015-11-04 0 48 26" src="https://cloud.githubusercontent.com/assets/4736016/10914081/1d934e3a-8294-11e5-8b5e-e3dc37aaced3.png"> Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #9437 from sarutak/SPARK-11480.
* [SPARKR][HOTFIX] Disable flaky SparkR package build testShivaram Venkataraman2015-11-161-1/+3
| | | | | | | | See https://github.com/apache/spark/pull/9390#issuecomment-157160063 and https://gist.github.com/shivaram/3a2fecce60768a603dac for more information Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #9744 from shivaram/sparkr-package-test-disable.
* [SPARK-11625][SQL] add java test for typed aggregateWenchen Fan2015-11-165-9/+92
| | | | | | Author: Wenchen Fan <wenchen@databricks.com> Closes #9591 from cloud-fan/agg-test.
* [SPARK-8658][SQL] AttributeReference's equals method compares all the membersgatorsmile2015-11-163-12/+14
| | | | | | | | This fix is to change the equals method to check all of the specified fields for equality of AttributeReference. Author: gatorsmile <gatorsmile@gmail.com> Closes #9216 from gatorsmile/namedExpressEqual.
* [SPARK-11553][SQL] Primitive Row accessors should not convert null to ↵Bartlomiej Alberski2015-11-163-23/+65
| | | | | | | | | | default value Invocation of getters for type extending AnyVal returns default value (if field value is null) instead of throwing NPE. Please check comments for SPARK-11553 issue for more details. Author: Bartlomiej Alberski <bartlomiej.alberski@allegrogroup.com> Closes #9642 from alberskib/bugfix/SPARK-11553.
* [SPARK-11742][STREAMING] Add the failure info to the batch listsShixiong Zhu2015-11-163-50/+120
| | | | | | | | <img width="1365" alt="screen shot 2015-11-13 at 9 57 43 pm" src="https://cloud.githubusercontent.com/assets/1000778/11162322/9b88e204-8a51-11e5-8c57-a44889cab713.png"> Author: Shixiong Zhu <shixiong@databricks.com> Closes #9711 from zsxwing/failure-info.
* Revert "[SPARK-11271][SPARK-11016][CORE] Use Spark BitSet instead of ↵Davies Liu2015-11-167-82/+33
| | | | | | RoaringBitmap to reduce memory usage" This reverts commit e209fa271ae57dc8849f8b1241bf1ea7d6d3d62c.
* [SPARK-11390][SQL] Query plan with/without filterPushdown indistinguishableZee Chen2015-11-163-4/+22
| | | | | | | | | | …ishable Propagate pushed filters to PhyicalRDD in DataSourceStrategy.apply Author: Zee Chen <zeechen@us.ibm.com> Closes #9679 from zeocio/spark-11390.
* [SPARK-11754][SQL] consolidate `ExpressionEncoder.tuple` and `Encoders.tuple`Wenchen Fan2015-11-163-120/+108
| | | | | | | | | | These 2 are very similar, we can consolidate them into one. Also add tests for it and fix a bug. Author: Wenchen Fan <wenchen@databricks.com> Closes #9729 from cloud-fan/tuple.
* [SPARK-11718][YARN][CORE] Fix explicitly killed executor dies silently issuejerryshao2015-11-163-8/+29
| | | | | | | | | | | | | | | | | | Currently if dynamic allocation is enabled, explicitly killing executor will not get response, so the executor metadata is wrong in driver side. Which will make dynamic allocation on Yarn fail to work. The problem is `disableExecutor` returns false for pending killing executors when `onDisconnect` is detected, so no further implementation is done. One solution is to bypass these explicitly killed executors to use `super.onDisconnect` to remove executor. This is simple. Another solution is still querying the loss reason for these explicitly kill executors. Since executor may get killed and informed in the same AM-RM communication, so current way of adding pending loss reason request is not worked (container complete is already processed), here we should store this loss reason for later query. Here this PR chooses solution 2. Please help to review. vanzin I think this part is changed by you previously, would you please help to review? Thanks a lot. Author: jerryshao <sshao@hortonworks.com> Closes #9684 from jerryshao/SPARK-11718.
* [SPARK-6328][PYTHON] Python API for StreamingListenerDaniel Jalova2015-11-165-2/+286
| | | | | | Author: Daniel Jalova <djalova@us.ibm.com> Closes #9186 from djalova/SPARK-6328.
* [SPARK-11731][STREAMING] Enable batching on Driver WriteAheadLog by defaultBurak Yavuz2015-11-165-7/+48
| | | | | | | | | | | | | Using batching on the driver for the WriteAheadLog should be an improvement for all environments and use cases. Users will be able to scale to much higher number of receivers with the BatchedWriteAheadLog. Therefore we should turn it on by default, and QA it in the QA period. I've also added some tests to make sure the default configurations are correct regarding recent additions: - batching on by default - closeFileAfterWrite off by default - parallelRecovery off by default Author: Burak Yavuz <brkyvz@gmail.com> Closes #9695 from brkyvz/enable-batch-wal.
* [SPARK-11743] [SQL] Add UserDefinedType support to RowEncoderLiang-Chi Hsieh2015-11-164-29/+139
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-11743 RowEncoder doesn't support UserDefinedType now. We should add the support for it. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #9712 from viirya/rowencoder-udt.
* [SPARK-11752] [SQL] fix timezone problem for DateTimeUtils.getSecondsWenchen Fan2015-11-162-7/+9
| | | | | | | | | | | | | | | | code snippet to reproduce it: ``` TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai")) val t = Timestamp.valueOf("1900-06-11 12:14:50.789") val us = fromJavaTimestamp(t) assert(getSeconds(us) === t.getSeconds) ``` it will be good to add a regression test for it, but the reproducing code need to change the default timezone, and even we change it back, the `lazy val defaultTimeZone` in `DataTimeUtils` is fixed. Author: Wenchen Fan <wenchen@databricks.com> Closes #9728 from cloud-fan/seconds.
* [SPARK-11522][SQL] input_file_name() returns "" for external tablesxin Wu2015-11-162-2/+98
| | | | | | | | | When computing partition for non-parquet relation, `HadoopRDD.compute` is used. but it does not set the thread local variable `inputFileName` in `NewSqlHadoopRDD`, like `NewSqlHadoopRDD.compute` does.. Yet, when getting the `inputFileName`, `NewSqlHadoopRDD.inputFileName` is exptected, which is empty now. Adding the setting inputFileName in HadoopRDD.compute resolves this issue. Author: xin Wu <xinwu@us.ibm.com> Closes #9542 from xwu0226/SPARK-11522.
* [SPARK-11692][SQL] Support for Parquet logical types, JSON and BSON ↵hyukjinkwon2015-11-162-1/+27
| | | | | | | | | | | | | (embedded types) Parquet supports some JSON and BSON datatypes. They are represented as binary for BSON and string (UTF-8) for JSON internally. I searched a bit and found Apache drill also supports both in this way, [link](https://drill.apache.org/docs/parquet-format/). Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #9658 from HyukjinKwon/SPARK-11692.
* [SPARK-11044][SQL] Parquet writer version fixed as version1hyukjinkwon2015-11-162-1/+35
| | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-11044 Spark writes a parquet file only with writer version1 ignoring the writer version given by user. So, in this PR, it keeps the writer version if given or sets version1 as default. Author: hyukjinkwon <gurwls223@gmail.com> Author: HyukjinKwon <gurwls223@gmail.com> Closes #9060 from HyukjinKwon/SPARK-11044.
* [SPARK-11745][SQL] Enable more JSON parsing optionsReynold Xin2015-11-169-106/+286
| | | | | | | | | | | | | | | | | | | | | | | | This patch adds the following options to the JSON data source, for dealing with non-standard JSON files: * `allowComments` (default `false`): ignores Java/C++ style comment in JSON records * `allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names * `allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes * `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers (e.g. 00012) To avoid passing a lot of options throughout the json package, I introduced a new JSONOptions case class to define all JSON config options. Also updated documentation to explain these options. Scala ![screen shot 2015-11-15 at 6 12 12 pm](https://cloud.githubusercontent.com/assets/323388/11172965/e3ace6ec-8bc4-11e5-805e-2d78f80d0ed6.png) Python ![screen shot 2015-11-15 at 6 11 28 pm](https://cloud.githubusercontent.com/assets/323388/11172964/e23ed6ee-8bc4-11e5-8216-312f5983acd5.png) Author: Reynold Xin <rxin@databricks.com> Closes #9724 from rxin/SPARK-11745.
* Revert "[SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called"Josh Rosen2015-11-151-3/+6
| | | | This reverts commit 3e0a6cf1e02a19b37c68d3026415d53bb57a576b.
* [SPARK-9928][SQL] Removal of LogicalLocalTablegatorsmile2015-11-151-22/+0
| | | | | | | | | | LogicalLocalTable in ExistingRDD.scala is replaced by localRelation in LocalRelation.scala? Do you know any reason why we still keep this class? Author: gatorsmile <gatorsmile@gmail.com> Closes #9717 from gatorsmile/LogicalLocalTable.
* [SPARK-10500][SPARKR] sparkr.zip cannot be created if /R/lib is unwritableSun Rui2015-11-1514-36/+121
| | | | | | | | | | | | | | | | | The basic idea is that: The archive of the SparkR package itself, that is sparkr.zip, is created during build process and is contained in the Spark binary distribution. No change to it after the distribution is installed as the directory it resides ($SPARK_HOME/R/lib) may not be writable. When there is R source code contained in jars or Spark packages specified with "--jars" or "--packages" command line option, a temporary directory is created by calling Utils.createTempDir() where the R packages built from the R source code will be installed. The temporary directory is writable, and won't interfere with each other when there are multiple SparkR sessions, and will be deleted when this SparkR session ends. The R binary packages installed in the temporary directory then are packed into an archive named rpkg.zip. sparkr.zip and rpkg.zip are distributed to the cluster in YARN modes. The distribution of rpkg.zip in Standalone modes is not supported in this PR, and will be address in another PR. Various R files are updated to accept multiple lib paths (one is for SparkR package, the other is for other R packages) so that these package can be accessed in R. Author: Sun Rui <rui.sun@intel.com> Closes #9390 from sun-rui/SPARK-10500.
* [SPARK-11086][SPARKR] Use dropFactors column-wise instead of nested loop ↵zero3232015-11-152-21/+49
| | | | | | | | | | | | | | | | when createDataFrame Use `dropFactors` column-wise instead of nested loop when `createDataFrame` from a `data.frame` At this moment SparkR createDataFrame is using nested loop to convert factors to character when called on a local data.frame. It works but is incredibly slow especially with data.table (~ 2 orders of magnitude compared to PySpark / Pandas version on a DateFrame of size 1M rows x 2 columns). A simple improvement is to apply `dropFactor `column-wise and then reshape output list. It should at least partially address [SPARK-8277](https://issues.apache.org/jira/browse/SPARK-8277). Author: zero323 <matthew.szymkiewicz@gmail.com> Closes #9099 from zero323/SPARK-11086.
* [SPARK-10181][SQL] Do kerberos login for credentials during hive client ↵Yu Gao2015-11-152-4/+37
| | | | | | | | | | | | | | | | initialization On driver process start up, UserGroupInformation.loginUserFromKeytab is called with the principal and keytab passed in, and therefore static var UserGroupInfomation,loginUser is set to that principal with kerberos credentials saved in its private credential set, and all threads within the driver process are supposed to see and use this login credentials to authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, UserGroupInformation class is not shared for hive metastore clients, and instead it is loaded separately and of course not able to see the prepared kerberos login credentials in the main thread. The first proposed fix would cause other classloader conflict errors, and is not an appropriate solution. This new change does kerberos login during hive client initialization, which will make credentials ready for the particular hive client instance. yhuai Please take a look and let me know. If you are not the right person to talk to, could you point me to someone responsible for this? Author: Yu Gao <ygao@us.ibm.com> Author: gaoyu <gaoyu@gaoyu-macbookpro.roam.corp.google.com> Author: Yu Gao <crystalgaoyu@gmail.com> Closes #9272 from yolandagao/master.
* [SPARK-11738] [SQL] Making ArrayType orderableYin Huai2015-11-1514-94/+335
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11738 Author: Yin Huai <yhuai@databricks.com> Closes #9718 from yhuai/makingArrayOrderable.
* [SPARK-11672][ML] set active SQLContext in JavaDefaultReadWriteSuiteXiangrui Meng2015-11-151-1/+6
| | | | | | | | The same as #9694, but for Java test suite. yhuai Author: Xiangrui Meng <meng@databricks.com> Closes #9719 from mengxr/SPARK-11672.4.
* [SPARK-11734][SQL] Rename TungstenProject -> Project, TungstenSort -> SortReynold Xin2015-11-1515-184/+148
| | | | | | | | I didn't remove the old Sort operator, since we still use it in randomized tests. I moved it into test module and renamed it ReferenceSort. Author: Reynold Xin <rxin@databricks.com> Closes #9700 from rxin/SPARK-11734.
* [SPARK-11736][SQL] Add monotonically_increasing_id to function registry.Yin Huai2015-11-142-1/+6
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11736 Author: Yin Huai <yhuai@databricks.com> Closes #9703 from yhuai/MonotonicallyIncreasingID.
* Typo in comment: use 2 seconds instead of 1Rohan Bhanderi2015-11-141-1/+1
| | | | | | | | Use 2 seconds batch size as duration specified in JavaStreamingContext constructor is 2000 ms Author: Rohan Bhanderi <rohan.bhanderi@sjsu.edu> Closes #9714 from RohanBhanderi/patch-2.
* [SPARK-11573] Correct 'reflective access of structural type member meth…Gábor Lipták2015-11-141-0/+1
| | | | | | | | …od should be enabled' Scala warnings Author: Gábor Lipták <gliptak@gmail.com> Closes #9550 from gliptak/SPARK-11573.
* [MINOR][DOCS] typo in docs/configuration.mdKai Jiang2015-11-141-5/+5
| | | | | | | | | | | `<\code>` end tag missing backslash in docs/configuration.md{L308-L339} ref #8795 Author: Kai Jiang <jiangkai@gmail.com> Closes #9715 from vectorijk/minor-typo-docs.
* [SPARK-11694][SQL] Parquet logical types are not being tested properlyhyukjinkwon2015-11-142-9/+47
| | | | | | | | | All the physical types are properly tested at `ParquetIOSuite` but logical type mapping is not being tested. Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #9660 from HyukjinKwon/SPARK-11694.
* [SPARK-7970] Skip closure cleaning for SQL operationsnitin goyal2015-11-139-20/+38
| | | | | | | | | Also introduces new spark private API in RDD.scala with name 'mapPartitionsInternal' which doesn't closure cleans the RDD elements. Author: nitin goyal <nitin.goyal@guavus.com> Author: nitin.goyal <nitin.goyal@guavus.com> Closes #9253 from nitin2goyal/master.
* [MINOR][ML] remove MLlibTestsSparkContext from ImpuritySuiteXiangrui Meng2015-11-131-2/+1
| | | | | | | | ImpuritySuite doesn't need SparkContext. Author: Xiangrui Meng <meng@databricks.com> Closes #9698 from mengxr/remove-mllib-test-context-in-impurity-suite.
* [SPARK-11336] Add links to example codesXusen Yin2015-11-131-2/+7
| | | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-11336 mengxr I add a hyperlink of Spark on Github and a hint of their existences in Spark code repo in each code example. I remove the config key for changing the example code dir, since we assume all examples should be in spark/examples. The hyperlink, though we cannot use it now, since the Spark v1.6.0 has not been released yet, can be used after the release. So it is not a problem. I add some screen shots, so you can get an instant feeling. <img width="949" alt="screen shot 2015-10-27 at 10 47 18 pm" src="https://cloud.githubusercontent.com/assets/2637239/10780634/bd20e072-7cfc-11e5-8960-def4fc62a8ea.png"> <img width="1144" alt="screen shot 2015-10-27 at 10 47 31 pm" src="https://cloud.githubusercontent.com/assets/2637239/10780636/c3f6e180-7cfc-11e5-80b2-233589f4a9a3.png"> Author: Xusen Yin <yinxusen@gmail.com> Closes #9320 from yinxusen/SPARK-11336.
* [SPARK-11672][ML] Set active SQLContext in MLlibTestSparkContext.beforeAllXiangrui Meng2015-11-132-2/+6
| | | | | | | | | | | | | | Still saw some error messages caused by `SQLContext.getOrCreate`: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3997/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.3,label=spark-test/testReport/junit/org.apache.spark.ml.util/JavaDefaultReadWriteSuite/testDefaultReadWrite/ This PR sets the active SQLContext in beforeAll, which is not automatically set in `new SQLContext`. This makes `SQLContext.getOrCreate` return the right SQLContext. cc: yhuai Author: Xiangrui Meng <meng@databricks.com> Closes #9694 from mengxr/SPARK-11672.3.
* [SPARK-11727][SQL] Split ExpressionEncoder into FlatEncoder and ProductEncoderWenchen Fan2015-11-1312-289/+766
| | | | | | | | | | | | | | | | | | | also add more tests for encoders, and fix bugs that I found: * when convert array to catalyst array, we can only skip element conversion for native types(e.g. int, long, boolean), not `AtomicType`(String is AtomicType but we need to convert it) * we should also handle scala `BigDecimal` when convert from catalyst `Decimal`. * complex map type should be supported other issues that still in investigation: * encode java `BigDecimal` and decode it back, seems we will loss precision info. * when encode case class that defined inside a object, `ClassNotFound` exception will be thrown. I'll remove unused code in a follow-up PR. Author: Wenchen Fan <wenchen@databricks.com> Closes #9693 from cloud-fan/split.
* [SPARK-11654][SQL][FOLLOW-UP] fix some mistakes and clean upWenchen Fan2015-11-137-15/+17
| | | | | | | | | | | * rename `AppendColumn` to `AppendColumns` to be consistent with the physical plan name. * clean up stale comments. * always pass in resolved encoder to `TypedColumn.withInputType`(test added) * enable a mistakenly disabled java test. Author: Wenchen Fan <wenchen@databricks.com> Closes #9688 from cloud-fan/follow.
* [SPARK-11690][PYSPARK] Add pivot to python apiAndrew Ray2015-11-131-1/+23
| | | | | | | | This PR adds pivot to the python api of GroupedData with the same syntax as Scala/Java. Author: Andrew Ray <ray.andrew@gmail.com> Closes #9653 from aray/sql-pivot-python.
* [SPARK-11723][ML][DOC] Use LibSVM data source rather than ↵Yanbo Liang2015-11-1326-130/+79
| | | | | | | | | | | | | | | | MLUtils.loadLibSVMFile to load DataFrame Use LibSVM data source rather than MLUtils.loadLibSVMFile to load DataFrame, include: * Use libSVM data source for all example codes under examples/ml, and remove unused import. * Use libSVM data source for user guides under ml-*** which were omitted by #8697. * Fix bug: We should use ```sqlContext.read().format("libsvm").load(path)``` at Java side, but the API doc and user guides misuse as ```sqlContext.read.format("libsvm").load(path)```. * Code cleanup. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #9690 from yanboliang/spark-11723.
* [SPARK-11445][DOCS] Replaced example code in mllib-ensembles.md using ↵Rishabh Bhardwaj2015-11-1313-514/+885
| | | | | | | | | | | include_example I have made the required changes and tested. Kindly review the changes. Author: Rishabh Bhardwaj <rbnext29@gmail.com> Closes #9407 from rishabhbhardwaj/SPARK-11445.
* [SPARK-11678][SQL] Partition discovery should stop at the root path of the ↵Yin Huai2015-11-1310-51/+235
| | | | | | | | | | | | table. https://issues.apache.org/jira/browse/SPARK-11678 The change of this PR is to pass root paths of table to the partition discovery logic. So, the process of partition discovery stops at those root paths instead of going all the way to the root path of the file system. Author: Yin Huai <yhuai@databricks.com> Closes #9651 from yhuai/SPARK-11678.
* [SPARK-11706][STREAMING] Fix the bug that Streaming Python tests cannot ↵Shixiong Zhu2015-11-135-18/+30
| | | | | | | | | | report failures This PR just checks the test results and returns 1 if the test fails, so that `run-tests.py` can mark it fail. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9669 from zsxwing/streaming-python-tests.
* [SPARK-8029] Robust shuffle writerDavies Liu2015-11-1216-52/+402
| | | | | | | | | | Currently, all the shuffle writer will write to target path directly, the file could be corrupted by other attempt of the same partition on the same executor. They should write to temporary file then rename to target path, as what we do in output committer. In order to make the rename atomic, the temporary file should be created in the same local directory (FileSystem). This PR is based on #9214 , thanks to squito . Closes #9214 Author: Davies Liu <davies@databricks.com> Closes #9610 from davies/safe_shuffle.
* [SPARK-11629][ML][PYSPARK][DOC] Python example code for Multilayer ↵Yanbo Liang2015-11-124-66/+206
| | | | | | | | | | Perceptron Classification Add Python example code for Multilayer Perceptron Classification, and make example code in user guide document testable. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #9594 from yanboliang/spark-11629.
* [SPARK-11717] Ignore R session and history files from gitLewuathe2015-11-121-0/+4
| | | | | | | | | | | see: https://issues.apache.org/jira/browse/SPARK-11717 SparkR generates R session data and history files under current directory. It might be useful to ignore these files even running SparkR on spark directory for test or development. Author: Lewuathe <lewuathe@me.com> Closes #9681 from Lewuathe/SPARK-11717.
* [SPARK-11263][SPARKR] lintr Throws Warnings on Commented Code in Documentationfelixcheung2015-11-128-1512/+1539
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | Clean out hundreds of `style: Commented code should be removed.` from lintr Like these: ``` /opt/spark-1.6.0-bin-hadoop2.6/R/pkg/R/DataFrame.R:513:3: style: Commented code should be removed. # sc <- sparkR.init() ^~~~~~~~~~~~~~~~~~~ /opt/spark-1.6.0-bin-hadoop2.6/R/pkg/R/DataFrame.R:514:3: style: Commented code should be removed. # sqlContext <- sparkRSQL.init(sc) ^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ /opt/spark-1.6.0-bin-hadoop2.6/R/pkg/R/DataFrame.R:515:3: style: Commented code should be removed. # path <- "path/to/file.json" ^~~~~~~~~~~~~~~~~~~~~~~~~~~ ``` tried without export or rdname, neither work instead, added this `#' noRd` to suppress .Rd file generation also updated `family` for DataFrame functions for longer descriptive text instead of `dataframe_funcs` ![image](https://cloud.githubusercontent.com/assets/8969467/10933937/17bf5b1e-8291-11e5-9777-40fc632105dc.png) this covers *most* of 'Commented code' but I left out a few that looks legitimate. Author: felixcheung <felixcheung_m@hotmail.com> Closes #9463 from felixcheung/rlintr.
* [SPARK-11672][ML] flaky spark.ml read/write testsXiangrui Meng2015-11-125-5/+7
| | | | | | | | | | We set `sqlContext = null` in `afterAll`. However, this doesn't change `SQLContext.activeContext` and then `SQLContext.getOrCreate` might use the `SparkContext` from previous test suite and hence causes the error. This PR calls `clearActive` in `beforeAll` and `afterAll` to avoid using an old context from other test suites. cc: yhuai Author: Xiangrui Meng <meng@databricks.com> Closes #9677 from mengxr/SPARK-11672.2.
* [SPARK-11681][STREAMING] Correctly update state timestamp even when state is ↵Tathagata Das2015-11-122-49/+192
| | | | | | | | | | | | | not updated Bug: Timestamp is not updated if there is data but the corresponding state is not updated. This is wrong, and timeout is defined as "no data for a while", not "not state update for a while". Fix: Update timestamp when timestamp when timeout is specified, otherwise no need. Also refactored the code for better testability and added unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9648 from tdas/SPARK-11681.
* [SPARK-11419][STREAMING] Parallel recovery for FileBasedWriteAheadLog + ↵Burak Yavuz2015-11-127-37/+268
| | | | | | | | | | | | | | minor recovery tweaks The support for closing WriteAheadLog files after writes was just merged in. Closing every file after a write is a very expensive operation as it creates many small files on S3. It's not necessary to enable it on HDFS anyway. However, when you have many small files on S3, recovery takes very long. In addition, files start stacking up pretty quickly, and deletes may not be able to keep up, therefore deletes can also be parallelized. This PR adds support for the two parallelization steps mentioned above, in addition to a couple more failures I encountered during recovery. Author: Burak Yavuz <brkyvz@gmail.com> Closes #9373 from brkyvz/par-recovery.
* [SPARK-11663][STREAMING] Add Java API for trackStateByKeyShixiong Zhu2015-11-1212-52/+485
| | | | | | | | | | | TODO - [x] Add Java API - [x] Add API tests - [x] Add a function test Author: Shixiong Zhu <shixiong@databricks.com> Closes #9636 from zsxwing/java-track.