[HUDI-1451] Support bulk insert v2 with Spark 3.0.0 (#2328)
Co-authored-by: Wenning Ding <wenningd@amazon.com> - Added support for bulk insert v2 with datasource v2 api in Spark 3.0.0.
This commit is contained in:
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* 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.spark3.internal;
|
||||
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.internal.BaseDefaultSource;
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper;
|
||||
|
||||
import org.apache.spark.sql.connector.catalog.Table;
|
||||
import org.apache.spark.sql.connector.catalog.TableProvider;
|
||||
import org.apache.spark.sql.connector.expressions.Transform;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* DataSource V2 implementation for managing internal write logic. Only called internally.
|
||||
* This class is only compatible with datasource V2 API in Spark 3.
|
||||
*/
|
||||
public class DefaultSource extends BaseDefaultSource implements TableProvider {
|
||||
|
||||
@Override
|
||||
public StructType inferSchema(CaseInsensitiveStringMap options) {
|
||||
return StructType.fromDDL(options.get(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Table getTable(StructType schema, Transform[] partitioning, Map<String, String> properties) {
|
||||
String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY);
|
||||
String path = properties.get("path");
|
||||
String tblName = properties.get(HoodieWriteConfig.TABLE_NAME);
|
||||
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, properties);
|
||||
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
|
||||
getConfiguration());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,64 @@
|
||||
/*
|
||||
* 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.spark3.internal;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.internal.BulkInsertDataInternalWriterHelper;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.connector.write.DataWriter;
|
||||
import org.apache.spark.sql.connector.write.WriterCommitMessage;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Hoodie's Implementation of {@link DataWriter<InternalRow>}. This is used in data source "hudi.spark3.internal" implementation for bulk insert.
|
||||
*/
|
||||
public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRow> {
|
||||
|
||||
private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper;
|
||||
|
||||
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||
String instantTime, int taskPartitionId, long taskId, StructType structType) {
|
||||
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
|
||||
writeConfig, instantTime, taskPartitionId, taskId, 0, structType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(InternalRow record) throws IOException {
|
||||
bulkInsertWriterHelper.write(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriterCommitMessage commit() throws IOException {
|
||||
return new HoodieWriterCommitMessage(bulkInsertWriterHelper.getWriteStatuses());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
bulkInsertWriterHelper.abort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
bulkInsertWriterHelper.close();
|
||||
}
|
||||
}
|
||||
@@ -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.spark3.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.connector.write.DataWriter;
|
||||
import org.apache.spark.sql.connector.write.DataWriterFactory;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
/**
|
||||
* Factory to assist in instantiating {@link HoodieBulkInsertDataInternalWriter}.
|
||||
*/
|
||||
public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFactory {
|
||||
|
||||
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> createWriter(int partitionId, long taskId) {
|
||||
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId,
|
||||
structType);
|
||||
}
|
||||
}
|
||||
@@ -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.spark3.internal;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.connector.write.DataWriterFactory;
|
||||
import org.apache.spark.sql.connector.write.WriterCommitMessage;
|
||||
import org.apache.spark.sql.connector.write.BatchWrite;
|
||||
import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Implementation of {@link BatchWrite} for datasource "hudi.spark3.internal" to be used in datasource implementation
|
||||
* of bulk insert.
|
||||
*/
|
||||
public class HoodieDataSourceInternalBatchWrite implements BatchWrite {
|
||||
|
||||
private final String instantTime;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
|
||||
|
||||
public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
||||
SparkSession jss, Configuration hadoopConfiguration) {
|
||||
this.instantTime = instantTime;
|
||||
this.writeConfig = writeConfig;
|
||||
this.structType = structType;
|
||||
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
|
||||
jss, hadoopConfiguration);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
|
||||
dataSourceInternalWriterHelper.createInflightCommit();
|
||||
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
|
||||
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
|
||||
writeConfig, instantTime, structType);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCommitCoordinator() {
|
||||
return dataSourceInternalWriterHelper.useCommitCoordinator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onDataWriterCommit(WriterCommitMessage message) {
|
||||
dataSourceInternalWriterHelper.onDataWriterCommit(message.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commit(WriterCommitMessage[] messages) {
|
||||
List<HoodieWriteStat> writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m)
|
||||
.flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList());
|
||||
dataSourceInternalWriterHelper.commit(writeStatList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(WriterCommitMessage[] messages) {
|
||||
dataSourceInternalWriterHelper.abort();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.spark3.internal;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.connector.write.BatchWrite;
|
||||
import org.apache.spark.sql.connector.write.WriteBuilder;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
/**
|
||||
* Implementation of {@link WriteBuilder} for datasource "hudi.spark3.internal" to be used in datasource implementation
|
||||
* of bulk insert.
|
||||
*/
|
||||
public class HoodieDataSourceInternalBatchWriteBuilder implements WriteBuilder {
|
||||
|
||||
private final String instantTime;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
private final SparkSession jss;
|
||||
private final Configuration hadoopConfiguration;
|
||||
|
||||
public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
||||
SparkSession jss, Configuration hadoopConfiguration) {
|
||||
this.instantTime = instantTime;
|
||||
this.writeConfig = writeConfig;
|
||||
this.structType = structType;
|
||||
this.jss = jss;
|
||||
this.hadoopConfiguration = hadoopConfiguration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BatchWrite buildForBatch() {
|
||||
return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss,
|
||||
hadoopConfiguration);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* 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.spark3.internal;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.connector.catalog.SupportsWrite;
|
||||
import org.apache.spark.sql.connector.catalog.TableCapability;
|
||||
import org.apache.spark.sql.connector.write.LogicalWriteInfo;
|
||||
import org.apache.spark.sql.connector.write.WriteBuilder;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Hoodie's Implementation of {@link SupportsWrite}. This is used in data source "hudi.spark3.internal" implementation for bulk insert.
|
||||
*/
|
||||
class HoodieDataSourceInternalTable implements SupportsWrite {
|
||||
|
||||
private final String instantTime;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
private final SparkSession jss;
|
||||
private final Configuration hadoopConfiguration;
|
||||
|
||||
public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config,
|
||||
StructType schema, SparkSession jss, Configuration hadoopConfiguration) {
|
||||
this.instantTime = instantTime;
|
||||
this.writeConfig = config;
|
||||
this.structType = schema;
|
||||
this.jss = jss;
|
||||
this.hadoopConfiguration = hadoopConfiguration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return this.getClass().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public StructType schema() {
|
||||
return structType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<TableCapability> capabilities() {
|
||||
return new HashSet<TableCapability>() {{
|
||||
add(TableCapability.BATCH_WRITE);
|
||||
add(TableCapability.TRUNCATE);
|
||||
}};
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) {
|
||||
return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss,
|
||||
hadoopConfiguration);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* 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.spark3.internal;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.internal.BaseWriterCommitMessage;
|
||||
import org.apache.spark.sql.connector.write.WriterCommitMessage;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Hoodie's {@link WriterCommitMessage} used in datasource "hudi.spark3.internal" implementation.
|
||||
*/
|
||||
public class HoodieWriterCommitMessage extends BaseWriterCommitMessage
|
||||
implements WriterCommitMessage {
|
||||
|
||||
public HoodieWriterCommitMessage(List<HoodieInternalWriteStatus> writeStatuses) {
|
||||
super(writeStatuses);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user