Skip to content

Conversation

@jiangxb1987
Copy link
Contributor

@jiangxb1987 jiangxb1987 commented Aug 12, 2018

What changes were proposed in this pull request?

Add method barrier() and getTaskInfos() in python TaskContext, these two methods are only allowed for barrier tasks.

How was this patch tested?

Add new tests in tests.py

logError(message)
throw new SparkException(message)
} else {
logDebug(s"Started GatewayServer to port BarrierTaskContext on port $boundPort.")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When isBarrier is false, I think we don't need show this?


# initialize global state
taskContext = TaskContext._getOrCreate()
isBarrier = read_bool(infile)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a comment indicating the following 3 inputs are only for barrier task?

_accumulatorRegistry.clear()

if isBarrier:
paras = GatewayParameters(port=boundPort, auth_token=secret, auto_convert=True)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe params instead of paras?

"""

_taskContext = None
_javaContext = None
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

_barrierContext?

} else {
logDebug(s"Started GatewayServer to port BarrierTaskContext on port $boundPort.")
}
// Write out the TaskContextInfo
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment should be moved too.

@SparkQA
Copy link

SparkQA commented Aug 12, 2018

Test build #94650 has finished for PR 22085 at commit 7b48829.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@mengxr mengxr left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should also add tests.

"""
return self._localProperties.get(key, None)

def barrier(self):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Create BarrierTaskContext that extends TaskContext and then move those two methods there.

raise Exception("Not supported to call getTaskInfos() inside a non-barrier task.")
else:
java_list = self._javaContext.getTaskInfos()
return [h for h in java_list]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Create BarrierTaskInfo class and wrap it over Java object.

.entryPoint(context.asInstanceOf[BarrierTaskContext])
.authToken(secret)
.javaPort(0)
.callbackClient(GatewayServer.DEFAULT_PYTHON_PORT, GatewayServer.defaultAddress(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Leave a TODO here. We do not have requests from Java to Python.

.javaPort(0)
.callbackClient(GatewayServer.DEFAULT_PYTHON_PORT, GatewayServer.defaultAddress(),
secret)
.build())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wait wait.. you guys sure have another Java gateway for each worker? (or did I rush to read this code?) Can you elaborate why this is needed? We should avoid this unless it's super required or necessary.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We have to port BarrierTaskContext from java to python side, otherwise there is no way to call BarrierTaskContext.barrier() from python side. Thus, of course, the JavaGateway is only initiated when the context is a BarrierTaskContext.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea, I read and understood if this is only initialised when the context is a BarrierTaskContext but this is super weird we start another Java gateway here. If it's a hard requirement, then I suspect the design issue. Should this be targeted to 2.4.0, @mengxr?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this should necessarily target 2.4.0, don't block by me since it's a new feature and probably we could consider another approach later but if we can avoid, I would suggest to avoid for now.

Let me try to track the design doc and changes about this. I think I need more time to check why it happened like this and if there's another way.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@HyukjinKwon Could you elaborate your concerns? Is it because resource usage or security?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Mainly the reason is about resource usage, unusual access pattern via Py4J at worker side, and the possibility of allowing JVM access within Python worker.

It pretty much looks an overkill to launch a Java gateway to allow access to call a function assuming from #22085 (comment). This pattern sounds pretty unusual - such cases, we usually send the data manually and read it in Python side, for instance TaskContext. Now, it opens a gateway for each worker if I am not mistaken.

I was thinking if we can avoid this. Can you elaborate why this is required and necessary? I haven't got enough time to look into this so was thinking about taking a look on this weekends.

This also now opens a possibility for an JVM access from worker side via BarrierTaskContext. For instance, I believe we can hack and access to JVM inside of UDFs.

Copy link
Member

@HyukjinKwon HyukjinKwon Aug 14, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, this is a design change https://cwiki.apache.org/confluence/display/SPARK/PySpark+Internals .. is this really required to open a gate there?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The major issue here is that we want to make the barrier() call blocking, the task shall wait until timeout or succeeded, do we have other ways to achieve this goal other than current approach here?

Copy link
Member

@HyukjinKwon HyukjinKwon Aug 15, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not yet. So I asked to hold it on for now since another gateway here looks the last choice, and was wondering if we can avoid to target 2.4.0. If this blocks, please go ahead. Will check it later on this weekends.

@HyukjinKwon
Copy link
Member

@mengxr, let me leave this link here again - #22085 (comment) in case this is missed by being folded.

@SparkQA
Copy link

SparkQA commented Aug 13, 2018

Test build #94681 has finished for PR 22085 at commit 289146d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class BarrierTaskContext(TaskContext):
  • class BarrierTaskInfo(object):


def __init__(self):
"""Construct a BarrierTaskContext, use get instead"""
pass
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would throw an exception BTW if this method should rather be banned

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This just follows TaskContext.__init__(), shall we update both?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, this is called in _getOrCreate. Sorry, I rushed to read. In this case, frankly I think we can remove this since that's the default constructor injected by Python or monkey patch to disallow the initialization (like we did for ImageSchema) but I guess we don't necessarily be super clever on this.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm okay as is.

"""

def __init__(self, info):
self.address = info.address
Copy link
Contributor

@mengxr mengxr Aug 14, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • should be info.address()
  • better to rename info to jobj to make it clear this is from Java

"""

def __init__(self, info):
self.address = info.address
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • should be info.address()
  • better to rename info to jobj to make it clear this is from Java

@SparkQA
Copy link

SparkQA commented Aug 14, 2018

Test build #94738 has finished for PR 22085 at commit 05c9609.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@mengxr
Copy link
Contributor

mengxr commented Aug 15, 2018

@HyukjinKwon Thanks for the feedback! We will replace the py4j route by a special implementation that can only trigger "context.barrier()" in JVM.

@HyukjinKwon
Copy link
Member

Thank you so much @mengxr and @jiangxb1987.

@jiangxb1987 jiangxb1987 changed the title [SPARK-25095][PySpark] Python support for BarrierTaskContext [WIP][SPARK-25095][PySpark] Python support for BarrierTaskContext Aug 17, 2018
/**
* Gateway to call BarrierTaskContext.barrier().
*/
def barrierAndServe(): Unit = {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not clear yet how to trigger this.

else:
_load_from_socket(self._port, self._secret)

def getTaskInfos(self):
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is not available temporarily.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

@SparkQA
Copy link

SparkQA commented Aug 17, 2018

Test build #94901 has finished for PR 22085 at commit e234a0a.

  • This patch fails from timeout after a configured wait of `340m`.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 20, 2018

Test build #94942 has finished for PR 22085 at commit 243a5a3.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jiangxb1987 jiangxb1987 changed the title [WIP][SPARK-25095][PySpark] Python support for BarrierTaskContext [SPARK-25095][PySpark] Python support for BarrierTaskContext Aug 20, 2018
@SparkQA
Copy link

SparkQA commented Aug 20, 2018

Test build #94951 has finished for PR 22085 at commit ba0ccad.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

import scala.collection.JavaConverters._

import org.apache.spark._
import org.apache.spark.{SparkException, _}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

_ should include SparkException already

return self._localProperties.get(key, None)


def _load_from_socket(port, auth_secret):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should document how this is different from the one in context.py.


// Authentication helper used when serving method calls via socket from Python side.
private lazy val authHelper = {
val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When SparkEnv.get returns null?

// Init a ServerSocket to accept method calls from Python side.
val isBarrier = context.isInstanceOf[BarrierTaskContext]
if (isBarrier) {
serverSocket = Some(new ServerSocket(0, 1, InetAddress.getByName("localhost")))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

minor: useful to add /* port */ and /* backlog */

sock = serverSocket.get.accept()
sock.setSoTimeout(10000)
val cmdString = readUtf8(sock)
if (cmdString.equals("run")) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we do not expect any other command from the socket, we should throw an exception

var sock: Socket = null
try {
sock = serverSocket.get.accept()
sock.setSoTimeout(10000)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should add a comment about this timeout.

barrierAndServe(sock)
}
} catch {
case _: SocketException =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the the timeout exception? I don't see any exception that we could silently ignore.

}

def writeUTF(str: String, dataOut: DataOutputStream) {
val bytes = str.getBytes(StandardCharsets.UTF_8)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: UTF_8 or always use StandardCharsets

@classmethod
def _getOrCreate(cls):
"""Internal function to get or create global BarrierTaskContext."""
if cls._taskContext is None:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Q: Does it handle python worker reuse?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIUC reuse python worker just means we start a python worker from a daemon thread, it shall not affect the input/output files related to worker.py.

import org.apache.spark.security.SocketAuthHelper
import org.apache.spark.util._


Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

tiny nit: I would remove this one back while addressing other comments.

@SparkQA
Copy link

SparkQA commented Aug 21, 2018

Test build #95037 has finished for PR 22085 at commit 2a8f3cb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 21, 2018

Test build #95040 has finished for PR 22085 at commit 1cacd40.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@mengxr
Copy link
Contributor

mengxr commented Aug 21, 2018

LGTM. I'm merging this into master. We might need a minor refactor for readability. But it shouldn't block developers testing this new feature. Thanks!

@asfgit asfgit closed this in ad45299 Aug 21, 2018
Load data from a given socket, this is a blocking method thus only return when the socket
connection has been closed.
This is copied from context.py, while modified the message protocol.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be nicer if we can deduplciate it later.

@HyukjinKwon
Copy link
Member

Thanks, @jiangxb1987 and @mengxr again.

}
// Close ServerSocket on task completion.
serverSocket.foreach { server =>
context.addTaskCompletionListener(_ => server.close())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is failing the Scala 2.12 build

[error] /Users/d_tsai/dev/apache-spark/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala:242: ambiguous reference to overloaded definition,
[error] both method addTaskCompletionListener in class TaskContext of type [U](f: org.apache.spark.TaskContext => U)org.apache.spark.TaskContext
[error] and  method addTaskCompletionListener in class TaskContext of type (listener: org.apache.spark.util.TaskCompletionListener)org.apache.spark.TaskContext
[error] match argument types (org.apache.spark.TaskContext => Unit)
[error]           context.addTaskCompletionListener(_ => server.close())
[error]                   ^
[error] one error found
[error] Compile failed at Aug 24, 2018 1:56:06 PM [31.582s]

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed in #22229

sock = serverSocket.get.accept()
// Wait for function call from python side.
sock.setSoTimeout(10000)
val input = new DataInputStream(sock.getInputStream())
Copy link
Contributor

@squito squito Aug 27, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why isn't authentication the first thing which happens on this connection? I don't think anything bad can happen in this case, but it just makes it more likely we leave a security hole here later on.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(I'd also like to do some refactoring of the socket setup code in python, and that can go further if we do authenticaion first here)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for catching this, yea I agree it would be better to move the authentication before recognising functions.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok I'm doing this -- SPARK-25253, will open a pr shortly

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants