aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-18613][ML] make spark.mllib LDA dependencies in spark.ml LDA privatesueann2017-02-101-6/+6
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? spark.ml.*LDAModel classes were exposing spark.mllib LDA models via protected methods. Made them package (clustering) private. ## How was this patch tested? ``` build/sbt doc # "millib.clustering" no longer appears in the docs for *LDA* classes build/sbt compile # compiles build/sbt > mllib/testOnly # tests pass ``` Author: sueann <sueann@databricks.com> Closes #16860 from sueann/SPARK-18613.
* [SPARK-19459][SQL] Add Hive datatype (char/varchar) to StructField metadataHerman van Hovell2017-02-106-25/+76
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Reading from an existing ORC table which contains `char` or `varchar` columns can fail with a `ClassCastException` if the table metadata has been created using Spark. This is caused by the fact that spark internally replaces `char` and `varchar` columns with a `string` column. This PR fixes this by adding the hive type to the `StructField's` metadata under the `HIVE_TYPE_STRING` key. This is picked up by the `HiveClient` and the ORC reader, see https://github.com/apache/spark/pull/16060 for more details on how the metadata is used. ## How was this patch tested? Added a regression test to `OrcSourceSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16804 from hvanhovell/SPARK-19459.
* [SPARK-19466][CORE][SCHEDULER] Improve Fair Scheduler LoggingEren Avsarogullari2017-02-101-21/+46
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | Fair Scheduler Logging for the following cases can be useful for the user. 1. If **valid** `spark.scheduler.allocation.file` property is set, user can be informed and aware which scheduler file is processed when `SparkContext` initializes. 2. If **invalid** `spark.scheduler.allocation.file` property is set, currently, the following stacktrace is shown to user. In addition to this, more meaningful message can be shown to user by emphasizing the problem at building level of fair scheduler. Also other potential issues can be covered at this level as **Fair Scheduler can not be built. + exception stacktrace** ``` Exception in thread "main" java.io.FileNotFoundException: INVALID_FILE (No such file or directory) at java.io.FileInputStream.open0(Native Method) at java.io.FileInputStream.open(FileInputStream.java:195) at java.io.FileInputStream.<init>(FileInputStream.java:138) at java.io.FileInputStream.<init>(FileInputStream.java:93) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$buildPools$1.apply(SchedulableBuilder.scala:76) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$buildPools$1.apply(SchedulableBuilder.scala:75) ``` 3. If `spark.scheduler.allocation.file` property is not set and **default** fair scheduler file (**fairscheduler.xml**) is found in classpath, it will be loaded but currently, user is not informed for using default file so logging can be useful as **Fair Scheduler file: fairscheduler.xml is found successfully and will be parsed.** 4. If **spark.scheduler.allocation.file** property is not set and **default** fair scheduler file does not exist in classpath, currently, user is not informed so logging can be useful as **No Fair Scheduler file found.** Also this PR is related with https://github.com/apache/spark/pull/15237 to emphasize fileName in warning logs when fair scheduler file has invalid minShare, weight or schedulingMode values. ## How was this patch tested? Added new Unit Tests. Author: erenavsarogullari <erenavsarogullari@gmail.com> Closes #16813 from erenavsarogullari/SPARK-19466.
* Encryption of shuffle filesHervé2017-02-101-5/+1
| | | | | | | | | | | | | | | | Hello According to my understanding of commits 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 & 8b325b17ecdf013b7a6edcb7ee3773546bd914df, one may now encrypt shuffle files regardless of the cluster manager in use. However I have limited understanding of the code, I'm not able to find out whether theses changes also comprise all "temporary local storage, such as shuffle files, cached data, and other application files". Please feel free to amend or reject my PR if I'm wrong. dud Author: Hervé <dud225@users.noreply.github.com> Closes #16885 from dud225/patch-1.
* [SPARK-10748][MESOS] Log error instead of crashing Spark Mesos dispatcher ↵Devaraj K2017-02-101-9/+19
| | | | | | | | | | | | | | | when a job is misconfigured ## What changes were proposed in this pull request? Now handling the spark exception which gets thrown for invalid job configuration, marking that job as failed and continuing to launch the other drivers instead of throwing the exception. ## How was this patch tested? I verified manually, now the misconfigured jobs move to Finished Drivers section in UI and continue to launch the other jobs. Author: Devaraj K <devaraj@apache.org> Closes #13077 from devaraj-kavali/SPARK-10748.
* [SPARK-19545][YARN] Fix compile issue for Spark on Yarn when building ↵jerryshao2017-02-102-8/+23
| | | | | | | | | | | | | | | | against Hadoop 2.6.0~2.6.3 ## What changes were proposed in this pull request? Due to the newly added API in Hadoop 2.6.4+, Spark builds against Hadoop 2.6.0~2.6.3 will meet compile error. So here still reverting back to use reflection to handle this issue. ## How was this patch tested? Manual verification. Author: jerryshao <sshao@hortonworks.com> Closes #16884 from jerryshao/SPARK-19545.
* [SPARK-19543] from_json fails when the input row is emptyBurak Yavuz2017-02-102-1/+9
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Using from_json on a column with an empty string results in: java.util.NoSuchElementException: head of empty list. This is because `parser.parse(input)` may return `Nil` when `input.trim.isEmpty` ## How was this patch tested? Regression test in `JsonExpressionsSuite` Author: Burak Yavuz <brkyvz@gmail.com> Closes #16881 from brkyvz/json-fix.
* [SPARK-19263] Fix race in SchedulerIntegrationSuite.jinxing2017-02-091-7/+7
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? All the process of offering resource and generating `TaskDescription` should be guarded by taskScheduler.synchronized in `reviveOffers`, otherwise there is race condition. ## How was this patch tested? Existing unit tests. Author: jinxing <jinxing@meituan.com> Closes #16831 from jinxing64/SPARK-19263-FixRaceInTest.
* [SPARK-19481] [REPL] [MAVEN] Avoid to leak SparkContext in ↵Shixiong Zhu2017-02-095-11/+20
| | | | | | | | | | | | | | | | | | Signaling.cancelOnInterrupt ## What changes were proposed in this pull request? `Signaling.cancelOnInterrupt` leaks a SparkContext per call and it makes ReplSuite unstable. This PR adds `SparkContext.getActive` to allow `Signaling.cancelOnInterrupt` to get the active `SparkContext` to avoid the leak. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16825 from zsxwing/SPARK-19481.
* [SPARK-16554][CORE] Automatically Kill Executors and Nodes when they are ↵José Hiram Soltren2017-02-0911-26/+248
| | | | | | | | | | | | | | | | | | | | | | | Blacklisted ## What changes were proposed in this pull request? In SPARK-8425, we introduced a mechanism for blacklisting executors and nodes (hosts). After a certain number of failures, these resources would be "blacklisted" and no further work would be assigned to them for some period of time. In some scenarios, it is better to fail fast, and to simply kill these unreliable resources. This changes proposes to do so by having the BlacklistTracker kill unreliable resources when they would otherwise be "blacklisted". In order to be thread safe, this code depends on the CoarseGrainedSchedulerBackend sending a message to the driver backend in order to do the actual killing. This also helps to prevent a race which would permit work to begin on a resource (executor or node), between the time the resource is marked for killing and the time at which it is finally killed. ## How was this patch tested? ./dev/run-tests Ran https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh, and checked logs to see executors and nodes being killed. Testing can likely be improved here; suggestions welcome. Author: José Hiram Soltren <jose@cloudera.com> Closes #16650 from jsoltren/SPARK-16554-submit.
* [SPARK-19025][SQL] Remove SQL builder for operatorsjiangxingbo2017-02-096-2196/+3
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? With the new approach of view resolution, we can get rid of SQL generation on view creation, so let's remove SQL builder for operators. Note that, since all sql generation for operators is defined in one file (org.apache.spark.sql.catalyst.SQLBuilder), it’d be trivial to recover it in the future. ## How was this patch tested? N/A Author: jiangxingbo <jiangxb1987@gmail.com> Closes #16869 from jiangxb1987/SQLBuilder.
* [SPARK-19512][SQL] codegen for compare structs failsBogdan Raducanu2017-02-093-4/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Set currentVars to null in GenerateOrdering.genComparisons before genCode is called. genCode ignores INPUT_ROW if currentVars is not null and in genComparisons we want it to use INPUT_ROW. ## How was this patch tested? Added test with 2 queries in WholeStageCodegenSuite Author: Bogdan Raducanu <bogdan.rdc@gmail.com> Closes #16852 from bogdanrdc/SPARK-19512.
* [SPARK-19514] Making range interruptible.Ala Luszczak2017-02-093-6/+52
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Previously range operator could not be interrupted. For example, using DAGScheduler.cancelStage(...) on a query with range might have been ineffective. This change adds periodic checks of TaskContext.isInterrupted to codegen version, and InterruptibleOperator to non-codegen version. I benchmarked the performance of codegen version on a sample query `spark.range(1000L * 1000 * 1000 * 10).count()` and there is no measurable difference. ## How was this patch tested? Adds a unit test. Author: Ala Luszczak <ala@databricks.com> Closes #16872 from ala/SPARK-19514b.
* [SPARK-17874][CORE] Add SSL port configuration.Marcelo Vanzin2017-02-097-89/+164
| | | | | | | | | | | | | | | | | | | | | | Make the SSL port configuration explicit, instead of deriving it from the non-SSL port, but retain the existing functionality in case anyone depends on it. The change starts the HTTPS and HTTP connectors separately, so that it's possible to use independent ports for each. For that to work, the initialization of the server needs to be shuffled around a bit. The change also makes it so the initialization of both connectors is similar, and end up using the same Scheduler - previously only the HTTP connector would use the correct one. Also fixed some outdated documentation about a couple of services that were removed long ago. Tested with unit tests and by running spark-shell with SSL configs. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16625 from vanzin/SPARK-17874.
* [MINOR][CORE] Fix incorrect documentation of WritableConverterLee Dongjin2017-02-091-8/+10
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `WritableConverter` and `WritableFactory` work in opposite directions. But both of them are documented with same description: > A class encapsulating how to convert some type T to Writable. It stores both the Writable class corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion. This error is a result of commit 2604939. As a note, `WritableFactory` was added from commit d37978d, which resolves [SPARK-4795](https://issues.apache.org/jira/browse/SPARK-4795) with the correct description. This PR fix the documentation of `WritableConverter`, along with some improvements on type description. ## How was this patch tested? `build/mvn clean checkstyle:checkstyle` Author: Lee Dongjin <dongjin@apache.org> Closes #16830 from dongjinleekr/feature/fix-writableconverter-doc.
* [SPARK-19265][SQL][FOLLOW-UP] Configurable `tableRelationCache` maximum sizeLiwei Lin2017-02-096-3/+63
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable. In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`. ## How was this patch tested? new test case: `test("conf entry: checkValue()")` Author: Liwei Lin <lwlin7@gmail.com> Closes #16736 from lw-lin/conf.
* [SPARK-19359][SQL] renaming partition should not leave useless directoriesWenchen Fan2017-02-093-15/+84
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Hive metastore is not case-preserving and keep partition columns with lower case names. If Spark SQL creates a table with upper-case partition column names using `HiveExternalCatalog`, when we rename partition, it first calls the HiveClient to renamePartition, which will create a new lower case partition path, then Spark SQL renames the lower case path to upper-case. However, when we rename a nested path, different file systems have different behaviors. e.g. in jenkins, renaming `a=1/b=2` to `A=2/B=2` will success, but leave an empty directory `a=1`. in mac os, the renaming doesn't work as expected and result to `a=1/B=2`. This PR renames the partition directory recursively from the first partition column in `HiveExternalCatalog`, to be most compatible with different file systems. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16837 from cloud-fan/partition.
* [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery (Aggregate, ↵Dilip Biswal2017-02-096-0/+885
| | | | | | | | | | | | | | | | | | | | | | | | | | Having, Orderby, Limit) ## What changes were proposed in this pull request? This PR adds the second set of tests for EXISTS subquery. File name | Brief description ------------------------| ----------------- exists-aggregate.sql |Tests aggregate expressions in outer query and EXISTS subquery. exists-having.sql|Tests HAVING clause in subquery. exists-orderby-limit.sql|Tests EXISTS subquery support with ORDER BY and LIMIT clauses. DB2 results are attached here as reference : [exists-aggregate-db2.txt](https://github.com/apache/spark/files/743287/exists-aggregate-db2.txt) [exists-having-db2.txt](https://github.com/apache/spark/files/743286/exists-having-db2.txt) [exists-orderby-limit-db2.txt](https://github.com/apache/spark/files/743288/exists-orderby-limit-db2.txt) ## How the patch was tested. The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #16760 from dilipbiswal/exists-pr2.
* [SPARK-19464][BUILD][HOTFIX] run-tests should use hadoop2.6Dongjoon Hyun2017-02-083-11/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? After SPARK-19464, **SparkPullRequestBuilder** fails because it still tries to use hadoop2.3. **BEFORE** https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72595/console ``` ======================================================================== Building Spark ======================================================================== [error] Could not find hadoop2.3 in the list. Valid options are ['hadoop2.6', 'hadoop2.7'] Attempting to post to Github... > Post successful. ``` **AFTER** https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72595/console ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark (w/Hive 1.2.1) using SBT with these arguments: -Phadoop-2.6 -Pmesos -Pkinesis-asl -Pyarn -Phive-thriftserver -Phive test:package streaming-kafka-0-8-assembly/assembly streaming-flume-assembly/assembly streaming-kinesis-asl-assembly/assembly Using /usr/java/jdk1.8.0_60 as default JAVA_HOME. Note, this will be overridden by -java-home if it is set. ``` ## How was this patch tested? Pass the existing test. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16858 from dongjoon-hyun/hotfix_run-tests.
* [SPARK-19400][ML] Allow GLM to handle intercept only modelactuaryzhang2017-02-083-1/+60
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Intercept-only GLM is failing for non-Gaussian family because of reducing an empty array in IWLS. The following code `val maxTolOfCoefficients = oldCoefficients.toArray.reduce { (x, y) => math.max(math.abs(x), math.abs(y))` fails in the intercept-only model because `oldCoefficients` is empty. This PR fixes this issue. yanboliang srowen imatiach-msft zhengruifeng ## How was this patch tested? New test for intercept only model. Author: actuaryzhang <actuaryzhang10@gmail.com> Closes #16740 from actuaryzhang/interceptOnly.
* [SPARK-19464][BUILD][HOTFIX][TEST-HADOOP2.6] Add back mockito test dep in ↵Sean Owen2017-02-081-0/+6
| | | | | | | | | | | | | | YARN module, as it ends up being required in a Maven build Add back mockito test dep in YARN module, as it ends up being required in a Maven build ## How was this patch tested? PR builder again, but also a local `mvn` run using the command that the broken Jenkins job uses Author: Sean Owen <sowen@cloudera.com> Closes #16853 from srowen/SPARK-19464.2.
* [SPARK-19279][SQL][FOLLOW-UP] Infer Schema for Hive Serde Tablesgatorsmile2017-02-083-1/+53
| | | | | | | | | | | | ### What changes were proposed in this pull request? `table.schema` is always not empty for partitioned tables, because `table.schema` also contains the partitioned columns, even if the original table does not have any column. This PR is to fix the issue. ### How was this patch tested? Added a test case Author: gatorsmile <gatorsmile@gmail.com> Closes #16848 from gatorsmile/inferHiveSerdeSchema.
* [SPARK-19409][BUILD][TEST-MAVEN] Fix ParquetAvroCompatibilitySuite failure ↵Dongjoon Hyun2017-02-082-0/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | due to test dependency on avro ## What changes were proposed in this pull request? After using Apache Parquet 1.8.2, `ParquetAvroCompatibilitySuite` fails on **Maven** test. It is because `org.apache.parquet.avro.AvroParquetWriter` in the test code used new `avro 1.8.0` specific class, `LogicalType`. This PR aims to fix the test dependency of `sql/core` module to use avro 1.8.0. https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/2530/consoleFull ``` ParquetAvroCompatibilitySuite: *** RUN ABORTED *** java.lang.NoClassDefFoundError: org/apache/avro/LogicalType at org.apache.parquet.avro.AvroParquetWriter.writeSupport(AvroParquetWriter.java:144) ``` ## How was this patch tested? Pass the existing test with **Maven**. ``` $ build/mvn -Pyarn -Phadoop-2.7 -Pkinesis-asl -Phive -Phive-thriftserver test ... [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 02:07 h [INFO] Finished at: 2017-02-04T05:41:43+00:00 [INFO] Final Memory: 77M/987M [INFO] ------------------------------------------------------------------------ ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16795 from dongjoon-hyun/SPARK-19409-2.
* [SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and ↵Sean Owen2017-02-0834-1201/+202
| | | | | | | | | | | | | | | | | | | earlier ## What changes were proposed in this pull request? - Remove support for Hadoop 2.5 and earlier - Remove reflection and code constructs only needed to support multiple versions at once - Update docs to reflect newer versions - Remove older versions' builds and profiles. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16810 from srowen/SPARK-19464.
* [SPARK-19488][SQL] fix csv infer schema when the field is Nan/Inf etcwindpiger2017-02-082-1/+13
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? when csv infer schema, it does not use user defined csvoptions to parse the field, such as `inf`, `-inf` which are should be parsed to DoubleType this pr add `options.nanValue`, `options.negativeInf`, `options.positiveIn` to check if the field is a DoubleType ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16834 from windpiger/fixinferInfSchemaCsv.
* [MINOR][DOC] Remove parenthesis in readStream() on kafka structured ↵manugarri2017-02-071-2/+2
| | | | | | | | | | | | streaming doc There is a typo in http://spark.apache.org/docs/latest/structured-streaming-kafka-integration.html#creating-a-kafka-source-stream , python example n1 uses `readStream()` instead of `readStream` Just removed the parenthesis. Author: manugarri <manuel.garrido.pena@gmail.com> Closes #16836 from manugarri/fix_kafka_python_doc.
* [SPARK-18873][SQL][TEST] New test cases for scalar subquery (part 1 of 2) - ↵Nattavut Sutyanyong2017-02-072-0/+328
| | | | | | | | | | | | | | scalar subquery in SELECT clause ## What changes were proposed in this pull request? This PR adds new test cases for scalar subquery in SELECT clause. ## How was this patch tested? The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct. Author: Nattavut Sutyanyong <nsy.can@gmail.com> Closes #16712 from nsyca/18873.
* [SPARK-19499][SS] Add more notes in the comments of Sink.addBatch()CodingCat2017-02-071-1/+4
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? addBatch method in Sink trait is supposed to be a synchronous method to coordinate with the fault-tolerance design in StreamingExecution (being different with the compute() method in DStream) We need to add more notes in the comments of this method to remind the developers ## How was this patch tested? existing tests Author: CodingCat <zhunansjtu@gmail.com> Closes #16840 from CodingCat/SPARK-19499.
* [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operationsTathagata Das2017-02-0718-36/+1059
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState` *Requirements* - Users should be able to specify a function that can do the following - Access the input row corresponding to a key - Access the previous state corresponding to a key - Optionally, update or remove the state - Output any number of new rows (or none at all) *Proposed API* ``` // ------------ New methods on KeyValueGroupedDataset ------------ class KeyValueGroupedDataset[K, V] { // Scala friendly def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U) def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U]) // Java friendly def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U]) def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U]) } // ------------------- New Java-friendly function classes ------------------- public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable { R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception; } public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable { Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception; } // ---------------------- Wrapper class for state data ---------------------- trait State[S] { def exists(): Boolean def get(): S // throws Exception is state does not exist def getOption(): Option[S] def update(newState: S): Unit def remove(): Unit // exists() will be false after this } ``` Key Semantics of the State class - The state can be null. - If the state.remove() is called, then state.exists() will return false, and getOption will returm None. - After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...). - None of the operations are thread-safe. This is to avoid memory barriers. *Usage* ``` val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => { val newCount = words.size + runningCount.getOption.getOrElse(0L) runningCount.update(newCount) (word, newCount) } dataset // type is Dataset[String] .groupByKey[String](w => w) // generates KeyValueGroupedDataset[String, String] .mapGroupsWithState[Long, (String, Long)](stateFunc) // returns Dataset[(String, Long)] ``` ## How was this patch tested? New unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16758 from tdas/mapWithState.
* [SPARK-19397][SQL] Make option names of LIBSVM and TEXT case insensitivegatorsmile2017-02-086-10/+136
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Prior to Spark 2.1, the option names are case sensitive for all the formats. Since Spark 2.1, the option key names become case insensitive except the format `Text` and `LibSVM `. This PR is to fix these issues. Also, add a check to know whether the input option vector type is legal for `LibSVM`. ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16737 from gatorsmile/libSVMTextOptions.
* [SPARK-18682][SS] Batch Source for KafkaTyson Condie2017-02-0712-430/+1180
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Today, you can start a stream that reads from kafka. However, given kafka's configurable retention period, it seems like sometimes you might just want to read all of the data that is available now. As such we should add a version that works with spark.read as well. The options should be the same as the streaming kafka source, with the following differences: startingOffsets should default to earliest, and should not allow latest (which would always be empty). endingOffsets should also be allowed and should default to latest. the same assign json format as startingOffsets should also be accepted. It would be really good, if things like .limit(n) were enough to prevent all the data from being read (this might just work). ## How was this patch tested? KafkaRelationSuite was added for testing batch queries via KafkaUtils. Author: Tyson Condie <tcondie@gmail.com> Closes #16686 from tcondie/SPARK-18682.
* [SPARK-18609][SPARK-18841][SQL] Fix redundant Alias removal in the optimizerHerman van Hovell2017-02-079-115/+302
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The optimizer tries to remove redundant alias only projections from the query plan using the `RemoveAliasOnlyProject` rule. The current rule identifies removes such a project and rewrites the project's attributes in the **entire** tree. This causes problems when parts of the tree are duplicated (for instance a self join on a temporary view/CTE) and the duplicated part contains the alias only project, in this case the rewrite will break the tree. This PR fixes these problems by using a blacklist for attributes that are not to be moved, and by making sure that attribute remapping is only done for the parent tree, and not for unrelated parts of the query plan. The current tree transformation infrastructure works very well if the transformation at hand requires little or a global contextual information. In this case we need to know both the attributes that were not to be moved, and we also needed to know which child attributes were modified. This cannot be done easily using the current infrastructure, and solutions typically involves transversing the query plan multiple times (which is super slow). I have moved around some code in `TreeNode`, `QueryPlan` and `LogicalPlan`to make this much more straightforward; this basically allows you to manually traverse the tree. This PR subsumes the following PRs by windpiger: Closes https://github.com/apache/spark/pull/16267 Closes https://github.com/apache/spark/pull/16255 ## How was this patch tested? I have added unit tests to `RemoveRedundantAliasAndProjectSuite` and I have added integration tests to the `SQLQueryTestSuite.union` and `SQLQueryTestSuite.cte` test cases. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16757 from hvanhovell/SPARK-18609.
* [SPARK-19495][SQL] Make SQLConf slightly more extensibleReynold Xin2017-02-075-137/+139
| | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request makes SQLConf slightly more extensible by removing the visibility limitations on the build* functions. ## How was this patch tested? N/A - there are no logic changes and everything should be covered by existing unit tests. Author: Reynold Xin <rxin@databricks.com> Closes #16835 from rxin/SPARK-19495.
* [SPARK-16609] Add to_date/to_timestamp with format functionsanabranch2017-02-0711-22/+296
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
* [SPARK-19447] Fixing input metrics for range operator.Ala Luszczak2017-02-076-75/+350
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change introduces a new metric "number of generated rows". It is used exclusively for Range, which is a leaf in the query tree, yet doesn't read any input data, and therefore cannot report "recordsRead". Additionally the way in which the metrics are reported by the JIT-compiled version of Range was changed. Previously, it was immediately reported that all the records were produced. This could be confusing for a user monitoring execution progress in the UI. Now, the metric is updated gradually. In order to avoid negative impact on Range performance, the code generation was reworked. The values are now produced in batches in the tighter inner loop, while the metrics are updated in the outer loop. The change also contains a number of unit tests, which should help ensure the correctness of metrics for various input sources. ## How was this patch tested? Unit tests. Author: Ala Luszczak <ala@databricks.com> Closes #16829 from ala/SPARK-19447.
* [SPARK-19118][SQL] Percentile support for frequency distribution tablegagan taneja2017-02-072-55/+141
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? I have a frequency distribution table with following entries Age, No of person 21, 10 22, 15 23, 18 .. .. 30, 14 Moreover it is common to have data in frequency distribution format to further calculate Percentile, Median. With current implementation It would be very difficult and complex to find the percentile. Therefore i am proposing enhancement to current Percentile and Approx Percentile implementation to take frequency distribution column into consideration ## How was this patch tested? 1) Enhanced /sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala to cover the additional functionality 2) Run some performance benchmark test with 20 million row in local environment and did not see any performance degradation Please review http://spark.apache.org/contributing.html before opening a pull request. Author: gagan taneja <tanejagagan@gagans-MacBook-Pro.local> Closes #16497 from tanejagagan/branch-18940.
* [SPARK-16101][SQL] Refactoring CSV schema inference path to be consistent ↵hyukjinkwon2017-02-0711-246/+271
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | with JSON ## What changes were proposed in this pull request? This PR refactors CSV schema inference path to be consistent with JSON data source and moves some filtering codes having the similar/same logics into `CSVUtils`. It makes the methods in classes have consistent arguments with JSON ones. (this PR renames `.../json/InferSchema.scala` → `.../json/JsonInferSchema.scala`) `CSVInferSchema` and `JsonInferSchema` ``` scala private[csv] object CSVInferSchema { ... def infer( csv: Dataset[String], caseSensitive: Boolean, options: CSVOptions): StructType = { ... ``` ``` scala private[sql] object JsonInferSchema { ... def infer( json: RDD[String], columnNameOfCorruptRecord: String, configOptions: JSONOptions): StructType = { ... ``` These allow schema inference from `Dataset[String]` directly, meaning the similar functionalities that use `JacksonParser`/`JsonInferSchema` for JSON can be easily implemented by `UnivocityParser`/`CSVInferSchema` for CSV. This completes refactoring CSV datasource and they are now pretty consistent. ## How was this patch tested? Existing tests should cover this and ``` ./dev/change-scala-version.sh 2.10 ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16680 from HyukjinKwon/SPARK-16101-schema-inference.
* [SPARK-19260] Spaces or "%20" in path parameter are not correctly handled ↵zuotingbing2017-02-072-3/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | with… JIRA Issue: https://issues.apache.org/jira/browse/SPARK-19260 ## What changes were proposed in this pull request? 1. “spark.history.fs.logDirectory” supports with space character and “%20” characters. 2. As usually, if the run classpath includes hdfs-site.xml and core-site.xml files, the supplied path eg."/test" which does not contain a scheme should be taken as a HDFS path rather than a local path since the path parameter is a Hadoop dir. ## How was this patch tested? Update Unit Test and take some manual tests local: .sbin/start-history-server.sh "file:/a b" .sbin/start-history-server.sh "/abc%20c" (without hdfs-site.xml,core-site.xml) .sbin/start-history-server.sh "/a b" (without hdfs-site.xml,core-site.xml) .sbin/start-history-server.sh "/a b/a bc%20c" (without hdfs-site.xml,core-site.xml) hdfs: .sbin/start-history-server.sh "hdfs:/namenode:9000/a b" .sbin/start-history-server.sh "/a b" (with hdfs-site.xml,core-site.xml) .sbin/start-history-server.sh "/a b/a bc%20c" (with hdfs-site.xml,core-site.xml) Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #16614 from zuotingbing/SPARK-19260.
* [SPARK-19444][ML][DOCUMENTATION] Fix imports not being present in documentationAseem Bansal2017-02-071-3/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-19444 imports not being present in documentation ## How was this patch tested? Manual ## Disclaimer Contribution is original work and I license the work to the project under the project’s open source license Author: Aseem Bansal <anshbansal@users.noreply.github.com> Closes #16789 from anshbansal/patch-1.
* [SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizerEyal Farago2017-02-077-4/+427
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It often happens that a complex object (struct/map/array) is created only to get elements from it in an subsequent expression. We can add an optimizer rule for this. ## How was this patch tested? unit-tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Eyal Farago <eyal@nrgene.com> Author: eyal farago <eyal.farago@gmail.com> Closes #16043 from eyalfa/SPARK-18601.
* [SPARK-18967][SCHEDULER] compute locality levels even if delay = 0Imran Rashid2017-02-063-8/+105
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Before this change, with delay scheduling off, spark would effectively ignore locality preferences for bulk scheduling. With this change, locality preferences are used when multiple offers are made simultaneously. ## How was this patch tested? Test case added which fails without this change. All unit tests run via jenkins. Author: Imran Rashid <irashid@cloudera.com> Closes #16376 from squito/locality_without_delay.
* [SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it ↵uncleGen2017-02-061-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | from uri scheme ## What changes were proposed in this pull request? ``` Caused by: java.lang.IllegalArgumentException: Wrong FS: s3a://**************/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649) at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82) at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606) at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824) at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601) at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426) at org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51) at org.apache.spark.sql.execution.streaming.StreamExecution.<init>(StreamExecution.scala:100) at org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232) at org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269) at org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262) ``` Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config. WorkAround --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16815 from uncleGen/SPARK-19407.
* [SPARK-19467][ML][PYTHON] Remove cyclic imports from pyspark.ml.pipelinezero3232017-02-061-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Remove cyclic imports between `pyspark.ml.pipeline` and `pyspark.ml`. ## How was this patch tested? Existing unit tests. Author: zero323 <zero323@users.noreply.github.com> Closes #16814 from zero323/SPARK-19467.
* [SPARK-19441][SQL] Remove IN type coercion from PromoteStringsgatorsmile2017-02-071-9/+0
| | | | | | | | | | | | ### What changes were proposed in this pull request? The removed codes for `IN` are not reachable, because the previous rule `InConversion` already resolves the type coercion issues. ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #16783 from gatorsmile/typeCoercionIn.
* [SPARK-19472][SQL] Parser should not mistake CASE WHEN(...) for a function callHerman van Hovell2017-02-062-1/+3
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The SQL parser can mistake a `WHEN (...)` used in `CASE` for a function call. This happens in cases like the following: ```sql select case when (1) + case when 1 > 0 then 1 else 0 end = 2 then 1 else 0 end from tb ``` This PR fixes this by re-organizing the case related parsing rules. ## How was this patch tested? Added a regression test to the `ExpressionParserSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16821 from hvanhovell/SPARK-19472.
* [SPARK-19398] Change one misleading log in TaskSetManager.Jin Xing2017-02-061-4/+8
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Log below is misleading: ``` if (successful(index)) { logInfo( s"Task ${info.id} in stage ${taskSet.id} (TID $tid) failed, " + "but another instance of the task has already succeeded, " + "so not re-queuing the task to be re-executed.") } ``` If fetch failed, the task is marked as successful in `TaskSetManager:: handleFailedTask`. Then log above will be printed. The `successful` just means task will not be scheduled any longer, not a real success. ## How was this patch tested? Existing unit tests can cover this. Author: jinxing <jinxing@meituan.com> Closes #16738 from jinxing64/SPARK-19398.
* [SPARK-19080][SQL] simplify data source analysisWenchen Fan2017-02-0720-199/+126
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current way of resolving `InsertIntoTable` and `CreateTable` is convoluted: sometimes we replace them with concrete implementation commands during analysis, sometimes during planning phase. And the error checking logic is also a mess: we may put it in extended analyzer rules, or extended checking rules, or `CheckAnalysis`. This PR simplifies the data source analysis: 1. `InsertIntoTable` and `CreateTable` are always unresolved and need to be replaced by concrete implementation commands during analysis. 2. The error checking logic is mainly in 2 rules: `PreprocessTableCreation` and `PreprocessTableInsertion`. ## How was this patch tested? existing test. Author: Wenchen Fan <wenchen@databricks.com> Closes #16269 from cloud-fan/ddl.
* [SPARK-17213][SQL][FOLLOWUP] Re-enable Parquet filter tests for binary and ↵hyukjinkwon2017-02-061-4/+2
| | | | | | | | | | | | | | | | | | string ## What changes were proposed in this pull request? This PR proposes to enable the tests for Parquet filter pushdown with binary and string. This was disabled in https://github.com/apache/spark/pull/16106 due to Parquet's issue but it is now revived in https://github.com/apache/spark/pull/16791 after upgrading Parquet to 1.8.2. ## How was this patch tested? Manually tested `ParquetFilterSuite` via IDE. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16817 from HyukjinKwon/SPARK-17213.
* [SPARK-17663][CORE] SchedulableBuilder should handle invalid data access via ↵erenavsarogullari2017-02-063-51/+182
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | scheduler.allocation.file ## What changes were proposed in this pull request? If `spark.scheduler.allocation.file` has invalid `minShare` or/and `weight` values, these cause : - `NumberFormatException` due to `toInt` function - `SparkContext` can not be initialized. - It does not show meaningful error message to user. In a nutshell, this functionality can be more robust by selecting one of the following flows : **1-** Currently, if `schedulingMode` has an invalid value, a warning message is logged and default value is set as `FIFO`. Same pattern can be used for `minShare`(default: 0) and `weight`(default: 1) as well **2-** Meaningful error message can be shown to the user for all invalid cases. PR offers : - `schedulingMode` handles just empty values. It also needs to be supported for **whitespace**, **non-uppercase**(fair, FaIr etc...) or `SchedulingMode.NONE` cases by setting default value(`FIFO`) - `minShare` and `weight` handle just empty values. They also need to be supported for **non-integer** cases by setting default values. - Some refactoring of `PoolSuite`. **Code to Reproduce :** ``` val conf = new SparkConf().setAppName("spark-fairscheduler").setMaster("local") conf.set("spark.scheduler.mode", "FAIR") conf.set("spark.scheduler.allocation.file", "src/main/resources/fairscheduler-invalid-data.xml") val sc = new SparkContext(conf) ``` **fairscheduler-invalid-data.xml :** ``` <allocations> <pool name="production"> <schedulingMode>FIFO</schedulingMode> <weight>invalid_weight</weight> <minShare>2</minShare> </pool> </allocations> ``` **Stacktrace :** ``` Exception in thread "main" java.lang.NumberFormatException: For input string: "invalid_weight" at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.lang.Integer.parseInt(Integer.java:580) at java.lang.Integer.parseInt(Integer.java:615) at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272) at scala.collection.immutable.StringOps.toInt(StringOps.scala:29) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$org$apache$spark$scheduler$FairSchedulableBuilder$$buildFairSchedulerPool$1.apply(SchedulableBuilder.scala:127) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$org$apache$spark$scheduler$FairSchedulableBuilder$$buildFairSchedulerPool$1.apply(SchedulableBuilder.scala:102) ``` ## How was this patch tested? Added Unit Test Case. Author: erenavsarogullari <erenavsarogullari@gmail.com> Closes #15237 from erenavsarogullari/SPARK-17663.
* [SPARK-19409][SPARK-17213] Cleanup Parquet workarounds/hacks due to bugs of ↵Cheng Lian2017-02-062-33/+2
| | | | | | | | | | | | | | | | old Parquet versions ## What changes were proposed in this pull request? We've already upgraded parquet-mr to 1.8.2. This PR does some further cleanup by removing a workaround of PARQUET-686 and a hack due to PARQUET-363 and PARQUET-278. All three Parquet issues are fixed in parquet-mr 1.8.2. ## How was this patch tested? Existing unit tests. Author: Cheng Lian <lian@databricks.com> Closes #16791 from liancheng/parquet-1.8.2-cleanup.