[HUDI-2875] Make HoodieParquetWriter Thread safe and memory executor exit gracefully (#4264)
This commit is contained in:
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user