- Jan 16, 2015
-
-
Reynold Xin authored
After the following patches, the main (Scala) API is now usable for Java users directly. https://github.com/apache/spark/pull/4056 https://github.com/apache/spark/pull/4054 https://github.com/apache/spark/pull/4049 https://github.com/apache/spark/pull/4030 https://github.com/apache/spark/pull/3965 https://github.com/apache/spark/pull/3958 Author: Reynold Xin <rxin@databricks.com> Closes #4065 from rxin/sql-java-api and squashes the following commits: b1fd860 [Reynold Xin] Fix Mima 6d86578 [Reynold Xin] Ok one more attempt in fixing Python... e8f1455 [Reynold Xin] Fix Python again... 3e53f91 [Reynold Xin] Fixed Python. 83735da [Reynold Xin] Fix BigDecimal test. e9f1de3 [Reynold Xin] Use scala BigDecimal. 500d2c4 [Reynold Xin] Fix Decimal. ba3bfa2 [Reynold Xin] Updated javadoc for RowFactory. c4ae1c5 [Reynold Xin] [SPARK-5193][SQL] Remove Spark SQL Java-specific API.
-
scwf authored
Adding optimization to simplify the And/Or condition in spark sql. There are two kinds of Optimization 1 Numeric condition optimization, such as: a < 3 && a > 5 ---- False a < 1 || a > 0 ---- True a > 3 && a > 5 => a > 5 (a < 2 || b > 5) && a < 2 => a < 2 2 optimizing the some query from a cartesian product into equi-join, such as this sql (one of hive-testbench): ``` select sum(l_extendedprice* (1 - l_discount)) as revenue from lineitem, part where ( p_partkey = l_partkey and p_brand = 'Brand#32' and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') and l_quantity >= 7 and l_quantity <= 7 + 10 and p_size between 1 and 5 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' ) or ( p_partkey = l_partkey and p_brand = 'Brand#35' and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') and l_quantity >= 15 and l_quantity <= 15 + 10 and p_size between 1 and 10 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' ) or ( p_partkey = l_partkey and p_brand = 'Brand#24' and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') and l_quantity >= 26 and l_quantity <= 26 + 10 and p_size between 1 and 15 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' ) ``` It has a repeated expression in Or, so we can optimize it by ``` (a && b) || (a && c) = a && (b || c)``` Before optimization, this sql hang in my locally test, and the physical plan is:  After optimization, this sql run successfully in 20+ seconds, and its physical plan is:  This PR focus on the second optimization and some simple ones of the first. For complex Numeric condition optimization, I will make a follow up PR. Author: scwf <wangfei1@huawei.com> Author: wangfei <wangfei1@huawei.com> Closes #3778 from scwf/filter1 and squashes the following commits: 58bcbc2 [scwf] minor format fix 9570211 [scwf] conflicts fix 527e6ce [scwf] minor comment improvements 5c6f134 [scwf] remove numeric optimizations and move to BooleanSimplification 546a82b [wangfei] style fix 825fa69 [wangfei] adding more tests a001e8c [wangfei] revert pom changes 32a595b [scwf] improvement and test fix e99a26c [wangfei] refactory And/Or optimization to make it more readable and clean
-
Ilya Ganelin authored
I've added documentation clarifying the particular lack of clarity highlighted in the relevant JIRA. I've also added code examples for this issue to clarify the explanation. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #4022 from ilganeli/SPARK-733 and squashes the following commits: 587def5 [Ilya Ganelin] Updated to clarify verbage df3afd7 [Ilya Ganelin] Revert "Partially updated task metrics to make some vars private" 3f6c512 [Ilya Ganelin] Revert "Completed refactoring to make vars in TaskMetrics class private" 58034fb [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-733 4dc2cdb [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-733 3a38db1 [Ilya Ganelin] Verified documentation update by building via jekyll 33b5a2d [Ilya Ganelin] Added code examples for java and python 1fd59b2 [Ilya Ganelin] Updated documentation for accumulators to highlight lazy evaluation issue 5525c20 [Ilya Ganelin] Completed refactoring to make vars in TaskMetrics class private c64da4f [Ilya Ganelin] Partially updated task metrics to make some vars private
-
Chip Senkbeil authored
As requested in [SPARK-4923](https://issues.apache.org/jira/browse/SPARK-4923), I've provided a rough DeveloperApi for the repl. I've only done this for Scala 2.10 because it does not appear that Scala 2.11 is implemented. The Scala 2.11 repl still has the old `scala.tools.nsc` package and the SparkIMain does not appear to have the class server needed for shipping code over (unless this functionality has been moved elsewhere?). I also left alone the `ExecutorClassLoader` and `ConstructorCleaner` as I have no experience working with those classes. This marks the majority of methods in `SparkIMain` as _private_ with a few special cases being _private[repl]_ as other classes within the same package access them. Any public method has been marked with `DeveloperApi` as suggested by pwendell and I took the liberty of writing up a Scaladoc for each one to further elaborate their usage. As the Scala 2.11 REPL [conforms]((https://github.com/scala/scala/pull/2206)) to [JSR-223](http://docs.oracle.com/javase/8/docs/technotes/guides/scripting/), the [Spark Kernel](https://github.com/ibm-et/spark-kernel) uses the SparkIMain of Scala 2.10 in the same manner. So, I've taken care to expose methods predominately related to necessary functionality towards a JSR-223 scripting engine implementation. 1. The ability to _get_ variables from the interpreter (and other information like class/symbol/type) 2. The ability to _put_ variables into the interpreter 3. The ability to _compile_ code 4. The ability to _execute_ code 5. The ability to get contextual information regarding the scripting environment Additional functionality that I marked as exposed included the following: 1. The blocking initialization method (needed to actually start SparkIMain instance) 2. The class server uri (needed to set the _spark.repl.class.uri_ property after initialization), reduced from the entire class server 3. The class output directory (beneficial for tools like ours that need to inspect and use the directory where class files are served) 4. Suppression (quiet/silence) mechanics for output 5. Ability to add a jar to the compile/runtime classpath 6. The reset/close functionality 7. Metric information (last variable assignment, "needed" for extracting results from last execution, real variable name for better debugging) 8. Execution wrapper (useful to have, but debatable) Aside from `SparkIMain`, I updated other classes/traits and their methods in the _repl_ package to be private/package protected where possible. A few odd cases (like the SparkHelper being in the scala.tools.nsc package to expose a private variable) still exist, but I did my best at labelling them. `SparkCommandLine` has proven useful to extract settings and `SparkJLineCompletion` has proven to be useful in implementing auto-completion in the [Spark Kernel](https://github.com/ibm-et/spark-kernel) project. Other than those - and `SparkIMain` - my experience has yielded that other classes/methods are not necessary for interactive applications taking advantage of the REPL API. Tested via the following: $ export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" $ mvn -Phadoop-2.3 -DskipTests clean package && mvn -Phadoop-2.3 test Also did a quick verification that I could start the shell and execute some code: $ ./bin/spark-shell ... scala> val x = 3 x: Int = 3 scala> sc.parallelize(1 to 10).reduce(_+_) ... res1: Int = 55 Author: Chip Senkbeil <rcsenkbe@us.ibm.com> Author: Chip Senkbeil <chip.senkbeil@gmail.com> Closes #4034 from rcsenkbeil/AddDeveloperApiToRepl and squashes the following commits: 053ca75 [Chip Senkbeil] Fixed failed build by adding missing DeveloperApi import c1b88aa [Chip Senkbeil] Added DeveloperApi to public classes in repl 6dc1ee2 [Chip Senkbeil] Added missing method to expose error reporting flag 26fd286 [Chip Senkbeil] Refactored other Scala 2.10 classes and methods to be private/package protected where possible 925c112 [Chip Senkbeil] Added DeveloperApi and Scaladocs to SparkIMain for Scala 2.10
-
Kousuke Saruta authored
When we decrease the width of browsers, the header of WebUI wraps and collapses like as following image.  Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #3995 from sarutak/fixed-collapse-webui-layout and squashes the following commits: 3e60b5b [Kousuke Saruta] Modified line-height property in webui.css 7bfb5fb [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fixed-collapse-webui-layout 5d83e18 [Kousuke Saruta] Fixed collapse of WebUI layout
-
Kousuke Saruta authored
History Server doesn't show collect job submission time. It's because `JobProgressListener` updates job submission time every time `onJobStart` method is invoked from `ReplayListenerBus`. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #4029 from sarutak/SPARK-5231 and squashes the following commits: 0af9e22 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5231 da8bd14 [Kousuke Saruta] Made submissionTime in SparkListenerJobStartas and completionTime in SparkListenerJobEnd as regular Long 0412a6a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5231 26b9b99 [Kousuke Saruta] Fixed the test cases 2d47bd3 [Kousuke Saruta] Fixed to record job submission time and completion time collectly
-
Sean Owen authored
This fixes a simple typo in the cogroup docs noted in http://mail-archives.apache.org/mod_mbox/spark-user/201501.mbox/%3CCAMAsSdJ8_24evMAMg7fOZCQjwimisbYWa9v8BN6Rc3JCauja6wmail.gmail.com%3E I didn't bother with a JIRA Author: Sean Owen <sowen@cloudera.com> Closes #4072 from srowen/CogroupDocFix and squashes the following commits: 43c850b [Sean Owen] Fix typo in return type of cogroup
-
Ye Xianjin authored
There is an int overflow in the ParallelCollectionRDD.slice method. That's originally reported by SaintBacchus. ``` sc.makeRDD(1 to (Int.MaxValue)).count // result = 0 sc.makeRDD(1 to (Int.MaxValue - 1)).count // result = 2147483646 = Int.MaxValue - 1 sc.makeRDD(1 until (Int.MaxValue)).count // result = 2147483646 = Int.MaxValue - 1 ``` see https://github.com/apache/spark/pull/2874 for more details. This pr try to fix the overflow. However, There's another issue I don't address. ``` val largeRange = Int.MinValue to Int.MaxValue largeRange.length // throws java.lang.IllegalArgumentException: -2147483648 to 2147483647 by 1: seqs cannot contain more than Int.MaxValue elements. ``` So, the range we feed to sc.makeRDD cannot contain more than Int.MaxValue elements. This is the limitation of Scala. However I think we may want to support that kind of range. But the fix is beyond this pr. srowen andrewor14 would you mind take a look at this pr? Author: Ye Xianjin <advancedxy@gmail.com> Closes #4002 from advancedxy/SPARk-5201 and squashes the following commits: 96265a1 [Ye Xianjin] Update slice method comment and some responding docs. e143d7a [Ye Xianjin] Update inclusive range check for splitting inclusive range. b3f5577 [Ye Xianjin] We can include the last element in the last slice in general for inclusive range, hence eliminate the need to check Int.MaxValue or Int.MinValue. 7d39b9e [Ye Xianjin] Convert the two cases pattern matching to one case. 651c959 [Ye Xianjin] rename sign to needsInclusiveRange. add some comments 196f8a8 [Ye Xianjin] Add test cases for ranges end with Int.MaxValue or Int.MinValue e66e60a [Ye Xianjin] Deal with inclusive and exclusive ranges in one case. If the range is inclusive and the end of the range is (Int.MaxValue or Int.MinValue), we should use inclusive range instead of exclusive
-
WangTaoTheTonic authored
Based on top of changes in https://github.com/apache/spark/pull/3806. https://issues.apache.org/jira/browse/SPARK-1507 `--driver-cores` and `spark.driver.cores` for all cluster modes and `spark.yarn.am.cores` for yarn client mode. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #4018 from WangTaoTheTonic/SPARK-1507 and squashes the following commits: 01419d3 [WangTaoTheTonic] amend the args name b255795 [WangTaoTheTonic] indet thing d86557c [WangTaoTheTonic] some comments amend 43c9392 [WangTao] fix compile error b39a100 [WangTao] specify # cores for ApplicationMaster
-
- Jan 15, 2015
-
-
Kostas Sakellis authored
When calculating the input metrics there was an assumption that one task only reads from one block - this is not true for some operations including coalesce. This patch simply increments the task's input metrics if previous ones existed of the same read method. A limitation to this patch is that if a task reads from two different blocks of different read methods, one will override the other. Author: Kostas Sakellis <kostas@cloudera.com> Closes #3120 from ksakellis/kostas-spark-4092 and squashes the following commits: 54e6658 [Kostas Sakellis] Drops metrics if conflicting read methods exist f0e0cc5 [Kostas Sakellis] Add bytesReadCallback to InputMetrics a2a36d4 [Kostas Sakellis] CR feedback 5a0c770 [Kostas Sakellis] [SPARK-4092] [CORE] Fix InputMetrics for coalesce'd Rdds
-
Kostas Sakellis authored
Adds onExecutorAdded and onExecutorRemoved events to the SparkListener. This will allow a client to get notified when an executor has been added/removed and provide additional information such as how many vcores it is consuming. In addition, this commit adds a SparkListenerAdapter to the Java API that provides default implementations to the SparkListener. This is to get around the fact that default implementations for traits don't work in Java. Having Java clients extend SparkListenerAdapter moving forward will prevent breakage in java when we add new events to SparkListener. Author: Kostas Sakellis <kostas@cloudera.com> Closes #3711 from ksakellis/kostas-spark-4857 and squashes the following commits: 946d2c5 [Kostas Sakellis] Added executorAdded/Removed events to MesosSchedulerBackend b1d054a [Kostas Sakellis] Remove executorInfo from ExecutorRemoved event 1727b38 [Kostas Sakellis] Renamed ExecutorDetails back to ExecutorInfo and other CR feedback 14fe78d [Kostas Sakellis] Added executor added/removed events to json protocol 93d087b [Kostas Sakellis] [SPARK-4857] [CORE] Adds Executor membership events to SparkListener
-
Kousuke Saruta authored
In BlockManager, there is a word `BlockTranserService` but I think it's typo for `BlockTransferService`. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #4046 from sarutak/fix-tiny-typo and squashes the following commits: a3e2a2f [Kousuke Saruta] Fixed tiny typo in BlockManager
-
Reynold Xin authored
As part of SPARK-5193: 1. Removed UDFRegistration as a mixin in SQLContext and made it a field ("udf"). 2. For Java UDFs, renamed dataType to returnType. 3. For Scala UDFs, added type tags. 4. Added all Java UDF registration methods to Scala's UDFRegistration. 5. Documentation Author: Reynold Xin <rxin@databricks.com> Closes #4056 from rxin/udf-registration and squashes the following commits: ae9c556 [Reynold Xin] Updated example. 675a3c9 [Reynold Xin] Style fix 47c24ff [Reynold Xin] Python fix. 5f00c45 [Reynold Xin] Restore data type position in java udf and added typetags. 032f006 [Reynold Xin] [SPARK-5193][SQL] Reconcile Java and Scala UDFRegistration.
-
Davies Liu authored
After the default batchSize changed to 0 (batched based on the size of object), but parallelize() still use BatchedSerializer with batchSize=1, this PR will use batchSize=1024 for parallelize by default. Also, BatchedSerializer did not work well with list and numpy.ndarray, this improve BatchedSerializer by using __len__ and __getslice__. Here is the benchmark for parallelize 1 millions int with list or ndarray: | before | after | improvements ------- | ------------ | ------------- | ------- list | 11.7 s | 0.8 s | 14x numpy.ndarray | 32 s | 0.7 s | 40x Author: Davies Liu <davies@databricks.com> Closes #4024 from davies/opt_numpy and squashes the following commits: 7618c7c [Davies Liu] improve performance of parallelize list/ndarray
-
- Jan 14, 2015
-
-
Reynold Xin authored
1. Removed the deprecated LocalHiveContext 2. Made private[sql] fields protected[sql] so they don't show up in javadoc. 3. Added javadoc to refreshTable. 4. Added Experimental tag to analyze command. Author: Reynold Xin <rxin@databricks.com> Closes #4054 from rxin/hivecontext-api and squashes the following commits: 25cc00a [Reynold Xin] Add implicit conversion back. cbca886 [Reynold Xin] [SPARK-5193][SQL] Tighten up HiveContext API
-
Xiangrui Meng authored
Forgot to remove this section in #4052. Author: Xiangrui Meng <meng@databricks.com> Closes #4053 from mengxr/SPARK-5254-update and squashes the following commits: f295bde [Xiangrui Meng] remove developers section from spark.ml guide
-
Reynold Xin authored
1. Removed 2 implicits (logicalPlanToSparkQuery and baseRelationToSchemaRDD) 2. Moved extraStrategies into ExperimentalMethods. 3. Made private methods protected[sql] so they don't show up in javadocs. 4. Removed createParquetFile. 5. Added Java version of applySchema to SQLContext. Author: Reynold Xin <rxin@databricks.com> Closes #4049 from rxin/sqlContext-refactor and squashes the following commits: a326a1a [Reynold Xin] Remove createParquetFile and add applySchema for Java to SQLContext. ecd6685 [Reynold Xin] Added baseRelationToSchemaRDD back. 4a38c9b [Reynold Xin] [SPARK-5193][SQL] Tighten up SQLContext API
-
Xiangrui Meng authored
The current statement in the user guide may deliver confusing messages to users. spark.ml contains high-level APIs for building ML pipelines. But it doesn't mean that spark.mllib is being deprecated. First of all, the pipeline API is in its alpha stage and we need to see more use cases from the community to stabilizes it, which may take several releases. Secondly, the components in spark.ml are simple wrappers over spark.mllib implementations. Neither the APIs or the implementations from spark.mllib are being deprecated. We expect users use spark.ml pipeline APIs to build their ML pipelines, but we will keep supporting and adding features to spark.mllib. For example, there are many features in review at https://spark-prs.appspot.com/#mllib. So users should be comfortable with using spark.mllib features and expect more coming. The user guide needs to be updated to make the message clear. Author: Xiangrui Meng <meng@databricks.com> Closes #4052 from mengxr/SPARK-5254 and squashes the following commits: 6d5f1d3 [Xiangrui Meng] typo 0cc935b [Xiangrui Meng] update user guide to position spark.ml better
-
Yuhao Yang authored
JIRA issue: https://issues.apache.org/jira/browse/SPARK-5234 simply add the call. Author: Yuhao Yang <yuhao@yuhaodevbox.sh.intel.com> Closes #4044 from hhbyyh/addscStop and squashes the following commits: c1f75ac [Yuhao Yang] add SparkContext.stop to 3 ml examples
-
Alex Baretta authored
Declare SQLConf to be serializable to fix "Task not serializable" exceptions in SparkSQL Author: Alex Baretta <alexbaretta@gmail.com> Closes #4031 from alexbaretta/SPARK-5235-SQLConf and squashes the following commits: c2103f5 [Alex Baretta] [SPARK-5235] Make SQLConf Serializable
-
Josh Rosen authored
`TaskContext.attemptId` is misleadingly-named, since it currently returns a taskId, which uniquely identifies a particular task attempt within a particular SparkContext, instead of an attempt number, which conveys how many times a task has been attempted. This patch deprecates `TaskContext.attemptId` and add `TaskContext.taskId` and `TaskContext.attemptNumber` fields. Prior to this change, it was impossible to determine whether a task was being re-attempted (or was a speculative copy), which made it difficult to write unit tests for tasks that fail on early attempts or speculative tasks that complete faster than original tasks. Earlier versions of the TaskContext docs suggest that `attemptId` behaves like `attemptNumber`, so there's an argument to be made in favor of changing this method's implementation. Since we've decided against making that change in maintenance branches, I think it's simpler to add better-named methods and retain the old behavior for `attemptId`; if `attemptId` behaved differently in different branches, then this would cause confusing build-breaks when backporting regression tests that rely on the new `attemptId` behavior. Most of this patch is fairly straightforward, but there is a bit of trickiness related to Mesos tasks: since there's no field in MesosTaskInfo to encode the attemptId, I packed it into the `data` field alongside the task binary. Author: Josh Rosen <joshrosen@databricks.com> Closes #3849 from JoshRosen/SPARK-4014 and squashes the following commits: 89d03e0 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 5cfff05 [Josh Rosen] Introduce wrapper for serializing Mesos task launch data. 38574d4 [Josh Rosen] attemptId -> taskAttemptId in PairRDDFunctions a180b88 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 1d43aa6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 eee6a45 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 0b10526 [Josh Rosen] Use putInt instead of putLong (silly mistake) 8c387ce [Josh Rosen] Use local with maxRetries instead of local-cluster. cbe4d76 [Josh Rosen] Preserve attemptId behavior and deprecate it: b2dffa3 [Josh Rosen] Address some of Reynold's minor comments 9d8d4d1 [Josh Rosen] Doc typo 1e7a933 [Josh Rosen] [SPARK-4014] Change TaskContext.attemptId to return attempt number instead of task ID. fd515a5 [Josh Rosen] Add failing test for SPARK-4014
-
Kousuke Saruta authored
In current WebUI, tables for Active Stages, Completed Stages, Skipped Stages and Failed Stages are hidden when they are empty while tables for Active Jobs, Completed Jobs and Failed Jobs are not hidden though they are empty. This is before my patch is applied.  And this is after my patch is applied.  Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #4028 from sarutak/SPARK-5228 and squashes the following commits: b1e6e8b [Kousuke Saruta] Fixed a small typo daab563 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5228 9493a1d [Kousuke Saruta] Modified AllJobPage.scala so that hide Active Jobs/Completed Jobs/Failed Jobs when they are empty
-
MechCoder authored
Slightly different than the scala code which converts the sparsevector into a densevector and then checks the index. I also hope I've added tests in the right place. Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #4025 from MechCoder/spark-2909 and squashes the following commits: 07d0f26 [MechCoder] STY: Rename item to index f02148b [MechCoder] [SPARK-2909] [Mlib] SparseVector in pyspark now supports indexing
-
Daoyuan Wang authored
Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #4000 from adrian-wang/comment and squashes the following commits: 9c24fc4 [Daoyuan Wang] some comments
-
Yin Huai authored
jira: https://issues.apache.org/jira/browse/SPARK-5211 Author: Yin Huai <yhuai@databricks.com> Closes #4026 from yhuai/SPARK-5211 and squashes the following commits: 15ee32b [Yin Huai] Remove extra line. c6c1651 [Yin Huai] Get back HiveMetastoreTypes.toDataType.
-
Daoyuan Wang authored
rxin follow up of #3732 Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #4041 from adrian-wang/decimal and squashes the following commits: aa3d738 [Daoyuan Wang] fix auto refactor 7777a58 [Daoyuan Wang] move sql.types.decimal.Decimal to sql.types.Decimal
-
Reynold Xin authored
Mostly just moving stuff around. This should still be source compatible since we type aliased Row previously in org.apache.spark.sql.Row. Added the following APIs to Row: ```scala def getMap[K, V](i: Int): scala.collection.Map[K, V] def getJavaMap[K, V](i: Int): java.util.Map[K, V] def getSeq[T](i: Int): Seq[T] def getList[T](i: Int): java.util.List[T] def getStruct(i: Int): StructType ``` Author: Reynold Xin <rxin@databricks.com> Closes #4030 from rxin/sql-row and squashes the following commits: 6c85c29 [Reynold Xin] Fixed style violation by adding a new line to Row.scala. 82b064a [Reynold Xin] [SPARK-5167][SQL] Move Row into sql package and make it usable for Java.
-
- Jan 13, 2015
-
-
Reynold Xin authored
Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box. As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code. This subsumes https://github.com/apache/spark/pull/3925 Author: Reynold Xin <rxin@databricks.com> Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits: 66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API).
-
Reynold Xin authored
This change should be binary and source backward compatible since we didn't change any user facing APIs. Author: Reynold Xin <rxin@databricks.com> Closes #3965 from rxin/SPARK-5168-sqlconf and squashes the following commits: 42eec09 [Reynold Xin] Fix default conf value. 0ef86cc [Reynold Xin] Fix constructor ordering. 4d7f910 [Reynold Xin] Properly override config. ccc8e6a [Reynold Xin] [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext
-
Yin Huai authored
With changes in this PR, users can persist metadata of tables created based on the data source API in metastore through DDLs. Author: Yin Huai <yhuai@databricks.com> Author: Michael Armbrust <michael@databricks.com> Closes #3960 from yhuai/persistantTablesWithSchema2 and squashes the following commits: 069c235 [Yin Huai] Make exception messages user friendly. c07cbc6 [Yin Huai] Get the location of test file in a correct way. 4456e98 [Yin Huai] Test data. 5315dfc [Yin Huai] rxin's comments. 7fc4b56 [Yin Huai] Add DDLStrategy and HiveDDLStrategy to plan DDLs based on the data source API. aeaf4b3 [Yin Huai] Add comments. 06f9b0c [Yin Huai] Revert unnecessary changes. feb88aa [Yin Huai] Merge remote-tracking branch 'apache/master' into persistantTablesWithSchema2 172db80 [Yin Huai] Fix unit test. 49bf1ac [Yin Huai] Unit tests. 8f8f1a1 [Yin Huai] [SPARK-4574][SQL] Adding support for defining schema in foreign DDL commands. #3431 f47fda1 [Yin Huai] Unit tests. 2b59723 [Michael Armbrust] Set external when creating tables c00bb1b [Michael Armbrust] Don't use reflection to read options 1ea6e7b [Michael Armbrust] Don't fail when trying to uncache a table that doesn't exist 6edc710 [Michael Armbrust] Add tests. d7da491 [Michael Armbrust] First draft of persistent tables.
-
Davies Liu authored
It will introduce problems if the object in dict/list/tuple can not support by py4j, such as Vector. Also, pickle may have better performance for larger object (less RPC). In some cases that the object in dict/list can not be pickled (such as JavaObject), we should still use MapConvert/ListConvert. This PR should be ported into branch-1.2 Author: Davies Liu <davies@databricks.com> Closes #4023 from davies/listconvert and squashes the following commits: 55d4ab2 [Davies Liu] fix MapConverter and ListConverter in MLlib
-
uncleGen authored
There is a discrepancy in WAL implementation and configuration doc. Author: uncleGen <hustyugm@gmail.com> Closes #3930 from uncleGen/master-clean-doc and squashes the following commits: 3a4245f [uncleGen] doc typo 8e407d3 [uncleGen] doc typo
-
WangTaoTheTonic authored
I found some arguments in yarn module take environment variables before system properties while the latter override the former in core module. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #3557 from WangTaoTheTonic/SPARK4697 and squashes the following commits: 836b9ef [WangTaoTheTonic] fix type mismatch e3e486a [WangTaoTheTonic] remove the comma 1262d57 [WangTaoTheTonic] handle spark.app.name and SPARK_YARN_APP_NAME in SparkSubmitArguments bee9447 [WangTaoTheTonic] wrong brace 81833bb [WangTaoTheTonic] rebase 40934b4 [WangTaoTheTonic] just switch blocks 5f43f45 [WangTao] System property can override environment variable
-
WangTaoTheTonic authored
https://issues.apache.org/jira/browse/SPARK-5006 I think the issue is produced in https://github.com/apache/spark/pull/1777. Not digging mesos's backend yet. Maybe should add same logic either. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits: 8cdf96d [WangTao] indent thing 2d86d65 [WangTaoTheTonic] fix line length 7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor 61a370d [WangTaoTheTonic] some minor fixes bc6e1ec [WangTaoTheTonic] rebase 67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg 29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable 396c226 [WangTaoTheTonic] make the grammar more like scala 191face [WangTaoTheTonic] invalid value name 62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work
-
- Jan 12, 2015
-
-
Gabe Mulley authored
When attempting to infer the schema of an RDD that contains namedtuples, pyspark fails to identify the records as namedtuples, resulting in it raising an error. Example: ```python from pyspark import SparkContext from pyspark.sql import SQLContext from collections import namedtuple import os sc = SparkContext() rdd = sc.textFile(os.path.join(os.getenv('SPARK_HOME'), 'README.md')) TextLine = namedtuple('TextLine', 'line length') tuple_rdd = rdd.map(lambda l: TextLine(line=l, length=len(l))) tuple_rdd.take(5) # This works sqlc = SQLContext(sc) # The following line raises an error schema_rdd = sqlc.inferSchema(tuple_rdd) ``` The error raised is: ``` File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/worker.py", line 107, in main process() File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/worker.py", line 98, in process serializer.dump_stream(func(split_index, iterator), outfile) File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/serializers.py", line 227, in dump_stream vs = list(itertools.islice(iterator, batch)) File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/rdd.py", line 1107, in takeUpToNumLeft yield next(iterator) File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/sql.py", line 816, in convert_struct raise ValueError("unexpected tuple: %s" % obj) TypeError: not all arguments converted during string formatting ``` Author: Gabe Mulley <gabe@edx.org> Closes #3978 from mulby/inferschema-namedtuple and squashes the following commits: 98c61cc [Gabe Mulley] Ensure exception message is populated correctly 375d96b [Gabe Mulley] Ensure schema can be inferred from a namedtuple
-
Michael Armbrust authored
Followup to #3870. Props to rahulaggarwalguavus for identifying the issue. Author: Michael Armbrust <michael@databricks.com> Closes #3990 from marmbrus/SPARK-5049 and squashes the following commits: dd03e4e [Michael Armbrust] Fill in the partition values of parquet scans instead of using JoinedRow
-
jerryshao authored
Currently WAL-backed block is read out from HDFS and put into BlockManger with storage level MEMORY_ONLY_SER by default, since WAL-backed block is already materialized in HDFS with fault-tolerance, no need to put into BlockManger again by default. Author: jerryshao <saisai.shao@intel.com> Closes #3906 from jerryshao/SPARK-4999 and squashes the following commits: b95f95e [jerryshao] Change storeInBlockManager to false by default
-
Sean Owen authored
In addition to the `hadoop-2.x` profiles in the parent POM, there is actually another set of profiles in `examples` that has to be activated differently to get the right Hadoop 1 vs 2 flavor of HBase. This wasn't actually used in making Hadoop 2 distributions, hence the problem. To reduce complexity, I suggest merging them with the parent POM profiles, which is possible now. You'll see this changes appears to update the HBase version, but actually, the default 0.94 version was not being used. HBase is only used in examples, and the examples POM always chose one profile or the other that updated the version to 0.98.x anyway. Author: Sean Owen <sowen@cloudera.com> Closes #3992 from srowen/SPARK-5172 and squashes the following commits: 17830d9 [Sean Owen] Control hbase hadoop1/2 flavor in the parent POM with existing hadoop-2.x profiles
-
Michael Armbrust authored
Current spark lets you set the ip address using SPARK_LOCAL_IP, but then this is given to akka after doing a reverse DNS lookup. This makes it difficult to run spark in Docker. You can already change the hostname that is used programmatically, but it would be nice to be able to do this with an environment variable as well. Author: Michael Armbrust <michael@databricks.com> Closes #3893 from marmbrus/localHostnameEnv and squashes the following commits: 85045b6 [Michael Armbrust] Optionally read from SPARK_LOCAL_HOSTNAME
-
Sean Owen authored
https://issues.apache.org/jira/browse/SPARK-4159 was resolved but as Sandy points out, the guidance in https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools under "Running Individual Tests" no longer quite works, not optimally. This minor change is not really the important change, which is an update to the wiki text. The correct way to run one Scala test suite in Maven is now: ``` mvn test -DwildcardSuites=org.apache.spark.io.CompressionCodecSuite -Dtests=none ``` The correct way to run one Java test is ``` mvn test -DwildcardSuites=none -Dtests=org.apache.spark.streaming.JavaAPISuite ``` Basically, you have to set two properties in order to suppress all of one type of test (with a non-existent test name like 'none') and all but one test of the other type. The change in the PR just prevents Surefire from barfing when it finds no "none" test. Author: Sean Owen <sowen@cloudera.com> Closes #3993 from srowen/SPARK-4159 and squashes the following commits: 83106d7 [Sean Owen] Default failIfNoTests to false to enable the -DwildcardSuites=... -Dtests=... syntax for running one test to work
-