I've finally fixed the issues identified here in the thread: The blob manager and the application master/job manager allocate their ports in a specified range.
You can now whitelist a port range in the firewall and Flink services will only allocate ports in that range: https://github.com/apache/flink/blob/master/docs/setup/yarn_setup.md#running-flink-on-yarn-behind-firewalls Please let me know if that fixes your issues. Note that the fix is only available in 1.0-SNAPSHOT. On Wed, Nov 25, 2015 at 6:58 PM, Robert Metzger <rmetz...@apache.org> wrote: > Hi, > I just wanted to let you know that I didn't forget about this! > > The BlobManager in 1.0-SNAPSHOT has already a configuration parameter to > use a certain range of ports. > I'm trying to add the same feature for YARN tomorrow. > Sorry for the delay. > > > On Tue, Nov 10, 2015 at 9:27 PM, Cory Monty <cory.mo...@getbraintree.com> > wrote: > >> Thanks, Stephan. >> >> I'll give those two workarounds a try! >> >> On Tue, Nov 10, 2015 at 2:18 PM, Stephan Ewen <se...@apache.org> wrote: >> >>> Hi Cory! >>> >>> There is no flag to define the BlobServer port right now, but we should >>> definitely add this: https://issues.apache.org/jira/browse/FLINK-2996 >>> >>> If your setup is such that the firewall problem is only between client >>> and master node (and the workers can reach the master on all ports), then >>> you can try two workarounds: >>> >>> 1) Start the program in the cluster (or on the master node, via ssh). >>> >>> 2) Add the program jar to the lib directory of Flink, and start your >>> program with the RemoteExecutor, without a jar attachment. Then it only >>> needs to communicate to the actor system (RPC) port, which is not random in >>> standalone mode (6123 by default). >>> >>> Stephan >>> >>> >>> >>> >>> On Tue, Nov 10, 2015 at 8:46 PM, Cory Monty <cory.mo...@getbraintree.com >>> > wrote: >>> >>>> I'm also running into an issue with a non-YARN cluster. When submitting >>>> a JAR to Flink, we'll need to have an arbitrary port open on all of the >>>> hosts, which we don't know about until the socket attempts to bind; a bit >>>> of a problem for us. >>>> >>>> Are there ways to submit a JAR to Flink that bypasses the need for the >>>> BlobServer's random port binding? Or, to control the port BlobServer binds >>>> to? >>>> >>>> Cheers, >>>> >>>> Cory >>>> >>>> On Thu, Nov 5, 2015 at 8:07 AM, Niels Basjes <ni...@basjes.nl> wrote: >>>> >>>>> That is what I tried. Couldn't find that port though. >>>>> >>>>> On Thu, Nov 5, 2015 at 3:06 PM, Robert Metzger <rmetz...@apache.org> >>>>> wrote: >>>>> >>>>>> Hi, >>>>>> >>>>>> cool, that's good news. >>>>>> >>>>>> The RM proxy is only for the web interface of the AM. >>>>>> >>>>>> I'm pretty sure that the MapReduce AM has at least two ports: >>>>>> - one for the web interface (accessible through the RM proxy, so >>>>>> behind the firewall) >>>>>> - one for the AM RPC (and that port is allocated within the >>>>>> configured range, open through the firewall). >>>>>> >>>>>> You can probably find the RPC port in the log file of the running >>>>>> MapReduce AM (to find that, identify the NodeManager running the AM, >>>>>> access >>>>>> the NM web interface and retrieve the logs of the container running the >>>>>> AM). >>>>>> >>>>>> Maybe the mapreduce client also logs the AM RPC port when querying >>>>>> the status of a running job. >>>>>> >>>>>> >>>>>> On Thu, Nov 5, 2015 at 2:59 PM, Niels Basjes <ni...@basjes.nl> wrote: >>>>>> >>>>>>> Hi, >>>>>>> >>>>>>> I checked and this setting has been set to a limited port range of >>>>>>> only 100 port numbers. >>>>>>> >>>>>>> I tried to find the actual port an AM is running on and couldn't >>>>>>> find it (I'm not the admin on that cluster) >>>>>>> >>>>>>> The url to the AM that I use to access it always looks like this: >>>>>>> >>>>>>> http://master-001.xxxxxx.net:8088/proxy/application_1443166961758_85492/index.html >>>>>>> >>>>>>> As you can see I never connect directly; always via the proxy that >>>>>>> runs over the master on a single fixed port. >>>>>>> >>>>>>> Niels >>>>>>> >>>>>>> On Thu, Nov 5, 2015 at 2:46 PM, Robert Metzger <rmetz...@apache.org> >>>>>>> wrote: >>>>>>> >>>>>>>> While discussing with my colleagues about the issue today, we came >>>>>>>> up with another approach to resolve the issue: >>>>>>>> >>>>>>>> d) Upload the job jar to HDFS (or another FS) and trigger the >>>>>>>> execution of the jar using an HTTP request to the web interface. >>>>>>>> >>>>>>>> We could add some tooling into the /bin/flink client to submit a >>>>>>>> job like this transparently, so users would not need to bother with the >>>>>>>> file upload and request sending. >>>>>>>> Also, Sachin started a discussion on the dev@ list to add support >>>>>>>> for submitting jobs over the web interface, so maybe we can base the >>>>>>>> fix >>>>>>>> for FLINK-2960 on that. >>>>>>>> >>>>>>>> I've also looked into the Hadoop MapReduce code and it seems they >>>>>>>> do the following: >>>>>>>> When submitting a job, they are uploading the job jar file to HDFS. >>>>>>>> They also upload a configuration file that contains all the config >>>>>>>> options >>>>>>>> of the job. Then, they submit this altogether as an application to >>>>>>>> YARN. >>>>>>>> So far, there has not been any firewall involved. They establish a >>>>>>>> connection between the JobClient and the ApplicationMaster when the >>>>>>>> user is >>>>>>>> querying the current job status, but I could not find any special code >>>>>>>> getting the status over HTTP. >>>>>>>> >>>>>>>> But I found the following configuration parameter: >>>>>>>> "yarn.app.mapreduce.am.job.client.port-range", so it seems that they >>>>>>>> try to >>>>>>>> allocate the AM port within that range (if specified). >>>>>>>> Niels, can you check if this configuration parameter is set in your >>>>>>>> environment? I assume your firewall allows outside connections from >>>>>>>> that >>>>>>>> port range. >>>>>>>> So we also have a new approach: >>>>>>>> >>>>>>>> f) Allocate the YARN application master (and blob manager) within a >>>>>>>> user-specified port-range. >>>>>>>> >>>>>>>> This would be really easy to implement, because we would just need >>>>>>>> to go through the range until we find an available port. >>>>>>>> >>>>>>>> >>>>>>>> On Tue, Nov 3, 2015 at 1:06 PM, Niels Basjes <ni...@basjes.nl> >>>>>>>> wrote: >>>>>>>> >>>>>>>>> Great! >>>>>>>>> >>>>>>>>> I'll watch the issue and give it a test once I see a working patch. >>>>>>>>> >>>>>>>>> Niels Basjes >>>>>>>>> >>>>>>>>> On Tue, Nov 3, 2015 at 1:03 PM, Maximilian Michels <m...@apache.org >>>>>>>>> > wrote: >>>>>>>>> >>>>>>>>>> Hi Niels, >>>>>>>>>> >>>>>>>>>> Thanks a lot for reporting this issue. I think it is a very >>>>>>>>>> common setup in corporate infrastructure to have restrictive firewall >>>>>>>>>> settings. For Flink 1.0 (and probably in a minor 0.10.X release) we >>>>>>>>>> will >>>>>>>>>> have to address this issue to ensure proper integration of Flink. >>>>>>>>>> >>>>>>>>>> I've created a JIRA to keep track: >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-2960 >>>>>>>>>> >>>>>>>>>> Best regards, >>>>>>>>>> Max >>>>>>>>>> >>>>>>>>>> On Tue, Nov 3, 2015 at 11:02 AM, Niels Basjes <ni...@basjes.nl> >>>>>>>>>> wrote: >>>>>>>>>> >>>>>>>>>>> Hi, >>>>>>>>>>> >>>>>>>>>>> I forgot to answer your other question: >>>>>>>>>>> >>>>>>>>>>> On Mon, Nov 2, 2015 at 4:34 PM, Robert Metzger < >>>>>>>>>>> rmetz...@apache.org> wrote: >>>>>>>>>>> >>>>>>>>>>>> so the problem is that you can not submit a job to Flink using >>>>>>>>>>>> the "/bin/flink" tool, right? >>>>>>>>>>>> I assume Flink and its TaskManagers properly start and connect >>>>>>>>>>>> to each other (the number of TaskManagers is shown correctly in >>>>>>>>>>>> the web >>>>>>>>>>>> interface). >>>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> Correct. Flink starts (i see the jobmanager UI) but the actual >>>>>>>>>>> job is not started. >>>>>>>>>>> >>>>>>>>>>> Niels Basjes >>>>>>>>>>> >>>>>>>>>> >>>>>>>>>> >>>>>>>>> >>>>>>>>> >>>>>>>>> -- >>>>>>>>> Best regards / Met vriendelijke groeten, >>>>>>>>> >>>>>>>>> Niels Basjes >>>>>>>>> >>>>>>>> >>>>>>>> >>>>>>> >>>>>>> >>>>>>> -- >>>>>>> Best regards / Met vriendelijke groeten, >>>>>>> >>>>>>> Niels Basjes >>>>>>> >>>>>> >>>>>> >>>>> >>>>> >>>>> -- >>>>> Best regards / Met vriendelijke groeten, >>>>> >>>>> Niels Basjes >>>>> >>>> >>>> >>> >> >