Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[KYUUBI #5952] Disconnect connections without running operations after engine maxlife time graceful period #6040

Closed
wants to merge 5 commits into from

Conversation

turboFei
Copy link
Member

@turboFei turboFei commented Feb 4, 2024

🔍 Description

Issue References 🔗

We found that, some kyuubi connections(maybe managed by jdbc connection pool likes hikari) always keep alive, and the engine can not be terminated after exceeds the max life time.

So, In this pr, I introduce a graceful period after spark engine max life time, after the graceful period, the connections without running operations will be disconnected forcibly.

Close #5952

Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

Types of changes 🔖

  • Bugfix (non-breaking change which fixes an issue)
  • New feature (non-breaking change which adds functionality)
  • Breaking change (fix or feature that would cause existing functionality to change)

Test Plan 🧪

Behavior Without This Pull Request ⚰️

Behavior With This Pull Request 🎉

Related Unit Tests


Checklist 📝

Be nice. Be informative.

ut

ut

revert

save

save

debug

Revert "debug"

This reverts commit fe46552.

retry
@turboFei turboFei force-pushed the close_on_nooperation_rebase branch from 739ccca to 45ad348 Compare February 4, 2024 01:52
@@ -68,7 +67,7 @@ abstract class ServiceDiscovery(
def stopGracefully(isLost: Boolean = false): Unit = {
while (fe.be.sessionManager.getOpenSessionCount > 0) {
info(s"${fe.be.sessionManager.getOpenSessionCount} connection(s) are active, delay shutdown")
Thread.sleep(1000 * 60)
Thread.sleep(TimeUnit.SECONDS.toMillis(5))
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

short the sleep interval to pass UT

@turboFei turboFei force-pushed the close_on_nooperation_rebase branch from 4fc4d71 to b39c9b3 Compare February 4, 2024 05:18
@turboFei turboFei self-assigned this Feb 4, 2024
@codecov-commenter
Copy link

codecov-commenter commented Feb 4, 2024

Codecov Report

Attention: 6 lines in your changes are missing coverage. Please review.

Comparison is base (576379c) 61.06% compared to head (4329a85) 61.07%.
Report is 5 commits behind head on master.

❗ Current head 4329a85 differs from pull request most recent head 0b05ddc. Consider uploading reports for the commit 0b05ddc to get more accurate results

Files Patch % Lines
...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala 72.22% 2 Missing and 3 partials ⚠️
...org/apache/kyuubi/ha/client/ServiceDiscovery.scala 0.00% 1 Missing ⚠️
Additional details and impacted files
@@            Coverage Diff            @@
##             master    #6040   +/-   ##
=========================================
  Coverage     61.06%   61.07%           
  Complexity       23       23           
=========================================
  Files           623      623           
  Lines         37160    37180   +20     
  Branches       5038     5038           
=========================================
+ Hits          22693    22708   +15     
- Misses        12012    12015    +3     
- Partials       2455     2457    +2     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@turboFei turboFei force-pushed the close_on_nooperation_rebase branch from 992aefd to 4329a85 Compare February 4, 2024 18:05
// for ETCD, the de-registering process might be blocked, so deregister it async
info(s"Spark engine has been running for more than $maxLifetime ms," +
s" deregistering from engine discovery space.")
frontendServices.flatMap(_.discoveryService).foreach(_.stop())
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

09:46:29.180 vertx-blocked-thread-checker WARN BlockedThreadChecker: Thread Thread[vert.x-eventloop-thread-0,5,main] has been blocked for 19018 ms, time limit is 2000 ms
io.vertx.core.VertxException: Thread blocked
        at java.lang.Thread.sleep(Native Method)
        at org.apache.kyuubi.ha.client.ServiceDiscovery.stopGracefully(ServiceDiscovery.scala:71)
        at org.apache.kyuubi.ha.client.etcd.EtcdDiscoveryClient$DeRegisterWatcher.$anonfun$onNext$2(EtcdDiscoveryClient.scala:384)
        at org.apache.kyuubi.ha.client.etcd.EtcdDiscoveryClient$DeRegisterWatcher.$anonfun$onNext$2$adapted(EtcdDiscoveryClient.scala:381)
        at org.apache.kyuubi.ha.client.etcd.EtcdDiscoveryClient$DeRegisterWatcher$$Lambda$2964/1803489267.apply(Unknown Source)
        at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
        at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
        at org.apache.kyuubi.ha.client.etcd.EtcdDiscoveryClient$DeRegisterWatcher.onNext(EtcdDiscoveryClient.scala:381)
        at io.etcd.jetcd.impl.WatchImpl$WatcherImpl.onNext(WatchImpl.java:307)
        at io.etcd.jetcd.impl.WatchImpl$WatcherImpl$$Lambda$2892/1233911532.handle(Unknown Source)
        at io.vertx.grpc.stub.StreamObserverReadStream.onNext(StreamObserverReadStream.java:37)
        at io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onMessage(ClientCalls.java:468)
        at io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInternal(ClientCallImpl.java:667)
        at io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInContext(ClientCallImpl.java:654)
        at io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
        at io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
        at io.vertx.grpc.VertxChannelBuilder.lambda$null$0(VertxChannelBuilder.java:305)
        at io.vertx.grpc.VertxChannelBuilder$$Lambda$2847/1899882869.handle(Unknown Source)
        at io.vertx.core.impl.ContextInternal.dispatch(ContextInternal.java:276)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need to deregister it async? The blocked vertx-blocked-thread-checker thread does not seem to be the checker thread.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

image

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, discoveryService.stop() blocks the subsequent close session operation, causing the EngineServiceDiscovery.stopGracefully method to continue waiting.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't know. It doesn't happen every time.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Anyway, the current solution looks good to me.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the async deregister may introduce issues, say for some reason the deregistering takes 3s to complete and we stop the frontend/backend services, new incoming sessions will get errors

Copy link
Member

@pan3793 pan3793 Feb 6, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if only etcd has such an issue, let's add an internal configuration to switch the sync/async behavior. the Guava's SameThreadExecutorService may help

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about we execute stopGracefully asynchronously in DeRegisterWatcher#onNext?

@turboFei
Copy link
Member Author

turboFei commented Feb 4, 2024

cc @bowenliang123 and @pan3793
It passed the UT eventually.

Copy link
Contributor

@bowenliang123 bowenliang123 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@wForget wForget requested a review from pan3793 February 6, 2024 03:04
@turboFei turboFei force-pushed the close_on_nooperation_rebase branch from dd34342 to 0b05ddc Compare February 6, 2024 06:52
Copy link
Member

@pan3793 pan3793 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, pending CI

@turboFei turboFei closed this in 8c3f471 Feb 7, 2024
@wForget wForget added this to the v1.8.1 milestone Feb 8, 2024
wForget pushed a commit to wForget/kyuubi that referenced this pull request Feb 8, 2024
…s after engine maxlife time graceful period

# 🔍 Description
## Issue References 🔗

We found that, some kyuubi connections(maybe managed by jdbc connection pool likes hikari) always keep alive, and the engine can not be terminated after exceeds the max life time.

So, In this pr, I introduce a graceful period after spark engine max life time, after the graceful period, the connections without running operations will be disconnected forcibly.

Close apache#5952

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [x] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes apache#6040 from turboFei/close_on_nooperation_rebase.

Closes apache#5952

0b05ddc [Fei Wang] comments
4329a85 [Fei Wang] async stop
b39c9b3 [Fei Wang] use short sleep
7c123f6 [Fei Wang] save
45ad348 [Fei Wang]  check no running operation

Authored-by: Fei Wang <[email protected]>
Signed-off-by: Fei Wang <[email protected]>

(cherry picked from commit 8c3f471)
pan3793 pushed a commit that referenced this pull request Feb 10, 2024
… after engine maxlife time graceful period

# 🔍 Description
## Issue References 🔗

We found that, some kyuubi connections(maybe managed by jdbc connection pool likes hikari) always keep alive, and the engine can not be terminated after exceeds the max life time.

So, In this pr, I introduce a graceful period after spark engine max life time, after the graceful period, the connections without running operations will be disconnected forcibly.

Close #5952

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [x] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes #6040 from turboFei/close_on_nooperation_rebase.

Closes #5952

0b05ddc [Fei Wang] comments
4329a85 [Fei Wang] async stop
b39c9b3 [Fei Wang] use short sleep
7c123f6 [Fei Wang] save
45ad348 [Fei Wang]  check no running operation

Authored-by: Fei Wang <fwang12ebay.com>
Signed-off-by: Fei Wang <fwang12ebay.com>

(cherry picked from commit 8c3f471)

# 🔍 Description
## Issue References 🔗

This pull request fixes #

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [ ] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes #6054 from wForget/cherry_pick_6040_to_1_8.

Closes #5952

0749d75 [Fei Wang] [KYUUBI #5952] Disconnect connections without running operations after engine maxlife time graceful period

Authored-by: Fei Wang <[email protected]>
Signed-off-by: Cheng Pan <[email protected]>
pan3793 pushed a commit that referenced this pull request Feb 17, 2024
… after engine maxlife time graceful period

# 🔍 Description
## Issue References 🔗

We found that, some kyuubi connections(maybe managed by jdbc connection pool likes hikari) always keep alive, and the engine can not be terminated after exceeds the max life time.

So, In this pr, I introduce a graceful period after spark engine max life time, after the graceful period, the connections without running operations will be disconnected forcibly.

Close #5952

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [x] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes #6040 from turboFei/close_on_nooperation_rebase.

Closes #5952

0b05ddc [Fei Wang] comments
4329a85 [Fei Wang] async stop
b39c9b3 [Fei Wang] use short sleep
7c123f6 [Fei Wang] save
45ad348 [Fei Wang]  check no running operation

Authored-by: Fei Wang <fwang12ebay.com>
Signed-off-by: Fei Wang <fwang12ebay.com>

(cherry picked from commit 8c3f471)

# 🔍 Description
## Issue References 🔗

This pull request fixes #

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [ ] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes #6054 from wForget/cherry_pick_6040_to_1_8.

Closes #5952

0749d75 [Fei Wang] [KYUUBI #5952] Disconnect connections without running operations after engine maxlife time graceful period

Authored-by: Fei Wang <[email protected]>
Signed-off-by: Cheng Pan <[email protected]>
zhaohehuhu pushed a commit to zhaohehuhu/incubator-kyuubi that referenced this pull request Mar 21, 2024
…s after engine maxlife time graceful period

# 🔍 Description
## Issue References 🔗

We found that, some kyuubi connections(maybe managed by jdbc connection pool likes hikari) always keep alive, and the engine can not be terminated after exceeds the max life time.

So, In this pr, I introduce a graceful period after spark engine max life time, after the graceful period, the connections without running operations will be disconnected forcibly.

Close apache#5952

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [x] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes apache#6040 from turboFei/close_on_nooperation_rebase.

Closes apache#5952

0b05ddc [Fei Wang] comments
4329a85 [Fei Wang] async stop
b39c9b3 [Fei Wang] use short sleep
7c123f6 [Fei Wang] save
45ad348 [Fei Wang]  check no running operation

Authored-by: Fei Wang <[email protected]>
Signed-off-by: Fei Wang <[email protected]>
beryllw pushed a commit to beryllw/incubator-kyuubi that referenced this pull request Jun 7, 2024
…ations after engine maxlife time graceful period

# 🔍 Description
## Issue References 🔗

We found that, some kyuubi connections(maybe managed by jdbc connection pool likes hikari) always keep alive, and the engine can not be terminated after exceeds the max life time.

So, In this pr, I introduce a graceful period after spark engine max life time, after the graceful period, the connections without running operations will be disconnected forcibly.

Close apache#5952

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [x] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes apache#6040 from turboFei/close_on_nooperation_rebase.

Closes apache#5952

0b05ddc [Fei Wang] comments
4329a85 [Fei Wang] async stop
b39c9b3 [Fei Wang] use short sleep
7c123f6 [Fei Wang] save
45ad348 [Fei Wang]  check no running operation

Authored-by: Fei Wang <fwang12ebay.com>
Signed-off-by: Fei Wang <fwang12ebay.com>

(cherry picked from commit 8c3f471)

# 🔍 Description
## Issue References 🔗

This pull request fixes #

## Describe Your Solution 🔧

Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. List any dependencies that are required for this change.

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [ ] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️

#### Behavior With This Pull Request 🎉

#### Related Unit Tests

---

# Checklist 📝

- [ ] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes apache#6054 from wForget/cherry_pick_6040_to_1_8.

Closes apache#5952

0749d75 [Fei Wang] [KYUUBI apache#5952] Disconnect connections without running operations after engine maxlife time graceful period

Authored-by: Fei Wang <[email protected]>
Signed-off-by: Cheng Pan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[FEATURE] Disconnect connections without running operations after engine maxlife time graceful period
5 participants