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

[#2241] improvement(server): Introduce storage flush operation timeout cancel to avoid disk hang #2247

Merged
merged 2 commits into from
Nov 14, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.uniffle.common.future;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Supplier;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.uniffle.common.exception.RssException;
import org.apache.uniffle.common.util.ThreadUtils;

public class CompletableFutureUtils {
private static final Logger LOGGER = LoggerFactory.getLogger(CompletableFutureUtils.class);

public static <T> T withTimeoutCancel(Supplier<T> supplier, long timeoutMills) throws Exception {
return withTimeoutCancel(supplier, timeoutMills, "");
}

public static <T> T withTimeoutCancel(
Supplier<T> supplier, long timeoutMills, String operationName) throws Exception {
CompletableFuture<T> future =
CompletableFuture.supplyAsync(
supplier,
Executors.newSingleThreadExecutor(ThreadUtils.getThreadFactory(operationName)));
future.exceptionally(
throwable -> {
throw new RssException(throwable);
});

CompletableFuture<T> extended =
CompletableFutureExtension.orTimeout(future, timeoutMills, TimeUnit.MILLISECONDS);
try {
return extended.get();
} catch (Exception e) {
if (e instanceof ExecutionException) {
Throwable internalThrowable = e.getCause();
if (internalThrowable instanceof TimeoutException) {
LOGGER.error(
"The operation of [{}] haven't finished in the {}(millis). Drop this execution!",
operationName,
timeoutMills);
throw new TimeoutException();
}
if (internalThrowable instanceof Exception) {
throw (Exception) internalThrowable;
}
}
throw e;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.uniffle.common.future;

import java.util.concurrent.TimeoutException;
import java.util.function.Supplier;

import org.junit.jupiter.api.Test;

import org.apache.uniffle.common.exception.RssException;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.fail;

public class CompletableFutureUtilsTest {

@Test
public void timeoutTest() {
// case1: legal operation
Supplier<Integer> supplier = () -> 1;
try {
int result = CompletableFutureUtils.withTimeoutCancel(supplier, 100);
assertEquals(1, result);
} catch (Exception e) {
fail();
}

// case2: illegal
supplier =
() -> {
try {
Thread.sleep(100000);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
return 10;
};
try {
int result = CompletableFutureUtils.withTimeoutCancel(supplier, 100);
fail();
} catch (Exception e) {
if (!(e instanceof TimeoutException)) {
fail();
}
}

// case3: fast fail when internal supplier throw exception
supplier =
() -> {
throw new RssException("Hello");
};
try {
int result = CompletableFutureUtils.withTimeoutCancel(supplier, 100);
fail();
} catch (Exception e) {
if (e instanceof RssException) {
// ignore
} else {
fail();
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,10 @@
import org.slf4j.LoggerFactory;

import org.apache.uniffle.common.config.RssBaseConf;
import org.apache.uniffle.common.exception.RssException;
import org.apache.uniffle.common.executor.ThreadPoolManager;
import org.apache.uniffle.common.function.ConsumerWithException;
import org.apache.uniffle.common.future.CompletableFutureUtils;
import org.apache.uniffle.common.util.ThreadUtils;
import org.apache.uniffle.server.flush.EventDiscardException;
import org.apache.uniffle.server.flush.EventInvalidException;
Expand All @@ -42,6 +44,7 @@
import org.apache.uniffle.storage.common.Storage;
import org.apache.uniffle.storage.util.StorageType;

import static org.apache.uniffle.server.ShuffleServerConf.STORAGE_FLUSH_OPERATION_TIMEOUT_SEC;
import static org.apache.uniffle.server.ShuffleServerMetrics.EVENT_QUEUE_SIZE;

public class DefaultFlushEventHandler implements FlushEventHandler {
Expand All @@ -56,6 +59,7 @@ public class DefaultFlushEventHandler implements FlushEventHandler {
protected final BlockingQueue<ShuffleDataFlushEvent> flushQueue = Queues.newLinkedBlockingQueue();
private ConsumerWithException<ShuffleDataFlushEvent> eventConsumer;
private final ShuffleServer shuffleServer;
private final long flushMaxWaitTimeoutSec;

private volatile boolean stopped = false;

Expand All @@ -65,6 +69,7 @@ public DefaultFlushEventHandler(
ShuffleServer shuffleServer,
ConsumerWithException<ShuffleDataFlushEvent> eventConsumer) {
this.shuffleServerConf = conf;
this.flushMaxWaitTimeoutSec = conf.getLong(STORAGE_FLUSH_OPERATION_TIMEOUT_SEC);
this.storageType =
StorageType.valueOf(shuffleServerConf.get(RssBaseConf.RSS_STORAGE_TYPE).name());
this.storageManager = storageManager;
Expand All @@ -83,6 +88,24 @@ public void handle(ShuffleDataFlushEvent event) {
}
}

private void consumeEvent(ShuffleDataFlushEvent event) throws Exception {
if (flushMaxWaitTimeoutSec <= 0) {
eventConsumer.accept(event);
return;
}

Supplier<Void> supplier =
() -> {
try {
this.eventConsumer.accept(event);
} catch (Exception e) {
throw new RssException(e);
}
return null;
};
CompletableFutureUtils.withTimeoutCancel(supplier, flushMaxWaitTimeoutSec * 1000);
}

/**
* @param event
* @param storage
Expand All @@ -95,7 +118,7 @@ private void handleEventAndUpdateMetrics(ShuffleDataFlushEvent event, Storage st
try {
readLock.lock();
try {
eventConsumer.accept(event);
consumeEvent(event);
} finally {
readLock.unlock();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -676,6 +676,13 @@ public class ShuffleServerConf extends RssBaseConf {
.defaultValue(10 * 60L)
.withDescription("The storage remove resource operation timeout.");

public static final ConfigOption<Long> STORAGE_FLUSH_OPERATION_TIMEOUT_SEC =
ConfigOptions.key("rss.server.storage.flushOperationTimeoutSec")
.longType()
.defaultValue(-1L)
.withDescription(
"The storage flush max timeout second, this will not be activated by default");

public static final ConfigOption<Boolean> SERVER_MERGE_ENABLE =
ConfigOptions.key("rss.server.merge.enable")
.booleanType()
Expand Down
Loading