- Oct 21, 2016
-
-
cody koeninger authored
## What changes were proposed in this pull request? Minor doc change to mention kafka configuration for larger spark batches. ## How was this patch tested? Doc change only, confirmed via jekyll. The configuration issue was discussed / confirmed with users on the mailing list. Author: cody koeninger <cody@koeninger.org> Closes #15570 from koeninger/kafka-doc-heartbeat.
-
cody koeninger authored
## What changes were proposed in this pull request? startingOffsets takes specific per-topicpartition offsets as a json argument, usable with any consumer strategy assign with specific topicpartitions as a consumer strategy ## How was this patch tested? Unit tests Author: cody koeninger <cody@koeninger.org> Closes #15504 from koeninger/SPARK-17812.
-
Wenchen Fan authored
## What changes were proposed in this pull request? In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again. This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #15581 from cloud-fan/stream.
-
w00228970 authored
## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-17929 Now `CoarseGrainedSchedulerBackend` reset will get the lock, ``` protected def reset(): Unit = synchronized { numPendingExecutors = 0 executorsPendingToRemove.clear() // Remove all the lingering executors that should be removed but not yet. The reason might be // because (1) disconnected event is not yet received; (2) executors die silently. executorDataMap.toMap.foreach { case (eid, _) => driverEndpoint.askWithRetry[Boolean]( RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) } } ``` but on removeExecutor also need the lock "CoarseGrainedSchedulerBackend.this.synchronized", this will cause deadlock. ``` private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { logDebug(s"Asked to remove executor $executorId with reason $reason") executorDataMap.get(executorId) match { case Some(executorInfo) => // This must be synchronized because variables mutated // in this block are read when requesting executors val killed = CoarseGrainedSchedulerBackend.this.synchronized { addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingLossReason -= executorId executorsPendingToRemove.remove(executorId).getOrElse(false) } ... ## How was this patch tested? manual test. Author: w00228970 <wangfei1@huawei.com> Closes #15481 from scwf/spark-17929.
-
Tathagata Das authored
## What changes were proposed in this pull request? StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`. ## How was this patch tested? New unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15476 from tdas/SPARK-17926.
-
Hossein authored
## What changes were proposed in this pull request? NA date values are serialized as "NA" and NA time values are serialized as NaN from R. In the backend we did not have proper logic to deal with them. As a result we got an IllegalArgumentException for Date and wrong value for time. This PR adds support for deserializing NA as Date and Time. ## How was this patch tested? * [x] TODO Author: Hossein <hossein@databricks.com> Closes #15421 from falaki/SPARK-17811.
-
Felix Cheung authored
## What changes were proposed in this pull request? Add crossJoin and do not default to cross join if joinExpr is left out ## How was this patch tested? unit test Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #15559 from felixcheung/rcrossjoin.
-
Felix Cheung authored
## What changes were proposed in this pull request? testthat library we are using for testing R is redirecting warning (and disabling `options("warn" = 2)`), we need to have a way to detect any new warning and fail ## How was this patch tested? manual testing, Jenkins Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #15576 from felixcheung/rtestwarning.
-
Josh Rosen authored
We should upgrade to the latest release of MiMa (0.1.11) in order to include a fix for a bug which led to flakiness in the MiMa checks (https://github.com/typesafehub/migration-manager/issues/115). Author: Josh Rosen <joshrosen@databricks.com> Closes #15571 from JoshRosen/SPARK-18034.
-
Alex Bozarth authored
## What changes were proposed in this pull request? Updated the Executors added/removed bubble in the time line so it's clearer where it starts. Now the bubble is left justified on the start time (still also denoted by the line) rather than center justified. ## How was this patch tested? Manually tested UI <img width="596" alt="screen shot 2016-10-17 at 6 04 36 pm" src="https://cloud.githubusercontent.com/assets/13952758/19496563/e6c9186e-953c-11e6-85e4-63309a553f65.png"> <img width="492" alt="screen shot 2016-10-17 at 5 54 09 pm" src="https://cloud.githubusercontent.com/assets/13952758/19496568/e9f06132-953c-11e6-8901-54405ebc7f5b.png"> Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #15536 from ajbozarth/spark13275.
-
Zheng RuiFeng authored
## What changes were proposed in this pull request? `Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays. Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates. cc srowen ## How was this patch tested? existing tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #15564 from zhengruifeng/avoid_0_length_array.
-
Jagadeesan authored
## What changes were proposed in this pull request? 1) Upgrade the Py4J version on the Java side 2) Update the py4j src zip file we bundle with Spark ## How was this patch tested? Existing doctests & unit tests pass Author: Jagadeesan <as2@us.ibm.com> Closes #15514 from jagadeesanas2/SPARK-17960.
-
- Oct 20, 2016
-
-
Wenchen Fan authored
## What changes were proposed in this pull request? In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it. ## How was this patch tested? the new `PruneFileSourcePartitionsSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #15569 from cloud-fan/partition-bug.
-
Felix Cheung authored
## What changes were proposed in this pull request? Fix for a bunch of test warnings that were added recently. We need to investigate why warnings are not turning into errors. ``` Warnings ----------------------------------------------------------------------- 1. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Sepal_Length instead of Sepal.Length as column name 2. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Sepal_Width instead of Sepal.Width as column name 3. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Petal_Length instead of Petal.Length as column name 4. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Petal_Width instead of Petal.Width as column name Consider adding importFrom("utils", "object.size") to your NAMESPACE file. ``` ## How was this patch tested? unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #15560 from felixcheung/rwarnings.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? My hunch is `mkdirs` fails. Just add more checks to collect more info. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #15577 from zsxwing/SPARK-18030-debug.
-
Mark Grover authored
## What changes were proposed in this pull request? Update docs to not suggest to package Spark before running tests. ## How was this patch tested? Not creating a JIRA since this pretty small. We haven't had the need to run mvn package before mvn test since 1.6 at least, or so I am told. So, updating the docs to not be misguiding. Author: Mark Grover <mark@apache.org> Closes #15572 from markgrover/doc_update.
-
Reynold Xin authored
## What changes were proposed in this pull request? Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing. On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions. This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically. There are also some other minor cleanups: - Removed the UUID passed through generic Configuration string - Some minor rewrites for better clarity - Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning ## How was this patch tested? This should be covered by existing data source tests. Author: Reynold Xin <rxin@databricks.com> Closes #15562 from rxin/SPARK-18021.
-
jerryshao authored
## What changes were proposed in this pull request? The newly implemented Structured Streaming `KafkaSource` did calculate the preferred locations for each topic partition, but didn't offer this information through RDD's `getPreferredLocations` method. So here propose to add this method in `KafkaSourceRDD`. ## How was this patch tested? Manual verification. Author: jerryshao <sshao@hortonworks.com> Closes #15545 from jerryshao/SPARK-17999.
-
Koert Kuipers authored
## What changes were proposed in this pull request? Add mapValues to KeyValueGroupedDataset ## How was this patch tested? New test in DatasetSuite for groupBy function, mapValues, flatMap Author: Koert Kuipers <koert@tresata.com> Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
-
Tejas Patil authored
## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-17698 `ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below: [0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91 eg. ``` val df = (1 until 10).toDF("id").coalesce(1) hc.sql("DROP TABLE IF EXISTS table1").collect df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1") hc.sql("DROP TABLE IF EXISTS table2").collect df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2") sqlContext.sql(""" SELECT a.id, b.id FROM table1 a FULL OUTER JOIN table2 b ON a.id = b.id AND a.id='1' AND b.id='1' """).explain(true) ``` BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job. ``` SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter :- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200) : +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int> +- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200) +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int> ``` AFTER : ``` SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0)) :- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int> +- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int> ``` ## How was this patch tested? - Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses` - Ran all the tests in `BucketedReadSuite` Author: Tejas Patil <tejasp@fb.com> Closes #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause.
-
Dilip Biswal authored
[SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration ## What changes were proposed in this pull request? SHOW COLUMNS command validates the user supplied database name with database name from qualified table name name to make sure both of them are consistent. This comparison should respect case sensitivity. ## How was this patch tested? Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #15423 from dilipbiswal/dkb_show_column_fix.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2. **Reported Error Scenario** ```scala scala> sql("CREATE TABLE t(a string)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t") org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*; ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #15376 from dongjoon-hyun/SPARK-17796.
-
Mike Ihbe authored
## What changes were proposed in this pull request? Add a SPARK_NO_DAEMONIZE environment variable flag to spark-daemon.sh that causes the process it would run to be run in the foreground. It looks like there has been some prior work in https://github.com/apache/spark/pull/3881, but there was some talk about these being refactored. I'm not sure if that happened or not, but that PR is almost 2 years old at this point so it was worth revisiting. ## How was this patch tested? ./dev/run-tests still seems to work. It doesn't look like these scripts have tests, but if I missed them just let me know. Author: Mike Ihbe <mikejihbe@gmail.com> Closes #15338 from mikejihbe/SPARK-11653.
-
Eric Liang authored
## What changes were proposed in this pull request? This should apply to non-converted metastore relations. WIP to see if this causes any test failures. ## How was this patch tested? Existing tests. Author: Eric Liang <ekl@databricks.com> Closes #15475 from ericl/try-enabling-pruning.
-
WeichenXu authored
## What changes were proposed in this pull request? - Fix bug of RDD `zipWithIndex` generating wrong result when one partition contains more than 2147483647 records. - Fix bug of RDD `zipWithUniqueId` generating wrong result when one partition contains more than 2147483647 records. ## How was this patch tested? test added. Author: WeichenXu <WeichenXu123@outlook.com> Closes #15550 from WeichenXu123/fix_rdd_zipWithIndex_overflow.
-
Reynold Xin authored
## What changes were proposed in this pull request? This patch refactors WriterContainer to simplify the logic and make control flow more obvious.The previous code setup made it pretty difficult to track the actual dependencies on variables and setups because the driver side and the executor side were using the same set of variables. ## How was this patch tested? N/A - this should be covered by existing tests. Author: Reynold Xin <rxin@databricks.com> Closes #15551 from rxin/writercontainer-refactor.
-
- Oct 19, 2016
-
-
hyukjinkwon authored
[SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException ## What changes were proposed in this pull request? This PR proposes to check the second argument, `ascendingOrder` rather than throwing `ClassCastException` exception message. ```sql select sort_array(array('b', 'd'), '1'); ``` **Before** ``` 16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')] java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85) at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185) at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297) ``` **After** ``` Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7; ``` ## How was this patch tested? Unit test in `DataFrameFunctionsSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15532 from HyukjinKwon/SPARK-17989.
-
Alex Bozarth authored
[SPARK-10541][WEB UI] Allow ApplicationHistoryProviders to provide their own text when there aren't any complete apps ## What changes were proposed in this pull request? I've added a method to `ApplicationHistoryProvider` that returns the html paragraph to display when there are no applications. This allows providers other than `FsHistoryProvider` to determine what is printed. The current hard coded text is now moved into `FsHistoryProvider` since it assumed that's what was being used before. I chose to make the function return html rather than text because the current text block had inline html in it and it allows a new implementation of `ApplicationHistoryProvider` more versatility. I did not see any security issues with this since injecting html here requires implementing `ApplicationHistoryProvider` and can't be done outside of code. ## How was this patch tested? Manual testing and dev/run-tests No visible changes to the UI Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #15490 from ajbozarth/spark10541.
-
Takuya UESHIN authored
## What changes were proposed in this pull request? `SerializationUtils.clone()` of commons-lang3 (<3.5) has a bug that breaks thread safety, which gets stack sometimes caused by race condition of initializing hash map. See https://issues.apache.org/jira/browse/LANG-1251. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #15548 from ueshin/issues/SPARK-17985.
-
- Oct 18, 2016
-
-
Tommy YU authored
## What changes were proposed in this pull request? In http://spark.apache.org/docs/latest/sql-programming-guide.html, Section "Untyped Dataset Operations (aka DataFrame Operations)" Link to R DataFrame doesn't work that return The requested URL /docs/latest/api/R/DataFrame.html was not found on this server. Correct link is SparkDataFrame.html for spark 2.0 ## How was this patch tested? Manual checked. Author: Tommy YU <tummyyu@163.com> Closes #15543 from Wenpei/spark-18001.
-
Wenchen Fan authored
[SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table) ## What changes were proposed in this pull request? Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`. However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO. This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc. ## How was this patch tested? The added back tests and some new tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #15434 from cloud-fan/revert.
-
Yu Peng authored
## What changes were proposed in this pull request? Fix hadoop2.2 compilation error. ## How was this patch tested? Existing tests. cc tdas zsxwing Author: Yu Peng <loneknightpy@gmail.com> Closes #15537 from loneknightpy/fix-17711.
-
Eric Liang authored
## What changes were proposed in this pull request? There was a bug introduced in https://github.com/apache/spark/pull/14690 which broke refreshByPath with converted hive tables (though, it turns out it was very difficult to refresh converted hive tables anyways, since you had to specify the exact path of one of the partitions). This changes refreshByPath to invalidate by prefix instead of exact match, and fixes the issue. cc sameeragarwal for refreshByPath changes mallman ## How was this patch tested? Extended unit test. Author: Eric Liang <ekl@databricks.com> Closes #15521 from ericl/fix-caching.
-
Tathagata Das authored
## What changes were proposed in this pull request? As per rxin request, here are further API changes - Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event` - Changed the fields in `StreamingQueryListener.on***` from `query*` to `event` ## How was this patch tested? Existing unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15530 from tdas/SPARK-17731-1.
-
Liang-Chi Hsieh authored
[SPARK-17817] [PYSPARK] [FOLLOWUP] PySpark RDD Repartitioning Results in Highly Skewed Partition Sizes ## What changes were proposed in this pull request? This change is a followup for #15389 which calls `_to_java_object_rdd()` to solve this issue. Due to the concern of the possible expensive cost of the call, we can choose to decrease the batch size to solve this issue too. Simple benchmark: import time num_partitions = 20000 a = sc.parallelize(range(int(1e6)), 2) start = time.time() l = a.repartition(num_partitions).glom().map(len).collect() end = time.time() print(end - start) Before: 419.447577953 _to_java_object_rdd(): 421.916361094 decreasing the batch size: 423.712255955 ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #15445 from viirya/repartition-batch-size.
-
cody koeninger authored
## What changes were proposed in this pull request? Actually drain commit queue rather than just iterating it. iterator() on a concurrent linked queue won't remove items from the queue, poll() will. ## How was this patch tested? Unit tests Author: cody koeninger <cody@koeninger.org> Closes #15407 from koeninger/SPARK-17841.
-
Reynold Xin authored
This reverts commit bfe7885a. The commit caused build failures on Hadoop 2.2 profile: ``` [error] /scratch/rxin/spark/core/src/main/scala/org/apache/spark/util/Utils.scala:1489: value read is not a member of object org.apache.commons.io.IOUtils [error] var numBytes = IOUtils.read(gzInputStream, buf) [error] ^ [error] /scratch/rxin/spark/core/src/main/scala/org/apache/spark/util/Utils.scala:1492: value read is not a member of object org.apache.commons.io.IOUtils [error] numBytes = IOUtils.read(gzInputStream, buf) [error] ^ ```
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR proposes to make `DataFrameReader.jdbc` call `DataFrameReader.format("jdbc").load` consistently with other APIs in `DataFrameReader`/`DataFrameWriter` and avoid calling `sparkSession.baseRelationToDataFrame(..)` here and there. The changes were mostly copied from `DataFrameWriter.jdbc()` which was recently updated. ```diff - val params = extraOptions.toMap ++ connectionProperties.asScala.toMap - val options = new JDBCOptions(url, table, params) - val relation = JDBCRelation(parts, options)(sparkSession) - sparkSession.baseRelationToDataFrame(relation) + this.extraOptions = this.extraOptions ++ connectionProperties.asScala + // explicit url and dbtable should override all + this.extraOptions += ("url" -> url, "dbtable" -> table) + format("jdbc").load() ``` ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15499 from HyukjinKwon/SPARK-17955.
-
Guoqiang Li authored
[SPARK-17930][CORE] The SerializerInstance instance used when deserializing a TaskResult is not reused ## What changes were proposed in this pull request? The following code is called when the DirectTaskResult instance is deserialized ```scala def value(): T = { if (valueObjectDeserialized) { valueObject } else { // Each deserialization creates a new instance of SerializerInstance, which is very time-consuming val resultSer = SparkEnv.get.serializer.newInstance() valueObject = resultSer.deserialize(valueBytes) valueObjectDeserialized = true valueObject } } ``` In the case of stage has a lot of tasks, reuse SerializerInstance instance can improve the scheduling performance of three times The test data is TPC-DS 2T (Parquet) and SQL statement as follows (query 2): ```sql select i_item_id, avg(ss_quantity) agg1, avg(ss_list_price) agg2, avg(ss_coupon_amt) agg3, avg(ss_sales_price) agg4 from store_sales, customer_demographics, date_dim, item, promotion where ss_sold_date_sk = d_date_sk and ss_item_sk = i_item_sk and ss_cdemo_sk = cd_demo_sk and ss_promo_sk = p_promo_sk and cd_gender = 'M' and cd_marital_status = 'M' and cd_education_status = '4 yr Degree' and (p_channel_email = 'N' or p_channel_event = 'N') and d_year = 2001 group by i_item_id order by i_item_id limit 100; ``` `spark-defaults.conf` file: ``` spark.master yarn-client spark.executor.instances 20 spark.driver.memory 16g spark.executor.memory 30g spark.executor.cores 5 spark.default.parallelism 100 spark.sql.shuffle.partitions 100000 spark.serializer org.apache.spark.serializer.KryoSerializer spark.driver.maxResultSize 0 spark.rpc.netty.dispatcher.numThreads 8 spark.executor.extraJavaOptions -XX:+UseG1GC -XX:+UseStringDeduplication -XX:G1HeapRegionSize=16M -XX:MetaspaceSize=256M spark.cleaner.referenceTracking.blocking true spark.cleaner.referenceTracking.blocking.shuffle true ``` Performance test results are as follows [SPARK-17930](https://github.com/witgo/spark/tree/SPARK-17930)| [ed146334](https://github.com/witgo/spark/commit/ed1463341455830b8867b721a1b34f291139baf3]) ------------ | ------------- 54.5 s|231.7 s ## How was this patch tested? Existing tests. Author: Guoqiang Li <witgo@qq.com> Closes #15512 from witgo/SPARK-17930.
-
Weiqing Yang authored
## What changes were proposed in this pull request? Add more built-in sources in sql-programming-guide.md. ## How was this patch tested? Manually. Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #15522 from weiqingy/dsDoc.
-