1
0

[HUDI-3430] Fix Deltastreamer to properly shut down the services upon failure (#4824)

This commit is contained in:
Y Ethan Guo
2022-02-18 05:44:56 -08:00
committed by GitHub
parent de8161ae96
commit fba5822ee3
4 changed files with 44 additions and 16 deletions

View File

@@ -82,10 +82,16 @@ public abstract class AsyncClusteringService extends HoodieAsyncTableService {
}
LOG.info("Clustering executor shutting down properly");
} catch (InterruptedException ie) {
hasError = true;
LOG.warn("Clustering executor got interrupted exception! Stopping", ie);
} catch (IOException e) {
LOG.error("Clustering executor failed", e);
hasError = true;
LOG.error("Clustering executor failed due to IOException", e);
throw new HoodieIOException(e.getMessage(), e);
} catch (Exception e) {
hasError = true;
LOG.error("Clustering executor failed", e);
throw e;
}
return true;
}, executor)).toArray(CompletableFuture[]::new)), executor);

View File

@@ -92,10 +92,16 @@ public abstract class AsyncCompactService extends HoodieAsyncTableService {
}
LOG.info("Compactor shutting down properly!!");
} catch (InterruptedException ie) {
hasError = true;
LOG.warn("Compactor executor thread got interrupted exception. Stopping", ie);
} catch (IOException e) {
LOG.error("Compactor executor failed", e);
hasError = true;
LOG.error("Compactor executor failed due to IOException", e);
throw new HoodieIOException(e.getMessage(), e);
} catch (Exception e) {
hasError = true;
LOG.error("Compactor executor failed", e);
throw e;
}
return true;
}, executor)).toArray(CompletableFuture[]::new)), executor);

View File

@@ -41,7 +41,10 @@ import java.util.function.Function;
public abstract class HoodieAsyncService implements Serializable {
private static final Logger LOG = LogManager.getLogger(HoodieAsyncService.class);
private static final long POLLING_SECONDS = 10;
// Flag indicating whether an error is incurred in the service
protected boolean hasError;
// Flag to track if the service is started.
private boolean started;
// Flag indicating shutdown is externally requested
@@ -82,9 +85,13 @@ public abstract class HoodieAsyncService implements Serializable {
return shutdown;
}
public boolean hasError() {
return hasError;
}
/**
* Wait till the service shutdown. If the service shutdown with exception, it will be thrown
*
*
* @throws ExecutionException
* @throws InterruptedException
*/
@@ -109,6 +116,7 @@ public abstract class HoodieAsyncService implements Serializable {
public void shutdown(boolean force) {
if (!shutdownRequested || force) {
shutdownRequested = true;
shutdown = true;
if (executor != null) {
if (force) {
executor.shutdownNow();
@@ -178,8 +186,8 @@ public abstract class HoodieAsyncService implements Serializable {
public void waitTillPendingAsyncServiceInstantsReducesTo(int numPending) throws InterruptedException {
try {
queueLock.lock();
while (!isShutdown() && (pendingInstants.size() > numPending)) {
consumed.await();
while (!isShutdown() && !hasError() && (pendingInstants.size() > numPending)) {
consumed.await(POLLING_SECONDS, TimeUnit.SECONDS);
}
} finally {
queueLock.unlock();
@@ -202,8 +210,8 @@ public abstract class HoodieAsyncService implements Serializable {
* @throws InterruptedException
*/
HoodieInstant fetchNextAsyncServiceInstant() throws InterruptedException {
LOG.info("Waiting for next instant upto 10 seconds");
HoodieInstant instant = pendingInstants.poll(10, TimeUnit.SECONDS);
LOG.info(String.format("Waiting for next instant up to %d seconds", POLLING_SECONDS));
HoodieInstant instant = pendingInstants.poll(POLLING_SECONDS, TimeUnit.SECONDS);
if (instant != null) {
try {
queueLock.lock();