- Sep 27, 2016
-
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? This PR introduces more compact representation for ```UnsafeArrayData```. ```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts ``` [numElements] [offsets] [values] ``` `Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`. This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts. ``` [numElements][null bits][values or offset&length][variable length portion] ``` In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries. In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries. The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison: 1024x1024 elements integer array Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes In summary, we got 1.0-2.6x performance improvements over the code before applying this PR. Here are performance results of [benchmark programs](https://github.com/kiszk/spark/blob/04d2e4b6dbdc4eff43ce18b3c9b776e0129257c7/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala): **Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 430 / 436 390.0 2.6 1.0X Double 456 / 485 367.8 2.7 0.9X With SPARK-15962 Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 252 / 260 666.1 1.5 1.0X Double 281 / 292 597.7 1.7 0.9X ```` **Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 203 / 273 103.4 9.7 1.0X Double 239 / 356 87.9 11.4 0.8X With SPARK-15962 Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 196 / 249 107.0 9.3 1.0X Double 227 / 367 92.3 10.8 0.9X ```` **Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 207 / 217 304.2 3.3 1.0X Double 257 / 363 245.2 4.1 0.8X With SPARK-15962 Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 151 / 198 415.8 2.4 1.0X Double 214 / 394 293.6 3.4 0.7X ```` **Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 340 / 385 185.1 5.4 1.0X Double 479 / 705 131.3 7.6 0.7X With SPARK-15962 Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 206 / 211 306.0 3.3 1.0X Double 232 / 406 271.6 3.7 0.9X ```` 1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala) over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ serialize 442 / 533 0.0 441927.1 1.0X deserialize 217 / 274 0.0 217087.6 2.0X With SPARK-15962 VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ serialize 265 / 318 0.0 265138.5 1.0X deserialize 155 / 197 0.0 154611.4 1.7X ```` ## How was this patch tested? Added unit tests into ```UnsafeArraySuite``` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #13680 from kiszk/SPARK-15962.
-
Ding Fei authored
## What changes were proposed in this pull request? Fix two comments since Actor is not used anymore. Author: Ding Fei <danis@danix> Closes #15251 from danix800/comment-fixing.
-
- Sep 26, 2016
-
-
Yanbo Liang authored
## What changes were proposed in this pull request? #15140 exposed ```JavaSparkContext.addFile(path: String, recursive: Boolean)``` to Python/R, then we can update SparkR ```spark.addFile``` to support adding directory recursively. ## How was this patch tested? Added unit test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #15216 from yanboliang/spark-17577-2.
-
Andrew Mills authored
Corrected a link to the configuration.html page, it was pointing to a page that does not exist (configurations.html). Documentation change, verified in preview. Author: Andrew Mills <ammills01@users.noreply.github.com> Closes #15244 from ammills01/master.
-
Sameer Agarwal authored
## What changes were proposed in this pull request? This minor patch fixes a confusing exception message while reserving additional capacity in the vectorized parquet reader. ## How was this patch tested? Exisiting Unit Tests Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #15225 from sameeragarwal/error-msg.
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? When reading file stream with non-globbing path, the results return data with all `null`s for the partitioned columns. E.g., case class A(id: Int, value: Int) val data = spark.createDataset(Seq( A(1, 1), A(2, 2), A(2, 3)) ) val url = "/tmp/test" data.write.partitionBy("id").parquet(url) spark.read.parquet(url).show +-----+---+ |value| id| +-----+---+ | 2| 2| | 3| 2| | 1| 1| +-----+---+ val s = spark.readStream.schema(spark.read.load(url).schema).parquet(url) s.writeStream.queryName("test").format("memory").start() sql("SELECT * FROM test").show +-----+----+ |value| id| +-----+----+ | 2|null| | 3|null| | 1|null| +-----+----+ ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #14803 from viirya/filestreamsource-option.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Log how many Spark events got dropped in LiveListenerBus so that the user can get insights on how to set a correct event queue size. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #15220 from zsxwing/SPARK-17649.
-
hyukjinkwon authored
## What changes were proposed in this pull request? This was suggested in https://github.com/apache/spark/commit/101663f1ae222a919fc40510aa4f2bad22d1be6f#commitcomment-17114968. This PR adds `testImplicits` to `MLlibTestSparkContext` so that some implicits such as `toDF()` can be sued across ml tests. This PR also changes all the usages of `spark.createDataFrame( ... )` to `toDF()` where applicable in ml tests in Scala. ## How was this patch tested? Existing tests should work. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14035 from HyukjinKwon/minor-ml-test.
-
Justin Pihony authored
## What changes were proposed in this pull request? This change modifies the implementation of DataFrameWriter.save such that it works with jdbc, and the call to jdbc merely delegates to save. ## How was this patch tested? This was tested via unit tests in the JDBCWriteSuite, of which I added one new test to cover this scenario. ## Additional details rxin This seems to have been most recently touched by you and was also commented on in the JIRA. This contribution is my original work and I license the work to the project under the project's open source license. Author: Justin Pihony <justin.pihony@gmail.com> Author: Justin Pihony <justin.pihony@typesafe.com> Closes #12601 from JustinPihony/jdbc_reconciliation.
-
Yanbo Liang authored
## What changes were proposed in this pull request? #14597 modified ```ChiSqSelector``` to support ```fpr``` type selector, however, it left some issue need to be addressed: * We should allow users to set selector type explicitly rather than switching them by using different setting function, since the setting order will involves some unexpected issue. For example, if users both set ```numTopFeatures``` and ```percentile```, it will train ```kbest``` or ```percentile``` model based on the order of setting (the latter setting one will be trained). This make users confused, and we should allow users to set selector type explicitly. We handle similar issues at other place of ML code base such as ```GeneralizedLinearRegression``` and ```LogisticRegression```. * Meanwhile, if there are more than one parameter except ```alpha``` can be set for ```fpr``` model, we can not handle it elegantly in the existing framework. And similar issues for ```kbest``` and ```percentile``` model. Setting selector type explicitly can solve this issue also. * If setting selector type explicitly by users is allowed, we should handle param interaction such as if users set ```selectorType = percentile``` and ```alpha = 0.1```, we should notify users the parameter ```alpha``` will take no effect. We should handle complex parameter interaction checks at ```transformSchema```. (FYI #11620) * We should use lower case of the selector type names to follow MLlib convention. * Add ML Python API. ## How was this patch tested? Unit test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #15214 from yanboliang/spark-17017.
-
Burak Yavuz authored
## What changes were proposed in this pull request? When a malformed URL was sent to Executors through `sc.addJar` and `sc.addFile`, the executors become unusable, because they constantly throw `MalformedURLException`s and can never acknowledge that the file or jar is just bad input. This PR tries to fix that problem by making sure MalformedURLs can never be submitted through `sc.addJar` and `sc.addFile`. Another solution would be to blacklist bad files and jars on Executors. Maybe fail the first time, and then ignore the second time (but print a warning message). ## How was this patch tested? Unit tests in SparkContextSuite Author: Burak Yavuz <brkyvz@gmail.com> Closes #15224 from brkyvz/SPARK-17650.
-
- Sep 25, 2016
-
-
xin wu authored
## What changes were proposed in this pull request? This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST). Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files. ## How was this patch tested? Added a new test case in DataFrameSuite. Author: petermaxlee <petermaxlee@gmail.com> Author: Xin Wu <xinwu@us.ibm.com> Closes #15123 from petermaxlee/SPARK-17551.
-
- Sep 24, 2016
-
-
Yanbo Liang authored
## What changes were proposed in this pull request? Add ```sparkr-vignettes.html``` to ```.gitignore```. ## How was this patch tested? No need test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #15215 from yanboliang/ignore.
-
Sean Owen authored
## What changes were proposed in this pull request? Match ProbabilisticClassifer.thresholds requirements to R randomForest cutoff, requiring all > 0 ## How was this patch tested? Jenkins tests plus new test cases Author: Sean Owen <sowen@cloudera.com> Closes #15149 from srowen/SPARK-17057.
-
Sean Owen authored
[SPARK-10835][ML] Word2Vec should accept non-null string array, in addition to existing null string array ## What changes were proposed in this pull request? To match Tokenizer and for compatibility with Word2Vec, output a nullable string array type in NGram ## How was this patch tested? Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #15179 from srowen/SPARK-10835.
-
- Sep 23, 2016
-
-
Shivaram Venkataraman authored
## What changes were proposed in this pull request? This PR sets the R package version while tagging releases. Note that since R doesn't accept `-SNAPSHOT` in version number field, we remove that while setting the next version ## How was this patch tested? Tested manually by running locally Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #15223 from shivaram/sparkr-version-change.
-
jisookim authored
Currently task metrics don't support executor CPU time, so there's no way to calculate how much CPU time a stage/task took from History Server metrics. This PR enables reporting CPU time. Author: jisookim <jisookim0513@gmail.com> Closes #10212 from jisookim0513/add-cpu-time-metric.
-
Michael Armbrust authored
For some sources, it is difficult to provide a global ordering based only on the data in the offset. Since we don't use comparison for correctness, lets remove it. Author: Michael Armbrust <michael@databricks.com> Closes #15207 from marmbrus/removeComparable.
-
Jeff Zhang authored
## What changes were proposed in this pull request? Spark will add sparkr.zip to archive only when it is yarn mode (SparkSubmit.scala). ``` if (args.isR && clusterManager == YARN) { val sparkRPackagePath = RUtils.localSparkRPackagePath if (sparkRPackagePath.isEmpty) { printErrorAndExit("SPARK_HOME does not exist for R application in YARN mode.") } val sparkRPackageFile = new File(sparkRPackagePath.get, SPARKR_PACKAGE_ARCHIVE) if (!sparkRPackageFile.exists()) { printErrorAndExit(s"$SPARKR_PACKAGE_ARCHIVE does not exist for R application in YARN mode.") } val sparkRPackageURI = Utils.resolveURI(sparkRPackageFile.getAbsolutePath).toString // Distribute the SparkR package. // Assigns a symbol link name "sparkr" to the shipped package. args.archives = mergeFileLists(args.archives, sparkRPackageURI + "#sparkr") // Distribute the R package archive containing all the built R packages. if (!RUtils.rPackages.isEmpty) { val rPackageFile = RPackageUtils.zipRLibraries(new File(RUtils.rPackages.get), R_PACKAGE_ARCHIVE) if (!rPackageFile.exists()) { printErrorAndExit("Failed to zip all the built R packages.") } val rPackageURI = Utils.resolveURI(rPackageFile.getAbsolutePath).toString // Assigns a symbol link name "rpkg" to the shipped package. args.archives = mergeFileLists(args.archives, rPackageURI + "#rpkg") } } ``` So it is necessary to pass spark.master from R process to JVM. Otherwise sparkr.zip won't be distributed to executor. Besides that I also pass spark.yarn.keytab/spark.yarn.principal to spark side, because JVM process need them to access secured cluster. ## How was this patch tested? Verify it manually in R Studio using the following code. ``` Sys.setenv(SPARK_HOME="/Users/jzhang/github/spark") .libPaths(c(file.path(Sys.getenv(), "R", "lib"), .libPaths())) library(SparkR) sparkR.session(master="yarn-client", sparkConfig = list(spark.executor.instances="1")) df <- as.DataFrame(mtcars) head(df) ``` … Author: Jeff Zhang <zjffdu@apache.org> Closes #14784 from zjffdu/SPARK-17210.
-
WeichenXu authored
[SPARK-17499][SPARKR][ML][MLLIB] make the default params in sparkR spark.mlp consistent with MultilayerPerceptronClassifier ## What changes were proposed in this pull request? update `MultilayerPerceptronClassifierWrapper.fit` paramter type: `layers: Array[Int]` `seed: String` update several default params in sparkR `spark.mlp`: `tol` --> 1e-6 `stepSize` --> 0.03 `seed` --> NULL ( when seed == NULL, the scala-side wrapper regard it as a `null` value and the seed will use the default one ) r-side `seed` only support 32bit integer. remove `layers` default value, and move it in front of those parameters with default value. add `layers` parameter validation check. ## How was this patch tested? tests added. Author: WeichenXu <WeichenXu123@outlook.com> Closes #15051 from WeichenXu123/update_py_mlp_default.
-
Holden Karau authored
## What changes were proposed in this pull request? Move the internals of the PySpark accumulator API from the old deprecated API on top of the new accumulator API. ## How was this patch tested? The existing PySpark accumulator tests (both unit tests and doc tests at the start of accumulator.py). Author: Holden Karau <holden@us.ibm.com> Closes #14467 from holdenk/SPARK-16861-refactor-pyspark-accumulator-api.
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR proposes to close some stale PRs and ones suggested to be closed by committer(s) Closes #12415 Closes #14765 Closes #15118 Closes #15184 Closes #15183 Closes #9440 Closes #15023 Closes #14643 Closes #14827 ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #15198 from HyukjinKwon/stale-prs.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Avoid using -1 as the default batchId for FileStreamSource.FileEntry so that we can make sure not writing any FileEntry(..., batchId = -1) into the log. This also avoids people misusing it in future (#15203 is an example). ## How was this patch tested? Jenkins. Author: Shixiong Zhu <shixiong@databricks.com> Closes #15206 from zsxwing/cleanup.
-
Joseph K. Bradley authored
## What changes were proposed in this pull request? RandomForest currently sends the entire forest to each worker on each iteration. This is because (a) the node queue is FIFO and (b) the closure references the entire array of trees (topNodes). (a) causes RFs to handle splits in many trees, especially early on in learning. (b) sends all trees explicitly. This PR: (a) Change the RF node queue to be FILO (a stack), so that RFs tend to focus on 1 or a few trees before focusing on others. (b) Change topNodes to pass only the trees required on that iteration. ## How was this patch tested? Unit tests: * Existing tests for correctness of tree learning * Manually modifying code and running tests to verify that a small number of trees are communicated on each iteration * This last item is hard to test via unit tests given the current APIs. Author: Joseph K. Bradley <joseph@databricks.com> Closes #14359 from jkbradley/rfs-fewer-trees.
-
- Sep 22, 2016
-
-
Marcelo Vanzin authored
This was missing, preventing code that uses javax.crypto to properly compile in Spark. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15204 from vanzin/SPARK-17639.
-
Yucai Yu authored
## What changes were proposed in this pull request? "agg_plan" are hardcoded in HashAggregateExec, which have potential issue, so removing them. ## How was this patch tested? existing tests. Author: Yucai Yu <yucai.yu@intel.com> Closes #15199 from yucai/agg_plan.
-
Burak Yavuz authored
## What changes were proposed in this pull request? A [PR](https://github.com/apache/spark/commit/a6aade0042d9c065669f46d2dac40ec6ce361e63) was merged concurrently that made the unit test for PR #15122 not test anything anymore. This PR fixes the test. ## How was this patch tested? Changed line https://github.com/apache/spark/blob/0d634875026ccf1eaf984996e9460d7673561f80/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L137 from `false` to `true` and made sure the unit test failed. Author: Burak Yavuz <brkyvz@gmail.com> Closes #15203 from brkyvz/fix-test.
-
Gayathri Murali authored
## What changes were proposed in this pull request? Allow Spark 2.x to load instances of LDA, LocalLDAModel, and DistributedLDAModel saved from Spark 1.6. ## How was this patch tested? I tested this manually, saving the 3 types from 1.6 and loading them into master (2.x). In the future, we can add generic tests for testing backwards compatibility across all ML models in SPARK-15573. Author: Joseph K. Bradley <joseph@databricks.com> Closes #15034 from jkbradley/lda-backwards.
-
Herman van Hovell authored
## What changes were proposed in this pull request? We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example: ```sql select grp, collect_list(col1), count(distinct col2) from tbl_a group by 1 ``` This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6). ## How was this patch tested? Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15187 from hvanhovell/SPARK-17616.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? When the Python process is dead, the JVM StreamingContext is still running. Hence we will see a lot of Py4jException before the JVM process exits. It's better to stop the JVM StreamingContext to avoid those annoying logs. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #15201 from zsxwing/stop-jvm-ssc.
-
Burak Yavuz authored
## What changes were proposed in this pull request? Consider you have a bucket as `s3a://some-bucket` and under it you have files: ``` s3a://some-bucket/file1.parquet s3a://some-bucket/file2.parquet ``` Getting the parent path of `s3a://some-bucket/file1.parquet` yields `s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map. When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list! This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics. ## How was this patch tested? Unit test in `FileCatalogSuite`. Author: Burak Yavuz <brkyvz@gmail.com> Closes #15169 from brkyvz/SPARK-17613.
-
Shivaram Venkataraman authored
## What changes were proposed in this pull request? When we build the docs separately we don't have the JAR files from the Spark build in the same tree. As the SparkR vignettes need to launch a SparkContext to be built, we skip building them if JAR files don't exist ## How was this patch tested? To test this we can run the following: ``` build/mvn -DskipTests -Psparkr clean ./R/create-docs.sh ``` You should see a line `Skipping R vignettes as Spark JARs not found` at the end Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #15200 from shivaram/sparkr-vignette-skip.
-
Dhruve Ashar authored
## What changes were proposed in this pull request? We are killing multiple executors together instead of iterating over expensive RPC calls to kill single executor. ## How was this patch tested? Executed sample spark job to observe executors being killed/removed with dynamic allocation enabled. Author: Dhruve Ashar <dashar@yahoo-inc.com> Author: Dhruve Ashar <dhruveashar@gmail.com> Closes #15152 from dhruve/impr/SPARK-17365.
-
Wenchen Fan authored
## What changes were proposed in this pull request? This comment went stale long time ago, this PR fixes it according to my understanding. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #15095 from cloud-fan/update-comment.
-
WeichenXu authored
## What changes were proposed in this pull request? Add treeAggregateDepth parameter for AFTSurvivalRegression to keep consistent with LiR/LoR. ## How was this patch tested? Existing tests. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14851 from WeichenXu123/add_treeAggregate_param_for_survival_regression.
-
frreiss authored
## What changes were proposed in this pull request? Modified the documentation to clarify that `build/mvn` and `pom.xml` always add Java 7-specific parameters to `MAVEN_OPTS`, and that developers can safely ignore warnings about `-XX:MaxPermSize` that may result from compiling or running tests with Java 8. ## How was this patch tested? Rebuilt HTML documentation, made sure that building-spark.html displays correctly in a browser. Author: frreiss <frreiss@us.ibm.com> Closes #15005 from frreiss/fred-17421a.
-
Zhenhua Wang authored
[SPARK-17625][SQL] set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation ## What changes were proposed in this pull request? We should set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation, otherwise the outputs of LogicalRelation are different from outputs of SimpleCatalogRelation - they have different exprId's. ## How was this patch tested? add a test case Author: Zhenhua Wang <wzh_zju@163.com> Closes #15182 from wzhfy/expectedAttributes.
-
gatorsmile authored
### What changes were proposed in this pull request? For data sources without extending `SchemaRelationProvider`, we expect users to not specify schemas when they creating tables. If the schema is input from users, an exception is issued. Since Spark 2.1, for any data source, to avoid infer the schema every time, we store the schema in the metastore catalog. Thus, when reading a cataloged data source table, the schema could be read from metastore catalog. In this case, we also got an exception. For example, ```Scala sql( s""" |CREATE TABLE relationProvierWithSchema |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', | To '10' |) """.stripMargin) spark.table(tableName).show() ``` ``` org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas.; ``` This PR is to fix the above issue. When building a data source, we introduce a flag `isSchemaFromUsers` to indicate whether the schema is really input from users. If true, we issue an exception. Otherwise, we will call the `createRelation` of `RelationProvider` to generate the `BaseRelation`, in which it contains the actual schema. ### How was this patch tested? Added a few cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #15046 from gatorsmile/tempViewCases.
-
Yadong Qi authored
[SPARK-17425][SQL] Override sameResult in HiveTableScanExec to make ReuseExchange work in text format table ## What changes were proposed in this pull request? The PR will override the `sameResult` in `HiveTableScanExec` to make `ReuseExchange` work in text format table. ## How was this patch tested? # SQL ```sql SELECT * FROM src t1 JOIN src t2 ON t1.key = t2.key JOIN src t3 ON t1.key = t3.key; ``` # Before ``` == Physical Plan == *BroadcastHashJoin [key#30], [key#34], Inner, BuildRight :- *BroadcastHashJoin [key#30], [key#32], Inner, BuildRight : :- *Filter isnotnull(key#30) : : +- HiveTableScan [key#30, value#31], MetastoreRelation default, src : +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))) : +- *Filter isnotnull(key#32) : +- HiveTableScan [key#32, value#33], MetastoreRelation default, src +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))) +- *Filter isnotnull(key#34) +- HiveTableScan [key#34, value#35], MetastoreRelation default, src ``` # After ``` == Physical Plan == *BroadcastHashJoin [key#2], [key#6], Inner, BuildRight :- *BroadcastHashJoin [key#2], [key#4], Inner, BuildRight : :- *Filter isnotnull(key#2) : : +- HiveTableScan [key#2, value#3], MetastoreRelation default, src : +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))) : +- *Filter isnotnull(key#4) : +- HiveTableScan [key#4, value#5], MetastoreRelation default, src +- ReusedExchange [key#6, value#7], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))) ``` cc: davies cloud-fan Author: Yadong Qi <qiyadong2010@gmail.com> Closes #14988 from watermen/SPARK-17425.
-
- Sep 21, 2016
-
-
Wenchen Fan authored
## What changes were proposed in this pull request? After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks. This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in #15054 , to make the code simpler. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #15160 from cloud-fan/exists.
-