-
Notifications
You must be signed in to change notification settings - Fork 594
HDDS-5341. Container report processing is single threaded #2338
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
Merged
Merged
Changes from all commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
9047b57
HDDS-5341. Container report processing is single threaded.
bharatviswa504 f325c4c
add to ozone-default
bharatviswa504 c61227a
fix dup method
bharatviswa504 b165db6
fix usage
bharatviswa504 e172f83
add metric for scheduled
bharatviswa504 4a6f518
add sample test using fixedthreadpool executor in event queue
bharatviswa504 fdc7f8a
fix test failure
bharatviswa504 69e7c83
remove unnecessary return in java doc.
bharatviswa504 2b2698c
fix cs
bharatviswa504 8c08059
fix cs
bharatviswa504 14616e5
fix cs
bharatviswa504 0fd7b20
fix tests
bharatviswa504 847f800
fix tests
bharatviswa504 8fedd12
fix test unnecessary changes
bharatviswa504 35e62a8
Merge branch 'master' into HDDS-5341
bharatviswa504 462f544
fix cs
bharatviswa504 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
134 changes: 134 additions & 0 deletions
134
...framework/src/main/java/org/apache/hadoop/hdds/server/events/FixedThreadPoolExecutor.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,134 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * <p> | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * <p> | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.hadoop.hdds.server.events; | ||
|
|
||
| import org.apache.hadoop.hdds.conf.OzoneConfiguration; | ||
| import org.apache.hadoop.util.StringUtils; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
|
|
||
| import org.apache.hadoop.metrics2.annotation.Metric; | ||
| import org.apache.hadoop.metrics2.annotation.Metrics; | ||
| import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; | ||
| import org.apache.hadoop.metrics2.lib.MutableCounterLong; | ||
|
|
||
| import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_EVENT_PREFIX; | ||
| import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_EVENT_THREAD_POOL_SIZE_DEFAULT; | ||
|
|
||
| /** | ||
| * Fixed thread pool EventExecutor to call all the event handler one-by-one. | ||
| * | ||
| * @param <P> the payload type of events | ||
| */ | ||
| @Metrics(context = "EventQueue") | ||
| public class FixedThreadPoolExecutor<P> implements EventExecutor<P> { | ||
|
|
||
| private static final String EVENT_QUEUE = "EventQueue"; | ||
|
|
||
| private static final Logger LOG = | ||
| LoggerFactory.getLogger(FixedThreadPoolExecutor.class); | ||
|
|
||
| private final String name; | ||
|
|
||
| private final ExecutorService executor; | ||
|
|
||
| @Metric | ||
| private MutableCounterLong queued; | ||
|
|
||
| @Metric | ||
| private MutableCounterLong done; | ||
|
|
||
| @Metric | ||
| private MutableCounterLong failed; | ||
|
|
||
| @Metric | ||
| private MutableCounterLong scheduled; | ||
|
|
||
| /** | ||
| * Create FixedThreadPoolExecutor. | ||
| * | ||
| * @param eventName | ||
| * @param name Unique name used in monitoring and metrics. | ||
| */ | ||
| public FixedThreadPoolExecutor(String eventName, String name) { | ||
| this.name = name; | ||
| DefaultMetricsSystem.instance() | ||
| .register(EVENT_QUEUE + name, "Event Executor metrics ", this); | ||
|
|
||
|
|
||
| OzoneConfiguration configuration = new OzoneConfiguration(); | ||
| int threadPoolSize = configuration.getInt(OZONE_SCM_EVENT_PREFIX + | ||
| StringUtils.camelize(eventName) + ".thread.pool.size", | ||
| OZONE_SCM_EVENT_THREAD_POOL_SIZE_DEFAULT); | ||
|
|
||
| executor = Executors.newFixedThreadPool(threadPoolSize, runnable -> { | ||
| Thread thread = new Thread(runnable); | ||
| thread.setName(EVENT_QUEUE + "-" + name); | ||
| return thread; | ||
| }); | ||
| } | ||
|
|
||
| @Override | ||
| public void onMessage(EventHandler<P> handler, P message, EventPublisher | ||
| publisher) { | ||
| queued.incr(); | ||
| executor.execute(() -> { | ||
| scheduled.incr(); | ||
| try { | ||
| handler.onMessage(message, publisher); | ||
| done.incr(); | ||
| } catch (Exception ex) { | ||
| LOG.error("Error on execution message {}", message, ex); | ||
| failed.incr(); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| @Override | ||
| public long failedEvents() { | ||
| return failed.value(); | ||
| } | ||
|
|
||
| @Override | ||
| public long successfulEvents() { | ||
| return done.value(); | ||
| } | ||
|
|
||
| @Override | ||
| public long queuedEvents() { | ||
| return queued.value(); | ||
| } | ||
|
|
||
| @Override | ||
| public long scheduledEvents() { | ||
| return scheduled.value(); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| executor.shutdown(); | ||
| } | ||
|
|
||
| @Override | ||
| public String getName() { | ||
| return name; | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.