1
0

[HUDI-1040] Make Hudi support Spark 3 (#2208)

* Fix flaky MOR unit test

* Update Spark APIs to make it be compatible with both spark2 & spark3

* Refactor bulk insert v2 part to make Hudi be able to compile with Spark3

* Add spark3 profile to handle fasterxml & spark version

* Create hudi-spark-common module & refactor hudi-spark related modules

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2020-12-09 15:52:23 -08:00
committed by GitHub
parent 3a91d26d62
commit fce1453fa6
79 changed files with 1040 additions and 172 deletions

View File

@@ -0,0 +1,91 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.sources.DataSourceRegister;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
import org.apache.spark.sql.sources.v2.DataSourceV2;
import org.apache.spark.sql.sources.v2.ReadSupport;
import org.apache.spark.sql.sources.v2.WriteSupport;
import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
import org.apache.spark.sql.types.StructType;
import java.util.Optional;
/**
* DataSource V2 implementation for managing internal write logic. Only called internally.
*/
public class DefaultSource implements DataSourceV2, ReadSupport, WriteSupport,
DataSourceRegister {
private static final Logger LOG = LogManager
.getLogger(DefaultSource.class);
private SparkSession sparkSession = null;
private Configuration configuration = null;
@Override
public String shortName() {
return "hudi_internal";
}
@Override
public DataSourceReader createReader(StructType schema, DataSourceOptions options) {
return null;
}
@Override
public DataSourceReader createReader(DataSourceOptions options) {
return null;
}
@Override
public Optional<DataSourceWriter> createWriter(String writeUUID, StructType schema, SaveMode mode,
DataSourceOptions options) {
String instantTime = options.get(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY).get();
String path = options.get("path").get();
String tblName = options.get(HoodieWriteConfig.TABLE_NAME).get();
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, options.asMap());
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
getConfiguration()));
}
private SparkSession getSparkSession() {
if (sparkSession == null) {
sparkSession = SparkSession.builder().getOrCreate();
}
return sparkSession;
}
private Configuration getConfiguration() {
if (configuration == null) {
this.configuration = getSparkSession().sparkContext().hadoopConfiguration();
}
return configuration;
}
}

View File

@@ -0,0 +1,119 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieRowCreateHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
/**
* Hoodie's Implementation of {@link DataWriter<InternalRow>}. This is used in data source implementation for bulk insert.
*/
public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRow> {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieBulkInsertDataInternalWriter.class);
private final String instantTime;
private final int taskPartitionId;
private final long taskId;
private final long taskEpochId;
private final HoodieTable hoodieTable;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
private HoodieRowCreateHandle handle;
private String lastKnownPartitionPath = null;
private String fileIdPrefix = null;
private int numFilesWritten = 0;
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
StructType structType) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.taskPartitionId = taskPartitionId;
this.taskId = taskId;
this.taskEpochId = taskEpochId;
this.structType = structType;
this.fileIdPrefix = UUID.randomUUID().toString();
}
@Override
public void write(InternalRow record) throws IOException {
try {
String partitionPath = record.getUTF8String(
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
LOG.info("Creating new file for partition path " + partitionPath);
createNewHandle(partitionPath);
lastKnownPartitionPath = partitionPath;
}
handle.write(record);
} catch (Throwable t) {
LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t);
throw t;
}
}
@Override
public WriterCommitMessage commit() throws IOException {
close();
return new HoodieWriterCommitMessage(writeStatusList);
}
@Override
public void abort() throws IOException {
}
private void createNewHandle(String partitionPath) throws IOException {
if (null != handle) {
close();
}
handle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType);
}
public void close() throws IOException {
if (null != handle) {
writeStatusList.add(handle.close());
}
}
protected String getNextFileId() {
return String.format("%s-%d", fileIdPrefix, numFilesWritten++);
}
}

View File

@@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
import org.apache.spark.sql.types.StructType;
/**
* Factory to assist in instantiating {@link HoodieBulkInsertDataInternalWriter}.
*/
public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFactory<InternalRow> {
private final String instantTime;
private final HoodieTable hoodieTable;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, StructType structType) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.structType = structType;
}
@Override
public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId,
structType);
}
}

View File

@@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.InternalRow;
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.List;
import java.util.stream.Collectors;
/**
* Implementation of {@link DataSourceWriter} for datasource "hudi.internal" to be used in datasource implementation
* of bulk insert.
*/
public class HoodieDataSourceInternalWriter implements DataSourceWriter {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieDataSourceInternalWriter.class);
public static final String INSTANT_TIME_OPT_KEY = "hoodie.instant.time";
private final String instantTime;
private final HoodieTableMetaClient metaClient;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final SparkRDDWriteClient writeClient;
private final HoodieTable hoodieTable;
private final WriteOperationType operationType;
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession sparkSession, Configuration configuration) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.operationType = WriteOperationType.BULK_INSERT;
this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true);
writeClient.setOperationType(operationType);
writeClient.startCommitWithTime(instantTime);
this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath());
this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient);
}
@Override
public DataWriterFactory<InternalRow> createWriterFactory() {
metaClient.getActiveTimeline().transitionRequestedToInflight(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty());
if (WriteOperationType.BULK_INSERT == operationType) {
return new HoodieBulkInsertDataInternalWriterFactory(hoodieTable, writeConfig, instantTime, structType);
} else {
throw new IllegalArgumentException("Write Operation Type + " + operationType + " not supported ");
}
}
@Override
public boolean useCommitCoordinator() {
return true;
}
@Override
public void onDataWriterCommit(WriterCommitMessage message) {
LOG.info("Received commit of a data writer =" + message);
}
@Override
public void commit(WriterCommitMessage[] messages) {
List<HoodieWriteStat> writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m)
.flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList());
try {
writeClient.commitStats(instantTime, writeStatList, Option.empty(),
DataSourceUtils.getCommitActionType(operationType, metaClient.getTableType()));
} catch (Exception ioe) {
throw new HoodieException(ioe.getMessage(), ioe);
} finally {
writeClient.close();
}
}
@Override
public void abort(WriterCommitMessage[] messages) {
LOG.error("Commit " + instantTime + " aborted ");
writeClient.rollback(instantTime);
writeClient.close();
}
}

View File

@@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import java.util.ArrayList;
import java.util.List;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
/**
* Hoodie's {@link WriterCommitMessage} used in datasource implementation.
*/
public class HoodieWriterCommitMessage implements WriterCommitMessage {
private List<HoodieInternalWriteStatus> writeStatuses = new ArrayList<>();
public HoodieWriterCommitMessage(List<HoodieInternalWriteStatus> writeStatuses) {
this.writeStatuses = writeStatuses;
}
public List<HoodieInternalWriteStatus> getWriteStatuses() {
return writeStatuses;
}
@Override
public String toString() {
return "HoodieWriterCommitMessage{" + "writeStatuses=" + writeStatuses + '}';
}
}

View File

@@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi
import org.apache.hudi.client.utils.SparkRowDeserializer
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
class Spark2RowDeserializer(val encoder: ExpressionEncoder[Row]) extends SparkRowDeserializer {
def deserializeRow(internalRow: InternalRow): Row = {
encoder.fromRow(internalRow)
}
}

View File

@@ -0,0 +1,218 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.spark.package$;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
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.getInternalRowWithError;
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.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
/**
* Unit tests {@link HoodieBulkInsertDataInternalWriter}.
*/
public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
// this test is only compatible with spark 2
assumeTrue(package$.MODULE$.SPARK_VERSION().startsWith("2."));
initSparkContexts("TestHoodieBulkInsertDataInternalWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testDataInternalWriter() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
// execute N rounds
for (int i = 0; i < 5; i++) {
String instantTime = "00" + i;
// init writer
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(1000);
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
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);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<String> fileAbsPaths = new ArrayList<>();
List<String> fileNames = new ArrayList<>();
// verify write statuses
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertOutput(totalInputRows, result, instantTime, fileNames);
}
}
/**
* Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch2 of invalid records which is expected
* to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk.
*/
@Test
public void testGlobalFailure() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
String instantTime = "001";
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(100);
int totalFailures = 5;
// Generate first batch of valid rows
Dataset<Row> inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false);
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// generate some failures rows
for (int i = 0; i < totalFailures; i++) {
internalRows.add(getInternalRowWithError(partitionPath));
}
// generate 2nd batch of valid rows
Dataset<Row> inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false);
internalRows.addAll(toInternalRows(inputRows2, ENCODER));
// issue writes
try {
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
fail("Should have failed");
} catch (Throwable e) {
// expected
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<String> fileAbsPaths = new ArrayList<>();
List<String> fileNames = new ArrayList<>();
// verify write statuses
assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertOutput(inputRows, result, instantTime, fileNames);
}
private void writeRows(Dataset<Row> inputRows, HoodieBulkInsertDataInternalWriter writer)
throws Exception {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
}
private void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size, List<String> fileAbsPaths, List<String> fileNames) {
assertEquals(batches, writeStatuses.size());
int counter = 0;
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
// verify write status
assertEquals(writeStatus.getTotalRecords(), size);
assertNull(writeStatus.getGlobalError());
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertNotNull(writeStatus.getFileId());
String fileId = writeStatus.getFileId();
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath());
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
fileNames.add(writeStatus.getStat().getPath().substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1));
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
}
}
private void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> fileNames) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
assertTrue(fileNames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}

View File

@@ -0,0 +1,324 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.spark.package$;
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.writer.DataWriter;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
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.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
/**
* Unit tests {@link HoodieDataSourceInternalWriter}.
*/
public class TestHoodieDataSourceInternalWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
// this test is only compatible with spark 2
assumeTrue(package$.MODULE$.SPARK_VERSION().startsWith("2."));
initSparkContexts("TestHoodieDataSourceInternalWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testDataSourceWriter() 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);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
List<String> partitionPathsAbs = new ArrayList<>();
for (String partitionPath : partitionPaths) {
partitionPathsAbs.add(basePath + "/" + partitionPath + "/*");
}
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);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
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);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
@Test
public void testMultipleDataSourceWrites() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
int partitionCounter = 0;
// execute N rounds
for (int i = 0; i < 5; i++) {
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
int size = 10 + RANDOM.nextInt(1000);
int batches = 5; // one batch per partition
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
}
@Test
public void testLargeWrites() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
int partitionCounter = 0;
// execute N rounds
for (int i = 0; i < 3; i++) {
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
int size = 10000 + RANDOM.nextInt(10000);
int batches = 3; // one batch per partition
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
}
/**
* Tests that DataSourceWriter.abort() will abort the written records of interest write and commit batch1 write and abort batch2 Read of entire dataset should show only records from batch1.
* commit batch1
* abort batch2
* verify only records from batch1 is available to read
*/
@Test
public void testAbort() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
String instantTime0 = "00" + 0;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
List<String> partitionPathsAbs = new ArrayList<>();
for (String partitionPath : partitionPaths) {
partitionPathsAbs.add(basePath + "/" + partitionPath + "/*");
}
int size = 10 + RANDOM.nextInt(100);
int batches = 1;
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);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
// commit 1st batch
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 rows
assertOutput(totalInputRows, result, instantTime0);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
// 2nd batch. abort in the end
String instantTime1 = "00" + 1;
dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
}
commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
// commit 1st batch
dataSourceInternalWriter.abort(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
// only rows from first batch should be present
assertOutput(totalInputRows, result, instantTime0);
}
private void writeRows(Dataset<Row> inputRows, DataWriter<InternalRow> writer) throws Exception {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
}
private void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size) {
assertEquals(batches, writeStatuses.size());
int counter = 0;
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
assertEquals(writeStatus.getPartitionPath(), HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]);
assertEquals(writeStatus.getTotalRecords(), size);
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertEquals(writeStatus.getTotalErrorRecords(), 0);
assertFalse(writeStatus.hasErrors());
assertNull(writeStatus.getGlobalError());
assertNotNull(writeStatus.getFileId());
String fileId = writeStatus.getFileId();
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
}
}
private void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}