1
0

[HUDI-2009] Fixing extra commit metadata in row writer path (#3075)

This commit is contained in:
Sivabalan Narayanan
2021-07-08 03:07:27 -04:00
committed by GitHub
parent 1d3cd06572
commit 8c0dbaa9b3
12 changed files with 180 additions and 29 deletions

View File

@@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.avro.Schema;
@@ -211,4 +212,22 @@ public class HoodieClientTestUtils {
return valuesAsList.stream();
}
public static Option<HoodieCommitMetadata> getCommitMetadataForLatestInstant(HoodieTableMetaClient metaClient) {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
if (timeline.lastInstant().isPresent()) {
return getCommitMetadataForInstant(metaClient, timeline.lastInstant().get());
} else {
return Option.empty();
}
}
private static Option<HoodieCommitMetadata> getCommitMetadataForInstant(HoodieTableMetaClient metaClient, HoodieInstant instant) {
try {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
byte[] data = timeline.getInstantDetails(instant).get();
return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class));
} catch (Exception e) {
throw new HoodieException("Failed to read schema from commit metadata", e);
}
}
}

View File

@@ -55,6 +55,7 @@ import org.apache.spark.sql.Row;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -132,6 +133,18 @@ public class DataSourceUtils {
}
}
public static Map<String, String> getExtraMetadata(Map<String, String> properties) {
Map<String, String> extraMetadataMap = new HashMap<>();
if (properties.containsKey(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key())) {
properties.entrySet().forEach(entry -> {
if (entry.getKey().startsWith(properties.get(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key()))) {
extraMetadataMap.put(entry.getKey(), entry.getValue());
}
});
}
return extraMetadataMap;
}
/**
* Create a payload class via reflection, do not ordering/precombine value.
*/

View File

@@ -38,8 +38,8 @@ import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.types.StructType;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Helper class for HoodieDataSourceInternalWriter used by Spark datasource v2.
@@ -54,14 +54,17 @@ public class DataSourceInternalWriterHelper {
private final SparkRDDWriteClient writeClient;
private final HoodieTable hoodieTable;
private final WriteOperationType operationType;
private Map<String, String> extraMetadata;
public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession sparkSession, Configuration configuration) {
SparkSession sparkSession, Configuration configuration, Map<String, String> extraMetadata) {
this.instantTime = instantTime;
this.operationType = WriteOperationType.BULK_INSERT;
this.extraMetadata = extraMetadata;
this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig);
writeClient.setOperationType(operationType);
writeClient.startCommitWithTime(instantTime);
this.metaClient = HoodieTableMetaClient.builder().setConf(configuration).setBasePath(writeConfig.getBasePath()).build();
this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient);
}
@@ -76,7 +79,7 @@ public class DataSourceInternalWriterHelper {
public void commit(List<HoodieWriteStat> writeStatList) {
try {
writeClient.commitStats(instantTime, writeStatList, Option.of(new HashMap<>()),
writeClient.commitStats(instantTime, writeStatList, Option.of(extraMetadata),
CommitUtils.getCommitActionType(operationType, metaClient.getTableType()));
} catch (Exception ioe) {
throw new HoodieException(ioe.getMessage(), ioe);

View File

@@ -67,6 +67,6 @@ public class DefaultSource extends BaseDefaultSource implements DataSourceV2,
options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).isPresent()
? options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).get() : null);
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
getConfiguration(), arePartitionRecordsSorted));
getConfiguration(), options, arePartitionRecordsSorted));
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
@@ -26,13 +27,16 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
import org.apache.spark.sql.types.StructType;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -46,15 +50,18 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
private final StructType structType;
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
private final Boolean arePartitionRecordsSorted;
private Map<String, String> extraMetadataMap = new HashMap<>();
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession sparkSession, Configuration configuration, boolean arePartitionRecordsSorted) {
SparkSession sparkSession, Configuration configuration, DataSourceOptions dataSourceOptions,
boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.extraMetadataMap = DataSourceUtils.getExtraMetadata(dataSourceOptions.asMap());
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
sparkSession, configuration);
sparkSession, configuration, extraMetadataMap);
}
@Override
@@ -89,4 +96,5 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
public void abort(WriterCommitMessage[] messages) {
dataSourceInternalWriterHelper.abort();
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
import java.util.List;

View File

@@ -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++) {

View File

@@ -53,6 +53,6 @@ public class DefaultSource extends BaseDefaultSource implements TableProvider {
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA.key()), path, tblName, properties);
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
getConfiguration(), arePartitionRecordsSorted);
getConfiguration(), properties, arePartitionRecordsSorted);
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.spark3.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
@@ -33,7 +34,9 @@ import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
import org.apache.spark.sql.types.StructType;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -47,15 +50,17 @@ public class HoodieDataSourceInternalBatchWrite implements BatchWrite {
private final StructType structType;
private final boolean arePartitionRecordsSorted;
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
private Map<String, String> extraMetadata = new HashMap<>();
public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession jss, Configuration hadoopConfiguration, boolean arePartitionRecordsSorted) {
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.extraMetadata = DataSourceUtils.getExtraMetadata(properties);
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
jss, hadoopConfiguration);
jss, hadoopConfiguration, extraMetadata);
}
@Override

View File

@@ -26,6 +26,8 @@ import org.apache.spark.sql.connector.write.BatchWrite;
import org.apache.spark.sql.connector.write.WriteBuilder;
import org.apache.spark.sql.types.StructType;
import java.util.Map;
/**
* Implementation of {@link WriteBuilder} for datasource "hudi.spark3.internal" to be used in datasource implementation
* of bulk insert.
@@ -38,20 +40,22 @@ public class HoodieDataSourceInternalBatchWriteBuilder implements WriteBuilder {
private final SparkSession jss;
private final Configuration hadoopConfiguration;
private final boolean arePartitionRecordsSorted;
private final Map<String, String> properties;
public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession jss, Configuration hadoopConfiguration, boolean arePartitionRecordsSorted) {
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.jss = jss;
this.hadoopConfiguration = hadoopConfiguration;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.properties = properties;
}
@Override
public BatchWrite buildForBatch() {
return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss,
hadoopConfiguration, arePartitionRecordsSorted);
hadoopConfiguration, properties, arePartitionRecordsSorted);
}
}

View File

@@ -29,6 +29,7 @@ import org.apache.spark.sql.connector.write.WriteBuilder;
import org.apache.spark.sql.types.StructType;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
/**
@@ -42,15 +43,18 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
private final SparkSession jss;
private final Configuration hadoopConfiguration;
private final boolean arePartitionRecordsSorted;
private final Map<String, String> properties;
public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config,
StructType schema, SparkSession jss, Configuration hadoopConfiguration, boolean arePartitionRecordsSorted) {
StructType schema, SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties,
boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = config;
this.structType = schema;
this.jss = jss;
this.hadoopConfiguration = hadoopConfiguration;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.properties = properties;
}
@Override
@@ -75,6 +79,6 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
@Override
public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) {
return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss,
hadoopConfiguration, arePartitionRecordsSorted);
hadoopConfiguration, properties, arePartitionRecordsSorted);
}
}

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.spark3.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;
@@ -34,13 +36,18 @@ 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 HoodieDataSourceInternalBatchWrite}.
@@ -50,13 +57,17 @@ public class TestHoodieDataSourceInternalBatchWrite 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();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String instantTime = "001";
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, extraMetadata, false);
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
@@ -84,11 +95,49 @@ public class TestHoodieDataSourceInternalBatchWrite extends
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
dataSourceInternalBatchWrite.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
@@ -103,8 +152,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong());
@@ -148,8 +196,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong());
@@ -195,8 +242,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
String instantTime0 = "00" + 0;
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
@@ -234,7 +280,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
// 2nd batch. abort in the end
String instantTime1 = "00" + 1;
dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong());
for (int j = 0; j < batches; j++) {