- Dec 08, 2016
-
-
Tathagata Das authored
## What changes were proposed in this pull request? - Changed FileStreamSource to use new FileStreamSourceOffset rather than LongOffset. The field is named as `logOffset` to make it more clear that this is a offset in the file stream log. - Fixed bug in FileStreamSourceLog, the field endId in the FileStreamSourceLog.get(startId, endId) was not being used at all. No test caught it earlier. Only my updated tests caught it. Other minor changes - Dont use batchId in the FileStreamSource, as calling it batch id is extremely miss leading. With multiple sources, it may happen that a new batch has no new data from a file source. So offset of FileStreamSource != batchId after that batch. ## How was this patch tested? Updated unit test. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16205 from tdas/SPARK-18776.
-
Shivaram Venkataraman authored
This PR changes the SparkR source release tarball to be built using the Hadoop 2.6 profile. Previously it was using the without hadoop profile which leads to an error as discussed in https://github.com/apache/spark/pull/16014#issuecomment-265843991 Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #16218 from shivaram/fix-sparkr-release-build.
-
Reynold Xin authored
Closes #16191 Closes #16198 Closes #14561 Closes #14223 Closes #7739 Closes #13026 Closes #16217
-
Reynold Xin authored
## What changes were proposed in this pull request? This patch fixes the format specification in explain for file sources (Parquet and Text formats are the only two that are different from the rest): Before: ``` scala> spark.read.text("test.text").explain() == Physical Plan == *FileScan text [value#15] Batched: false, Format: org.apache.spark.sql.execution.datasources.text.TextFileFormatxyz, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string> ``` After: ``` scala> spark.read.text("test.text").explain() == Physical Plan == *FileScan text [value#15] Batched: false, Format: Text, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string> ``` Also closes #14680. ## How was this patch tested? Verified in spark-shell. Author: Reynold Xin <rxin@databricks.com> Closes #16187 from rxin/SPARK-18760.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? When `SparkContext.stop` is called in `Utils.tryOrStopSparkContext` (the following three places), it will cause deadlock because the `stop` method needs to wait for the thread running `stop` to exit. - ContextCleaner.keepCleaning - LiveListenerBus.listenerThread.run - TaskSchedulerImpl.start This PR adds `SparkContext.stopInNewThread` and uses it to eliminate the potential deadlock. I also removed my changes in #15775 since they are not necessary now. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16178 from zsxwing/fix-stop-deadlock.
-
Felix Cheung authored
## What changes were proposed in this pull request? This PR has 2 key changes. One, we are building source package (aka bundle package) for SparkR which could be released on CRAN. Two, we should include in the official Spark binary distributions SparkR installed from this source package instead (which would have help/vignettes rds needed for those to work when the SparkR package is loaded in R, whereas earlier approach with devtools does not) But, because of various differences in how R performs different tasks, this PR is a fair bit more complicated. More details below. This PR also includes a few minor fixes. ### more details These are the additional steps in make-distribution; please see [here](https://github.com/apache/spark/blob/master/R/CRAN_RELEASE.md) on what's going to a CRAN release, which is now run during make-distribution.sh. 1. package needs to be installed because the first code block in vignettes is `library(SparkR)` without lib path 2. `R CMD build` will build vignettes (this process runs Spark/SparkR code and captures outputs into pdf documentation) 3. `R CMD check` on the source package will install package and build vignettes again (this time from source packaged) - this is a key step required to release R package on CRAN (will skip tests here but tests will need to pass for CRAN release process to success - ideally, during release signoff we should install from the R source package and run tests) 4. `R CMD Install` on the source package (this is the only way to generate doc/vignettes rds files correctly, not in step # 1) (the output of this step is what we package into Spark dist and sparkr.zip) Alternatively, R CMD build should already be installing the package in a temp directory though it might just be finding this location and set it to lib.loc parameter; another approach is perhaps we could try calling `R CMD INSTALL --build pkg` instead. But in any case, despite installing the package multiple times this is relatively fast. Building vignettes takes a while though. ## How was this patch tested? Manually, CI. Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16014 from felixcheung/rdist.
-
Andrew Ray authored
## What changes were proposed in this pull request? Fixes a bug in the python implementation of rdd cartesian product related to batching that showed up in repeated cartesian products with seemingly random results. The root cause being multiple iterators pulling from the same stream in the wrong order because of logic that ignored batching. `CartesianDeserializer` and `PairDeserializer` were changed to implement `_load_stream_without_unbatching` and borrow the one line implementation of `load_stream` from `BatchedSerializer`. The default implementation of `_load_stream_without_unbatching` was changed to give consistent results (always an iterable) so that it could be used without additional checks. `PairDeserializer` no longer extends `CartesianDeserializer` as it was not really proper. If wanted a new common super class could be added. Both `CartesianDeserializer` and `PairDeserializer` now only extend `Serializer` (which has no `dump_stream` implementation) since they are only meant for *de*serialization. ## How was this patch tested? Additional unit tests (sourced from #14248) plus one for testing a cartesian with zip. Author: Andrew Ray <ray.andrew@gmail.com> Closes #16121 from aray/fix-cartesian.
-
Ergin Seyfe authored
## What changes were proposed in this pull request? - Removed the`attempt.completed ` filter so cleaner would include the orphan inprogress files. - Use loading time for inprogress files as lastUpdated. Keep using the modTime for completed files. First one will prevent deletion of inprogress job files. Second one will ensure that lastUpdated time won't change for completed jobs in an event of HistoryServer reboot. ## How was this patch tested? Added new unittests and via existing tests. Author: Ergin Seyfe <eseyfe@fb.com> Closes #16165 from seyfe/clear_old_inprogress_files.
-
Marcelo Vanzin authored
These directories are added to the classpath of applications when testing or using SPARK_PREPEND_CLASSES, otherwise updated classes are not seen. Also, add the mesos directory which was missing. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16202 from vanzin/SPARK-18662.
-
Liang-Chi Hsieh authored
[SPARK-18667][PYSPARK][SQL] Change the way to group row in BatchEvalPythonExec so input_file_name function can work with UDF in pyspark ## What changes were proposed in this pull request? `input_file_name` doesn't return filename when working with UDF in PySpark. An example shows the problem: from pyspark.sql.functions import * from pyspark.sql.types import * def filename(path): return path sourceFile = udf(filename, StringType()) spark.read.json("tmp.json").select(sourceFile(input_file_name())).show() +---------------------------+ |filename(input_file_name())| +---------------------------+ | | +---------------------------+ The cause of this issue is, we group rows in `BatchEvalPythonExec` for batching processing of PythonUDF. Currently we group rows first and then evaluate expressions on the rows. If the data is less than the required number of rows for a group, the iterator will be consumed to the end before the evaluation. However, once the iterator reaches the end, we will unset input filename. So the input_file_name expression can't return correct filename. This patch fixes the approach to group the batch of rows. We evaluate the expression first and then group evaluated results to batch. ## How was this patch tested? Added unit test to PySpark. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16115 from viirya/fix-py-udf-input-filename.
-
hyukjinkwon authored
[SPARK-18718][TESTS] Skip some test failures due to path length limitation and fix tests to pass on Windows ## What changes were proposed in this pull request? There are some tests failed on Windows due to the wrong format of path and the limitation of path length as below: This PR proposes both to fix the failed tests by fixing the path for the tests below: - `InsertSuite` ``` Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.sources.InsertSuite *** ABORTED *** (12 seconds, 547 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-177945ef-9128-42b4-8c07-de31f78bbbd6; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) ``` - `PathOptionSuite` ``` - path option also exist for write path *** FAILED *** (1 second, 93 milliseconds) "C:[projectsspark arget mp]spark-5ab34a58-df8d-..." did not equal "C:[\projects\spark\target\tmp\]spark-5ab34a58-df8d-..." (PathOptionSuite.scala:93) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) ... ``` - `UDFSuite` ``` - SPARK-8005 input_file_name *** FAILED *** (2 seconds, 234 milliseconds) "file:///C:/projects/spark/target/tmp/spark-e4e5720a-2006-48f9-8b11-797bf59794bf/part-00001-26fb05e4-603d-471d-ae9d-b9549e0c7765.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-e4e5720a-2006-48f9-8b11-797bf59794bf" (UDFSuite.scala:67) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) ... ``` and to skip the tests belows which are being failed on Windows due to path length limitation. - `SparkLauncherSuite` ``` Test org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher failed: java.lang.AssertionError: expected:<0> but was:<1>, took 0.062 sec at org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher(SparkLauncherSuite.java:177) ... ``` The stderr from the process is `The filename or extension is too long` which is equivalent to the one below. - `BroadcastJoinSuite` ``` 04:09:40.882 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor java.io.IOException: Cannot run program "C:\Progra~1\Java\jdk1.8.0\bin\java" (in directory "C:\projects\spark\work\app-20161205040542-0000\51658"): CreateProcess error=206, The filename or extension is too long at java.lang.ProcessBuilder.start(ProcessBuilder.java:1048) at org.apache.spark.deploy.worker.ExecutorRunner.org$apache$spark$deploy$worker$ExecutorRunner$$fetchAndRunExecutor(ExecutorRunner.scala:167) at org.apache.spark.deploy.worker.ExecutorRunner$$anon$1.run(ExecutorRunner.scala:73) Caused by: java.io.IOException: CreateProcess error=206, The filename or extension is too long at java.lang.ProcessImpl.create(Native Method) at java.lang.ProcessImpl.<init>(ProcessImpl.java:386) at java.lang.ProcessImpl.start(ProcessImpl.java:137) at java.lang.ProcessBuilder.start(ProcessBuilder.java:1029) ... 2 more 04:09:40.929 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor (appearently infinite same error messages) ... ``` ## How was this patch tested? Manually tested via AppVeyor. **Before** `InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/148-InsertSuite-pr `PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/139-PathOptionSuite-pr `UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/143-UDFSuite-pr `SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/141-SparkLauncherSuite-pr `BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/145-BroadcastJoinSuite-pr **After** `PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/140-PathOptionSuite-pr `SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/142-SparkLauncherSuite-pr `UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/144-UDFSuite-pr `InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/147-InsertSuite-pr `BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/149-BroadcastJoinSuite-pr Author: hyukjinkwon <gurwls223@gmail.com> Closes #16147 from HyukjinKwon/fix-tests.
-
Yanbo Liang authored
## What changes were proposed in this pull request? * Add all R examples for ML wrappers which were added during 2.1 release cycle. * Split the whole ```ml.R``` example file into individual example for each algorithm, which will be convenient for users to rerun them. * Add corresponding examples to ML user guide. * Update ML section of SparkR user guide. Note: MLlib Scala/Java/Python examples will be consistent, however, SparkR examples may different from them, since R users may use the algorithms in a different way, for example, using R ```formula``` to specify ```featuresCol``` and ```labelCol```. ## How was this patch tested? Run all examples manually. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16148 from yanboliang/spark-18325.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? When `ignoreCorruptFiles` is enabled, it's better to also ignore non-existing files. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16203 from zsxwing/ignore-file-not-found.
-
Reynold Xin authored
Closes #15689 Closes #14640 Closes #15917 Closes #16188 Closes #16206
-
- Dec 07, 2016
-
-
Yanbo Liang authored
## What changes were proposed in this pull request? Reviewing SparkR ML wrappers API for 2.1 release, mainly two issues: * Remove ```probabilityCol``` from the argument list of ```spark.logit``` and ```spark.randomForest```. Since it was used when making prediction and should be an argument of ```predict```, and we will work on this at [SPARK-18618](https://issues.apache.org/jira/browse/SPARK-18618) in the next release cycle. * Fix ```spark.als``` params to make it consistent with MLlib. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16169 from yanboliang/spark-18326.
-
sethah authored
## What changes were proposed in this pull request? WeightedLeastSquares now supports L1 and elastic net penalties and has an additional solver option: QuasiNewton. The docs are updated to reflect this change. ## How was this patch tested? Docs only. Generated documentation to make sure Latex looks ok. Author: sethah <seth.hendrickson16@gmail.com> Closes #16139 from sethah/SPARK-18705.
-
Tathagata Das authored
[SPARK-18758][SS] StreamingQueryListener events from a StreamingQuery should be sent only to the listeners in the same session as the query ## What changes were proposed in this pull request? Listeners added with `sparkSession.streams.addListener(l)` are added to a SparkSession. So events only from queries in the same session as a listener should be posted to the listener. Currently, all the events gets rerouted through the Spark's main listener bus, that is, - StreamingQuery posts event to StreamingQueryListenerBus. Only the queries associated with the same session as the bus posts events to it. - StreamingQueryListenerBus posts event to Spark's main LiveListenerBus as a SparkEvent. - StreamingQueryListenerBus also subscribes to LiveListenerBus events thus getting back the posted event in a different thread. - The received is posted to the registered listeners. The problem is that *all StreamingQueryListenerBuses in all sessions* gets the events and posts them to their listeners. This is wrong. In this PR, I solve it by making StreamingQueryListenerBus track active queries (by their runIds) when a query posts the QueryStarted event to the bus. This allows the rerouted events to be filtered using the tracked queries. Note that this list needs to be maintained separately from the `StreamingQueryManager.activeQueries` because a terminated query is cleared from `StreamingQueryManager.activeQueries` as soon as it is stopped, but the this ListenerBus must clear a query only after the termination event of that query has been posted lazily, much after the query has been terminated. Credit goes to zsxwing for coming up with the initial idea. ## How was this patch tested? Updated test harness code to use the correct session, and added new unit test. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16186 from tdas/SPARK-18758.
-
wm624@hotmail.com authored
## What changes were proposed in this pull request? Logistic Regression summary is added in Python API. We need to add example and document for summary. The newly added example is consistent with Scala and Java examples. ## How was this patch tested? Manually tests: Run the example with spark-submit; copy & paste code into pyspark; build document and check the document. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16064 from wangmiao1981/py.
-
Nathan Howell authored
## What changes were proposed in this pull request? `makeRootConverter` is only called with a `StructType` value. By making this method less general we can remove pattern matches, which are never actually hit outside of the test suite. ## How was this patch tested? The existing tests. Author: Nathan Howell <nhowell@godaddy.com> Closes #16084 from NathanHowell/SPARK-18654.
-
Michael Armbrust authored
Based on an informal survey, users find this option easier to understand / remember. Author: Michael Armbrust <michael@databricks.com> Closes #16182 from marmbrus/renameRecentProgress.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Fixed the following failures: ``` org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 3745 times over 1.0000790851666665 minutes. Last failure message: assertion failed: failOnDataLoss-0 not deleted after timeout. ``` ``` sbt.ForkMain$ForkError: org.apache.spark.sql.streaming.StreamingQueryException: Query query-66 terminated with exception: null at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:252) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:146) Caused by: sbt.ForkMain$ForkError: java.lang.NullPointerException: null at java.util.ArrayList.addAll(ArrayList.java:577) at org.apache.kafka.clients.Metadata.getClusterForCurrentTopics(Metadata.java:257) at org.apache.kafka.clients.Metadata.update(Metadata.java:177) at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.handleResponse(NetworkClient.java:605) at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeHandleCompletedReceive(NetworkClient.java:582) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:450) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.awaitPendingRequests(ConsumerNetworkClient.java:260) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:222) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.ensurePartitionAssignment(ConsumerCoordinator.java:366) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:978) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:938) at ... ``` ## How was this patch tested? Tested in #16048 by running many times. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16109 from zsxwing/fix-kafka-flaky-test.
-
sarutak authored
## What changes were proposed in this pull request? When SSL is enabled, the Spark shell shows: ``` Spark context Web UI available at https://192.168.99.1:4040 ``` This is wrong because 4040 is http, not https. It redirects to the https port. More importantly, this introduces several broken links in the UI. For example, in the master UI, the worker link is https:8081 instead of http:8081 or https:8481. CC: mengxr liancheng I manually tested accessing by accessing MasterPage, WorkerPage and HistoryServer with SSL enabled. Author: sarutak <sarutak@oss.nttdata.co.jp> Closes #16190 from sarutak/SPARK-18761.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? It's better to add a warning log when skipping a corrupted file. It will be helpful when we want to finish the job first, then find them in the log and fix these files. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16192 from zsxwing/SPARK-18764.
-
Andrew Ray authored
## What changes were proposed in this pull request? Fixes AnalysisException for pivot queries that have group by columns that are expressions and not attributes by substituting the expressions output attribute in the second aggregation and final projection. ## How was this patch tested? existing and additional unit tests Author: Andrew Ray <ray.andrew@gmail.com> Closes #16177 from aray/SPARK-17760.
-
Jie Xiong authored
## What changes were proposed in this pull request? BytesToBytesMap currently does not release the in-memory storage (the longArray variable) after it spills to disk. This is typically not a problem during aggregation because the longArray should be much smaller than the pages, and because we grow the longArray at a conservative rate. However this can lead to an OOM when an already running task is allocated more than its fair share, this can happen because of a scheduling delay. In this case the longArray can grow beyond the fair share of memory for the task. This becomes problematic when the task spills and the long array is not freed, that causes subsequent memory allocation requests to be denied by the memory manager resulting in an OOM. This PR fixes this issuing by freeing the longArray when the BytesToBytesMap spills. ## How was this patch tested? Existing tests and tested on realworld workloads. Author: Jie Xiong <jiexiong@fb.com> Author: jiexiong <jiexiong@gmail.com> Closes #15722 from jiexiong/jie_oom_fix.
-
Sean Owen authored
## What changes were proposed in this pull request? Fix reservoir sampling bias for small k. An off-by-one error meant that the probability of replacement was slightly too high -- k/(l-1) after l element instead of k/l, which matters for small k. ## How was this patch tested? Existing test plus new test case. Author: Sean Owen <sowen@cloudera.com> Closes #16129 from srowen/SPARK-18678.
-
actuaryzhang authored
Poisson GLM fails for many standard data sets (see example in test or JIRA). The issue is incorrect initialization leading to almost zero probability and weights. Specifically, the mean is initialized as the response, which could be zero. Applying the log link results in very negative numbers (protected against -Inf), which again leads to close to zero probability and weights in the weighted least squares. Fix and test are included in the commits. ## What changes were proposed in this pull request? Update initialization in Poisson GLM ## How was this patch tested? Add test in GeneralizedLinearRegressionSuite srowen sethah yanboliang HyukjinKwon mengxr Author: actuaryzhang <actuaryzhang10@gmail.com> Closes #16131 from actuaryzhang/master.
-
Yanbo Liang authored
## What changes were proposed in this pull request? Several cleanup and improvements for ```spark.logit```: * ```summary``` should return coefficients matrix, and should output labels for each class if the model is multinomial logistic regression model. * ```summary``` should not return ```areaUnderROC, roc, pr, ...```, since most of them are DataFrame which are less important for R users. Meanwhile, these metrics ignore instance weights (setting all to 1.0) which will be changed in later Spark version. In case it will introduce breaking changes, we do not expose them currently. * SparkR test improvement: comparing the training result with native R glmnet. * Remove argument ```aggregationDepth``` from ```spark.logit```, since it's an expert Param(related with Spark architecture and job execution) that would be used rarely by R users. ## How was this patch tested? Unit tests. The ```summary``` output after this change: multinomial logistic regression: ``` > df <- suppressWarnings(createDataFrame(iris)) > model <- spark.logit(df, Species ~ ., regParam = 0.5) > summary(model) $coefficients versicolor virginica setosa (Intercept) 1.514031 -2.609108 1.095077 Sepal_Length 0.02511006 0.2649821 -0.2900921 Sepal_Width -0.5291215 -0.02016446 0.549286 Petal_Length 0.03647411 0.1544119 -0.190886 Petal_Width 0.000236092 0.4195804 -0.4198165 ``` binomial logistic regression: ``` > df <- suppressWarnings(createDataFrame(iris)) > training <- df[df$Species %in% c("versicolor", "virginica"), ] > model <- spark.logit(training, Species ~ ., regParam = 0.5) > summary(model) $coefficients Estimate (Intercept) -6.053815 Sepal_Length 0.2449379 Sepal_Width 0.1648321 Petal_Length 0.4730718 Petal_Width 1.031947 ``` Author: Yanbo Liang <ybliang8@gmail.com> Closes #16117 from yanboliang/spark-18686.
-
- Dec 06, 2016
-
-
Tathagata Das authored
## What changes were proposed in this pull request? Maven compilation seem to not allow resource is sql/test to be easily referred to in kafka-0-10-sql tests. So moved the kafka-source-offset-version-2.1.0 from sql test resources to kafka-0-10-sql test resources. ## How was this patch tested? Manually ran maven test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16183 from tdas/SPARK-18671-1.
-
Reynold Xin authored
Closes #14537. Closes #16181. Closes #8318. Closes #6848. Closes #7265. Closes #9543.
-
c-sahuja authored
## What changes were proposed in this pull request? Although, currently, the saveAsTable does not provide an API to save the table as an external table from a DataFrame, we can achieve this functionality by using options on DataFrameWriter where the key for the map is the String: "path" and the value is another String which is the location of the external table itself. This can be provided before the call to saveAsTable is performed. ## How was this patch tested? Documentation was reviewed for formatting and content after the push was performed on the branch.  Author: c-sahuja <sahuja@cloudera.com> Closes #16185 from c-sahuja/createExternalTable.
-
Tathagata Das authored
[SPARK-18734][SS] Represent timestamp in StreamingQueryProgress as formatted string instead of millis ## What changes were proposed in this pull request? Easier to read while debugging as a formatted string (in ISO8601 format) than in millis ## How was this patch tested? Updated unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16166 from tdas/SPARK-18734.
-
-
Anirudh authored
## What changes were proposed in this pull request? * Moves yarn and mesos scheduler backends to resource-managers/ sub-directory (in preparation for https://issues.apache.org/jira/browse/SPARK-18278) * Corresponding change in top-level pom.xml. Ref: https://github.com/apache/spark/pull/16061#issuecomment-263649340 ## How was this patch tested? * Manual tests /cc rxin Author: Anirudh <ramanathana@google.com> Closes #16092 from foxish/fix-scheduler-structure-2.
-
Shuai Lin authored
[SPARK-18171][MESOS] Show correct framework address in mesos master web ui when the advertised address is used ## What changes were proposed in this pull request? In [SPARK-4563](https://issues.apache.org/jira/browse/SPARK-4563) we added the support for the driver to advertise a different hostname/ip (`spark.driver.host` to the executors other than the hostname/ip the driver actually binds to (`spark.driver.bindAddress`). But in the mesos webui's frameworks page, it still shows the driver's binds hostname/ip (though the web ui link is correct). We should fix it to make them consistent. Before:  After:  This PR doesn't affect the behavior on the spark side, only makes the display on the mesos master side more consistent. ## How was this patch tested? Manual test. - Build the package and build a docker image (spark:2.1-test) ``` sh ./dev/make-distribution.sh -Phadoop-2.6 -Phive -Phive-thriftserver -Pyarn -Pmesos ``` - Then run the spark driver inside a docker container. ``` sh docker run --rm -it \ --name=spark \ -p 30000-30010:30000-30010 \ -e LIBPROCESS_ADVERTISE_IP=172.17.42.1 \ -e LIBPROCESS_PORT=30000 \ -e MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos-1.0.0.so \ -e MESOS_NATIVE_JAVA_LIBRARY=/usr/local/lib/libmesos-1.0.0.so \ -e SPARK_HOME=/opt/dist \ spark:2.1-test ``` - Inside the container, launch the spark driver, making use of the advertised address: ``` sh /opt/dist/bin/spark-shell \ --master mesos://zk://172.17.42.1:2181/mesos \ --conf spark.driver.host=172.17.42.1 \ --conf spark.driver.bindAddress=172.17.0.1 \ --conf spark.driver.port=30001 \ --conf spark.driver.blockManager.port=30002 \ --conf spark.ui.port=30003 \ --conf spark.mesos.coarse=true \ --conf spark.cores.max=2 \ --conf spark.executor.cores=1 \ --conf spark.executor.memory=1g \ --conf spark.mesos.executor.docker.image=spark:2.1-test ``` - Run several spark jobs to ensure everything is running fine. ``` scala val rdd = sc.textFile("file:///opt/dist/README.md") rdd.cache().count ``` Author: Shuai Lin <linshuai2012@gmail.com> Closes #15684 from lins05/spark-18171-show-correct-host-name-in-mesos-master-web-ui.
-
Weiqing Yang authored
## What changes were proposed in this pull request? This PR is to upgrade sbt plugins. The following sbt plugins will be upgraded: ``` sbt-assembly: 0.11.2 -> 0.14.3 sbteclipse-plugin: 4.0.0 -> 5.0.1 sbt-mima-plugin: 0.1.11 -> 0.1.12 org.ow2.asm/asm: 5.0.3 -> 5.1 org.ow2.asm/asm-commons: 5.0.3 -> 5.1 ``` All other plugins are up-to-date. ## How was this patch tested? Pass the Jenkins build. Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #16159 from weiqingy/SPARK-18697.
-
Shuai Lin authored
## What changes were proposed in this pull request? Since we already include the python examples in the pyspark package, we should include the example data with it as well. We should also include the third-party licences since we distribute their jars with the pyspark package. ## How was this patch tested? Manually tested with python2.7 and python3.4 ```sh $ ./build/mvn -DskipTests -Phive -Phive-thriftserver -Pyarn -Pmesos clean package $ cd python $ python setup.py sdist $ pip install dist/pyspark-2.1.0.dev0.tar.gz $ ls -1 /usr/local/lib/python2.7/dist-packages/pyspark/data/ graphx mllib streaming $ du -sh /usr/local/lib/python2.7/dist-packages/pyspark/data/ 600K /usr/local/lib/python2.7/dist-packages/pyspark/data/ $ ls -1 /usr/local/lib/python2.7/dist-packages/pyspark/licenses/|head -5 LICENSE-AnchorJS.txt LICENSE-DPark.txt LICENSE-Mockito.txt LICENSE-SnapTree.txt LICENSE-antlr.txt ``` Author: Shuai Lin <linshuai2012@gmail.com> Closes #16082 from lins05/include-data-in-pyspark-dist.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? We added some codes in https://github.com/apache/spark/pull/14961 because of https://github.com/netty/netty/issues/5833 Now we can remove them as it's fixed in Netty 4.0.42.Final. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16167 from zsxwing/remove-netty-workaround.
-
Yuhao authored
## What changes were proposed in this pull request? Currently English stop words list in MLlib contains only the argumented words after removing all the apostrophes, so "wouldn't" become "wouldn" and "t". Yet by default Tokenizer and RegexTokenizer don't split on apostrophes or quotes. Adding original form to stop words list to match the behavior of Tokenizer and StopwordsRemover. Also remove "won" from list. see more discussion in the jira: https://issues.apache.org/jira/browse/SPARK-18374 ## How was this patch tested? existing ut Author: Yuhao <yuhao.yang@intel.com> Author: Yuhao Yang <hhbyyh@gmail.com> Closes #16103 from hhbyyh/addstopwords.
-
Tathagata Das authored
[SPARK-18671][SS][TEST] Added tests to ensure stability of that all Structured Streaming log formats ## What changes were proposed in this pull request? To be able to restart StreamingQueries across Spark version, we have already made the logs (offset log, file source log, file sink log) use json. We should added tests with actual json files in the Spark such that any incompatible changes in reading the logs is immediately caught. This PR add tests for FileStreamSourceLog, FileStreamSinkLog, and OffsetSeqLog. ## How was this patch tested? new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16128 from tdas/SPARK-18671.
-