[HUDI-1922] Bulk insert with row writer supports mor table (#2981)
This commit is contained in:
@@ -25,14 +25,12 @@ import org.apache.hudi.common.model.WriteOperationType;
|
|||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
|
||||||
import org.apache.hudi.common.util.CommitUtils;
|
import org.apache.hudi.common.util.CommitUtils;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.table.HoodieSparkTable;
|
import org.apache.hudi.table.HoodieSparkTable;
|
||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
@@ -95,7 +93,9 @@ public class DataSourceInternalWriterHelper {
|
|||||||
|
|
||||||
public void createInflightCommit() {
|
public void createInflightCommit() {
|
||||||
metaClient.getActiveTimeline().transitionRequestedToInflight(
|
metaClient.getActiveTimeline().transitionRequestedToInflight(
|
||||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty());
|
new HoodieInstant(State.REQUESTED,
|
||||||
|
CommitUtils.getCommitActionType(operationType, metaClient.getTableType()),
|
||||||
|
instantTime), Option.empty());
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieTable getHoodieTable() {
|
public HoodieTable getHoodieTable() {
|
||||||
|
|||||||
@@ -117,7 +117,9 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("test bulk insert dataset with datasource impl") {
|
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||||
|
.foreach(tableType => {
|
||||||
|
test("test bulk insert dataset with datasource impl for " + tableType) {
|
||||||
initSparkContext("test_bulk_insert_datasource")
|
initSparkContext("test_bulk_insert_datasource")
|
||||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||||
try {
|
try {
|
||||||
@@ -127,6 +129,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
|||||||
//create a new table
|
//create a new table
|
||||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||||
|
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
|
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
|
||||||
@@ -167,6 +170,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
|||||||
FileUtils.deleteDirectory(path.toFile)
|
FileUtils.deleteDirectory(path.toFile)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
})
|
||||||
|
|
||||||
test("test insert dataset without precombine field") {
|
test("test insert dataset without precombine field") {
|
||||||
val session = SparkSession.builder()
|
val session = SparkSession.builder()
|
||||||
|
|||||||
Reference in New Issue
Block a user