[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:
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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++);
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
@@ -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 + '}';
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user