@@ -19,12 +19,9 @@ package org.apache.spark.sql.raydp
1919
2020import com .intel .raydp .shims .SparkShimLoader
2121import io .ray .api .{ActorHandle , ObjectRef , Ray }
22- import io .ray .api .PyActorHandle
23- import io .ray .api .function .PyActorMethod
2422import io .ray .runtime .AbstractRayRuntime
25- import io .ray .runtime .config .RayConfig
2623import java .io .ByteArrayOutputStream
27- import java .util .{List , Optional , UUID }
24+ import java .util .{List , UUID }
2825import java .util .concurrent .{ConcurrentHashMap , ConcurrentLinkedQueue }
2926import java .util .function .{Function => JFunction }
3027import org .apache .arrow .vector .VectorSchemaRoot
@@ -39,7 +36,6 @@ import org.apache.spark.deploy.raydp._
3936import org .apache .spark .executor .RayDPExecutor
4037import org .apache .spark .network .util .JavaUtils
4138import org .apache .spark .raydp .{RayDPUtils , RayExecutorUtils }
42- import org .apache .spark .raydp .SparkOnRayConfigs
4339import org .apache .spark .sql .DataFrame
4440import org .apache .spark .sql .execution .arrow .ArrowWriter
4541import org .apache .spark .sql .execution .python .BatchIterator
@@ -66,76 +62,33 @@ class ObjectStoreWriter(@transient val df: DataFrame) extends Serializable {
6662 def writeToRay (
6763 data : Array [Byte ],
6864 numRecords : Int ,
69- queue : ObjectRefHolder .Queue ,
7065 ownerName : String ): RecordBatch = {
7166
72- // Owner-transfer only implementation:
73- // - ownerName must always be provided (non-empty) and refer to a Python actor
74- // implemented RayDPBlockStoreActorRegistry.
75- // - JVM never creates/handles Ray ObjectRefs for the dataset blocks.
76- // - JVM returns only a per-batch key encoded in RecordBatch.objectId (bytes),
77- // and Python will fetch the real ObjectRefs from the owner actor by key.
78-
67+ // Single-owner implementation:
68+ // - Spark executor JVM actor produces Arrow IPC bytes (data).
69+ // - Bytes are buffered inside the executor actor process keyed by batchKey.
70+ // - JVM returns (executorActorName, batchKey) to Python.
71+ // - A Python single owner actor later calls executorActor.popArrowIPC(batchKey),
72+ // decodes into pyarrow.Table and becomes the Ray owner of the resulting Dataset blocks.
73+ //
74+ // We keep ownerName non-empty for API consistency and to avoid accidental usage without
75+ // a dedicated owner actor on the Python side.
7976 if (ownerName == null || ownerName.isEmpty) {
8077 throw new RayDPException (" ownerName must be set for Spark->Ray conversion." )
8178 }
8279
83- val registryActorOptional = Ray .getActor(ownerName).asInstanceOf [Optional [AnyRef ]]
84- if (! registryActorOptional.isPresent) {
85- throw new RayDPException (s " Blobstore registry actor $ownerName not found. " )
86- }
87- val registryActorHandle : AnyRef = registryActorOptional.get()
88- if (! registryActorHandle.isInstanceOf [PyActorHandle ]) {
89- throw new RayDPException (
90- s " Blobstore registry actor $ownerName is not a Python actor. " )
91- }
92-
93- val appName = SparkEnv .get.conf.get(" spark.app.name" , " raydp" )
94- val blockStoreActorName =
95- ObjectStoreWriter .getBlockStoreActorName(appName, SparkEnv .get.executorId)
96- val pyHandle = registryActorHandle.asInstanceOf [PyActorHandle ]
97- val getActorMethod = PyActorMethod .of(
98- " get_or_create_blockstore_actor" , classOf [java.lang.Boolean ])
99-
100- // Get config inside to retain backward compatibility since this is a public API.
101- val nodeIp = RayConfig .create().nodeIp
102- val cpuOpt =
103- SparkEnv .get.conf.getOption(SparkOnRayConfigs .BLOCKSTORE_ACTOR_RESOURCE_CPU )
104- val memOpt =
105- SparkEnv .get.conf.getOption(SparkOnRayConfigs .BLOCKSTORE_ACTOR_RESOURCE_MEMORY )
106- val nodeAffinityOpt =
107- SparkEnv .get.conf.getOption(SparkOnRayConfigs .BLOCKSTORE_ACTOR_NODE_AFFINITY_RESOURCE )
108- val numCpus = cpuOpt.map(_.toDouble).getOrElse(0.0 )
109- val memory = memOpt.map(ObjectStoreWriter .parseMemoryBytes).getOrElse(0.0 )
110- val nodeAffinity = nodeAffinityOpt.map(_.toDouble).getOrElse(0.001 )
111-
112- pyHandle
113- .task(
114- getActorMethod,
115- blockStoreActorName,
116- nodeIp,
117- Double .box(numCpus),
118- Double .box(memory),
119- Double .box(nodeAffinity))
120- .remote()
121- .get()
122- val blockStorageActorHandleOpt =
123- Ray .getActor(blockStoreActorName).asInstanceOf [Optional [PyActorHandle ]]
124- if (! blockStorageActorHandleOpt.isPresent) {
125- throw new RayDPException (s " Actor $blockStoreActorName not found when putting dataset block. " )
126- }
127- val blockStorageActorHandle = blockStorageActorHandleOpt.get()
128-
80+ val executorId = SparkEnv .get.executorId
81+ val executorActorName = s " raydp-executor- ${executorId}"
12982 val batchKey = UUID .randomUUID().toString
13083
131- // put_arrow_ipc(batchKey, arrowBytes) -> boolean ack
132- val putArrowIPCMethod = PyActorMethod .of( " put_arrow_ipc " , classOf [java.lang. Boolean ])
133- blockStorageActorHandle.task(putArrowIPCMethod, batchKey, data).remote().get( )
84+ // Buffer bytes in the executor actor process. The Python owner actor will pop them via
85+ // cross-language actor call later.
86+ RayDPExecutor .putArrowIPC( batchKey, data)
13487
135- // RecordBatch payload is an application-level locator (not Ray object metadata) :
136- // - ownerAddress encodes the BlockStore actor name (UTF-8)
88+ // RecordBatch payload:
89+ // - ownerAddress encodes the RayDPExecutor actor name (UTF-8)
13790 // - objectId encodes the batch key (UTF-8)
138- RecordBatch (blockStoreActorName .getBytes(" UTF-8" ), batchKey.getBytes(" UTF-8" ), numRecords)
91+ RecordBatch (executorActorName .getBytes(" UTF-8" ), batchKey.getBytes(" UTF-8" ), numRecords)
13992 }
14093
14194 /**
@@ -151,8 +104,6 @@ class ObjectStoreWriter(@transient val df: DataFrame) extends Serializable {
151104 val schema = df.schema
152105
153106 val objectIds = df.queryExecution.toRdd.mapPartitions{ iter =>
154- val queue = ObjectRefHolder .getQueue(uuid)
155-
156107 // DO NOT use iter.grouped(). See BatchIterator.
157108 val batchIter = if (batchSize > 0 ) {
158109 new BatchIterator (iter, batchSize)
@@ -196,7 +147,7 @@ class ObjectStoreWriter(@transient val df: DataFrame) extends Serializable {
196147
197148 // get the wrote ByteArray and save to Ray ObjectStore
198149 val byteArray = byteOut.toByteArray
199- results += writeToRay(byteArray, numRecords, queue, ownerName)
150+ results += writeToRay(byteArray, numRecords, ownerName)
200151 // end writes footer to the output stream and doesn't clean any resources.
201152 // It could throw exception if the output stream is closed, so it should be
202153 // in the try block.
@@ -267,20 +218,6 @@ object ObjectStoreWriter {
267218 }
268219 }
269220
270- private def sanitizeActorName (name : String ): String = {
271- if (name == null || name.isEmpty) {
272- " raydp"
273- } else {
274- // Ray named actor names should be reasonably simple; normalize to [A-Za-z0-9_].
275- name.replaceAll(" [^A-Za-z0-9_]" , " _" )
276- }
277- }
278-
279- private [spark] def getBlockStoreActorName (appName : String , executorId : String ): String = {
280- val safeAppName = sanitizeActorName(appName)
281- s " ${safeAppName}_BLOCKSTORE_ ${executorId}"
282- }
283-
284221 def getAddress (): Array [Byte ] = {
285222 if (address == null ) {
286223 val objectRef = Ray .put(1 )
0 commit comments