aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17442][SPARKR] Additional arguments in write.df are not passed to ↵Felix Cheung2016-09-082-1/+12
| | | | | | | | | | | | | | | | | data source ## What changes were proposed in this pull request? additional options were not passed down in write.df. ## How was this patch tested? unit tests falaki shivaram Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #15010 from felixcheung/testreadoptions.
* [SPARK-17432][SQL] PreprocessDDL should respect case sensitivity when ↵Wenchen Fan2016-09-082-1/+13
| | | | | | | | | | | | | | | | | | checking duplicated columns ## What changes were proposed in this pull request? In `PreprocessDDL` we will check if table columns are duplicated. However, this checking ignores case sensitivity config(it's always case-sensitive) and lead to different result between `HiveExternalCatalog` and `InMemoryCatalog`. `HiveExternalCatalog` will throw exception because hive metastore is always case-nonsensitive, and `InMemoryCatalog` is fine. This PR fixes it. ## How was this patch tested? a new test in DDLSuite Author: Wenchen Fan <wenchen@databricks.com> Closes #14994 from cloud-fan/check-dup.
* [SPARK-17052][SQL] Remove Duplicate Test Cases auto_join from ↵gatorsmile2016-09-071-24/+26
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | HiveCompatibilitySuite.scala ### What changes were proposed in this pull request? The original [JIRA Hive-1642](https://issues.apache.org/jira/browse/HIVE-1642) delivered the test cases `auto_joinXYZ` for verifying the results when the joins are automatically converted to map-join. Basically, most of them are just copied from the corresponding `joinXYZ`. After comparison between `auto_joinXYZ` and `joinXYZ`, below is a list of duplicate cases: ``` "auto_join0", "auto_join1", "auto_join10", "auto_join11", "auto_join12", "auto_join13", "auto_join14", "auto_join14_hadoop20", "auto_join15", "auto_join17", "auto_join18", "auto_join2", "auto_join20", "auto_join21", "auto_join23", "auto_join24", "auto_join3", "auto_join4", "auto_join5", "auto_join6", "auto_join7", "auto_join8", "auto_join9" ``` We can remove all of them without affecting the test coverage. ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #14635 from gatorsmile/removeAuto.
* [SPARK-17370] Shuffle service files not invalidated when a slave is lostEric Liang2016-09-0712-31/+92
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? DAGScheduler invalidates shuffle files when an executor loss event occurs, but not when the external shuffle service is enabled. This is because when shuffle service is on, the shuffle file lifetime can exceed the executor lifetime. However, it also doesn't invalidate shuffle files when the shuffle service itself is lost (due to whole slave loss). This can cause long hangs when slaves are lost since the file loss is not detected until a subsequent stage attempts to read the shuffle files. The proposed fix is to also invalidate shuffle files when an executor is lost due to a `SlaveLost` event. ## How was this patch tested? Unit tests, also verified on an actual cluster that slave loss invalidates shuffle files immediately as expected. cc mateiz Author: Eric Liang <ekl@databricks.com> Closes #14931 from ericl/sc-4439.
* [MINOR][SQL] Fixing the typo in unit testSrinivasa Reddy Vundela2016-09-071-2/+2
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixing the typo in the unit test of CodeGenerationSuite.scala ## How was this patch tested? Ran the unit test after fixing the typo and it passes Author: Srinivasa Reddy Vundela <vsr@cloudera.com> Closes #14989 from vundela/typo_fix.
* [SPARK-17427][SQL] function SIZE should return -1 when parameter is nullDaoyuan Wang2016-09-073-14/+28
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `select size(null)` returns -1 in Hive. In order to be compatible, we should return `-1`. ## How was this patch tested? unit test in `CollectionFunctionsSuite` and `DataFrameFunctionsSuite`. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #14991 from adrian-wang/size.
* [SPARK-17339][SPARKR][CORE] Fix some R tests and use Path.toUri in ↵hyukjinkwon2016-09-072-6/+12
| | | | | | | | | | | | | | | | | | | | SparkContext for Windows paths in SparkR ## What changes were proposed in this pull request? This PR fixes the Windows path issues in several APIs. Please refer https://issues.apache.org/jira/browse/SPARK-17339 for more details. ## How was this patch tested? Tests via AppVeyor CI - https://ci.appveyor.com/project/HyukjinKwon/spark/build/82-SPARK-17339-fix-r Also, manually, ![2016-09-06 3 14 38](https://cloud.githubusercontent.com/assets/6477701/18263406/b93a98be-7444-11e6-9521-b28ee65a4771.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #14960 from HyukjinKwon/SPARK-17339.
* [SPARK-17359][SQL][MLLIB] Use ArrayBuffer.+=(A) instead of ↵Liwei Lin2016-09-0725-61/+60
| | | | | | | | | | | | | | | | ArrayBuffer.append(A) in performance critical paths ## What changes were proposed in this pull request? We should generally use `ArrayBuffer.+=(A)` rather than `ArrayBuffer.append(A)`, because `append(A)` would involve extra boxing / unboxing. ## How was this patch tested? N/A Author: Liwei Lin <lwlin7@gmail.com> Closes #14914 from lw-lin/append_to_plus_eq_v2.
* [SPARK-16785] R dapply doesn't return array or raw columnsClark Fitzgerald2016-09-065-1/+72
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixed bug in `dapplyCollect` by changing the `compute` function of `worker.R` to explicitly handle raw (binary) vectors. cc shivaram ## How was this patch tested? Unit tests Author: Clark Fitzgerald <clarkfitzg@gmail.com> Closes #14783 from clarkfitzg/SPARK-16785.
* [SPARK-17372][SQL][STREAMING] Avoid serialization issues by using Arrays to ↵Tathagata Das2016-09-066-18/+65
| | | | | | | | | | | | | | | | | | | | | | | | | | save file names in FileStreamSource ## What changes were proposed in this pull request? When we create a filestream on a directory that has partitioned subdirs (i.e. dir/x=y/), then ListingFileCatalog.allFiles returns the files in the dir as Seq[String] which internally is a Stream[String]. This is because of this [line](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L93), where a LinkedHashSet.values.toSeq returns Stream. Then when the [FileStreamSource](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L79) filters this Stream[String] to remove the seen files, it creates a new Stream[String], which has a filter function that has a $outer reference to the FileStreamSource (in Scala 2.10). Trying to serialize this Stream[String] causes NotSerializableException. This will happened even if there is just one file in the dir. Its important to note that this behavior is different in Scala 2.11. There is no $outer reference to FileStreamSource, so it does not throw NotSerializableException. However, with a large sequence of files (tested with 10000 files), it throws StackOverflowError. This is because how Stream class is implemented. Its basically like a linked list, and attempting to serialize a long Stream requires *recursively* going through linked list, thus resulting in StackOverflowError. In short, across both Scala 2.10 and 2.11, serialization fails when both the following conditions are true. - file stream defined on a partitioned directory - directory has 10k+ files The right solution is to convert the seq to an array before writing to the log. This PR implements this fix in two ways. - Changing all uses for HDFSMetadataLog to ensure Array is used instead of Seq - Added a `require` in HDFSMetadataLog such that it is never used with type Seq ## How was this patch tested? Added unit test that test that ensures the file stream source can handle with 10000 files. This tests fails in both Scala 2.10 and 2.11 with different failures as indicated above. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14987 from tdas/SPARK-17372.
* [SPARK-17238][SQL] simplify the logic for converting data source table into ↵Wenchen Fan2016-09-071-14/+18
| | | | | | | | | | | | | | | | | | | | | | | | | | | hive compatible format ## What changes were proposed in this pull request? Previously we have 2 conditions to decide whether a data source table is hive-compatible: 1. the data source is file-based and has a corresponding Hive serde 2. have a `path` entry in data source options/storage properties However, if condition 1 is true, condition 2 must be true too, as we will put the default table path into data source options/storage properties for managed data source tables. There is also a potential issue: we will set the `locationUri` even for managed table. This PR removes the condition 2 and only set the `locationUri` for external data source tables. Note: this is also a first step to unify the `path` of data source tables and `locationUri` of hive serde tables. For hive serde tables, `locationUri` is only set for external table. For data source tables, `path` is always set. We can make them consistent after this PR. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #14809 from cloud-fan/minor2.
* [SPARK-17408][TEST] Flaky test: org.apache.spark.sql.hive.StatisticsSuitegatorsmile2016-09-071-61/+80
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64956/testReport/junit/org.apache.spark.sql.hive/StatisticsSuite/test_statistics_of_LogicalRelation_converted_from_MetastoreRelation/ ``` org.apache.spark.sql.hive.StatisticsSuite.test statistics of LogicalRelation converted from MetastoreRelation Failing for the past 1 build (Since Failed#64956 ) Took 1.4 sec. Error Message org.scalatest.exceptions.TestFailedException: 6871 did not equal 4236 Stacktrace sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: 6871 did not equal 4236 at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) ``` This fix does not check the exact value of `sizeInBytes`. Instead, we compare whether it is larger than zero and compare the values between different values. In addition, we also combine `checkMetastoreRelationStats` and `checkLogicalRelationStats` into the same checking function. ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #14978 from gatorsmile/spark17408.
* [SPARK-17371] Resubmitted shuffle outputs can get deleted by zombie map tasksEric Liang2016-09-063-5/+0
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It seems that old shuffle map tasks hanging around after a stage resubmit will delete intended shuffle output files on stop(), causing downstream stages to fail even after successful resubmit completion. This can happen easily if the prior map task is waiting for a network timeout when its stage is resubmitted. This can cause unnecessary stage resubmits, sometimes multiple times as fetch fails cause a cascade of shuffle file invalidations, and confusing FetchFailure messages that report shuffle index files missing from the local disk. Given that IndexShuffleBlockResolver commits data atomically, it seems unnecessary to ever delete committed task output: even in the rare case that a task is failed after it finishes committing shuffle output, it should be safe to retain that output. ## How was this patch tested? Prior to the fix proposed in https://github.com/apache/spark/pull/14931, I was able to reproduce this behavior by killing slaves in the middle of a large shuffle. After this patch, stages were no longer resubmitted multiple times due to shuffle index loss. cc JoshRosen vanzin Author: Eric Liang <ekl@databricks.com> Closes #14932 from ericl/dont-remove-committed-files.
* [SPARK-17316][CORE] Fix the 'ask' type parameter in 'removeExecutor'Shixiong Zhu2016-09-061-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix the 'ask' type parameter in 'removeExecutor' to eliminate a lot of error logs `Cannot cast java.lang.Boolean to scala.runtime.Nothing$` ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #14983 from zsxwing/SPARK-17316-3.
* [SPARK-15891][YARN] Clean up some logging in the YARN AM.Marcelo Vanzin2016-09-064-63/+82
| | | | | | | | | | | | | | | | | | | | | | | | | To make the log file more readable, rework some of the logging done by the AM: - log executor command / env just once, since they're all almost the same; the information that changes, such as executor ID, is already available in other log messages. - avoid printing logs when nothing happens, especially when updating the container requests in the allocator. - print fewer log messages when requesting many unlocalized executors, instead of repeating the same message multiple times. - removed some logs that seemed unnecessary. In the process, I slightly fixed up the wording in a few log messages, and did some minor clean up of method arguments that were redundant. Tested by running existing unit tests, and analyzing the logs of an application that exercises dynamic allocation by forcing executors to be allocated and be killed in waves. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14943 from vanzin/SPARK-15891.
* [SPARK-17296][SQL] Simplify parser join processing.Herman van Hovell2016-09-074-58/+102
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Join processing in the parser relies on the fact that the grammar produces a right nested trees, for instance the parse tree for `select * from a join b join c` is expected to produce a tree similar to `JOIN(a, JOIN(b, c))`. However there are cases in which this (invariant) is violated, like: ```sql SELECT COUNT(1) FROM test T1 CROSS JOIN test T2 JOIN test T3 ON T3.col = T1.col JOIN test T4 ON T4.col = T1.col ``` In this case the parser returns a tree in which Joins are located on both the left and the right sides of the parent join node. This PR introduces a different grammar rule which does not make this assumption. The new rule takes a relation and searches for zero or more joined relations. As a bonus processing is much easier. ## How was this patch tested? Existing tests and I have added a regression test to the plan parser suite. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14867 from hvanhovell/SPARK-17296.
* [SPARK-17110] Fix StreamCorruptionException in BlockManager.getRemoteValues()Josh Rosen2016-09-066-16/+22
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes a `java.io.StreamCorruptedException` error affecting remote reads of cached values when certain data types are used. The problem stems from #11801 / SPARK-13990, a patch to have Spark automatically pick the "best" serializer when caching RDDs. If PySpark cached a PythonRDD, then this would be cached as an `RDD[Array[Byte]]` and the automatic serializer selection would pick KryoSerializer for replication and block transfer. However, the `getRemoteValues()` / `getRemoteBytes()` code path did not pass proper class tags in order to enable the same serializer to be used during deserialization, causing Java to be inappropriately used instead of Kryo, leading to the StreamCorruptedException. We already fixed a similar bug in #14311, which dealt with similar issues in block replication. Prior to that patch, it seems that we had no tests to ensure that block replication actually succeeded. Similarly, prior to this bug fix patch it looks like we had no tests to perform remote reads of cached data, which is why this bug was able to remain latent for so long. This patch addresses the bug by modifying `BlockManager`'s `get()` and `getRemoteValues()` methods to accept ClassTags, allowing the proper class tag to be threaded in the `getOrElseUpdate` code path (which is used by `rdd.iterator`) ## How was this patch tested? Extended the caching tests in `DistributedSuite` to exercise the `getRemoteValues` path, plus manual testing to verify that the PySpark bug reproduction in SPARK-17110 is fixed. Author: Josh Rosen <joshrosen@databricks.com> Closes #14952 from JoshRosen/SPARK-17110.
* [MINOR] Remove unnecessary check in MLSerDeZheng RuiFeng2016-09-061-5/+4
| | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, remove unnecessary `require()`, because it will make following check useless. 2, update the error msg. ## How was this patch tested? no test Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #14972 from zhengruifeng/del_unnecessary_check.
* [SPARK-17299] TRIM/LTRIM/RTRIM should not strips characters other than spacesSandeep Singh2016-09-063-8/+18
| | | | | | | | | | | | ## What changes were proposed in this pull request? TRIM/LTRIM/RTRIM should not strips characters other than spaces, we were trimming all chars small than ASCII 0x20(space) ## How was this patch tested? fixed existing tests. Author: Sandeep Singh <sandeep@techaddict.me> Closes #14924 from techaddict/SPARK-17299.
* [SPARK-17378][BUILD] Upgrade snappy-java to 1.1.2.6Adam Roberts2016-09-066-6/+6
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Upgrades the Snappy version to 1.1.2.6 from 1.1.2.4, release notes: https://github.com/xerial/snappy-java/blob/master/Milestone.md mention "Fix a bug in SnappyInputStream when reading compressed data that happened to have the same first byte with the stream magic header (#142)" ## How was this patch tested? Existing unit tests using the latest IBM Java 8 on Intel, Power and Z architectures (little and big-endian) Author: Adam Roberts <aroberts@uk.ibm.com> Closes #14958 from a-roberts/master.
* [SPARK-16922] [SPARK-17211] [SQL] make the address of values portable in ↵Davies Liu2016-09-062-8/+75
| | | | | | | | | | | | | | | | | | LongToUnsafeRowMap ## What changes were proposed in this pull request? In LongToUnsafeRowMap, we use offset of a value as pointer, stored in a array also in the page for chained values. The offset is not portable, because Platform.LONG_ARRAY_OFFSET will be different with different JVM Heap size, then the deserialized LongToUnsafeRowMap will be corrupt. This PR will change to use portable address (without Platform.LONG_ARRAY_OFFSET). ## How was this patch tested? Added a test case with random generated keys, to improve the coverage. But this test is not a regression test, that could require a Spark cluster that have at least 32G heap in driver or executor. Author: Davies Liu <davies@databricks.com> Closes #14927 from davies/longmap.
* [SPARK-17374][SQL] Better error messages when parsing JSON using DataFrameReaderSean Zhong2016-09-062-2/+66
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds better error messages for malformed record when reading a JSON file using DataFrameReader. For example, for query: ``` import org.apache.spark.sql.types._ val corruptRecords = spark.sparkContext.parallelize("""{"a":{, b:3}""" :: Nil) val schema = StructType(StructField("a", StringType, true) :: Nil) val jsonDF = spark.read.schema(schema).json(corruptRecords) ``` **Before change:** We silently replace corrupted line with null ``` scala> jsonDF.show +----+ | a| +----+ |null| +----+ ``` **After change:** Add an explicit warning message: ``` scala> jsonDF.show 16/09/02 14:43:16 WARN JacksonParser: Found at least one malformed records (sample: {"a":{, b:3}). The JSON reader will replace all malformed records with placeholder null in current PERMISSIVE parser mode. To find out which corrupted records have been replaced with null, please use the default inferred schema instead of providing a custom schema. Code example to print all malformed records (scala): =================================================== // The corrupted record exists in column _corrupt_record. val parsedJson = spark.read.json("/path/to/json/file/test.json") +----+ | a| +----+ |null| +----+ ``` ### ## How was this patch tested? Unit test. Author: Sean Zhong <seanzhong@databricks.com> Closes #14929 from clockfly/logwarning_if_schema_not_contain_corrupted_record.
* [MINOR][ML] Correct weights doc of MultilayerPerceptronClassificationModel.Yanbo Liang2016-09-062-2/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? ```weights``` of ```MultilayerPerceptronClassificationModel``` should be the output weights of layers rather than initial weights, this PR correct it. ## How was this patch tested? Doc change. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14967 from yanboliang/mlp-weights.
* [SPARK-17356][SQL] Fix out of memory issue when generating JSON for TreeNodeSean Zhong2016-09-062-2/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? class `org.apache.spark.sql.types.Metadata` is widely used in mllib to store some ml attributes. `Metadata` is commonly stored in `Alias` expression. ``` case class Alias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, val explicitMetadata: Option[Metadata] = None, override val isGenerated: java.lang.Boolean = false) ``` The `Metadata` can take a big memory footprint since the number of attributes is big ( in scale of million). When `toJSON` is called on `Alias` expression, the `Metadata` will also be converted to a big JSON string. If a plan contains many such kind of `Alias` expressions, it may trigger out of memory error when `toJSON` is called, since converting all `Metadata` references to JSON will take huge memory. With this PR, we will skip scanning Metadata when doing JSON conversion. For a reproducer of the OOM, and analysis, please look at jira https://issues.apache.org/jira/browse/SPARK-17356. ## How was this patch tested? Existing tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #14915 from clockfly/json_oom.
* [SPARK-17361][SQL] file-based external table without path should not be createdWenchen Fan2016-09-069-44/+48
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Using the public `Catalog` API, users can create a file-based data source table, without giving the path options. For this case, currently we can create the table successfully, but fail when we read it. Ideally we should fail during creation. This is because when we create data source table, we resolve the data source relation without validating path: `resolveRelation(checkPathExist = false)`. Looking back to why we add this trick(`checkPathExist`), it's because when we call `resolveRelation` for managed table, we add the path to data source options but the path is not created yet. So why we add this not-yet-created path to data source options? This PR fix the problem by adding path to options after we call `resolveRelation`. Then we can remove the `checkPathExist` parameter in `DataSource.resolveRelation` and do some related cleanups. ## How was this patch tested? existing tests and new test in `CatalogSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14921 from cloud-fan/check-path.
* [SPARK-17358][SQL] Cached table(parquet/orc) should be shard between beelinesYadong Qi2016-09-064-9/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Cached table(parquet/orc) couldn't be shard between beelines, because the `sameResult` method used by `CacheManager` always return false(`sparkSession` are different) when compare two `HadoopFsRelation` in different beelines. So we make `sparkSession` a curry parameter. ## How was this patch tested? Beeline1 ``` 1: jdbc:hive2://localhost:10000> CACHE TABLE src_pqt; +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (5.143 seconds) 1: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt; +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | plan | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | == Physical Plan == InMemoryTableScan [key#49, value#50] +- InMemoryRelation [key#49, value#50], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt` +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,value:string> | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ ``` Beeline2 ``` 0: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt; +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | plan | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | == Physical Plan == InMemoryTableScan [key#68, value#69] +- InMemoryRelation [key#68, value#69], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt` +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,value:string> | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ ``` Author: Yadong Qi <qiyadong2010@gmail.com> Closes #14913 from watermen/SPARK-17358.
* [SPARK-17369][SQL] MetastoreRelation toJSON throws AssertException due to ↵Sean Zhong2016-09-062-1/+40
| | | | | | | | | | | | | | | | | | missing otherCopyArgs ## What changes were proposed in this pull request? `TreeNode.toJSON` requires a subclass to explicitly override otherCopyArgs to include currying construction arguments, otherwise it reports AssertException telling that the construction argument values' count doesn't match the construction argument names' count. For class `MetastoreRelation`, it has a currying construction parameter `client: HiveClient`, but Spark forgets to add it to the list of otherCopyArgs. ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #14928 from clockfly/metastore_relation_toJSON.
* [SPARK-17279][SQL] better error message for exceptions during ScalaUDF executionWenchen Fan2016-09-063-22/+86
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? If `ScalaUDF` throws exceptions during executing user code, sometimes it's hard for users to figure out what's wrong, especially when they use Spark shell. An example ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 12 in stage 325.0 failed 4 times, most recent failure: Lost task 12.3 in stage 325.0 (TID 35622, 10.0.207.202): java.lang.NullPointerException at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(<console>:40) at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(<console>:40) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) ... ``` We should catch these exceptions and rethrow them with better error message, to say that the exception is happened in scala udf. This PR also does some clean up for `ScalaUDF` and add a unit test suite for it. ## How was this patch tested? the new test suite Author: Wenchen Fan <wenchen@databricks.com> Closes #14850 from cloud-fan/npe.
* [SPARK-17072][SQL] support table-level statistics generation and storing ↵wangzhenhua2016-09-0516-108/+363
| | | | | | | | | | | | | | | | | | | | | | | into/loading from metastore ## What changes were proposed in this pull request? 1. Support generation table-level statistics for - hive tables in HiveExternalCatalog - data source tables in HiveExternalCatalog - data source tables in InMemoryCatalog. 2. Add a property "catalogStats" in CatalogTable to hold statistics in Spark side. 3. Put logics of statistics transformation between Spark and Hive in HiveClientImpl. 4. Extend Statistics class by adding rowCount (will add estimatedSize when we have column stats). ## How was this patch tested? add unit tests Author: wangzhenhua <wangzhenhua@huawei.com> Author: Zhenhua Wang <wangzhenhua@huawei.com> Closes #14712 from wzhfy/tableStats.
* [SPARK-17394][SQL] should not allow specify database in table/view name ↵Wenchen Fan2016-09-056-72/+32
| | | | | | | | | | | | | | | | | | | after RENAME TO ## What changes were proposed in this pull request? It's really weird that we allow users to specify database in both from table name and to table name in `ALTER TABLE RENAME TO`, while logically we can't support rename a table to a different database. Both postgres and MySQL disallow this syntax, it's reasonable to follow them and simply our code. ## How was this patch tested? new test in `DDLCommandSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14955 from cloud-fan/rename.
* [SPARK-17393][SQL] Error Handling when CTAS Against the Same Data Source ↵gatorsmile2016-09-052-26/+71
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Table Using Overwrite Mode ### What changes were proposed in this pull request? When we trying to read a table and then write to the same table using the `Overwrite` save mode, we got a very confusing error message: For example, ```Scala Seq((1, 2)).toDF("i", "j").write.saveAsTable("tab1") table("tab1").write.mode(SaveMode.Overwrite).saveAsTable("tab1") ``` ``` Job aborted. org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1.apply$mcV$sp ... Caused by: org.apache.spark.SparkException: Task failed while writing rows at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:266) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143) at org.apache.spark.sql.execution.datasources ``` After the PR, we will issue an `AnalysisException`: ``` Cannot overwrite table `tab1` that is also being read from ``` ### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #14954 from gatorsmile/ctasQueryAnalyze.
* [MINOR][ML][MLLIB] Remove work around for breeze sparse matrix.Yanbo Liang2016-09-042-18/+2
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Since we have updated breeze version to 0.12, we should remove work around for bug of breeze sparse matrix in v0.11. I checked all mllib code and found this is the only work around for breeze 0.11. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14953 from yanboliang/matrices.
* [SPARK-17311][MLLIB] Standardize Python-Java MLlib API to accept optional ↵Sean Owen2016-09-042-11/+13
| | | | | | | | | | | | | | | | | | | | long seeds in all cases ## What changes were proposed in this pull request? Related to https://github.com/apache/spark/pull/14524 -- just the 'fix' rather than a behavior change. - PythonMLlibAPI methods that take a seed now always take a `java.lang.Long` consistently, allowing the Python API to specify "no seed" - .mllib's Word2VecModel seemed to be an odd man out in .mllib in that it picked its own random seed. Instead it defaults to None, meaning, letting the Scala implementation pick a seed - BisectingKMeansModel arguably should not hard-code a seed for consistency with .mllib, I think. However I left it. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #14826 from srowen/SPARK-16832.2.
* [SPARK-17308] Improved the spark core code by replacing all pattern match on ↵Shivansh2016-09-047-43/+44
| | | | | | | | | | | | | | | boolean value by if/else block. ## What changes were proposed in this pull request? Improved the code quality of spark by replacing all pattern match on boolean value by if/else block. ## How was this patch tested? By running the tests Author: Shivansh <shiv4nsh@gmail.com> Closes #14873 from shiv4nsh/SPARK-17308.
* [SPARK-17324][SQL] Remove Direct Usage of HiveClient in InsertIntoHiveTablegatorsmile2016-09-0410-68/+168
| | | | | | | | | | | | ### What changes were proposed in this pull request? This is another step to get rid of HiveClient from `HiveSessionState`. All the metastore interactions should be through `ExternalCatalog` interface. However, the existing implementation of `InsertIntoHiveTable ` still requires Hive clients. This PR is to remove HiveClient by moving the metastore interactions into `ExternalCatalog`. ### How was this patch tested? Existing test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14888 from gatorsmile/removeClientFromInsertIntoHiveTable.
* [SPARK-16829][SPARKR] sparkR sc.setLogLevel doesn't workwm624@hotmail.com2016-09-031-1/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | (Please fill in changes proposed in this fix) ./bin/sparkR Launching java with spark-submit command /Users/mwang/spark_ws_0904/bin/spark-submit "sparkr-shell" /var/folders/s_/83b0sgvj2kl2kwq4stvft_pm0000gn/T//RtmpQxJGiZ/backend_porte9474603ed1e Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). > sc.setLogLevel("INFO") Error: could not find function "sc.setLogLevel" sc.setLogLevel doesn't exist. R has a function setLogLevel. I rename the setLogLevel function to sc.setLogLevel. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Change unit test. Run unit tests. Manually tested it in sparkR shell. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #14433 from wangmiao1981/sc.
* [SPARK-17315][SPARKR] Kolmogorov-Smirnov test SparkR wrapperJunyang Qian2016-09-035-2/+205
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR tries to add Kolmogorov-Smirnov Test wrapper to SparkR. This wrapper implementation only supports one sample test against normal distribution. ## How was this patch tested? R unit test. Author: Junyang Qian <junyangq@databricks.com> Closes #14881 from junyangq/SPARK-17315.
* [SPARK-17335][SQL] Fix ArrayType and MapType CatalogString.Herman van Hovell2016-09-034-75/+133
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? the `catalogString` for `ArrayType` and `MapType` currently calls the `simpleString` method on its children. This is a problem when the child is a struct, the `struct.simpleString` implementation truncates the number of fields it shows (25 at max). This breaks the generation of a proper `catalogString`, and has shown to cause errors while writing to Hive. This PR fixes this by providing proper `catalogString` implementations for `ArrayData` or `MapData`. ## How was this patch tested? Added testing for `catalogString` to `DataTypeSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14938 from hvanhovell/SPARK-17335.
* [MINOR][SQL] Not dropping all necessary tablesSandeep Singh2016-09-031-7/+6
| | | | | | | | | | | | ## What changes were proposed in this pull request? was not dropping table `parquet_t3` ## How was this patch tested? tested `LogicalPlanToSQLSuite` locally Author: Sandeep Singh <sandeep@techaddict.me> Closes #13767 from techaddict/minor-8.
* [SPARK-17347][SQL][EXAMPLES] Encoder in Dataset example has incorrect typeCodingCat2016-09-031-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We propose to fix the Encoder type in the Dataset example ## How was this patch tested? The PR will be tested with the current unit test cases Author: CodingCat <zhunansjtu@gmail.com> Closes #14901 from CodingCat/SPARK-17347.
* [SPARK-17363][ML][MLLIB] fix MultivariantOnlineSummerizer.numNonZerosWeichenXu2016-09-032-3/+3
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? fix `MultivariantOnlineSummerizer.numNonZeros` method, return `nnz` array, instead of `weightSum` array ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14923 from WeichenXu123/fix_MultivariantOnlineSummerizer_numNonZeros.
* [SPARKR][MINOR] Fix docs for sparkR.session and countJunyang Qian2016-09-023-4/+7
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR tries to add some more explanation to `sparkR.session`. It also modifies doc for `count` so when grouped in one doc, the description doesn't confuse users. ## How was this patch tested? Manual test. ![screen shot 2016-09-02 at 1 21 36 pm](https://cloud.githubusercontent.com/assets/15318264/18217198/409613ac-7110-11e6-8dae-cb0c8df557bf.png) Author: Junyang Qian <junyangq@databricks.com> Closes #14942 from junyangq/fixSparkRSessionDoc.
* [SPARK-17298][SQL] Require explicit CROSS join for cartesian productsSrinath Shankar2016-09-0342-127/+465
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Require the use of CROSS join syntax in SQL (and a new crossJoin DataFrame API) to specify explicit cartesian products between relations. By cartesian product we mean a join between relations R and S where there is no join condition involving columns from both R and S. If a cartesian product is detected in the absence of an explicit CROSS join, an error must be thrown. Turning on the "spark.sql.crossJoin.enabled" configuration flag will disable this check and allow cartesian products without an explicit CROSS join. The new crossJoin DataFrame API must be used to specify explicit cross joins. The existing join(DataFrame) method will produce a INNER join that will require a subsequent join condition. That is df1.join(df2) is equivalent to select * from df1, df2. ## How was this patch tested? Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used. Author: Srinath Shankar <srinath@databricks.com> Closes #14866 from srinathshankar/crossjoin.
* [SPARK-16334] Reusing same dictionary column for decoding consecutive row ↵Sameer Agarwal2016-09-021-16/+38
| | | | | | | | | | | | | | | | groups shouldn't throw an error ## What changes were proposed in this pull request? This patch fixes a bug in the vectorized parquet reader that's caused by re-using the same dictionary column vector while reading consecutive row groups. Specifically, this issue manifests for a certain distribution of dictionary/plain encoded data while we read/populate the underlying bit packed dictionary data into a column-vector based data structure. ## How was this patch tested? Manually tested on datasets provided by the community. Thanks to Chris Perluss and Keith Kraus for their invaluable help in tracking down this issue! Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14941 from sameeragarwal/parquet-exception-2.
* [SPARK-17230] [SQL] Should not pass optimized query into QueryExecution in ↵Davies Liu2016-09-027-6/+25
| | | | | | | | | | | | | | | | | | | | | | | | DataFrameWriter ## What changes were proposed in this pull request? Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. Ideally, we should make all the analyzer rules all idempotent, that may require lots of effort to double checking them one by one (may be not easy). An easier approach could be never feed a optimized plan into Analyzer, this PR fix the case for RunnableComand, they will be optimized, during execution, the passed `query` will also be passed into QueryExecution again. This PR make these `query` not part of the children, so they will not be optimized and analyzed again. Right now, we did not know a logical plan is optimized or not, we could introduce a flag for that, and make sure a optimized logical plan will not be analyzed again. ## How was this patch tested? Added regression tests. Author: Davies Liu <davies@databricks.com> Closes #14797 from davies/fix_writer.
* [SPARK-17376][SPARKR] followup - change since versionFelix Cheung2016-09-021-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? change since version in doc ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #14939 from felixcheung/rsparkversion2.
* [SPARK-16711] YarnShuffleService doesn't re-init properly on YARN rolling ↵Thomas Graves2016-09-026-138/+311
| | | | | | | | | | | | | | upgrade The Spark Yarn Shuffle Service doesn't re-initialize the application credentials early enough which causes any other spark executors trying to fetch from that node during a rolling upgrade to fail with "java.lang.NullPointerException: Password cannot be null if SASL is enabled". Right now the spark shuffle service relies on the Yarn nodemanager to re-register the applications, unfortunately this is after we open the port for other executors to connect. If other executors connected before the re-register they get a null pointer exception which isn't a re-tryable exception and cause them to fail pretty quickly. To solve this I added another leveldb file so that it can save and re-initialize all the applications before opening the port for other executors to connect to it. Adding another leveldb was simpler from the code structure point of view. Most of the code changes are moving things to common util class. Patch was tested manually on a Yarn cluster with rolling upgrade was happing while spark job was running. Without the patch I consistently get the NullPointerException, with the patch the job gets a few Connection refused exceptions but the retries kick in and the it succeeds. Author: Thomas Graves <tgraves@staydecay.corp.gq1.yahoo.com> Closes #14718 from tgravescs/SPARK-16711.
* [SPARKR][DOC] regexp_extract should doc that it returns empty string when ↵Felix Cheung2016-09-021-1/+2
| | | | | | | | | | | | | | | | match fails ## What changes were proposed in this pull request? Doc change - see https://issues.apache.org/jira/browse/SPARK-16324 ## How was this patch tested? manual check Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #14934 from felixcheung/regexpextractdoc.
* [SPARK-17376][SPARKR] Spark version should be available in RFelix Cheung2016-09-023-6/+32
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add sparkR.version() API. ``` > sparkR.version() [1] "2.1.0-SNAPSHOT" ``` ## How was this patch tested? manual, unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #14935 from felixcheung/rsparksessionversion.
* [SPARK-17261] [PYSPARK] Using HiveContext after re-creating SparkContext in ↵Jeff Zhang2016-09-021-0/+1
| | | | | | | | | | | | | | | | | | | | | Spark 2.0 throws "Java.lang.illegalStateException: Cannot call methods on a stopped sparkContext" ## What changes were proposed in this pull request? Set SparkSession._instantiatedContext as None so that we can recreate SparkSession again. ## How was this patch tested? Tested manually using the following command in pyspark shell ``` spark.stop() spark = SparkSession.builder.enableHiveSupport().getOrCreate() spark.sql("show databases").show() ``` Author: Jeff Zhang <zjffdu@apache.org> Closes #14857 from zjffdu/SPARK-17261.