Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,9 @@ object MimaExcludes {
ProblemFilters.exclude[MissingMethodProblem](
"org.apache.spark.mllib.linalg.Matrices.rand")
) ++ Seq(
// SPARK-3325
ProblemFilters.exclude[MissingMethodProblem](
"org.apache.spark.streaming.api.java.JavaDStreamLike.print"),
// SPARK-2757
ProblemFilters.exclude[IncompatibleResultTypeProblem](
"org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." +
Expand Down
12 changes: 7 additions & 5 deletions python/pyspark/streaming/dstream.py
Original file line number Diff line number Diff line change
Expand Up @@ -157,18 +157,20 @@ def foreachRDD(self, func):
api = self._ssc._jvm.PythonDStream
api.callForeachRDD(self._jdstream, jfunc)

def pprint(self):
def pprint(self, num=10):
"""
Print the first ten elements of each RDD generated in this DStream.
Print the first num elements of each RDD generated in this DStream.

@param num: the number of elements from the first will be printed.
"""
def takeAndPrint(time, rdd):
taken = rdd.take(11)
taken = rdd.take(num + 1)
print "-------------------------------------------"
print "Time: %s" % time
print "-------------------------------------------"
for record in taken[:10]:
for record in taken[:num]:
print record
if len(taken) > 10:
if len(taken) > num:
print "..."
print

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,15 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* operator, so this DStream will be registered as an output stream and there materialized.
*/
def print(): Unit = {
dstream.print()
print(10)
}

/**
* Print the first num elements of each RDD generated in this DStream. This is an output
* operator, so this DStream will be registered as an output stream and there materialized.
*/
def print(num: Int): Unit = {
dstream.print(num)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -605,13 +605,21 @@ abstract class DStream[T: ClassTag] (
* operator, so this DStream will be registered as an output stream and there materialized.
*/
def print() {
print(10)
}

/**
* Print the first num elements of each RDD generated in this DStream. This is an output
* operator, so this DStream will be registered as an output stream and there materialized.
*/
def print(num: Int) {
def foreachFunc = (rdd: RDD[T], time: Time) => {
val first11 = rdd.take(11)
val firstNum = rdd.take(num + 1)
println ("-------------------------------------------")
println ("Time: " + time)
println ("-------------------------------------------")
first11.take(10).foreach(println)
if (first11.size > 10) println("...")
firstNum.take(num).foreach(println)
if (firstNum.size > num) println("...")
println()
}
new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register()
Expand Down