[HUDI-2009] Fixing extra commit metadata in row writer path (#3075)
This commit is contained in:
committed by
GitHub
parent
1d3cd06572
commit
8c0dbaa9b3
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.internal;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -26,18 +28,24 @@ import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.sources.v2.DataSourceOptions;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataWriter;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieDataSourceInternalWriter}.
|
||||
@@ -47,12 +55,16 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
|
||||
@Test
|
||||
public void testDataSourceWriter() throws Exception {
|
||||
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP);
|
||||
}
|
||||
|
||||
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
String instantTime = "001";
|
||||
// init writer
|
||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(extraMetadata), false);
|
||||
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
|
||||
|
||||
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
|
||||
@@ -64,7 +76,6 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
int size = 10 + RANDOM.nextInt(1000);
|
||||
int batches = 5;
|
||||
Dataset<Row> totalInputRows = null;
|
||||
|
||||
for (int j = 0; j < batches; j++) {
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
|
||||
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
|
||||
@@ -80,11 +91,50 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||
commitMessages.add(commitMetadata);
|
||||
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
|
||||
// verify output
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty());
|
||||
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
|
||||
|
||||
// verify extra metadata
|
||||
Option<HoodieCommitMetadata> commitMetadataOption = HoodieClientTestUtils.getCommitMetadataForLatestInstant(metaClient);
|
||||
assertTrue(commitMetadataOption.isPresent());
|
||||
Map<String, String> actualExtraMetadata = new HashMap<>();
|
||||
commitMetadataOption.get().getExtraMetadata().entrySet().stream().filter(entry ->
|
||||
!entry.getKey().equals(HoodieCommitMetadata.SCHEMA_KEY)).forEach(entry -> actualExtraMetadata.put(entry.getKey(), entry.getValue()));
|
||||
assertEquals(actualExtraMetadata, expectedExtraMetadata);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataSourceWriterExtraCommitMetadata() throws Exception {
|
||||
|
||||
String commitExtraMetaPrefix = "commit_extra_meta_";
|
||||
Map<String, String> extraMeta = new HashMap<>();
|
||||
extraMeta.put(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key(), commitExtraMetaPrefix);
|
||||
extraMeta.put(commitExtraMetaPrefix + "a", "valA");
|
||||
extraMeta.put(commitExtraMetaPrefix + "b", "valB");
|
||||
extraMeta.put("commit_extra_c", "valC"); // should not be part of commit extra metadata
|
||||
|
||||
Map<String, String> expectedMetadata = new HashMap<>();
|
||||
expectedMetadata.putAll(extraMeta);
|
||||
expectedMetadata.remove(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key());
|
||||
expectedMetadata.remove("commit_extra_c");
|
||||
|
||||
testDataSourceWriterInternal(extraMeta, expectedMetadata);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataSourceWriterEmptyExtraCommitMetadata() throws Exception {
|
||||
String commitExtraMetaPrefix = "commit_extra_meta_";
|
||||
Map<String, String> extraMeta = new HashMap<>();
|
||||
extraMeta.put(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key(), commitExtraMetaPrefix);
|
||||
extraMeta.put("keyA", "valA");
|
||||
extraMeta.put("keyB", "valB");
|
||||
extraMeta.put("commit_extra_c", "valC");
|
||||
// none of the keys has commit metadata key prefix.
|
||||
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -98,8 +148,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
String instantTime = "00" + i;
|
||||
// init writer
|
||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||
|
||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
|
||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||
Dataset<Row> totalInputRows = null;
|
||||
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
|
||||
@@ -142,8 +191,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
String instantTime = "00" + i;
|
||||
// init writer
|
||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||
|
||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
|
||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||
Dataset<Row> totalInputRows = null;
|
||||
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
|
||||
@@ -189,7 +237,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
String instantTime0 = "00" + 0;
|
||||
// init writer
|
||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
|
||||
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
|
||||
|
||||
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
|
||||
@@ -227,7 +275,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
||||
// 2nd batch. abort in the end
|
||||
String instantTime1 = "00" + 1;
|
||||
dataSourceInternalWriter =
|
||||
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
|
||||
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
|
||||
|
||||
for (int j = 0; j < batches; j++) {
|
||||
|
||||
Reference in New Issue
Block a user