Would it make sense to add the annotations to the task manager and job
manager? In a non-istio environment it’d be a noop.

mTLS as a requirement is more complicated but having some docs around using
cert-manager might be enough depending on the orgs requirement.

On Mon, Jun 20, 2022 at 06:18, Őrhidi Mátyás <matyas.orh...@gmail.com>
wrote:

> It seems Istio must be configured to allow Akka cluster communication to
> bypass the Istio sidecar proxy:
> https://doc.akka.io/docs/akka-management/current/bootstrap/istio.html
>
> On Mon, Jun 20, 2022 at 11:30 AM Sigalit Eliazov <e.siga...@gmail.com>
> wrote:
>
>> Hi,
>> we have enabled HA as suggested, the task manager tries to reach the job
>> manager via pod id as expected but
>> the task manager is unable to connect to the job manager:
>>
>>
>> 2022-06-19 22:14:45,101 INFO  
>> org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Connecting 
>> to ResourceManager akka.tcp://
>> flink@192.168.3.144:6123/user/rpc/resourcemanager_0(8a98fdb734615089485c685afb0f402d)
>> .
>>
>>
>> 2022-06-19 22:14:45,242 WARN  akka.remote.transport.netty.NettyTransport     
>>               [] - Remote connection to [/
>> 192.168.3.144:6123
>> ] failed with java.io.IOException: Connection reset by peer
>>
>>
>> 2022-06-19 22:14:45,249 WARN  akka.remote.ReliableDeliverySupervisor         
>>               [] - Association with remote system [akka.tcp://
>> flink@192.168.3.144:6123
>> ] has failed, address is now gated for [50] ms. Reason: [Association failed 
>> with [akka.tcp://
>> flink@192.168.3.144:6123
>> ]] Caused by: [The remote system explicitly disassociated (reason unknown).]
>>
>>
>> 2022-06-19 22:14:45,255 INFO  
>> org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Could not 
>> resolve ResourceManager address akka.tcp://
>> flink@192.168.3.144:6123/user/rpc/resourcemanager_0
>> , retrying in 10000 ms: Could not connect to rpc endpoint under address 
>> akka.tcp://
>> flink@192.168.3.144:6123/user/rpc/resourcemanager_0.
>>
>> 2022-06-
>>
>>
>> Are there any additional definitions required for that?
>>
>>
>> thanks
>>
>> Sigalit
>>
>> On Thu, Jun 16, 2022 at 2:28 PM Yang Wang <danrtsey...@gmail.com> wrote:
>>
>>> Could you please have a try with high availability enabled[1]?
>>>
>>> If HA enabled, the internal jobmanager rpc service will not be created.
>>> Instead, the TaskManager retrieves the JobManager address via HA services
>>> and connects to it via pod ip.
>>>
>>> [1].
>>> https://github.com/apache/flink-kubernetes-operator/blob/main/examples/basic-checkpoint-ha.yaml
>>>
>>>
>>> Best,
>>> Yang
>>>
>>> Elisha, Moshe (Nokia - IL/Kfar Sava) <moshe.eli...@nokia.com>
>>> 于2022年6月16日周四 15:24写道:
>>>
>>>> Hello,
>>>>
>>>>
>>>>
>>>> We are launching Flink deployments using the Flink Kubernetes Operator
>>>> <https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-stable/>
>>>> on a Kubernetes cluster with Istio and mTLS enabled.
>>>>
>>>>
>>>>
>>>> We found that the TaskManager is unable to communicate with the
>>>> JobManager on the jobmanager-rpc port:
>>>>
>>>>
>>>>
>>>> 2022-06-15 15:25:40,508 WARN  akka.remote.ReliableDeliverySupervisor
>>>>                     [] - Association with remote system
>>>> [akka.tcp://flink@amf-events-to-inference-and-central.nwdaf-edge:6123]
>>>> has failed, address is now gated for [50] ms. Reason: [Association failed
>>>> with 
>>>> [akka.tcp://flink@amf-events-to-inference-and-central.nwdaf-edge:6123]]
>>>> Caused by: [The remote system explicitly disassociated (reason unknown).]
>>>>
>>>>
>>>>
>>>> The reason for the issue is that the JobManager service port
>>>> definitions are not following the Istio guidelines
>>>> https://istio.io/latest/docs/ops/configuration/traffic-management/protocol-selection/
>>>> (see example below).
>>>>
>>>>
>>>>
>>>> We believe a change to the default port definitions is needed but for
>>>> now, is there an immediate action we can take to work around the issue?
>>>> Perhaps overriding the default port definitions somehow?
>>>>
>>>>
>>>>
>>>> Thanks.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> flink-kubernetes-operator 1.0.0
>>>>
>>>> Flink 1.14-java11
>>>>
>>>> Kubernetes v1.19.5
>>>>
>>>> Istio 1.7.6
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> # k get service inference-results-to-analytics-engine -o yaml
>>>>
>>>> apiVersion: v1
>>>>
>>>> kind: Service
>>>>
>>>> metadata:
>>>>
>>>> ...
>>>>
>>>>   labels:
>>>>
>>>>     app: inference-results-to-analytics-engine
>>>>
>>>>     type: flink-native-kubernetes
>>>>
>>>>   name: inference-results-to-analytics-engine
>>>>
>>>> spec:
>>>>
>>>>   clusterIP: None
>>>>
>>>>   ports:
>>>>
>>>>   - name: jobmanager-rpc # should start with “tcp-“ or add "appProtocol"
>>>> property
>>>>
>>>>     port: 6123
>>>>
>>>>     protocol: TCP
>>>>
>>>>     targetPort: 6123
>>>>
>>>>   - name: blobserver # should start with "tcp-" or add "appProtocol"
>>>> property
>>>>
>>>>     port: 6124
>>>>
>>>>     protocol: TCP
>>>>
>>>>     targetPort: 6124
>>>>
>>>>   selector:
>>>>
>>>>     app: inference-results-to-analytics-engine
>>>>
>>>>     component: jobmanager
>>>>
>>>>     type: flink-native-kubernetes
>>>>
>>>>   sessionAffinity: None
>>>>
>>>>   type: ClusterIP
>>>>
>>>> status:
>>>>
>>>>   loadBalancer: {}
>>>>
>>>>
>>>>
>>>

Reply via email to