Skip to content

Commit

Permalink
[HUDI-2875] Make HoodieParquetWriter Thread safe and memory executor …
Browse files Browse the repository at this point in the history
…exit gracefully (#4264)
  • Loading branch information
guanziyue authored May 5, 2022
1 parent d794f4f commit abb4893
Show file tree
Hide file tree
Showing 17 changed files with 121 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import javax.annotation.concurrent.NotThreadSafe;

import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
Expand Down Expand Up @@ -66,6 +68,7 @@
* Users should ensure there are no duplicates when "insert" operation is used and if the respective config is enabled. So, above scenario should not
* happen and every batch should have new records to be inserted. Above example is for illustration purposes only.
*/
@NotThreadSafe
public class HoodieConcatHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieMergeHandle<T, I, K, O> {

private static final Logger LOG = LogManager.getLogger(HoodieConcatHandle.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,12 +42,15 @@
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import javax.annotation.concurrent.NotThreadSafe;

import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;

@NotThreadSafe
public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {

private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,8 @@
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import javax.annotation.concurrent.NotThreadSafe;

import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
Expand Down Expand Up @@ -91,6 +93,7 @@
*
* </p>
*/
@NotThreadSafe
public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {

private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@

import org.apache.avro.generic.GenericRecord;

import javax.annotation.concurrent.NotThreadSafe;

import java.io.IOException;
import java.util.Iterator;
import java.util.List;
Expand All @@ -45,6 +47,7 @@
* The implementation performs a merge-sort by comparing the key of the record being written to the list of
* keys in newRecordKeys (sorted in-memory).
*/
@NotThreadSafe
public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieMergeHandle<T, I, K, O> {

private final Queue<String> newRecordKeysSorted = new PriorityQueue<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,11 +28,14 @@
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import javax.annotation.concurrent.NotThreadSafe;

/**
* A HoodieCreateHandle which writes all data into a single file.
* <p>
* Please use this with caution. This can end up creating very large files if not used correctly.
*/
@NotThreadSafe
public class HoodieUnboundedCreateHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieCreateHandle<T, I, K, O> {

private static final Logger LOG = LogManager.getLogger(HoodieUnboundedCreateHandle.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,18 @@
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter;

import javax.annotation.concurrent.NotThreadSafe;

import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;

/**
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides a way to check if
* the current file can take more records with the <code>canWrite()</code>
*
* ATTENTION: HoodieParquetWriter is not thread safe and developer should take care of the order of write and close
*/
@NotThreadSafe
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
extends ParquetWriter<IndexedRecord> implements HoodieFileWriter<R> {

Expand Down Expand Up @@ -106,4 +111,9 @@ public void writeAvro(String key, IndexedRecord object) throws IOException {
writeSupport.add(key);
}
}

@Override
public void close() throws IOException {
super.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -148,13 +148,16 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
} catch (Exception e) {
throw new HoodieException(e);
} finally {
// HUDI-2875: mergeHandle is not thread safe, we should totally terminate record inputting
// and executor firstly and then close mergeHandle.
if (reader != null) {
reader.close();
}
mergeHandle.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
mergeHandle.close();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ protected List<WriteStatus> computeNext() {
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
bufferedIteratorExecutor.awaitTermination();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,13 +102,16 @@ public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List
} catch (Exception e) {
throw new HoodieException(e);
} finally {
// HUDI-2875: mergeHandle is not thread safe, we should totally terminate record inputting
// and executor firstly and then close mergeHandle.
if (reader != null) {
reader.close();
}
mergeHandle.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
mergeHandle.close();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ protected List<WriteStatus> computeNext() {
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
bufferedIteratorExecutor.awaitTermination();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,13 +102,16 @@ public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List
} catch (Exception e) {
throw new HoodieException(e);
} finally {
// HUDI-2875: mergeHandle is not thread safe, we should totally terminate record inputting
// and executor firstly and then close mergeHandle.
if (reader != null) {
reader.close();
}
mergeHandle.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
mergeHandle.close();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ protected List<WriteStatus> computeNext() {
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
bufferedIteratorExecutor.awaitTermination();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,10 +80,11 @@ void executeBootstrap(HoodieBootstrapHandle<?, ?, ?, ?> bootstrapHandle, Path so
} catch (Exception e) {
throw new HoodieException(e);
} finally {
bootstrapHandle.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
bootstrapHandle.close();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,9 +68,9 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException {
void executeBootstrap(HoodieBootstrapHandle<?, ?, ?, ?> bootstrapHandle,
Path sourceFilePath, KeyGeneratorInterface keyGenerator, String partitionPath, Schema avroSchema) throws Exception {
BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void> wrapper = null;
ParquetReader<IndexedRecord> reader =
AvroParquetReader.<IndexedRecord>builder(sourceFilePath).withConf(table.getHadoopConf()).build();
try {
ParquetReader<IndexedRecord> reader =
AvroParquetReader.<IndexedRecord>builder(sourceFilePath).withConf(table.getHadoopConf()).build();
wrapper = new BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config.getWriteBufferLimitBytes(),
new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> {
String recKey = keyGenerator.getKey(inp).getRecordKey();
Expand All @@ -84,10 +84,12 @@ void executeBootstrap(HoodieBootstrapHandle<?, ?, ?, ?> bootstrapHandle,
} catch (Exception e) {
throw new HoodieException(e);
} finally {
bootstrapHandle.close();
reader.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
bootstrapHandle.close();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,15 @@
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.testutils.HoodieClientTestHarness;

import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

import java.util.Iterator;
import java.util.List;

import scala.Tuple2;
Expand Down Expand Up @@ -105,6 +107,7 @@ protected Integer getResult() {
} finally {
if (executor != null) {
executor.shutdownNow();
executor.awaitTermination();
}
}
}
Expand Down Expand Up @@ -152,7 +155,49 @@ protected Integer getResult() {
} finally {
if (executor != null) {
executor.shutdownNow();
executor.awaitTermination();
}
}
}

@Test
public void testExecutorTermination() {
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
Iterator<GenericRecord> unboundedRecordIter = new Iterator<GenericRecord>() {
@Override
public boolean hasNext() {
return true;
}

@Override
public GenericRecord next() {
return dataGen.generateGenericRecord();
}
};

BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
new BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
@Override
protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> record) {
}

@Override
protected void finish() {
}

@Override
protected Integer getResult() {
return 0;
}
};

BoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor =
new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), unboundedRecordIter,
consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA),
getPreExecuteRunnable());
executor.shutdownNow();
boolean terminatedGracefully = executor.awaitTermination();
assertTrue(terminatedGracefully);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
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;

Expand All @@ -48,7 +49,7 @@
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.
Expand Down Expand Up @@ -168,6 +169,27 @@ public boolean isRemaining() {
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() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -860,12 +860,14 @@ public boolean deleteExistingKeyIfPresent(HoodieKey key) {
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;
}

Expand Down

0 comments on commit abb4893

Please sign in to comment.