[
https://issues.apache.org/jira/browse/KYLIN-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Gabor Arki updated KYLIN-4689:
------------------------------
Affects Version/s: v3.0.1
v3.0.2
> Deadlock in Kylin job execution
> -------------------------------
>
> Key: KYLIN-4689
> URL: https://issues.apache.org/jira/browse/KYLIN-4689
> Project: Kylin
> Issue Type: Bug
> Components: Job Engine
> Affects Versions: v3.0.0, v3.1.0, v3.0.1, v3.0.2
> Reporter: Gabor Arki
> Priority: Critical
>
> h4. Reproduction steps
> * Install Kylin 3.1.0
> * Deploy a streaming cube
> * Enable the cube having historical data present in the Kafka topic
> * Note: in our case, we had 3 cubes deployed, each consuming ~20-20 hourly
> segments from Kafka when the cubes were enabled
> h4. Expected result
> * Kylin is starting to process stream segments with stream jobs, eventually
> processing the older segments and catching up with the stream
> h4. Actual result
> * After a short time, all jobs are completely stuck without any progress.
> Some in running state, some in pending state.
> * The following logs are continously written:
> {code:java}
> 2020-08-06 06:16:22 INFO [Scheduler 116797841 Job
> 12750aea-3b96-c817-64e8-bf893d8c120f-254] MapReduceExecutable:409 -
> 12750aea-3b96-c817-64e8-bf893d8c120f-00, parent lock
> path(/cube_job_lock/cube_vm) is locked by other job result is true ,ephemeral
> lock path :/cube_job_ephemeral_lock/cube_vm is locked by other job result is
> true,will try after one minute
> 2020-08-06 06:16:33 WARN [FetcherRunner 787667774-43] FetcherRunner:56 -
> There are too many jobs running, Job Fetch will wait until next schedule time
> {code}
> * Zookeeper indicates the following locks are in place:
> {code:java}
> ls /kylin/kylin_metadata/cube_job_ephemeral_lock
> [cube_cm, cube_vm, cube_jm]
> ls /kylin/kylin_metadata/cube_job_ephemeral_lock/cube_cm
> []
> ls /kylin/kylin_metadata/cube_job_ephemeral_lock/cube_vm
> []
> ls /kylin/kylin_metadata/cube_job_ephemeral_lock/cube_jm
> []
> ls /kylin/kylin_metadata/cube_job_lock
> [cube_cm, cube_vm, cube_jm]
> ls /kylin/kylin_metadata/cube_job_lock/cube_cm
> [f888380e-9ff4-98f5-2df4-1ae71e045f93]
> ls /kylin/kylin_metadata/cube_job_lock/cube_vm
> [fc186bd9-1186-6ed4-e58c-bbbf6dd8ef74]
> ls /kylin/kylin_metadata/cube_job_lock/cube_jm
> [d1a6475a-9ab2-5ee4-6714-f395e20cfc01]
> {code}
> * The job IDs for the running jobs:
> *
> ** 169f75fa-a02f-221b-fc48-037bc7a842d0
> ** 0b5dae1b-6faf-66c5-71dc-86f5b820f1c4
> ** 00924699-8b51-8091-6e71-34ccfeba3a98
> ** 4620192a-71e1-16dd-3b05-44d7f9144ad4
> ** 416355c2-a3d7-57eb-55c6-c042aa256510
> ** 12750aea-3b96-c817-64e8-bf893d8c120f
> ** 42819dde-5857-fd6b-b075-439952f47140
> ** 00128937-bd4a-d6c1-7a4e-744dee946f67
> ** 46a0233f-217e-9155-725b-c815ad77ba2c
> ** 062150ba-bacd-6644-4801-3a51b260d1c5
> As you can see, the 10 jobs that are actually running do not possess the
> locks thus cannot actually do anything. On the other hand, the 3 jobs
> possessing the locks cannot resume running because there are already 10 jobs
> in running state, thus cannot proceed and release the locks. This is a
> deadlock and the cluster is completely stuck.
> We have been observing this behavior in 3.0.0 (where rolling back
> https://issues.apache.org/jira/browse/KYLIN-4165 resolved the issue), and now
> in 3.1.0 as well. It has been originally reported in the comments of
> https://issues.apache.org/jira/browse/KYLIN-4348 but I'm not sure that it's
> related to that bug/epic.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)