- Jul 10, 2017
-
-
Bryan Cutler authored
## What changes were proposed in this pull request? Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`. This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process. The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame. Data types except complex, date, timestamp, and decimal are currently supported, otherwise an `UnsupportedOperation` exception is thrown. Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served. A package private class/object `ArrowConverters` that provide data type mappings and conversion routines. In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default). ## How was this patch tested? Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types. The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data. This will ensure that the schema and data has been converted correctly. Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow. A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas. Author: Bryan Cutler <cutlerb@gmail.com> Author: Li Jin <ice.xelloss@gmail.com> Author: Li Jin <li.jin@twosigma.com> Author: Wes McKinney <wes.mckinney@twosigma.com> Closes #18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR supports schema in a DDL formatted string for `from_json` in R/Python and `dapply` and `gapply` in R, which are commonly used and/or consistent with Scala APIs. Additionally, this PR exposes `structType` in R to allow working around in other possible corner cases. **Python** `from_json` ```python from pyspark.sql.functions import from_json data = [(1, '''{"a": 1}''')] df = spark.createDataFrame(data, ("key", "value")) df.select(from_json(df.value, "a INT").alias("json")).show() ``` **R** `from_json` ```R df <- sql("SELECT named_struct('name', 'Bob') as people") df <- mutate(df, people_json = to_json(df$people)) head(select(df, from_json(df$people_json, "name STRING"))) ``` `structType.character` ```R structType("a STRING, b INT") ``` `dapply` ```R dapply(createDataFrame(list(list(1.0)), "a"), function(x) {x}, "a DOUBLE") ``` `gapply` ```R gapply(createDataFrame(list(list(1.0)), "a"), "a", function(key, x) { x }, "a DOUBLE") ``` ## How was this patch tested? Doc tests for `from_json` in Python and unit tests `test_sparkSQL.R` in R. Author: hyukjinkwon <gurwls223@gmail.com> Closes #18498 from HyukjinKwon/SPARK-21266.
-
Juliusz Sompolski authored
## What changes were proposed in this pull request? Updating numOutputRows metric was missing from one return path of LeftAnti SortMergeJoin. ## How was this patch tested? Non-zero output rows manually seen in metrics. Author: Juliusz Sompolski <julek@databricks.com> Closes #18494 from juliuszsompolski/SPARK-21272.
-
jinxing authored
## What changes were proposed in this pull request? When `RetryingBlockFetcher` retries fetching blocks. There could be two `DownloadCallback`s download the same content to the same target file. It could cause `ShuffleBlockFetcherIterator` reading a partial result. This pr proposes to create and delete the tmp files in `OneForOneBlockFetcher` Author: jinxing <jinxing6042@126.com> Author: Shixiong Zhu <zsxwing@gmail.com> Closes #18565 from jinxing64/SPARK-21342.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr made it more consistent to handle column name duplication. In the current master, error handling is different when hitting column name duplication: ``` // json scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil) scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data") scala> spark.read.format("json").schema(schema).load("/tmp/data").show org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#12, a#13.; at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153) scala> spark.read.format("json").load("/tmp/data").show org.apache.spark.sql.AnalysisException: Duplicate column(s) : "a" found, cannot save to JSON format; at org.apache.spark.sql.execution.datasources.json.JsonDataSource.checkConstraints(JsonDataSource.scala:81) at org.apache.spark.sql.execution.datasources.json.JsonDataSource.inferSchema(JsonDataSource.scala:63) at org.apache.spark.sql.execution.datasources.json.JsonFileFormat.inferSchema(JsonFileFormat.scala:57) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176) // csv scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil) scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data") scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#41, a#42.; at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152) // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) scala> spark.read.format("csv").option("header", true).load("/tmp/data").show +---+---+ | a0| a1| +---+---+ | 1| 1| +---+---+ // parquet scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil) scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data") scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#110, a#111.; at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) ``` When this patch applied, the results change to; ``` // json scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil) scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data") scala> spark.read.format("json").schema(schema).load("/tmp/data").show org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a"; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33) at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368) scala> spark.read.format("json").load("/tmp/data").show org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a"; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33) at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:156) // csv scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil) scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data") scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a"; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33) at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178) scala> spark.read.format("csv").option("header", true).load("/tmp/data").show +---+---+ | a0| a1| +---+---+ | 1| 1| +---+---+ // parquet scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil) scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data") scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a"; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33) at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368) ``` ## How was this patch tested? Added tests in `DataFrameReaderWriterSuite` and `SQLQueryTestSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17758 from maropu/SPARK-20460.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? Since this document became obsolete, we had better remove this for Apache Spark 2.3.0. The original document is removed via SPARK-12735 on January 2016, and currently it's just redirection page. The only reference in Apache Spark website will go directly to the destination in https://github.com/apache/spark-website/pull/54. ## How was this patch tested? N/A. This is a removal of documentation. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #18578 from dongjoon-hyun/SPARK-REMOVE-EC2.
-
Eric Vandenberg authored
## What changes were proposed in this pull request? There's a race condition in the current TaskSetManager where a failed task is added for retry (addPendingTask), and can asynchronously be assigned to an executor *prior* to the blacklist state (updateBlacklistForFailedTask), the result is the task might re-execute on the same executor. This is particularly problematic if the executor is shutting down since the retry task immediately becomes a lost task (ExecutorLostFailure). Another side effect is that the actual failure reason gets obscured by the retry task which never actually executed. There are sample logs showing the issue in the https://issues.apache.org/jira/browse/SPARK-21219 The fix is to change the ordering of the addPendingTask and updatingBlackListForFailedTask calls in TaskSetManager.handleFailedTask ## How was this patch tested? Implemented a unit test that verifies the task is black listed before it is added to the pending task. Ran the unit test without the fix and it fails. Ran the unit test with the fix and it passes. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Eric Vandenberg <ericvandenberg@fb.com> Closes #18427 from ericvandenbergfb/blacklistFix.
-
Wenchen Fan authored
## What changes were proposed in this pull request? Some code cleanup and adding comments to make the code more readable. Changed the way to generate result rows, to be more clear. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #18570 from cloud-fan/summary.
-
- Jul 09, 2017
-
-
jerryshao authored
## What changes were proposed in this pull request? Spark provides several ways to set configurations, either from configuration file, or from `spark-submit` command line options, or programmatically through `SparkConf` class. It may confuses beginners why some configurations set through `SparkConf` cannot take affect. So here add some docs to address this problems and let beginners know how to correctly set configurations. ## How was this patch tested? N/A Author: jerryshao <sshao@hortonworks.com> Closes #18552 from jerryshao/improve-doc.
-
Wenchen Fan authored
[SPARK-18016][SQL][FOLLOWUP] merge declareAddedFunctions, initNestedClasses and declareNestedClasses ## What changes were proposed in this pull request? These 3 methods have to be used together, so it makes more sense to merge them into one method and then the caller side only need to call one method. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #18579 from cloud-fan/minor.
-
- Jul 08, 2017
-
-
hyukjinkwon authored
## What changes were proposed in this pull request? This is a retry for #18320. This PR was reverted due to unexpected test failures with -10 error code. I was unable to reproduce in MacOS, CentOS and Ubuntu but only in Jenkins. So, the tests proceeded to verify this and revert the past try here - https://github.com/apache/spark/pull/18456 This new approach was tested in https://github.com/apache/spark/pull/18463. **Test results**: - With the part of suspicious change in the past try (https://github.com/apache/spark/pull/18463/commits/466325d3fd353668583f3bde38ae490d9db0b189) Tests ran 4 times and 2 times passed and 2 time failed. - Without the part of suspicious change in the past try (https://github.com/apache/spark/pull/18463/commits/466325d3fd353668583f3bde38ae490d9db0b189) Tests ran 5 times and they all passed. - With this new approach (https://github.com/apache/spark/pull/18463/commits/0a7589c09f53dfc2094497d8d3e59d6407569417) Tests ran 5 times and they all passed. It looks the cause is as below (see https://github.com/apache/spark/pull/18463/commits/466325d3fd353668583f3bde38ae490d9db0b189): ```diff + exitCode <- 1 ... + data <- parallel:::readChild(child) + if (is.raw(data)) { + if (unserialize(data) == exitCode) { ... + } + } ... - parallel:::mcexit(0L) + parallel:::mcexit(0L, send = exitCode) ``` Two possibilities I think - `parallel:::mcexit(.. , send = exitCode)` https://stat.ethz.ch/R-manual/R-devel/library/parallel/html/mcfork.html > It sends send to the master (unless NULL) and then shuts down the child process. However, it looks possible that the parent attemps to terminate the child right after getting our custom exit code. So, the child gets terminated between "send" and "shuts down", failing to exit properly. - A bug between `parallel:::mcexit(..., send = ...)` and `parallel:::readChild`. **Proposal**: To resolve this, I simply decided to avoid both possibilities with this new approach here (https://github.com/apache/spark/pull/18465/commits/9ff89a7859cb9f427fc774f33c3521c7d962b723). To support this idea, I explained with some quotation of the documentation as below: https://stat.ethz.ch/R-manual/R-devel/library/parallel/html/mcfork.html > `readChild` and `readChildren` return a raw vector with a "pid" attribute if data were available, an integer vector of length one with the process ID if a child terminated or `NULL` if the child no longer exists (no children at all for `readChildren`). `readChild` returns "an integer vector of length one with the process ID if a child terminated" so we can check if it is `integer` and the same selected "process ID". I believe this makes sure that the children are exited. In case that children happen to send any data manually to parent (which is why we introduced the suspicious part of the change (https://github.com/apache/spark/pull/18463/commits/466325d3fd353668583f3bde38ae490d9db0b189)), this should be raw bytes and will be discarded (and then will try to read the next and check if it is `integer` in the next loop). ## How was this patch tested? Manual tests and Jenkins tests. Author: hyukjinkwon <gurwls223@gmail.com> Closes #18465 from HyukjinKwon/SPARK-21093-retry-1.
-
Xiao Li authored
## What changes were proposed in this pull request? Since we do not set active sessions when parsing the plan, we are unable to correctly use SQLConf.get to find the correct active session. Since https://github.com/apache/spark/pull/18531 breaks the build, I plan to revert it at first. ## How was this patch tested? The existing test cases Author: Xiao Li <gatorsmile@gmail.com> Closes #18568 from gatorsmile/revert18531.
-
jinxing authored
## What changes were proposed in this pull request? In current code, reducer can break the old shuffle service when `spark.reducer.maxReqSizeShuffleToMem` is enabled. Let's refine document. Author: jinxing <jinxing6042@126.com> Closes #18566 from jinxing64/SPARK-21343.
-
Marcelo Vanzin authored
This makes sures that listeners get updated task information; otherwise it's possible to write incomplete task information into event logs, for example, making the information in a replayed UI inconsistent with the original application. Added a new unit test to try to detect the problem, but it's not guaranteed to fail since it's a race; but it fails pretty reliably for me without the scheduler changes. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #18393 from vanzin/SPARK-20342.try2.
-
Zhenhua Wang authored
## What changes were proposed in this pull request? We should be able to store zero size and row count after analyzing empty table. This pr also enhances the test cases for re-analyzing tables. ## How was this patch tested? Added a new test case and enhanced some test cases. Author: Zhenhua Wang <wangzhenhua@huawei.com> Closes #18292 from wzhfy/analyzeNewColumn.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? `SparkSessionBuilderSuite` should clean up stopped sessions. Otherwise, it leaves behind some stopped `SparkContext`s interfereing with other test suites using `ShardSQLContext`. Recently, master branch fails consequtively. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/ ## How was this patch tested? Pass the Jenkins with a updated suite. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #18567 from dongjoon-hyun/SPARK-SESSION.
-
caoxuewen authored
[SPARK-20609][MLLIB][TEST] manually cleared 'spark.local.dir' before/after a test in ALSCleanerSuite ## What changes were proposed in this pull request? This PR is similar to #17869. Once` 'spark.local.dir'` is set. Unless this is manually cleared before/after a test. it could return the same directory even if this property is configured. and add before/after for each likewise in ALSCleanerSuite. ## How was this patch tested? existing test. Author: caoxuewen <cao.xuewen@zte.com.cn> Closes #18537 from heary-cao/ALSCleanerSuite.
-
Joachim Hereth authored
## What changes were proposed in this pull request? Some link fixes for the documentation [Running Spark on Mesos](https://spark.apache.org/docs/latest/running-on-mesos.html): * Updated Link to Mesos Frameworks (Projects built on top of Mesos) * Update Link to Mesos binaries from Mesosphere (former link was redirected to dcos install page) ## How was this patch tested? Documentation was built and changed page manually/visually inspected. No code was changed, hence no dev tests. Since these changes are rather trivial I did not open a new JIRA ticket. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Joachim Hereth <joachim.hereth@numberfour.eu> Closes #18564 from daten-kieker/mesos_doc_fixes.
-
Michael Patterson authored
## What changes were proposed in this pull request? This adds documentation to many functions in pyspark.sql.functions.py: `upper`, `lower`, `reverse`, `unix_timestamp`, `from_unixtime`, `rand`, `randn`, `collect_list`, `collect_set`, `lit` Add units to the trigonometry functions. Renames columns in datetime examples to be more informative. Adds links between some functions. ## How was this patch tested? `./dev/lint-python` `python python/pyspark/sql/functions.py` `./python/run-tests.py --module pyspark-sql` Author: Michael Patterson <map222@gmail.com> Closes #17865 from map222/spark-20456.
-
wangmiao1981 authored
[SPARK-20307][SPARKR] SparkR: pass on setHandleInvalid to spark.mllib functions that use StringIndexer ## What changes were proposed in this pull request? For randomForest classifier, if test data contains unseen labels, it will throw an error. The StringIndexer already has the handleInvalid logic. The patch add a new method to set the underlying StringIndexer handleInvalid logic. This patch should also apply to other classifiers. This PR focuses on the main logic and randomForest classifier. I will do follow-up PR for other classifiers. ## How was this patch tested? Add a new unit test based on the error case in the JIRA. Author: wangmiao1981 <wm624@hotmail.com> Closes #18496 from wangmiao1981/handle.
-
Prashant Sharma authored
## What changes were proposed in this pull request? SPARK-20979 added a new structured streaming source: Rate source. This patch adds the corresponding documentation to programming guide. ## How was this patch tested? Tested by running jekyll locally. Author: Prashant Sharma <prashant@apache.org> Author: Prashant Sharma <prashsh1@in.ibm.com> Closes #18562 from ScrapCodes/spark-21069/rate-source-docs.
-
Marcelo Vanzin authored
This change exposes the internal code path in SparkConf that allows configs to be read with variable substitution applied, and uses that new method in SSLOptions so that SSL configs can reference other variables, and more importantly, environment variables, providing a secure way to provide passwords to Spark when using SSL. The approach is a little bit hacky, but is the smallest change possible. Otherwise, the concept of "namespaced configs" would have to be added to the config system, which would create a lot of noise for not much gain at this point. Tested with added unit tests, and on a real cluster with SSL enabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #18394 from vanzin/SPARK-20379.try2.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr modified code to use string types by default if `array` and `map` in functions have no argument. This behaviour is the same with Hive one; ``` hive> CREATE TEMPORARY TABLE t1 AS SELECT map(); hive> DESCRIBE t1; _c0 map<string,string> hive> CREATE TEMPORARY TABLE t2 AS SELECT array(); hive> DESCRIBE t2; _c0 array<string> ``` ## How was this patch tested? Added tests in `DataFrameFunctionsSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #18516 from maropu/SPARK-21281.
-
Andrew Ray authored
[SPARK-21100][SQL] Add summary method as alternative to describe that gives quartiles similar to Pandas ## What changes were proposed in this pull request? Adds method `summary` that allows user to specify which statistics and percentiles to calculate. By default it include the existing statistics from `describe` and quartiles (25th, 50th, and 75th percentiles) similar to Pandas. Also changes the implementation of `describe` to delegate to `summary`. ## How was this patch tested? additional unit test Author: Andrew Ray <ray.andrew@gmail.com> Closes #18307 from aray/SPARK-21100.
-
- Jul 07, 2017
-
-
Wang Gengliang authored
## What changes were proposed in this pull request? Revise rand comparison in BatchEvalPythonExecSuite In BatchEvalPythonExecSuite, there are two cases using the case "rand() > 3" Rand() generates a random value in [0, 1), it is wired to be compared with 3, use 0.3 instead ## How was this patch tested? unit test Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Wang Gengliang <ltnwgl@gmail.com> Closes #18560 from gengliangwang/revise_BatchEvalPythonExecSuite.
-
CodingCat authored
[SPARK-19358][CORE] LiveListenerBus shall log the event name when dropping them due to a fully filled queue ## What changes were proposed in this pull request? Some dropped event will make the whole application behaves unexpectedly, e.g. some UI problem...we shall log the dropped event name to facilitate the debugging ## How was this patch tested? Existing tests Author: CodingCat <zhunansjtu@gmail.com> Closes #16697 from CodingCat/SPARK-19358.
-
Wenchen Fan authored
## What changes were proposed in this pull request? un-aliased subquery is supported by Spark SQL for a long time. Its semantic was not well defined and had confusing behaviors, and it's not a standard SQL syntax, so we disallowed it in https://issues.apache.org/jira/browse/SPARK-20690 . However, this is a breaking change, and we do have existing queries using un-aliased subquery. We should add the support back and fix its semantic. This PR fixes the un-aliased subquery by assigning a default alias name. After this PR, there is no syntax change from branch 2.2 to master, but we invalid a weird use case: `SELECT v.i from (SELECT i FROM v)`. Now this query will throw analysis exception because users should not be able to use the qualifier inside a subquery. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #18559 from cloud-fan/sub-query.
-
Yan Facai (颜发才) authored
## What changes were proposed in this pull request? add the column name in the exception which is raised by unsupported data type. ## How was this patch tested? + [x] pass all tests. Author: Yan Facai (颜发才) <facai.yan@gmail.com> Closes #18523 from facaiy/ENH/vectorassembler_add_col.
-
Jacek Laskowski authored
## What changes were proposed in this pull request? Add `toString` with options for `ConsoleSink` so it shows nicely in query progress. **BEFORE** ``` "sink" : { "description" : "org.apache.spark.sql.execution.streaming.ConsoleSink4b340441" } ``` **AFTER** ``` "sink" : { "description" : "ConsoleSink[numRows=10, truncate=false]" } ``` /cc zsxwing tdas ## How was this patch tested? Local build Author: Jacek Laskowski <jacek@japila.pl> Closes #18539 from jaceklaskowski/SPARK-21313-ConsoleSink-toString.
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? Remove time metrics since it seems no way to measure it in non per-row tracking. ## How was this patch tested? Existing tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #18558 from viirya/SPARK-20703-followup.
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? This PR implements bulk-copy for `ColumnVector.Array.to<type>Array()` methods (e.g. `toIntArray()`) in `ColumnVector.Array` by using `System.arrayCopy()` or `Platform.copyMemory()`. Before this PR, when one of these method is called, the generic method in `ArrayData` is called. It is not fast since element-wise copy is performed. This PR can improve performance of a benchmark program by 1.9x and 3.2x. Without this PR ``` OpenJDK 64-Bit Server VM 1.8.0_131-8u131-b11-0ubuntu1.16.04.2-b11 on Linux 4.4.0-66-generic Intel(R) Xeon(R) CPU E5-2667 v3 3.20GHz Int Array Best/Avg Time(ms) Rate(M/s) Per Row(ns) ------------------------------------------------------------------------------------------------ ON_HEAP 586 / 628 14.3 69.9 OFF_HEAP 893 / 902 9.4 106.5 ``` With this PR ``` OpenJDK 64-Bit Server VM 1.8.0_131-8u131-b11-0ubuntu1.16.04.2-b11 on Linux 4.4.0-66-generic Intel(R) Xeon(R) CPU E5-2667 v3 3.20GHz Int Array Best/Avg Time(ms) Rate(M/s) Per Row(ns) ------------------------------------------------------------------------------------------------ ON_HEAP 306 / 331 27.4 36.4 OFF_HEAP 282 / 287 29.8 33.6 ``` Source program ``` (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { val len = 8 * 1024 * 1024 val column = ColumnVector.allocate(len * 2, new ArrayType(IntegerType, false), memMode) val data = column.arrayData var i = 0 while (i < len) { data.putInt(i, i) i += 1 } column.putArray(0, 0, len) val benchmark = new Benchmark("Int Array", len, minNumIters = 20) benchmark.addCase(s"$memMode") { iter => var i = 0 while (i < 50) { column.getArray(0).toIntArray i += 1 } } benchmark.run }} ``` ## How was this patch tested? Added test suite Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #18425 from kiszk/SPARK-21217.
-
Takuya UESHIN authored
[SPARK-21327][SQL][PYSPARK] ArrayConstructor should handle an array of typecode 'l' as long rather than int in Python 2. ## What changes were proposed in this pull request? Currently `ArrayConstructor` handles an array of typecode `'l'` as `int` when converting Python object in Python 2 into Java object, so if the value is larger than `Integer.MAX_VALUE` or smaller than `Integer.MIN_VALUE` then the overflow occurs. ```python import array data = [Row(longarray=array.array('l', [-9223372036854775808, 0, 9223372036854775807]))] df = spark.createDataFrame(data) df.show(truncate=False) ``` ``` +----------+ |longarray | +----------+ |[0, 0, -1]| +----------+ ``` This should be: ``` +----------------------------------------------+ |longarray | +----------------------------------------------+ |[-9223372036854775808, 0, 9223372036854775807]| +----------------------------------------------+ ``` ## How was this patch tested? Added a test and existing tests. Author: Takuya UESHIN <ueshin@databricks.com> Closes #18553 from ueshin/issues/SPARK-21327.
-
- Jul 06, 2017
-
-
hyukjinkwon authored
[SPARK-21326][SPARK-21066][ML] Use TextFileFormat in LibSVMFileFormat and allow multiple input paths for determining numFeatures ## What changes were proposed in this pull request? This is related with [SPARK-19918](https://issues.apache.org/jira/browse/SPARK-19918) and [SPARK-18362](https://issues.apache.org/jira/browse/SPARK-18362). This PR proposes to use `TextFileFormat` and allow multiple input paths (but with a warning) when determining the number of features in LibSVM data source via an extra scan. There are three points here: - The main advantage of this change should be to remove file-listing bottlenecks in driver side. - Another advantage is ones from using `FileScanRDD`. For example, I guess we can use `spark.sql.files.ignoreCorruptFiles` option when determining the number of features. - We can unify the schema inference code path in text based data sources. This is also a preparation for [SPARK-21289](https://issues.apache.org/jira/browse/SPARK-21289). ## How was this patch tested? Unit tests in `LibSVMRelationSuite`. Closes #18288 Author: hyukjinkwon <gurwls223@gmail.com> Closes #18556 from HyukjinKwon/libsvm-schema.
-
Jacek Laskowski authored
## What changes were proposed in this pull request? Making EventTimeWatermarkExec explicitly UnaryExecNode /cc tdas zsxwing ## How was this patch tested? Local build. Author: Jacek Laskowski <jacek@japila.pl> Closes #18509 from jaceklaskowski/EventTimeWatermarkExec-UnaryExecNode.
-
Wenchen Fan authored
## What changes were proposed in this pull request? SparkContext is shared by all sessions, we should not update its conf for only one session. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #18536 from cloud-fan/config.
-
Tathagata Das authored
## What changes were proposed in this pull request? Few changes to the Structured Streaming documentation - Clarify that the entire stream input table is not materialized - Add information for Ganglia - Add Kafka Sink to the main docs - Removed a couple of leftover experimental tags - Added more associated reading material and talk videos. In addition, https://github.com/apache/spark/pull/16856 broke the link to the RDD programming guide in several places while renaming the page. This PR fixes those sameeragarwal cloud-fan. - Added a redirection to avoid breaking internal and possible external links. - Removed unnecessary redirection pages that were there since the separate scala, java, and python programming guides were merged together in 2013 or 2014. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #18485 from tdas/SPARK-21267.
-
Wang Gengliang authored
## What changes were proposed in this pull request? Rename org.apache.spark.sql.catalyst.plans.logical.statsEstimation.Range to ValueInterval. The current naming is identical to logical operator "range". Refactoring it to ValueInterval is more accurate. ## How was this patch tested? unit test Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Wang Gengliang <ltnwgl@gmail.com> Closes #18549 from gengliangwang/ValueInterval.
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? Currently we can't produce a `Dataset` containing `Set` in SparkSQL. This PR tries to support serialization/deserialization of `Set`. Because there's no corresponding internal data type in SparkSQL for a `Set`, the most proper choice for serializing a set should be an array. ## How was this patch tested? Added unit tests. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #18416 from viirya/SPARK-21204.
-
Bogdan Raducanu authored
## What changes were proposed in this pull request? When data type is struct, InSet now uses TypeUtils.getInterpretedOrdering (similar to EqualTo) to build a TreeSet. In other cases it will use a HashSet as before (which should be faster). Similarly, In.eval uses Ordering.equiv instead of equals. ## How was this patch tested? New test in SQLQuerySuite. Author: Bogdan Raducanu <bogdan@databricks.com> Closes #18455 from bogdanrdc/SPARK-21228.
-
caoxuewen authored
## What changes were proposed in this pull request? This PR Improvement in two: 1.With spark.shuffle.spill.diskWriteBufferSize configure diskWriteBufferSize of ShuffleExternalSorter. when change the size of the diskWriteBufferSize to test `forceSorterToSpill` The average performance of running 10 times is as follows:(their unit is MS). ``` diskWriteBufferSize: 1M 512K 256K 128K 64K 32K 16K 8K 4K --------------------------------------------------------------------------------------- RecordSize = 2.5M 742 722 694 686 667 668 671 669 683 RecordSize = 1M 294 293 292 287 283 285 281 279 285 ``` 2.Remove outputBufferSizeInBytes and inputBufferSizeInBytes to initialize in mergeSpillsWithFileStream function. ## How was this patch tested? The unit test. Author: caoxuewen <cao.xuewen@zte.com.cn> Closes #18174 from heary-cao/buffersize.
-