Skip to content

Commit 2cf08bb

Browse files
Merge branch 'master' into caseWhen
Conflicts: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
2 parents 9f84b40 + fe78b8b commit 2cf08bb

File tree

148 files changed

+4097
-586
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

148 files changed

+4097
-586
lines changed

.gitignore

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,7 @@
77
sbt/*.jar
88
.settings
99
.cache
10-
.generated-mima-excludes
10+
.generated-mima*
1111
/build/
1212
work/
1313
out/

bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo
8080
test("large number of iterations") {
8181
// This tests whether jobs with a large number of iterations finish in a reasonable time,
8282
// because non-memoized recursion in RDD or DAGScheduler used to cause them to hang
83-
failAfter(10 seconds) {
83+
failAfter(30 seconds) {
8484
sc = new SparkContext("local", "test")
8585
val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0))))
8686
val msgs = sc.parallelize(Array[(String, TestMessage)]())
@@ -101,7 +101,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo
101101
sc = new SparkContext("local", "test")
102102
val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0))))
103103
val msgs = sc.parallelize(Array[(String, TestMessage)]())
104-
val numSupersteps = 50
104+
val numSupersteps = 20
105105
val result =
106106
Bagel.run(sc, verts, msgs, sc.defaultParallelism, StorageLevel.DISK_ONLY) {
107107
(self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) =>

bin/pyspark

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@ fi
4545
. $FWDIR/bin/load-spark-env.sh
4646

4747
# Figure out which Python executable to use
48-
if [ -z "$PYSPARK_PYTHON" ] ; then
48+
if [[ -z "$PYSPARK_PYTHON" ]]; then
4949
PYSPARK_PYTHON="python"
5050
fi
5151
export PYSPARK_PYTHON
@@ -59,7 +59,7 @@ export OLD_PYTHONSTARTUP=$PYTHONSTARTUP
5959
export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py
6060

6161
# If IPython options are specified, assume user wants to run IPython
62-
if [ -n "$IPYTHON_OPTS" ]; then
62+
if [[ -n "$IPYTHON_OPTS" ]]; then
6363
IPYTHON=1
6464
fi
6565

@@ -76,6 +76,16 @@ for i in "$@"; do
7676
done
7777
export PYSPARK_SUBMIT_ARGS
7878

79+
# For pyspark tests
80+
if [[ -n "$SPARK_TESTING" ]]; then
81+
if [[ -n "$PYSPARK_DOC_TEST" ]]; then
82+
exec "$PYSPARK_PYTHON" -m doctest $1
83+
else
84+
exec "$PYSPARK_PYTHON" $1
85+
fi
86+
exit
87+
fi
88+
7989
# If a python file is provided, directly run spark-submit.
8090
if [[ "$1" =~ \.py$ ]]; then
8191
echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -823,9 +823,11 @@ class SparkContext(config: SparkConf) extends Logging {
823823
}
824824

825825
/**
826+
* :: DeveloperApi ::
826827
* Return information about what RDDs are cached, if they are in mem or on disk, how much space
827828
* they take, etc.
828829
*/
830+
@DeveloperApi
829831
def getRDDStorageInfo: Array[RDDInfo] = {
830832
StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
831833
}
@@ -837,8 +839,10 @@ class SparkContext(config: SparkConf) extends Logging {
837839
def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
838840

839841
/**
842+
* :: DeveloperApi ::
840843
* Return information about blocks stored in all of the slaves
841844
*/
845+
@DeveloperApi
842846
def getExecutorStorageStatus: Array[StorageStatus] = {
843847
env.blockManager.master.getStorageStatus
844848
}
Lines changed: 129 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,129 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.api.python
19+
20+
import org.apache.spark.rdd.RDD
21+
import org.apache.spark.Logging
22+
import org.apache.hadoop.conf.Configuration
23+
import org.apache.hadoop.io._
24+
import scala.util.{Failure, Success, Try}
25+
import org.apache.spark.annotation.Experimental
26+
27+
28+
/**
29+
* :: Experimental ::
30+
* A trait for use with reading custom classes in PySpark. Implement this trait and add custom
31+
* transformation code by overriding the convert method.
32+
*/
33+
@Experimental
34+
trait Converter[T, U] extends Serializable {
35+
def convert(obj: T): U
36+
}
37+
38+
private[python] object Converter extends Logging {
39+
40+
def getInstance(converterClass: Option[String]): Converter[Any, Any] = {
41+
converterClass.map { cc =>
42+
Try {
43+
val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]]
44+
logInfo(s"Loaded converter: $cc")
45+
c
46+
} match {
47+
case Success(c) => c
48+
case Failure(err) =>
49+
logError(s"Failed to load converter: $cc")
50+
throw err
51+
}
52+
}.getOrElse { new DefaultConverter }
53+
}
54+
}
55+
56+
/**
57+
* A converter that handles conversion of common [[org.apache.hadoop.io.Writable]] objects.
58+
* Other objects are passed through without conversion.
59+
*/
60+
private[python] class DefaultConverter extends Converter[Any, Any] {
61+
62+
/**
63+
* Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or
64+
* object representation
65+
*/
66+
private def convertWritable(writable: Writable): Any = {
67+
import collection.JavaConversions._
68+
writable match {
69+
case iw: IntWritable => iw.get()
70+
case dw: DoubleWritable => dw.get()
71+
case lw: LongWritable => lw.get()
72+
case fw: FloatWritable => fw.get()
73+
case t: Text => t.toString
74+
case bw: BooleanWritable => bw.get()
75+
case byw: BytesWritable => byw.getBytes
76+
case n: NullWritable => null
77+
case aw: ArrayWritable => aw.get().map(convertWritable(_))
78+
case mw: MapWritable => mapAsJavaMap(mw.map { case (k, v) =>
79+
(convertWritable(k), convertWritable(v))
80+
}.toMap)
81+
case other => other
82+
}
83+
}
84+
85+
def convert(obj: Any): Any = {
86+
obj match {
87+
case writable: Writable =>
88+
convertWritable(writable)
89+
case _ =>
90+
obj
91+
}
92+
}
93+
}
94+
95+
/** Utilities for working with Python objects <-> Hadoop-related objects */
96+
private[python] object PythonHadoopUtil {
97+
98+
/**
99+
* Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]]
100+
*/
101+
def mapToConf(map: java.util.Map[String, String]): Configuration = {
102+
import collection.JavaConversions._
103+
val conf = new Configuration()
104+
map.foreach{ case (k, v) => conf.set(k, v) }
105+
conf
106+
}
107+
108+
/**
109+
* Merges two configurations, returns a copy of left with keys from right overwriting
110+
* any matching keys in left
111+
*/
112+
def mergeConfs(left: Configuration, right: Configuration): Configuration = {
113+
import collection.JavaConversions._
114+
val copy = new Configuration(left)
115+
right.iterator().foreach(entry => copy.set(entry.getKey, entry.getValue))
116+
copy
117+
}
118+
119+
/**
120+
* Converts an RDD of key-value pairs, where key and/or value could be instances of
121+
* [[org.apache.hadoop.io.Writable]], into an RDD[(K, V)]
122+
*/
123+
def convertRDD[K, V](rdd: RDD[(K, V)],
124+
keyConverter: Converter[Any, Any],
125+
valueConverter: Converter[Any, Any]): RDD[(Any, Any)] = {
126+
rdd.map { case (k, v) => (keyConverter.convert(k), valueConverter.convert(v)) }
127+
}
128+
129+
}

0 commit comments

Comments
 (0)