1
0

[HUDI-2875] Make HoodieParquetWriter Thread safe and memory executor exit gracefully (#4264)

This commit is contained in:
guanziyue
2022-05-06 04:49:34 +08:00
committed by GitHub
parent d794f4fbf9
commit abb4893b25
17 changed files with 121 additions and 12 deletions

View File

@@ -95,6 +95,7 @@ public class SparkLazyInsertIterable<T extends HoodieRecordPayload> extends Hood
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
bufferedIteratorExecutor.awaitTermination();
}
}
}

View File

@@ -80,10 +80,11 @@ class OrcBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
} catch (Exception e) {
throw new HoodieException(e);
} finally {
bootstrapHandle.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
bootstrapHandle.close();
}
}
}

View File

@@ -68,9 +68,9 @@ class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
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();
@@ -84,10 +84,12 @@ class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
} catch (Exception e) {
throw new HoodieException(e);
} finally {
bootstrapHandle.close();
reader.close();
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
}
bootstrapHandle.close();
}
}
}

View File

@@ -28,6 +28,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
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$;
@@ -35,6 +36,7 @@ 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;
@@ -105,6 +107,7 @@ public class TestBoundedInMemoryExecutorInSpark extends HoodieClientTestHarness
} finally {
if (executor != null) {
executor.shutdownNow();
executor.awaitTermination();
}
}
}
@@ -152,7 +155,49 @@ public class TestBoundedInMemoryExecutorInSpark extends HoodieClientTestHarness
} 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);
}
}