-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-34205][SQL][SS] Add pipe to Dataset to enable Streaming Dataset pipe #31296
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2889,6 +2889,24 @@ class Dataset[T] private[sql]( | |
| flatMap(func)(encoder) | ||
| } | ||
|
|
||
| /** | ||
| * Return a new Dataset of string created by piping elements to a forked external process. | ||
| * The resulting Dataset is computed by executing the given process once per partition. | ||
| * All elements of each input partition are written to a process's stdin as lines of input | ||
| * separated by a newline. The resulting partition consists of the process's stdout output, with | ||
| * each line of stdout resulting in one element of the output partition. A process is invoked | ||
| * even for empty partitions. | ||
| * | ||
| * @param command command to run in forked process. | ||
| * | ||
| * @group typedrel | ||
| * @since 3.2.0 | ||
| */ | ||
| def pipe(command: String): Dataset[String] = { | ||
|
||
| implicit val stringEncoder = Encoders.STRING | ||
| withTypedPlan[String](PipeElements[T](command, logicalPlan)) | ||
| } | ||
|
|
||
| /** | ||
| * Applies a function `f` to all rows. | ||
| * | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,7 +25,7 @@ import scala.language.existentials | |
| import org.apache.spark.api.java.function.MapFunction | ||
| import org.apache.spark.api.r._ | ||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.rdd.{PipedRDD, RDD} | ||
| import org.apache.spark.sql.Row | ||
| import org.apache.spark.sql.api.r.SQLUtils._ | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
|
|
@@ -624,3 +624,32 @@ case class CoGroupExec( | |
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Piping elements to a forked external process. | ||
| * The output of its child must be a single-field row containing the input object. | ||
| */ | ||
| case class PipeElementsExec( | ||
| outputObjAttr: Attribute, | ||
| command: String, | ||
| child: SparkPlan) | ||
| extends ObjectConsumerExec with ObjectProducerExec { | ||
|
|
||
| override protected def doExecute(): RDD[InternalRow] = { | ||
| val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) | ||
| val printRDDElement: (InternalRow, String => Unit) => Unit = (row, printFunc) => { | ||
| printFunc(getObject(row).toString) | ||
| } | ||
|
|
||
| child.execute() | ||
| .pipe(command = PipedRDD.tokenize(command), printRDDElement = printRDDElement) | ||
| .mapPartitionsInternal { iter => | ||
| val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) | ||
| iter.map(ele => outputObject(ele)) | ||
|
||
| } | ||
| } | ||
|
|
||
| override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
|
||
| override def outputPartitioning: Partitioning = child.outputPartitioning | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,7 +24,7 @@ import org.scalatest.Assertions._ | |
| import org.scalatest.exceptions.TestFailedException | ||
| import org.scalatest.prop.TableDrivenPropertyChecks._ | ||
|
|
||
| import org.apache.spark.{SparkException, TaskContext} | ||
| import org.apache.spark.{SparkException, TaskContext, TestUtils} | ||
| import org.apache.spark.sql.catalyst.{FooClassWithEnum, FooEnum, ScroogeLikeExample} | ||
| import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} | ||
| import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} | ||
|
|
@@ -2007,6 +2007,28 @@ class DatasetSuite extends QueryTest | |
|
|
||
| checkAnswer(withUDF, Row(Row(1), null, null) :: Row(Row(1), null, null) :: Nil) | ||
| } | ||
|
|
||
| test("SPARK-34205: Pipe Dataset") { | ||
| assume(TestUtils.testCommandAvailable("cat")) | ||
|
|
||
| val nums = spark.range(4) | ||
| val piped = nums.pipe("cat").toDF | ||
|
|
||
| checkAnswer(piped, Row("0") :: Row("1") :: Row("2") :: Row("3") :: Nil) | ||
|
|
||
| val piped2 = nums.pipe("wc -l").toDF.collect() | ||
| assert(piped2.size == 2) | ||
| assert(piped2(0).getString(0).trim == "2") | ||
| assert(piped2(1).getString(0).trim == "2") | ||
| } | ||
|
|
||
| test("SPARK-34205: pipe Dataset with empty partition") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you for making this sure! |
||
| val data = Seq(123, 4567).toDF("num").repartition(8, $"num") | ||
| val piped = data.pipe("wc -l") | ||
| assert(piped.count == 8) | ||
| val lineCounts = piped.map(_.trim.toInt).collect().toSet | ||
| assert(Set(0, 1, 1) == lineCounts) | ||
| } | ||
| } | ||
|
|
||
| case class Bar(a: Int) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1264,6 +1264,20 @@ class StreamSuite extends StreamTest { | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-34205: Pipe Streaming Dataset") { | ||
| assume(TestUtils.testCommandAvailable("cat")) | ||
|
|
||
| val inputData = MemoryStream[Int] | ||
| val piped = inputData.toDS() | ||
| .pipe("cat").toDF | ||
|
|
||
| testStream(piped)( | ||
| AddData(inputData, 1, 2, 3), | ||
| CheckAnswer(Row("1"), Row("2"), Row("3")), | ||
| AddData(inputData, 4), | ||
| CheckAnswer(Row("1"), Row("2"), Row("3"), Row("4"))) | ||
|
||
| } | ||
| } | ||
|
|
||
| abstract class FakeSource extends StreamSourceProvider { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.