[HUDI-2875] Make HoodieParquetWriter Thread safe and memory executor exit gracefully (#4264)
This commit is contained in:
@@ -37,6 +37,7 @@ import java.util.concurrent.ExecutorCompletionService;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@@ -48,7 +49,7 @@ import java.util.stream.Collectors;
|
||||
public class BoundedInMemoryExecutor<I, O, E> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class);
|
||||
|
||||
private static final long TERMINATE_WAITING_TIME_SECS = 60L;
|
||||
// Executor service used for launching write thread.
|
||||
private final ExecutorService producerExecutorService;
|
||||
// Executor service used for launching read thread.
|
||||
@@ -168,6 +169,27 @@ public class BoundedInMemoryExecutor<I, O, E> {
|
||||
public void shutdownNow() {
|
||||
producerExecutorService.shutdownNow();
|
||||
consumerExecutorService.shutdownNow();
|
||||
// close queue to force producer stop
|
||||
queue.close();
|
||||
}
|
||||
|
||||
public boolean awaitTermination() {
|
||||
// if current thread has been interrupted before awaitTermination was called, we still give
|
||||
// executor a chance to proceeding. So clear the interrupt flag and reset it if needed before return.
|
||||
boolean interruptedBefore = Thread.interrupted();
|
||||
boolean producerTerminated = false;
|
||||
boolean consumerTerminated = false;
|
||||
try {
|
||||
producerTerminated = producerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS);
|
||||
consumerTerminated = consumerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS);
|
||||
} catch (InterruptedException ie) {
|
||||
// fail silently for any other interruption
|
||||
}
|
||||
// reset interrupt flag if needed
|
||||
if (interruptedBefore) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
return producerTerminated && consumerTerminated;
|
||||
}
|
||||
|
||||
public BoundedInMemoryQueue<I, O> getQueue() {
|
||||
|
||||
@@ -860,12 +860,14 @@ public class HoodieTestDataGenerator implements AutoCloseable {
|
||||
return false;
|
||||
}
|
||||
|
||||
public GenericRecord generateGenericRecord() {
|
||||
return generateGenericRecord(genPseudoRandomUUID(rand).toString(), "0",
|
||||
genPseudoRandomUUID(rand).toString(), genPseudoRandomUUID(rand).toString(), rand.nextLong());
|
||||
}
|
||||
|
||||
public List<GenericRecord> generateGenericRecords(int numRecords) {
|
||||
List<GenericRecord> list = new ArrayList<>();
|
||||
IntStream.range(0, numRecords).forEach(i -> {
|
||||
list.add(generateGenericRecord(genPseudoRandomUUID(rand).toString(), "0",
|
||||
genPseudoRandomUUID(rand).toString(), genPseudoRandomUUID(rand).toString(), rand.nextLong()));
|
||||
});
|
||||
IntStream.range(0, numRecords).forEach(i -> list.add(generateGenericRecord()));
|
||||
return list;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user