- Feb 02, 2016
-
-
Wenchen Fan authored
when we generate map, we first randomly pick a length, then create a seq of key value pair with the expected length, and finally call `toMap`. However, `toMap` will remove all duplicated keys, which makes the actual map size much less than we expected. This PR fixes this problem by put keys in a set first, to guarantee we have enough keys to build a map with expected length. Author: Wenchen Fan <wenchen@databricks.com> Closes #10930 from cloud-fan/random-generator.
-
Davies Liu authored
Author: Davies Liu <davies@databricks.com> Closes #11037 from davies/disable_flaky.
-
Kevin (Sangwoo) Kim authored
The example will throw error like <console>:20: error: not found: value StructType Need to add this line: import org.apache.spark.sql.types._ Author: Kevin (Sangwoo) Kim <sangwookim.me@gmail.com> Closes #10141 from swkimme/patch-1.
-
Gabriele Nizzoli authored
Already merged into 1.6 branch, this PR is to commit to master the same change Author: Gabriele Nizzoli <mail@nizzoli.net> Closes #11028 from gabrielenizzoli/patch-1.
-
Davies Liu authored
As benchmarked and discussed here: https://github.com/apache/spark/pull/10786/files#r50038294, benefits from codegen, the declarative aggregate function could be much faster than imperative one. Author: Davies Liu <davies@databricks.com> Closes #10960 from davies/stddev.
-
Reynold Xin authored
ddl.scala is defined in the execution package, and yet its reference of "UnaryNode" and "Command" are logical. This was fairly confusing when I was trying to understand the ddl code. Author: Reynold Xin <rxin@databricks.com> Closes #11021 from rxin/SPARK-13138.
-
Grzegorz Chilkiewicz authored
Fixes problem and verifies fix by test suite. Also - adds optional parameter: nullable (Boolean) to: SchemaUtils.appendColumn and deduplicates SchemaUtils.appendColumn functions. Author: Grzegorz Chilkiewicz <grzegorz.chilkiewicz@codilime.com> Closes #10741 from grzegorz-chilkiewicz/master.
-
Daoyuan Wang authored
Jira: https://issues.apache.org/jira/browse/SPARK-13056 Create a map like { "a": "somestring", "b": null} Query like SELECT col["b"] FROM t1; NPE would be thrown. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #10964 from adrian-wang/npewriter.
-
Bryan Cutler authored
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent. This is for the clustering module. Author: Bryan Cutler <cutlerb@gmail.com> Closes #10610 from BryanCutler/param-desc-consistent-cluster-SPARK-12631.
-
hyukjinkwon authored
https://issues.apache.org/jira/browse/SPARK-13114 This PR adds a test for tokens more than the fields in schema. Author: hyukjinkwon <gurwls223@gmail.com> Closes #11020 from HyukjinKwon/SPARK-13114.
-
Michael Armbrust authored
Author: Michael Armbrust <michael@databricks.com> Closes #11014 from marmbrus/seqEncoders.
-
Michael Armbrust authored
This is a follow up to 9aadcffa that extends Spark SQL to allow users to _repeatedly_ optimize and execute structured queries. A `ContinuousQuery` can be expressed using SQL, DataFrames or Datasets. The purpose of this PR is only to add some initial infrastructure which will be extended in subsequent PRs. ## User-facing API - `sqlContext.streamFrom` and `df.streamTo` return builder objects that are analogous to the `read/write` interfaces already available to executing queries in a batch-oriented fashion. - `ContinuousQuery` provides an interface for interacting with a query that is currently executing in the background. ## Internal Interfaces - `StreamExecution` - executes streaming queries in micro-batches The following are currently internal, but public APIs will be provided in a future release. - `Source` - an interface for providers of continually arriving data. A source must have a notion of an `Offset` that monotonically tracks what data has arrived. For fault tolerance, a source must be able to replay data given a start offset. - `Sink` - an interface that accepts the results of a continuously executing query. Also responsible for tracking the offset that should be resumed from in the case of a failure. ## Testing - `MemoryStream` and `MemorySink` - simple implementations of source and sink that keep all data in memory and have methods for simulating durability failures - `StreamTest` - a framework for performing actions and checking invariants on a continuous query Author: Michael Armbrust <michael@databricks.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <rosenville@gmail.com> Closes #11006 from marmbrus/structured-streaming.
-
Michael Armbrust authored
It is not valid to call `toAttribute` on a `NamedExpression` unless we know for sure that the child produced that `NamedExpression`. The current code worked fine when the grouping expressions were simple, but when they were a derived value this blew up at execution time. Author: Michael Armbrust <michael@databricks.com> Closes #11013 from marmbrus/groupByFunction-master.
-
Reynold Xin authored
-
Reynold Xin authored
1. Use lower case 2. Change long prefixes to something shorter (in this case I am changing only one: TungstenAggregate -> agg). Author: Reynold Xin <rxin@databricks.com> Closes #11017 from rxin/SPARK-13130.
-
- Feb 01, 2016
-
-
felixcheung authored
Removed isLegacyLogDirectory code path and updated tests andrewor14 Author: felixcheung <felixcheung_m@hotmail.com> Closes #10860 from felixcheung/historyserverformat.
-
Sean Owen authored
Improve printing of StageInfo in onStageCompleted See also https://github.com/apache/spark/pull/10585 Author: Sean Owen <sowen@cloudera.com> Closes #10922 from srowen/SPARK-12637.
-
Reynold Xin authored
This pull request creates an internal catalog API. The creation of this API is the first step towards consolidating SQLContext and HiveContext. I envision we will have two different implementations in Spark 2.0: (1) a simple in-memory implementation, and (2) an implementation based on the current HiveClient (ClientWrapper). I took a look at what Hive's internal metastore interface/implementation, and then created this API based on it. I believe this is the minimal set needed in order to achieve all the needed functionality. Author: Reynold Xin <rxin@databricks.com> Closes #10982 from rxin/SPARK-13078.
-
Jacek Laskowski authored
…rBatch Fixes build for Scala 2.11. Author: Jacek Laskowski <jacek@japila.pl> Closes #10946 from jaceklaskowski/SPARK-12854-fix.
-
Nong Li authored
This includes: float, boolean, short, decimal and calendar interval. Decimal is mapped to long or byte array depending on the size and calendar interval is mapped to a struct of int and long. The only remaining type is map. The schema mapping is straightforward but we might want to revisit how we deal with this in the rest of the execution engine. Author: Nong Li <nong@databricks.com> Closes #10961 from nongli/spark-13043.
-
Iulian Dragos authored
The driver filesystem is likely different from where the executors will run, so resolving paths (and symlinks, etc.) will lead to invalid paths on executors. Author: Iulian Dragos <jaguarul@gmail.com> Closes #10923 from dragos/issue/canonical-paths.
-
Nilanjan Raychaudhuri authored
This takes over #10729 and makes sure that `spark-shell` fails with a proper error message. There is a slight behavioral change: before this change `spark-shell` would exit, while now the REPL is still there, but `sc` and `sqlContext` are not defined and the error is visible to the user. Author: Nilanjan Raychaudhuri <nraychaudhuri@gmail.com> Author: Iulian Dragos <jaguarul@gmail.com> Closes #10921 from dragos/pr/10729.
-
Timothy Chen authored
[SPARK-12463][SPARK-12464][SPARK-12465][SPARK-10647][MESOS] Fix zookeeper dir with mesos conf and add docs. Fix zookeeper dir configuration used in cluster mode, and also add documentation around these settings. Author: Timothy Chen <tnachen@gmail.com> Closes #10057 from tnachen/fix_mesos_dir.
-
Lewuathe authored
In [ml-guide](https://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation), there is invalid reference to `MulticlassClassificationEvaluator` apidoc. https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.ml.evaluation.MultiClassClassificationEvaluator Author: Lewuathe <lewuathe@me.com> Closes #10996 from Lewuathe/fix-typo-in-ml-guide.
-
Takeshi YAMAMURO authored
ISTM `lib` is better because `datanucleus` jars are located in `lib` for release builds. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #10901 from maropu/DocFix.
-
gatorsmile authored
JIRA: https://issues.apache.org/jira/browse/SPARK-12705 **Scope:** This PR is a general fix for sorting reference resolution when the child's `outputSet` does not have the order-by attributes (called, *missing attributes*): - UnaryNode support is limited to `Project`, `Window`, `Aggregate`, `Distinct`, `Filter`, `RepartitionByExpression`. - We will not try to resolve the missing references inside a subquery, unless the outputSet of this subquery contains it. **General Reference Resolution Rules:** - Jump over the nodes with the following types: `Distinct`, `Filter`, `RepartitionByExpression`. Do not need to add missing attributes. The reason is their `outputSet` is decided by their `inputSet`, which is the `outputSet` of their children. - Group-by expressions in `Aggregate`: missing order-by attributes are not allowed to be added into group-by expressions since it will change the query result. Thus, in RDBMS, it is not allowed. - Aggregate expressions in `Aggregate`: if the group-by expressions in `Aggregate` contains the missing attributes but aggregate expressions do not have it, just add them into the aggregate expressions. This can resolve the analysisExceptions thrown by the three TCPDS queries. - `Project` and `Window` are special. We just need to add the missing attributes to their `projectList`. **Implementation:** 1. Traverse the whole tree in a pre-order manner to find all the resolvable missing order-by attributes. 2. Traverse the whole tree in a post-order manner to add the found missing order-by attributes to the node if their `inputSet` contains the attributes. 3. If the origins of the missing order-by attributes are different nodes, each pass only resolves the missing attributes that are from the same node. **Risk:** Low. This rule will be trigger iff ```!s.resolved && child.resolved``` is true. Thus, very few cases are affected. Author: gatorsmile <gatorsmile@gmail.com> Closes #10678 from gatorsmile/sortWindows.
-
gatorsmile authored
JIRA: https://issues.apache.org/jira/browse/SPARK-12989 In the rule `ExtractWindowExpressions`, we simply replace alias by the corresponding attribute. However, this will cause an issue exposed by the following case: ```scala val data = Seq(("a", "b", "c", 3), ("c", "b", "a", 3)).toDF("A", "B", "C", "num") .withColumn("Data", struct("A", "B", "C")) .drop("A") .drop("B") .drop("C") val winSpec = Window.partitionBy("Data.A", "Data.B").orderBy($"num".desc) data.select($"*", max("num").over(winSpec) as "max").explain(true) ``` In this case, both `Data.A` and `Data.B` are `alias` in `WindowSpecDefinition`. If we replace these alias expression by their alias names, we are unable to know what they are since they will not be put in `missingExpr` too. Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #10963 from gatorsmile/seletStarAfterColDrop.
-
Shixiong Zhu authored
[SPARK-6847][CORE][STREAMING] Fix stack overflow issue when updateStateByKey is followed by a checkpointed dstream Add a local property to indicate if checkpointing all RDDs that are marked with the checkpoint flag, and enable it in Streaming Author: Shixiong Zhu <shixiong@databricks.com> Closes #10934 from zsxwing/recursive-checkpoint.
-
Wenchen Fan authored
The current implementation is sub-optimal: * If an expression is always nullable, e.g. `Unhex`, we can still remove null check for children if they are not nullable. * If an expression has some non-nullable children, we can still remove null check for these children and keep null check for others. This PR improves this by making the null check elimination more fine-grained. Author: Wenchen Fan <wenchen@databricks.com> Closes #10987 from cloud-fan/null-check.
-
- Jan 31, 2016
-
-
Herman van Hovell authored
This PR adds the ability to specify the ```ignoreNulls``` option to the functions dsl, e.g: ```df.select($"id", last($"value", ignoreNulls = true).over(Window.partitionBy($"id").orderBy($"other"))``` This PR is some where between a bug fix (see the JIRA) and a new feature. I am not sure if we should backport to 1.6. cc yhuai Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #10957 from hvanhovell/SPARK-13049.
-
Liang-Chi Hsieh authored
JIRA: https://issues.apache.org/jira/browse/SPARK-12689 DDLParser processes three commands: createTable, describeTable and refreshTable. This patch migrates the three commands to newly absorbed parser. Author: Liang-Chi Hsieh <viirya@gmail.com> Author: Liang-Chi Hsieh <viirya@appier.com> Closes #10723 from viirya/migrate-ddl-describe.
-
Cheng Lian authored
Make sure we throw better error messages when Parquet schema merging fails. Author: Cheng Lian <lian@databricks.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #10979 from viirya/schema-merging-failure-message.
-
- Jan 30, 2016
-
-
wangyang authored
In jdk1.7 TimeZone.getTimeZone() is synchronized, so use an instance variable to hold an GMT TimeZone object instead of instantiate it every time. Author: wangyang <wangyang@haizhi.com> Closes #10994 from wangyang1992/datetimeUtil.
-
Josh Rosen authored
This patch changes Spark's build to make Scala 2.11 the default Scala version. To be clear, this does not mean that Spark will stop supporting Scala 2.10: users will still be able to compile Spark for Scala 2.10 by following the instructions on the "Building Spark" page; however, it does mean that Scala 2.11 will be the default Scala version used by our CI builds (including pull request builds). The Scala 2.11 compiler is faster than 2.10, so I think we'll be able to look forward to a slight speedup in our CI builds (it looks like it's about 2X faster for the Maven compile-only builds, for instance). After this patch is merged, I'll update Jenkins to add new compile-only jobs to ensure that Scala 2.10 compilation doesn't break. Author: Josh Rosen <joshrosen@databricks.com> Closes #10608 from JoshRosen/SPARK-6363.
-
Wenchen Fan authored
This class is only used for serialization of Python DataFrame. However, we don't require internal row there, so `GenericRowWithSchema` can also do the job. Author: Wenchen Fan <wenchen@databricks.com> Closes #10992 from cloud-fan/python.
-
- Jan 29, 2016
-
-
Davies Liu authored
This PR add support for grouping keys for generated TungstenAggregate. Spilling and performance improvements for BytesToBytesMap will be done by followup PR. Author: Davies Liu <davies@databricks.com> Closes #10855 from davies/gen_keys.
-
Andrew Or authored
This issue is causing tests to fail consistently in master with Hadoop 2.6 / 2.7. This is because for Hadoop 2.5+ we overwrite existing values of `InputMetrics#bytesRead` in each call to `HadoopRDD#compute`. In the case of coalesce, e.g. ``` sc.textFile(..., 4).coalesce(2).count() ``` we will call `compute` multiple times in the same task, overwriting `bytesRead` values from previous calls to `compute`. For a regression test, see `InputOutputMetricsSuite.input metrics for old hadoop with coalesce`. I did not add a new regression test because it's impossible without significant refactoring; there's a lot of existing duplicate code in this corner of Spark. This was caused by #10835. Author: Andrew Or <andrew@databricks.com> Closes #10973 from andrewor14/fix-input-metrics-coalesce.
-
Andrew Or authored
Apparently chrome removed `SVGElement.prototype.getTransformToElement`, which is used by our JS library dagre-d3 when creating edges. The real diff can be found here: https://github.com/andrewor14/dagre-d3/commit/7d6c0002e4c74b82a02c5917876576f71e215590, which is taken from the fix in the main repo: https://github.com/cpettitt/dagre-d3/commit/1ef067f1c6ad2e0980f6f0ca471bce998784b7b2 Upstream issue: https://github.com/cpettitt/dagre-d3/issues/202 Author: Andrew Or <andrew@databricks.com> Closes #10986 from andrewor14/fix-dag-viz.
-
Andrew Or authored
Previously we would assert things before all events are guaranteed to have been processed. To fix this, just block until all events are actually processed, i.e. until the listener queue is empty. https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/79/testReport/junit/org.apache.spark.util.collection/ExternalAppendOnlyMapSuite/spilling/ Author: Andrew Or <andrew@databricks.com> Closes #10990 from andrewor14/accum-suite-less-flaky.
-
Reynold Xin authored
And ClientWrapper -> HiveClientImpl. I have some followup pull requests to introduce a new internal catalog, and I think this new naming reflects better the functionality of the two classes. Author: Reynold Xin <rxin@databricks.com> Closes #10981 from rxin/SPARK-13076.
-