aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19727][SQL] Fix for round function that modifies original columnWojtek Szymanski2017-03-087-25/+54
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix for SQL round function that modifies original column when underlying data frame is created from a local product. import org.apache.spark.sql.functions._ case class NumericRow(value: BigDecimal) val df = spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789")))) df.show() +--------------------+ | value| +--------------------+ |1.234567890000000000| +--------------------+ df.withColumn("value_rounded", round('value)).show() // before +--------------------+-------------+ | value|value_rounded| +--------------------+-------------+ |1.000000000000000000| 1| +--------------------+-------------+ // after +--------------------+-------------+ | value|value_rounded| +--------------------+-------------+ |1.234567890000000000| 1| +--------------------+-------------+ ## How was this patch tested? New unit test added to existing suite `org.apache.spark.sql.MathFunctionsSuite` Author: Wojtek Szymanski <wk.szymanski@gmail.com> Closes #17075 from wojtek-szymanski/SPARK-19727.
* [SPARK-19864][SQL][TEST] provide a makeQualifiedPath functions to optimize ↵windpiger2017-03-087-49/+32
| | | | | | | | | | | | | | | some code ## What changes were proposed in this pull request? Currently there are lots of places to make the path qualified, it is better to provide a function to do this, then the code will be more simple. ## How was this patch tested? N/A Author: windpiger <songjun@outlook.com> Closes #17204 from windpiger/addQualifiledPathUtil.
* [SPARK-19843][SQL][FOLLOWUP] Classdoc for `IntWrapper` and `LongWrapper`Tejas Patil2017-03-081-4/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is as per suggestion by rxin at : https://github.com/apache/spark/pull/17184#discussion_r104841735 ## How was this patch tested? NA as this is a documentation change Author: Tejas Patil <tejasp@fb.com> Closes #17205 from tejasapatil/SPARK-19843_followup.
* [SPARK-19601][SQL] Fix CollapseRepartition rule to preserve shuffle-enabled ↵Xiao Li2017-03-087-49/+178
| | | | | | | | | | | | | | | | | | | | | | | | | | | Repartition ### What changes were proposed in this pull request? Observed by felixcheung in https://github.com/apache/spark/pull/16739, when users use the shuffle-enabled `repartition` API, they expect the partition they got should be the exact number they provided, even if they call shuffle-disabled `coalesce` later. Currently, `CollapseRepartition` rule does not consider whether shuffle is enabled or not. Thus, we got the following unexpected result. ```Scala val df = spark.range(0, 10000, 1, 5) val df2 = df.repartition(10) assert(df2.coalesce(13).rdd.getNumPartitions == 5) assert(df2.coalesce(7).rdd.getNumPartitions == 5) assert(df2.coalesce(3).rdd.getNumPartitions == 3) ``` This PR is to fix the issue. We preserve shuffle-enabled Repartition. ### How was this patch tested? Added a test case Author: Xiao Li <gatorsmile@gmail.com> Closes #16933 from gatorsmile/CollapseRepartition.
* [SPARK-19865][SQL] remove the view identifier in SubqueryAliasjiangxingbo2017-03-0816-46/+42
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before. ## How was this patch tested? Update the related test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #17210 from jiangxb1987/SubqueryAlias.
* [SPARK-17080][SQL] join reorderwangzhenhua2017-03-088-4/+521
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Reorder the joins using a dynamic programming algorithm (Selinger paper): First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them. When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows: ``` level 1: p({A}), p({B}), p({C}), p({D}) level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D}) level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D}) level 4: p({A, B, C, D}) ``` where p({A, B, C, D}) is the final output plan. For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs. ## How was this patch tested? add test cases Author: wangzhenhua <wangzhenhua@huawei.com> Author: Zhenhua Wang <wzh_zju@163.com> Closes #17138 from wzhfy/joinReorder.
* [SPARK-16440][MLLIB] Ensure broadcasted variables are destroyed even in case ↵Anthony Truchet2017-03-081-3/+15
| | | | | | | | | | | | | | | of exception ## What changes were proposed in this pull request? Ensure broadcasted variable are destroyed even in case of exception ## How was this patch tested? Word2VecSuite was run locally Author: Anthony Truchet <a.truchet@criteo.com> Closes #14299 from AnthonyTruchet/SPARK-16440.
* [SPARK-19693][SQL] Make the SET mapreduce.job.reduces automatically ↵Yuming Wang2017-03-083-0/+33
| | | | | | | | | | | | | | | converted to spark.sql.shuffle.partitions ## What changes were proposed in this pull request? Make the `SET mapreduce.job.reduces` automatically converted to `spark.sql.shuffle.partitions`, it's similar to `SET mapred.reduce.tasks`. ## How was this patch tested? unit tests Author: Yuming Wang <wgyumg@gmail.com> Closes #17020 from wangyum/SPARK-19693.
* [SPARK-19806][ML][PYSPARK] PySpark GeneralizedLinearRegression supports ↵Yanbo Liang2017-03-083-12/+77
| | | | | | | | | | | | | | tweedie distribution. ## What changes were proposed in this pull request? PySpark ```GeneralizedLinearRegression``` supports tweedie distribution. ## How was this patch tested? Add unit tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17146 from yanboliang/spark-19806.
* [ML][MINOR] Separate estimator and model params for read/write test.Yanbo Liang2017-03-0823-54/+59
| | | | | | | | | | | | ## What changes were proposed in this pull request? Since we allow ```Estimator``` and ```Model``` not always share same params (see ```ALSParams``` and ```ALSModelParams```), we should pass in test params for estimator and model separately in function ```testEstimatorAndModelReadWrite```. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17151 from yanboliang/test-rw.
* [SPARK-18055][SQL] Use correct mirror in ExpresionEncoderMichael Armbrust2017-03-082-2/+13
| | | | | | | | | | Previously, we were using the mirror of passed in `TypeTag` when reflecting to build an encoder. This fails when the outer class is built in (i.e. `Seq`'s default mirror is based on root classloader) but inner classes (i.e. `A` in `Seq[A]`) are defined in the REPL or a library. This patch changes us to always reflect based on a mirror created using the context classloader. Author: Michael Armbrust <michael@databricks.com> Closes #17201 from marmbrus/replSeqEncoder.
* [SPARK-17629][ML] methods to return synonyms directlyAsher Krim2017-03-072-12/+45
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? provide methods to return synonyms directly, without wrapping them in a dataframe In performance sensitive applications (such as user facing apis) the roundtrip to and from dataframes is costly and unnecessary The methods are named ``findSynonymsArray`` to make the return type clear, which also implies a local datastructure ## How was this patch tested? updated word2vec tests Author: Asher Krim <akrim@hubspot.com> Closes #16811 from Krimit/w2vFindSynonymsLocal.
* [SPARK-19859][SS] The new watermark should override the old oneShixiong Zhu2017-03-072-0/+21
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The new watermark should override the old one. Otherwise, we just pick up the first column which has a watermark, it may be unexpected. ## How was this patch tested? The new test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #17199 from zsxwing/SPARK-19859.
* [SPARK-19841][SS] watermarkPredicate should filter based on keysShixiong Zhu2017-03-072-8/+39
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `StreamingDeduplicateExec.watermarkPredicate` should filter based on keys. Otherwise, it may generate a wrong answer if the watermark column in `keyExpression` has a different position in the row. `StateStoreSaveExec` has the same codes but its parent can makes sure the watermark column positions in `keyExpression` and `row` are the same. ## How was this patch tested? The added test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #17183 from zsxwing/SPARK-19841.
* [SPARK-18389][SQL] Disallow cyclic view referencejiangxingbo2017-03-072-6/+90
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Disallow cyclic view references, a cyclic view reference may be created by the following queries: ``` CREATE VIEW testView AS SELECT id FROM tbl CREATE VIEW testView2 AS SELECT id FROM testView ALTER VIEW testView AS SELECT * FROM testView2 ``` In the above example, a reference cycle (testView -> testView2 -> testView) exsits. We disallow cyclic view references by checking that in ALTER VIEW command, when the `analyzedPlan` contains the same `View` node with the altered view, we should prevent the behavior and throw an AnalysisException. ## How was this patch tested? Test by `SQLViewSuite.test("correctly handle a cyclic view reference")`. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #17152 from jiangxb1987/cyclic-view.
* [SPARK-19843][SQL] UTF8String => (int / long) conversion expensive for ↵Tejas Patil2017-03-073-82/+247
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | invalid inputs ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-19843 Created wrapper classes (`IntWrapper`, `LongWrapper`) to wrap the result of parsing (which are primitive types). In case of problem in parsing, the method would return a boolean. ## How was this patch tested? - Added new unit tests - Ran a prod job which had conversion from string -> int and verified the outputs ## Performance Tiny regression when all strings are valid integers ``` conversion to int: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------- trunk 502 / 522 33.4 29.9 1.0X SPARK-19843 493 / 503 34.0 29.4 1.0X ``` Huge gain when all strings are invalid integers ``` conversion to int: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------- trunk 33913 / 34219 0.5 2021.4 1.0X SPARK-19843 154 / 162 108.8 9.2 220.0X ``` Author: Tejas Patil <tejasp@fb.com> Closes #17184 from tejasapatil/SPARK-19843_is_numeric_maybe.
* Revert "[SPARK-19561] [PYTHON] cast TimestampType.toInternal output to long"Wenchen Fan2017-03-072-7/+1
| | | | This reverts commit 711addd46e98e42deca97c5b9c0e55fddebaa458.
* [SPARK-19857][YARN] Correctly calculate next credential update time.Marcelo Vanzin2017-03-071-3/+4
| | | | | | | | | | | | Add parentheses so that both lines form a single statement; also add a log message so that the issue becomes more explicit if it shows up again. Tested manually with integration test that exercises the feature. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #17198 from vanzin/SPARK-19857.
* [SPARK-19702][MESOS] Increase default refuse_seconds timeout in the Mesos ↵Michael Gummelt2017-03-077-105/+187
| | | | | | | | | | | | | | | | | | Spark Dispatcher ## What changes were proposed in this pull request? Increase default refuse_seconds timeout, and make it configurable. See JIRA for details on how this reduces the risk of starvation. ## How was this patch tested? Unit tests, Manual testing, and Mesos/Spark integration test suite cc susanxhuynh skonto jmlvanre Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #17031 from mgummelt/SPARK-19702-suppress-revive.
* [SPARK-19561] [PYTHON] cast TimestampType.toInternal output to longJason White2017-03-072-1/+7
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Cast the output of `TimestampType.toInternal` to long to allow for proper Timestamp creation in DataFrames near the epoch. ## How was this patch tested? Added a new test that fails without the change. dongjoon-hyun davies Mind taking a look? The contribution is my original work and I license the work to the project under the project’s open source license. Author: Jason White <jason.white@shopify.com> Closes #16896 from JasonMWhite/SPARK-19561.
* [SPARK-19803][TEST] flaky BlockManagerReplicationSuite test failureuncleGen2017-03-071-5/+5
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 200ms may be too short. Give more time for replication to happen and new block be reported to master ## How was this patch tested? test manully Author: uncleGen <hustyugm@gmail.com> Author: dylon <hustyugm@gmail.com> Closes #17144 from uncleGen/SPARK-19803.
* [SPARK-19516][DOC] update public doc to use SparkSession instead of SparkContextWenchen Fan2017-03-072-103/+76
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? After Spark 2.0, `SparkSession` becomes the new entry point for Spark applications. We should update the public documents to reflect this. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16856 from cloud-fan/doc.
* [SPARK-17498][ML] StringIndexer enhancement for handling unseen labelsVinceShieh2017-03-074-30/+95
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is an enhancement to ML StringIndexer. Before this PR, String Indexer only supports "skip"/"error" options to deal with unseen records. But those unseen records might still be useful and user would like to keep the unseen labels in certain use cases, This PR enables StringIndexer to support keeping unseen labels as indices [numLabels]. '''Before StringIndexer().setHandleInvalid("skip") StringIndexer().setHandleInvalid("error") '''After support the third option "keep" StringIndexer().setHandleInvalid("keep") ## How was this patch tested? Test added in StringIndexerSuite Signed-off-by: VinceShieh <vincent.xieintel.com> (Please fill in changes proposed in this fix) Author: VinceShieh <vincent.xie@intel.com> Closes #16883 from VinceShieh/spark-17498.
* [SPARK-19765][SPARK-18549][SQL] UNCACHE TABLE should un-cache all cached ↵Wenchen Fan2017-03-079-98/+119
| | | | | | | | | | | | | | | | | | | | | plans that refer to this table ## What changes were proposed in this pull request? When un-cache a table, we should not only remove the cache entry for this table, but also un-cache any other cached plans that refer to this table. This PR also includes some refactors: 1. use `java.util.LinkedList` to store the cache entries, so that it's safer to remove elements while iterating 2. rename `invalidateCache` to `recacheByPlan`, which is more obvious about what it does. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17097 from cloud-fan/cache.
* [SPARK-19637][SQL] Add to_json in FunctionRegistryTakeshi Yamamuro2017-03-075-2/+136
| | | | | | | | | | | | ## What changes were proposed in this pull request? This pr added entries in `FunctionRegistry` and supported `to_json` in SQL. ## How was this patch tested? Added tests in `JsonFunctionsSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #16981 from maropu/SPARK-19637.
* [SPARK-17075][SQL][FOLLOWUP] fix filter estimation issueswangzhenhua2017-03-062-274/+297
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. support boolean type in binary expression estimation. 2. deal with compound Not conditions. 3. avoid convert BigInt/BigDecimal directly to double unless it's within range (0, 1). 4. reorganize test code. ## How was this patch tested? modify related test cases. Author: wangzhenhua <wangzhenhua@huawei.com> Author: Zhenhua Wang <wzh_zju@163.com> Closes #17148 from wzhfy/fixFilter.
* [SPARK-19832][SQL] DynamicPartitionWriteTask get partitionPath should escape ↵windpiger2017-03-063-2/+58
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | the partition name ## What changes were proposed in this pull request? Currently in DynamicPartitionWriteTask, when we get the paritionPath of a parition, we just escape the partition value, not escape the partition name. this will cause some problems for some special partition name situation, for example : 1) if the partition name contains '%' etc, there will be two partition path created in the filesytem, one is for escaped path like '/path/a%25b=1', another is for unescaped path like '/path/a%b=1'. and the data inserted stored in unescaped path, while the show partitions table will return 'a%25b=1' which the partition name is escaped. So here it is not consist. And I think the data should be stored in the escaped path in filesystem, which Hive2.0.0 also have the same action. 2) if the partition name contains ':', there will throw exception that new Path("/path","a:b"), this is illegal which has a colon in the relative path. ``` java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: a:b at org.apache.hadoop.fs.Path.initialize(Path.java:205) at org.apache.hadoop.fs.Path.<init>(Path.java:171) at org.apache.hadoop.fs.Path.<init>(Path.java:88) ... 48 elided Caused by: java.net.URISyntaxException: Relative path in absolute URI: a:b at java.net.URI.checkPath(URI.java:1823) at java.net.URI.<init>(URI.java:745) at org.apache.hadoop.fs.Path.initialize(Path.java:202) ... 50 more ``` ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #17173 from windpiger/fixDatasourceSpecialCharPartitionName.
* [SPARK-19818][SPARKR] rbind should check for name consistency of input data ↵actuaryzhang2017-03-062-1/+14
| | | | | | | | | | | | | | frames ## What changes were proposed in this pull request? Added checks for name consistency of input data frames in union. ## How was this patch tested? new test. Author: actuaryzhang <actuaryzhang10@gmail.com> Closes #17159 from actuaryzhang/sparkRUnion.
* [SPARK-19350][SQL] Cardinality estimation of Limit and Samplewangzhenhua2017-03-064-103/+145
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Before this pr, LocalLimit/GlobalLimit/Sample propagates the same row count and column stats from its child, which is incorrect. We can get the correct rowCount in Statistics for GlobalLimit/Sample whether cbo is enabled or not. We don't know the rowCount for LocalLimit because we don't know the partition number at that time. Column stats should not be propagated because we don't know the distribution of columns after Limit or Sample. ## How was this patch tested? Added test cases. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #16696 from wzhfy/limitEstimation.
* [SPARK-19719][SS] Kafka writer for both structured streaming and batch queiresTyson Condie2017-03-065-5/+753
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add a new Kafka Sink and Kafka Relation for writing streaming and batch queries, respectively, to Apache Kafka. ### Streaming Kafka Sink - When addBatch is called -- If batchId is great than the last written batch --- Write batch to Kafka ---- Topic will be taken from the record, if present, or from a topic option, which overrides topic in record. -- Else ignore ### Batch Kafka Sink - KafkaSourceProvider will implement CreatableRelationProvider - CreatableRelationProvider#createRelation will write the passed in Dataframe to a Kafka - Topic will be taken from the record, if present, or from topic option, which overrides topic in record. - Save modes Append and ErrorIfExist supported under identical semantics. Other save modes result in an AnalysisException tdas zsxwing ## How was this patch tested? ### The following unit tests will be included - write to stream with topic field: valid stream write with data that includes an existing topic in the schema - write structured streaming aggregation w/o topic field, with default topic: valid stream write with data that does not include a topic field, but the configuration includes a default topic - write data with bad schema: various cases of writing data that does not conform to a proper schema e.g., 1. no topic field or default topic, and 2. no value field - write data with valid schema but wrong types: data with a complete schema but wrong types e.g., key and value types are integers. - write to non-existing topic: write a stream to a topic that does not exist in Kafka, which has been configured to not auto-create topics. - write batch to kafka: simple write batch to Kafka, which goes through the same code path as streaming scenario, so validity checks will not be redone here. ### Examples ```scala // Structured Streaming val writer = inputStringStream.map(s => s.get(0).toString.getBytes()).toDF("value") .selectExpr("value as key", "value as value") .writeStream .format("kafka") .option("checkpointLocation", checkpointDir) .outputMode(OutputMode.Append) .option("kafka.bootstrap.servers", brokerAddress) .option("topic", topic) .queryName("kafkaStream") .start() // Batch val df = spark .sparkContext .parallelize(Seq("1", "2", "3", "4", "5")) .map(v => (topic, v)) .toDF("topic", "value") df.write .format("kafka") .option("kafka.bootstrap.servers",brokerAddress) .option("topic", topic) .save() ``` Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Closes #17043 from tcondie/kafka-writer.
* [SPARK-19709][SQL] Read empty file with CSV data sourceWojtek Szymanski2017-03-062-38/+40
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Bugfix for reading empty file with CSV data source. Instead of throwing `NoSuchElementException`, an empty data frame is returned. ## How was this patch tested? Added new unit test in `org.apache.spark.sql.execution.datasources.csv.CSVSuite` Author: Wojtek Szymanski <wk.szymanski@gmail.com> Closes #17068 from wojtek-szymanski/SPARK-19709.
* [SPARK-19382][ML] Test sparse vectors in LinearSVCSuitewm624@hotmail.com2017-03-061-2/+22
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add unit tests for testing SparseVector. We can't add mixed DenseVector and SparseVector test case, as discussed in JIRA 19382. def merge(other: MultivariateOnlineSummarizer): this.type = { if (this.totalWeightSum != 0.0 && other.totalWeightSum != 0.0) { require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + s"Expecting $n but got $ {other.n} .") ## How was this patch tested? Unit tests Author: wm624@hotmail.com <wm624@hotmail.com> Author: Miao Wang <wangmiao1981@users.noreply.github.com> Closes #16784 from wangmiao1981/bk.
* [SPARK-19211][SQL] Explicitly prevent Insert into View or Create View As Insertjiangxingbo2017-03-064-2/+26
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we don't explicitly forbid the following behaviors: 1. The statement CREATE VIEW AS INSERT INTO throws the following exception: ``` scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")") org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.hadoop.hive.ql.metadata.HiveException: at least one column must be specified for the table; scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")") org.apache.spark.sql.AnalysisException: The number of columns produced by the SELECT clause (num: `0`) does not match the number of column names specified by CREATE VIEW (num: `2`).; ``` 2. The statement INSERT INTO view VALUES throws the following exception from checkAnalysis: ``` scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")") org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;; 'InsertIntoTable View (`default`.`testView`, [a#16,b#17]), false, false +- LocalRelation [col1#14, col2#15] ``` After this PR, the behavior changes to: ``` scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")") org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO; scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")") org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO; scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")") org.apache.spark.sql.AnalysisException: `default`.`testView` is a view, inserting into a view is not allowed; ``` ## How was this patch tested? Add a new test case in `SparkSqlParserSuite`; Update the corresponding test case in `SQLViewSuite`. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #17125 from jiangxb1987/insert-with-view.
* [SPARK-19796][CORE] Fix serialization of long property values in TaskDescriptionImran Rashid2017-03-062-2/+26
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The properties that are serialized with a TaskDescription can have very long values (eg. "spark.job.description" which is set to the full sql statement with the thrift-server). DataOutputStream.writeUTF() does not work well for long strings, so this changes the way those values are serialized to handle longer strings. ## How was this patch tested? Updated existing unit test to reproduce the issue. All unit tests via jenkins. Author: Imran Rashid <irashid@cloudera.com> Closes #17140 from squito/SPARK-19796.
* [SPARK-19257][SQL] location for table/partition/database should be java.net.URIwindpiger2017-03-0633-155/+460
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we treat the location of table/partition/database as URI string. It will be safer if we can make the type of location as java.net.URI. In this PR, there are following classes changes: **1. CatalogDatabase** ``` case class CatalogDatabase( name: String, description: String, locationUri: String, properties: Map[String, String]) ---> case class CatalogDatabase( name: String, description: String, locationUri: URI, properties: Map[String, String]) ``` **2. CatalogStorageFormat** ``` case class CatalogStorageFormat( locationUri: Option[String], inputFormat: Option[String], outputFormat: Option[String], serde: Option[String], compressed: Boolean, properties: Map[String, String]) ----> case class CatalogStorageFormat( locationUri: Option[URI], inputFormat: Option[String], outputFormat: Option[String], serde: Option[String], compressed: Boolean, properties: Map[String, String]) ``` Before and After this PR, it is transparent for user, there is no change that the user should concern. The `String` to `URI` just happened in SparkSQL internally. Here list some operation related location: **1. whitespace in the location** e.g. `/a/b c/d` For both table location and partition location, After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b c/d'` , then `DESC EXTENDED t ` show the location is `/a/b c/d`, and the real path in the FileSystem also show `/a/b c/d` **2. colon(:) in the location** e.g. `/a/b:c/d` For both table location and partition location, when `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b:c/d'` , **In linux file system** `DESC EXTENDED t ` show the location is `/a/b:c/d`, and the real path in the FileSystem also show `/a/b:c/d` **in HDFS** throw exception: `java.lang.IllegalArgumentException: Pathname /a/b:c/d from hdfs://iZbp1151s8hbnnwriekxdeZ:9000/a/b:c/d is not a valid DFS filename.` **while** After `INSERT INTO TABLE t PARTITION(a="a:b") SELECT 1` then `DESC EXTENDED t ` show the location is `/xxx/a=a%3Ab`, and the real path in the FileSystem also show `/xxx/a=a%3Ab` **3. percent sign(%) in the location** e.g. `/a/b%c/d` For both table location and partition location, After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b%c/d'` , then `DESC EXTENDED t ` show the location is `/a/b%c/d`, and the real path in the FileSystem also show `/a/b%c/d` **4. encoded(%25) in the location** e.g. `/a/b%25c/d` For both table location and partition location, After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b%25c/d'` , then `DESC EXTENDED t ` show the location is `/a/b%25c/d`, and the real path in the FileSystem also show `/a/b%25c/d` **while** After `INSERT INTO TABLE t PARTITION(a="%25") SELECT 1` then `DESC EXTENDED t ` show the location is `/xxx/a=%2525`, and the real path in the FileSystem also show `/xxx/a=%2525` **Additionally**, except the location, there are two other factors will affect the location of the table/partition. one is the table name which does not allowed to have special characters, and the other is `partition name` which have the same actions with `partition value`, and `partition name` with special character situation has add some testcase and resolve a bug in [PR](https://github.com/apache/spark/pull/17173) ### Summary: After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION path`, the path which we get from `DESC TABLE` and `real path in FileSystem` are all the same with the `CREATE TABLE` command(different filesystem has different action that allow what kind of special character to create the path, e.g. HDFS does not allow colon, but linux filesystem allow it ). `DataBase` also have the same logic with `CREATE TABLE` while if the `partition value` has some special character like `%` `:` `#` etc, then we will get the path with encoded `partition value` like `/xxx/a=A%25B` from `DESC TABLE` and `real path in FileSystem` In this PR, the core change code is using `new Path(str).toUri` and `new Path(uri).toString` which transfrom `str to uri `or `uri to str`. for example: ``` val str = '/a/b c/d' val uri = new Path(str).toUri --> '/a/b%20c/d' val strFromUri = new Path(uri).toString -> '/a/b c/d' ``` when we restore table/partition from metastore, or get the location from `CREATE TABLE` command, we can use it as above to change string to uri `new Path(str).toUri ` ## How was this patch tested? unit test added. The `current master branch` also `passed all the test cases` added in this PR by a litter change. https://github.com/apache/spark/pull/17149/files#diff-b7094baa12601424a5d19cb930e3402fR1764 here `toURI` -> `toString` when test in master branch. This can show that this PR is transparent for user. Author: windpiger <songjun@outlook.com> Closes #17149 from windpiger/changeStringToURI.
* [SPARK-19304][STREAMING][KINESIS] fix kinesis slow checkpoint recoveryGaurav2017-03-064-11/+25
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? added a limit to getRecords api call call in KinesisBackedBlockRdd. This helps reduce the amount of data returned by kinesis api call making the recovery considerably faster As we are storing the `fromSeqNum` & `toSeqNum` in checkpoint metadata, we can also store the number of records. Which can later be used for api call. ## How was this patch tested? The patch was manually tested Apologies for any silly mistakes, opening first pull request Author: Gaurav <gaurav@techtinium.com> Closes #16842 from Gauravshah/kinesis_checkpoint_recovery_fix_2_1_0.
* [SPARK-19737][SQL] New analysis rule for reporting unregistered functions ↵Cheng Lian2017-03-063-1/+48
| | | | | | | | | | | | | | | | | | | | | | without relying on relation resolution ## What changes were proposed in this pull request? This PR adds a new `Once` analysis rule batch consists of a single analysis rule `LookupFunctions` that performs simple existence check over `UnresolvedFunctions` without actually resolving them. The benefit of this rule is that it doesn't require function arguments to be resolved first and therefore doesn't rely on relation resolution, which may incur potentially expensive partition/schema discovery cost. Please refer to [SPARK-19737][1] for more details about the motivation. ## How was this patch tested? New test case added in `AnalysisErrorSuite`. [1]: https://issues.apache.org/jira/browse/SPARK-19737 Author: Cheng Lian <lian@databricks.com> Closes #17168 from liancheng/spark-19737-lookup-functions.
* [SPARK-17495][SQL] Support Decimal type in Hive-hashTejas Patil2017-03-062-3/+99
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Hive hash to support Decimal datatype. [Hive internally normalises decimals](https://github.com/apache/hive/blob/4ba713ccd85c3706d195aeef9476e6e6363f1c21/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java#L307) and I have ported that logic as-is to HiveHash. ## How was this patch tested? Added unit tests Author: Tejas Patil <tejasp@fb.com> Closes #17056 from tejasapatil/SPARK-17495_decimal.
* [SPARK-19822][TEST] CheckpointSuite.testCheckpointedOperation: should not ↵uncleGen2017-03-051-3/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | filter checkpointFilesOfLatestTime with the PATH string. ## What changes were proposed in this pull request? https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73800/testReport/ ``` sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 617 times over 10.003740484 seconds. Last failure message: 8 did not equal 2. at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:336) at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) at org.apache.spark.streaming.DStreamCheckpointTester$class.generateOutput(CheckpointSuite .scala:172) at org.apache.spark.streaming.CheckpointSuite.generateOutput(CheckpointSuite.scala:211) ``` the check condition is: ``` val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter { _.toString.contains(clock.getTimeMillis.toString) } // Checkpoint files are written twice for every batch interval. So assert that both // are written to make sure that both of them have been written. assert(checkpointFilesOfLatestTime.size === 2) ``` the path string may contain the `clock.getTimeMillis.toString`, like `3500` : ``` file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-500 file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1000 file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1500 file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2000 file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2500 file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3000 file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500.bk file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500 ▲▲▲▲ ``` so we should only check the filename, but not the whole path. ## How was this patch tested? Jenkins. Author: uncleGen <hustyugm@gmail.com> Closes #17167 from uncleGen/flaky-CheckpointSuite.
* [SPARK-19701][SQL][PYTHON] Throws a correct exception for 'in' operator ↵hyukjinkwon2017-03-052-1/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | against column ## What changes were proposed in this pull request? This PR proposes to remove incorrect implementation that has been not executed so far (at least from Spark 1.5.2) for `in` operator and throw a correct exception rather than saying it is a bool. I tested the codes above in 1.5.2, 1.6.3, 2.1.0 and in the master branch as below: **1.5.2** ```python >>> df = sqlContext.createDataFrame([[1]]) >>> 1 in df._1 Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark-1.5.2-bin-hadoop2.6/python/pyspark/sql/column.py", line 418, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **1.6.3** ```python >>> 1 in sqlContext.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark-1.6.3-bin-hadoop2.6/python/pyspark/sql/column.py", line 447, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **2.1.0** ```python >>> 1 in spark.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark-2.1.0-bin-hadoop2.7/python/pyspark/sql/column.py", line 426, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **Current Master** ```python >>> 1 in spark.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/sql/column.py", line 452, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **After** ```python >>> 1 in spark.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/sql/column.py", line 184, in __contains__ raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' " ValueError: Cannot apply 'in' operator against a column: please use 'contains' in a string column or 'array_contains' function for an array column. ``` In more details, It seems the implementation intended to support this ```python 1 in df.column ``` However, currently, it throws an exception as below: ```python Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/sql/column.py", line 426, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` What happens here is as below: ```python class Column(object): def __contains__(self, item): print "I am contains" return Column() def __nonzero__(self): raise Exception("I am nonzero.") >>> 1 in Column() I am contains Traceback (most recent call last): File "<stdin>", line 1, in <module> File "<stdin>", line 6, in __nonzero__ Exception: I am nonzero. ``` It seems it calls `__contains__` first and then `__nonzero__` or `__bool__` is being called against `Column()` to make this a bool (or int to be specific). It seems `__nonzero__` (for Python 2), `__bool__` (for Python 3) and `__contains__` forcing the the return into a bool unlike other operators. There are few references about this as below: https://bugs.python.org/issue16011 http://stackoverflow.com/questions/12244074/python-source-code-for-built-in-in-operator/12244378#12244378 http://stackoverflow.com/questions/38542543/functionality-of-python-in-vs-contains/38542777 It seems we can't overwrite `__nonzero__` or `__bool__` as a workaround to make this working because these force the return type as a bool as below: ```python class Column(object): def __contains__(self, item): print "I am contains" return Column() def __nonzero__(self): return "a" >>> 1 in Column() I am contains Traceback (most recent call last): File "<stdin>", line 1, in <module> TypeError: __nonzero__ should return bool or int, returned str ``` ## How was this patch tested? Added unit tests in `tests.py`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17160 from HyukjinKwon/SPARK-19701.
* [SPARK-19535][ML] RecommendForAllUsers RecommendForAllItems for ALS on DataframeSue Ann Hong2017-03-054-9/+297
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a simple implementation of RecommendForAllUsers & RecommendForAllItems for the Dataframe version of ALS. It uses Dataframe operations (not a wrapper on the RDD implementation). Haven't benchmarked against a wrapper, but unit test examples do work. ## How was this patch tested? Unit tests ``` $ build/sbt > mllib/testOnly *ALSSuite -- -z "recommendFor" > mllib/testOnly ``` Author: Your Name <you@example.com> Author: sueann <sueann@databricks.com> Closes #17090 from sueann/SPARK-19535.
* [SPARK-19595][SQL] Support json array in from_jsonhyukjinkwon2017-03-055-17/+186
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to both, **Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.** Currently, it only reads the single row when the input is a json array. So, the codes below: ```scala import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val schema = StructType(StructField("a", IntegerType) :: Nil) Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show() ``` prints ``` +--------------------+ |jsontostruct(struct)| +--------------------+ | [1]| +--------------------+ ``` This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements ``` +--------------------+ |jsontostruct(struct)| +--------------------+ | null| +--------------------+ ``` **Support json arrays in `from_json` with `ArrayType` as the schema.** ```scala import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show() ``` prints ``` +-------------------+ |jsontostruct(array)| +-------------------+ | [[1], [2]]| +-------------------+ ``` ## How was this patch tested? Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16929 from HyukjinKwon/disallow-array.
* [SPARK-19795][SPARKR] add column functions to_json, from_jsonFelix Cheung2017-03-054-7/+103
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add column functions: to_json, from_json, and tests covering error cases. ## How was this patch tested? unit tests, manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17134 from felixcheung/rtojson.
* [SPARK-19254][SQL] Support Seq, Map, and Struct in functions.litTakeshi Yamamuro2017-03-054-20/+121
| | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr is to support Seq, Map, and Struct in functions.lit; it adds a new IF named `lit2` with `TypeTag` for avoiding type erasure. ## How was this patch tested? Added tests in `LiteralExpressionSuite` Author: Takeshi Yamamuro <yamamuro@apache.org> Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16610 from maropu/SPARK-19254.
* [SPARK-19805][TEST] Log the row type when query result dose not matchuncleGen2017-03-051-0/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? improve the log message when query result does not match. before pr: ``` == Results == !== Correct Answer - 3 == == Spark Answer - 3 == [1] [1] [2] [2] [3] [3] ``` after pr: ~~== Results == !== Correct Answer - 3 == == Spark Answer - 3 == !RowType[string] RowType[integer] [1] [1] [2] [2] [3] [3]~~ ``` == Results == !== Correct Answer - 3 == == Spark Answer - 3 == !struct<value:string> struct<value:int> [1] [1] [2] [2] [3] [3] ``` ## How was this patch tested? Jenkins Author: uncleGen <hustyugm@gmail.com> Closes #17145 from uncleGen/improve-test-result.
* [SPARK-19792][WEBUI] In the Master Page,the column named “Memory per ↵liuxian2017-03-051-1/+1
| | | | | | | | | | | | | | | | | | Node” ,I think it is not all right Signed-off-by: liuxian <liu.xian3zte.com.cn> ## What changes were proposed in this pull request? Open the spark web page,in the Master Page ,have two tables:Running Applications table and Completed Applications table, to the column named “Memory per Node” ,I think it is not all right ,because a node may be not have only one executor.So I think that should be named as “Memory per Executor”.Otherwise easy to let the user misunderstanding ## How was this patch tested? N/A Author: liuxian <liu.xian3@zte.com.cn> Closes #17132 from 10110346/wid-lx-0302.
* [SPARK-19550][SPARKR][DOCS] Update R document to use JDK8Yuming Wang2017-03-041-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Update R document to use JDK8. ## How was this patch tested? manual tests Author: Yuming Wang <wgyumg@gmail.com> Closes #17162 from wangyum/SPARK-19550.
* [SPARK-19816][SQL][TESTS] Fix an issue that DataFrameCallbackSuite doesn't ↵Shixiong Zhu2017-03-031-3/+1
| | | | | | | | | | | | | | | | | | recover the log level ## What changes were proposed in this pull request? "DataFrameCallbackSuite.execute callback functions when a DataFrame action failed" sets the log level to "fatal" but doesn't recover it. Hence, tests running after it won't output any logs except fatal logs. This PR uses `testQuietly` instead to avoid changing the log level. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17156 from zsxwing/SPARK-19816.
* [SPARK-19084][SQL] Ensure context class loader is set when initializing Hive.Marcelo Vanzin2017-03-031-3/+8
| | | | | | | | | | | | | | | | A change in Hive 2.2 (most probably HIVE-13149) causes this code path to fail, since the call to "state.getConf.setClassLoader" does not actually change the context's class loader. Spark doesn't yet officially support Hive 2.2, but some distribution-specific metastore client libraries may have that change (as certain versions of CDH already do), and this also makes it easier to support 2.2 when it comes out. Tested with existing unit tests; we've also used this patch extensively with Hive metastore client jars containing the offending patch. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #17154 from vanzin/SPARK-19804.
* [SPARK-19718][SS] Handle more interrupt cases properly for HadoopShixiong Zhu2017-03-032-10/+119
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? [SPARK-19617](https://issues.apache.org/jira/browse/SPARK-19617) changed `HDFSMetadataLog` to enable interrupts when using the local file system. However, now we hit [HADOOP-12074](https://issues.apache.org/jira/browse/HADOOP-12074): `Shell.runCommand` converts `InterruptedException` to `new IOException(ie.toString())` before Hadoop 2.8. This is the Hadoop patch to fix HADOOP-1207: https://github.com/apache/hadoop/commit/95c73d49b1bb459b626a9ac52acadb8f5fa724de This PR adds new logic to handle the following cases related to `InterruptedException`. - Check if the message of IOException starts with `java.lang.InterruptedException`. If so, treat it as `InterruptedException`. This is for pre-Hadoop 2.8. - Treat `InterruptedIOException` as `InterruptedException`. This is for Hadoop 2.8+ and other places that may throw `InterruptedIOException` when the thread is interrupted. ## How was this patch tested? The new unit test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #17044 from zsxwing/SPARK-19718.