Thanks Till!
Do you know what is 1.11.0 release date?

On Mon, May 18, 2020 at 12:49 PM Till Rohrmann <trohrm...@apache.org> wrote:

> Hi Omar,
>
> with FLINK-15154 [1] which will be released with the upcoming 1.11.0
> release, it will be possible to bind the Blob server to the hostname
> specified via jobmanager.bind-host. Per default it will still bind to the
> wildcard address but with this option you can bind it to localhost, for
> example. Be aware, though, that the Blob server needs to be accessible from
> all TaskManager processes. Hence, if you run a distributed cluster, then
> binding the blob server to localhost won't work.
>
> [1] https://issues.apache.org/jira/browse/FLINK-15154
>
> Cheers,
> Till
>
> On Wed, May 13, 2020 at 10:10 AM Dawid Wysakowicz <dwysakow...@apache.org>
> wrote:
>
>> Hi Omar,
>>
>> Theoretically I think it could be possible to change the address on which
>> the BlobServer runs (even to localhost). There is no configuration option
>> for it now and the BlobServer always binds to the wildcard. One important
>> aspect to consider here is that the BlobServer must be accessible from all
>> the components of the cluster: taskmanagers, jobmanager (if I am not
>> mistaken).
>>
>> @Arvid  Wouldn't changing the line 192 in BlobServer:
>>
>>         this.serverSocket = NetUtils.createSocketFromPorts(ports,
>>                 (port) -> socketFactory.createServerSocket(port,
>> finalBacklog));
>>
>> to e.g.
>>
>>         this.serverSocket = NetUtils.createSocketFromPorts(ports,
>>               (port) -> socketFactory.createServerSocket(port,
>> finalBacklog, InetAddress.getByName(configuration.get(BLOB_HOSTNAME))));
>>
>> do the trick?
>>
>> That said I think for now your only option is what Arvid suggested.
>> Remember though that by default BlobServer is exposed on os chosen port, so
>> it might change if you restart your cluster. You can set a staticport/range
>> with 'blob.server.port' configuration option. If you feel strong about the
>> requirement to configure the host as well, feel free to open a jira ticket.
>> On 12/05/2020 13:34, Arvid Heise wrote:
>>
>> Hi Omar,
>>
>> wouldn't it be possible to just create an iptable rule that allows access
>> to 1098 only from localhost? I don't think you can open a socket just for
>> localhost programmatically (at least not from Java).
>>
>> Best,
>>
>> Arvid
>>
>> On Tue, May 12, 2020 at 12:51 PM Omar Gawi <omar.g...@gmail.com> wrote:
>>
>>> Hi All,
>>>
>>> I have Apache Flink running as part of our java program , on a linux
>>> machine.
>>> The Flink runs on thread(s) within the same java process.
>>> I see that the machine has the BLOB server port 1098 exposed to the
>>> outside :
>>>
>>> davc@sdavc:~$ netstat -anp | grep LISTEN
>>>
>>> (Not all processes could be identified, non-owned process info
>>>
>>> will not be shown, you would have to be root to see it all.)
>>>
>>> tcp        0      0 0.0.0.0:22              0.0.0.0:*
>>> LISTEN      -
>>>
>>> tcp        0      0 127.0.0.1:5432          0.0.0.0:*
>>> LISTEN      311/postgres
>>>
>>> tcp6       0      0 :::8080                 :::*
>>> LISTEN      -
>>>
>>> tcp6       0      0 :::21                   :::*
>>> LISTEN      -
>>>
>>> tcp6       0      0 :::22                   :::*
>>> LISTEN      -
>>>
>>> tcp6       0      0 ::1:5432                :::*
>>> LISTEN      311/postgres
>>>
>>> tcp6       0      0 :::8443                 :::*
>>> LISTEN      -
>>> *tcp6       0      0 :::1098                 :::*
>>> LISTEN      -*
>>>
>>>
>>> This bring to our team security concerns , when other external
>>> user/system open connection (for telnet or other protocols) to this port
>>> (accidentally or not), we get below error in the java app log:
>>>
>>> 2020-04-23 07:54:58 ERROR BlobServerConnection:131 - Error while
>>> executing BLOB connection.
>>>
>>> java.io.IOException: Unknown operation 3
>>>
>>>                at
>>> org.apache.flink.runtime.blob.BlobServerConnection.run(BlobServerConnection.java:122)
>>>
>>>
>>> My question if is there a way to avoid exposing this port  to the
>>> outside, and keep it available only for it's original purpose : serving the
>>> localhost/127.0.0.1 requests which come from the flink engine.
>>>
>>>
>>> Thank you and stay safe.
>>>
>>> Omar
>>>
>>
>>
>> --
>>
>> Arvid Heise | Senior Java Developer
>>
>> <https://www.ververica.com/>
>>
>> Follow us @VervericaData
>>
>> --
>>
>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
>> Conference
>>
>> Stream Processing | Event Driven | Real Time
>>
>> --
>>
>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>>
>> --
>> Ververica GmbH Registered at Amtsgericht Charlottenburg: HRB 158244 B 
>> Managing
>> Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng
>>
>>
>>

Reply via email to