Hi Marvin,
The discussion was not resolved yet.
Can you tell us a bit about your use case? Maybe that would help with the
decision.
Which of the discussed approaches would work best for your use case and why?
Thanks, Fabian
2018-06-25 13:27 GMT+02:00 Marvin777 :
> Hi, Shuyi:
>
> What is the pro
Hi, Shuyi:
What is the progress of the discussion? We also look forward to this
feature.
Thanks.
Shuyi Chen 于2018年6月8日周五 下午3:04写道:
> Thanks a lot for the comments, Till and Fabian.
>
> The RemoteEnvrionment does provide a way to specify jar files at
> construction, but we want the jar files to
Thanks a lot for the comments, Till and Fabian.
The RemoteEnvrionment does provide a way to specify jar files at
construction, but we want the jar files to be specified dynamically in the
user code, e.g. in a DDL statement, and the jar files might be in a remote
DFS. As we discussed, I think there
We could also offer a feature that users can request classloaders with
additional jars.
This could work as follows:
1) Users register jar files in the ExecutionEnvironment (similar to cached
files) with a name, e.g., env.registerJarFile("~/myJar.jar", "myName");
2) In a function, the user can requ
I see Shuyi's point that it would nice to allow adding jar files which
should be part of the user code classloader programmatically. Actually, we
expose this functionality in the `RemoteEnvironment` where you can specify
additional jars which shall be shipped to the cluster in the constructor. I
as
Hi Aljoscha, Fabian, Rong, Ted and Timo,
Thanks a lot for the feedback. Let me clarify the usage scenario in a bit
more detail. The context is that we want to add support for SQL DDL to load
UDF from external JARs located either in local filesystem or HDFS or a HTTP
endpoint in Flink SQL. The loca
I think the question here is whether registering Jar files (or other
executable files) during job submission is sufficient for @shuyi's use
case.
If I understand correctly regarding the part of dynamic distribution of the
external libraries in runtime. This is used to deal with DDL/DSL such as:
Yes, we are using the addJar functionionality of the JobGraph as well
for the SQL Client.
I think the execution environment is not the right place to specify
jars. The location of the jars depends on the submission method. If a
local path is specified in the main() method of a packaged Flink j
I think this functionality is already there, we just have to expose it in the
right places: ClusterClient.submitJob() takes a JobGraph, JobGraph has method
addJar() for adding jars that need to be in the classloader for executing a
user program.
> On 16. May 2018, at 12:34, Fabian Hueske wrote
Hi Ted,
The design doc is in late draft status and proposes support for SQL DDL
statements (CREATE TABLE, CREATE FUNCTION, etc.).
The question about registering JARs came up because we need a way to
distribute JAR files that contain the code of user-defined functions.
The design doc will soon be
bq. In a design document, Timo mentioned that we can ship multiple JAR files
Mind telling us where the design doc can be retrieved ?
Thanks
On Wed, May 16, 2018 at 1:29 AM, Fabian Hueske wrote:
> Hi,
>
> I'm not sure if we need to modify the existing method.
> What we need is a bit different f
Hi,
I'm not sure if we need to modify the existing method.
What we need is a bit different from what registerCachedFile() provides.
The method ensures that a file is copied to each TaskManager and can be
locally accessed from a function's RuntimeContext.
In our case, we don't need to access the fi
+1. This could be very useful for "dynamic" UDF.
Just to clarify, if I understand correctly, we are tying to use an ENUM
indicator to
(1) Replace the current Boolean isExecutable flag.
(2) Provide additional information used by ExecutionEnvironment to decide
when/where to use the DistributedCached
Hi Flink devs,
In an effort to support loading external libraries and creating UDFs from
external libraries using DDL in Flink SQL, we want to use Flink’s Blob
Server to distribute the external libraries in runtime and load those
libraries into the user code classloader automatically.
However, th
14 matches
Mail list logo