Skip to content
Snippets Groups Projects
  1. Oct 07, 2015
    • Josh Rosen's avatar
      [SPARK-9702] [SQL] Use Exchange to implement logical Repartition operator · 7e2e2682
      Josh Rosen authored
      This patch allows `Repartition` to support UnsafeRows. This is accomplished by implementing the logical `Repartition` operator in terms of `Exchange` and a new `RoundRobinPartitioning`.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      Author: Liang-Chi Hsieh <viirya@appier.com>
      
      Closes #8083 from JoshRosen/SPARK-9702.
      7e2e2682
    • Davies Liu's avatar
      [SPARK-10980] [SQL] fix bug in create Decimal · 37526aca
      Davies Liu authored
      The created decimal is wrong if using `Decimal(unscaled, precision, scale)` with unscaled > 1e18 and and precision > 18 and scale > 0.
      
      This bug exists since the beginning.
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #9014 from davies/fix_decimal.
      37526aca
    • Yanbo Liang's avatar
      [SPARK-10490] [ML] Consolidate the Cholesky solvers in WeightedLeastSquares and ALS · 7bf07faa
      Yanbo Liang authored
      Consolidate the Cholesky solvers in WeightedLeastSquares and ALS.
      
      Author: Yanbo Liang <ybliang8@gmail.com>
      
      Closes #8936 from yanboliang/spark-10490.
      7bf07faa
    • Reynold Xin's avatar
      [SPARK-10982] [SQL] Rename ExpressionAggregate -> DeclarativeAggregate. · 6dbfd7ec
      Reynold Xin authored
      DeclarativeAggregate matches more closely with ImperativeAggregate we already have.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #9013 from rxin/SPARK-10982.
      6dbfd7ec
    • Evan Chen's avatar
      [SPARK-10779] [PYSPARK] [MLLIB] Set initialModel for KMeans model in PySpark (spark.mllib) · da936fbb
      Evan Chen authored
      Provide initialModel param for pyspark.mllib.clustering.KMeans
      
      Author: Evan Chen <chene@us.ibm.com>
      
      Closes #8967 from evanyc15/SPARK-10779-pyspark-mllib.
      da936fbb
    • navis.ryu's avatar
      [SPARK-10679] [CORE] javax.jdo.JDOFatalUserException in executor · 713e4f44
      navis.ryu authored
      HadoopRDD throws exception in executor, something like below.
      {noformat}
      5/09/17 18:51:21 INFO metastore.HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
      15/09/17 18:51:21 INFO metastore.ObjectStore: ObjectStore, initialize called
      15/09/17 18:51:21 WARN metastore.HiveMetaStore: Retrying creating default database after error: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
      javax.jdo.JDOFatalUserException: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
      	at javax.jdo.JDOHelper.invokeGetPersistenceManagerFactoryOnImplementation(JDOHelper.java:1175)
      	at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:808)
      	at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:701)
      	at org.apache.hadoop.hive.metastore.ObjectStore.getPMF(ObjectStore.java:365)
      	at org.apache.hadoop.hive.metastore.ObjectStore.getPersistenceManager(ObjectStore.java:394)
      	at org.apache.hadoop.hive.metastore.ObjectStore.initialize(ObjectStore.java:291)
      	at org.apache.hadoop.hive.metastore.ObjectStore.setConf(ObjectStore.java:258)
      	at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73)
      	at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
      	at org.apache.hadoop.hive.metastore.RawStoreProxy.<init>(RawStoreProxy.java:57)
      	at org.apache.hadoop.hive.metastore.RawStoreProxy.getProxy(RawStoreProxy.java:66)
      	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.newRawStore(HiveMetaStore.java:593)
      	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:571)
      	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:620)
      	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:461)
      	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:66)
      	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.getProxy(RetryingHMSHandler.java:72)
      	at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762)
      	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:199)
      	at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.<init>(SessionHiveMetaStoreClient.java:74)
      	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
      	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
      	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
      	at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
      	at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1521)
      	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:86)
      	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:132)
      	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:104)
      	at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3005)
      	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3024)
      	at org.apache.hadoop.hive.ql.metadata.Hive.getAllDatabases(Hive.java:1234)
      	at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:174)
      	at org.apache.hadoop.hive.ql.metadata.Hive.<clinit>(Hive.java:166)
      	at org.apache.hadoop.hive.ql.plan.PlanUtils.configureJobPropertiesForStorageHandler(PlanUtils.java:803)
      	at org.apache.hadoop.hive.ql.plan.PlanUtils.configureInputJobPropertiesForStorageHandler(PlanUtils.java:782)
      	at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:298)
      	at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
      	at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
      	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
      	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
      	at scala.Option.map(Option.scala:145)
      	at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:176)
      	at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:220)
      	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:216)
      	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
      	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
      	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
      	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
      	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
      	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
      	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
      	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
      	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
      	at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:87)
      	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
      	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
      	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
      	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
      	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
      	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
      	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
      	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
      	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
      	at org.apache.spark.scheduler.Task.run(Task.scala:88)
      	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
      	at java.lang.Thread.run(Thread.java:745)
      {noformat}
      
      Author: navis.ryu <navis@apache.org>
      
      Closes #8804 from navis/SPARK-10679.
      713e4f44
    • Liang-Chi Hsieh's avatar
      [SPARK-10856][SQL] Mapping TimestampType to DATETIME for SQL Server jdbc dialect · c14aee4d
      Liang-Chi Hsieh authored
      JIRA: https://issues.apache.org/jira/browse/SPARK-10856
      
      For Microsoft SQL Server, TimestampType should be mapped to DATETIME instead of TIMESTAMP.
      
      Related information for the datatype mapping: https://msdn.microsoft.com/en-us/library/ms378878(v=sql.110).aspx
      
      Author: Liang-Chi Hsieh <viirya@appier.com>
      
      Closes #8978 from viirya/mysql-jdbc-timestamp.
      c14aee4d
    • Marcelo Vanzin's avatar
      [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py. · 94fc57af
      Marcelo Vanzin authored
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #8775 from vanzin/SPARK-10300.
      94fc57af
    • Josh Rosen's avatar
      [SPARK-10941] [SQL] Refactor AggregateFunction2 and AlgebraicAggregate... · a9ecd061
      Josh Rosen authored
      [SPARK-10941] [SQL] Refactor AggregateFunction2 and AlgebraicAggregate interfaces to improve code clarity
      
      This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.
      
      The biggest change is a refactoring of the `AggregateFunction2` class hierarchy. In the old code, we had a class named `AlgebraicAggregate` that inherited from `AggregateFunction2`, added a new set of methods, then banned the use of the inherited methods. I found this to be fairly confusing because.
      
      If you look carefully at the existing code, you'll see that subclasses of `AggregateFunction2` fall into two disjoint categories: imperative aggregation functions which directly extended `AggregateFunction2` and declarative, expression-based aggregate functions which extended `AlgebraicAggregate`. In order to make this more explicit, this patch refactors things so that `AggregateFunction2` is a sealed abstract class with two subclasses, `ImperativeAggregateFunction` and `ExpressionAggregateFunction`. The superclass, `AggregateFunction2`, now only contains methods and fields that are common to both subclasses.
      
      After making this change, I updated the various AggregationIterator classes to comply with this new naming scheme. I also performed several small renamings in the aggregate interfaces themselves in order to improve clarity and rewrote or expanded a number of comments.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #8973 from JoshRosen/tungsten-agg-comments.
      a9ecd061
    • Holden Karau's avatar
      [SPARK-9841] [ML] Make clear public · 5be5d247
      Holden Karau authored
      It is currently impossible to clear Param values once set. It would be helpful to be able to.
      
      Author: Holden Karau <holden@pigscanfly.ca>
      
      Closes #8619 from holdenk/SPARK-9841-params-clear-needs-to-be-public.
      5be5d247
    • Marcelo Vanzin's avatar
      [SPARK-10964] [YARN] Correctly register the AM with the driver. · 6ca27f85
      Marcelo Vanzin authored
      The `self` method returns null when called from the constructor;
      instead, registration should happen in the `onStart` method, at
      which point the `self` reference has already been initialized.
      
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #9005 from vanzin/SPARK-10964.
      6ca27f85
    • Marcelo Vanzin's avatar
      [SPARK-10812] [YARN] Fix shutdown of token renewer. · 4b747551
      Marcelo Vanzin authored
      A recent change to fix the referenced bug caused this exception in
      the `SparkContext.stop()` path:
      
      org.apache.spark.SparkException: YarnSparkHadoopUtil is not available in non-YARN mode!
              at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$.get(YarnSparkHadoopUtil.scala:167)
              at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.stop(YarnClientSchedulerBackend.scala:182)
              at org.apache.spark.scheduler.TaskSchedulerImpl.stop(TaskSchedulerImpl.scala:440)
              at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1579)
              at org.apache.spark.SparkContext$$anonfun$stop$7.apply$mcV$sp(SparkContext.scala:1730)
              at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1185)
              at org.apache.spark.SparkContext.stop(SparkContext.scala:1729)
      
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #8996 from vanzin/SPARK-10812.
      4b747551
    • Michael Armbrust's avatar
      [SPARK-10966] [SQL] Codegen framework cleanup · f5d154bc
      Michael Armbrust authored
      This PR is mostly cosmetic and cleans up some warts in codegen (nearly all of which were inherited from the original quasiquote version).
       - Add lines numbers to errors (in stacktraces when debug logging is on, and always for compile fails)
       - Use a variable for input row instead of hardcoding "i" everywhere
       - rename `primitive` -> `value` (since its often actually an object)
      
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #9006 from marmbrus/codegen-cleanup.
      f5d154bc
    • Kevin Cox's avatar
      [SPARK-10952] Only add hive to classpath if HIVE_HOME is set. · 9672602c
      Kevin Cox authored
      Currently if it isn't set it scans `/lib/*` and adds every dir to the
      classpath which makes the env too large and every command called
      afterwords fails.
      
      Author: Kevin Cox <kevincox@kevincox.ca>
      
      Closes #8994 from kevincox/kevincox-only-add-hive-to-classpath-if-var-is-set.
      9672602c
    • Sun Rui's avatar
      [SPARK-10752] [SPARKR] Implement corr() and cov in DataFrameStatFunctions. · f57c63d4
      Sun Rui authored
      Author: Sun Rui <rui.sun@intel.com>
      
      Closes #8869 from sun-rui/SPARK-10752.
      f57c63d4
    • Xin Ren's avatar
      [SPARK-10669] [DOCS] Link to each language's API in codetabs in ML docs: spark.mllib · 27cdde2f
      Xin Ren authored
      In the Markdown docs for the spark.mllib Programming Guide, we have code examples with codetabs for each language. We should link to each language's API docs within the corresponding codetab, but we are inconsistent about this. For an example of what we want to do, see the "ChiSqSelector" section in https://github.com/apache/spark/blob/64743870f23bffb8d96dcc8a0181c1452782a151/docs/mllib-feature-extraction.md
      This JIRA is just for spark.mllib, not spark.ml.
      
      Please let me know if more work is needed, thanks a lot.
      
      Author: Xin Ren <iamshrek@126.com>
      
      Closes #8977 from keypointt/SPARK-10669.
      27cdde2f
  2. Oct 06, 2015
  3. Oct 05, 2015
    • zsxwing's avatar
      [SPARK-10900] [STREAMING] Add output operation events to StreamingListener · be7c5ff1
      zsxwing authored
      Add output operation events to StreamingListener so as to implement the following UI features:
      
      1. Progress bar of a batch in the batch list.
      2. Be able to display output operation `description` and `duration` when there is no spark job in a Streaming job.
      
      Author: zsxwing <zsxwing@gmail.com>
      
      Closes #8958 from zsxwing/output-operation-events.
      be7c5ff1
    • Wenchen Fan's avatar
      [SPARK-10934] [SQL] handle hashCode of unsafe array correctly · a609eb20
      Wenchen Fan authored
      `Murmur3_x86_32.hashUnsafeWords` only accepts word-aligned bytes, but unsafe array is not.
      
      Author: Wenchen Fan <cloud0fan@163.com>
      
      Closes #8987 from cloud-fan/hash.
      a609eb20
    • Wenchen Fan's avatar
      [SPARK-10585] [SQL] only copy data once when generate unsafe projection · c4871369
      Wenchen Fan authored
      This PR is a completely rewritten of GenerateUnsafeProjection, to accomplish the goal of copying data only once. The old code of GenerateUnsafeProjection is still there to reduce review difficulty.
      
      Instead of creating unsafe conversion code for struct, array and map, we create code of writing the content to the global row buffer.
      
      Author: Wenchen Fan <cloud0fan@163.com>
      Author: Wenchen Fan <cloud0fan@outlook.com>
      
      Closes #8747 from cloud-fan/copy-once.
      c4871369
  4. Oct 04, 2015
  5. Oct 03, 2015
  6. Oct 02, 2015
  7. Oct 01, 2015
    • Takeshi YAMAMURO's avatar
      [SPARK-9867] [SQL] Move utilities for binary data into ByteArray · 2272962e
      Takeshi YAMAMURO authored
      The utilities such as Substring#substringBinarySQL and BinaryPrefixComparator#computePrefix for binary data are put together in ByteArray for easy-to-read.
      
      Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
      
      Closes #8122 from maropu/CleanUpForBinaryType.
      2272962e
    • Cheng Lian's avatar
      [SPARK-10400] [SQL] Renames SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC · 01cd688f
      Cheng Lian authored
      We introduced SQL option `spark.sql.parquet.followParquetFormatSpec` while working on implementing Parquet backwards-compatibility rules in SPARK-6777. It indicates whether we should use legacy Parquet format adopted by Spark 1.4 and prior versions or the standard format defined in parquet-format spec to write Parquet files.
      
      This option defaults to `false` and is marked as a non-public option (`isPublic = false`) because we haven't finished refactored Parquet write path. The problem is, the name of this option is somewhat confusing, because it's not super intuitive why we shouldn't follow the spec. Would be nice to rename it to `spark.sql.parquet.writeLegacyFormat`, and invert its default value (the two option names have opposite meanings).
      
      Although this option is private in 1.5, we'll make it public in 1.6 after refactoring Parquet write path. So that users can decide whether to write Parquet files in standard format or legacy format.
      
      Author: Cheng Lian <lian@databricks.com>
      
      Closes #8566 from liancheng/spark-10400/deprecate-follow-parquet-format-spec.
      01cd688f
Loading