1
0

[HUDI-3172] Refactor hudi existing modules to make more code reuse in V2 Implementation (#4514)

* Introduce hudi-spark3-common and hudi-spark2-common modules to place classes that would be reused in different spark versions, also introduce hudi-spark3.1.x to support spark 3.1.x.
* Introduce hudi format under hudi-spark2, hudi-spark3, hudi-spark3.1.x modules and change the hudi format in original hudi-spark module to hudi_v1 format.
* Manually tested on Spark 3.1.2 and Spark 3.2.0 SQL.
* Added a README.md file under hudi-spark-datasource module.
This commit is contained in:
leesf
2022-01-14 13:42:35 +08:00
committed by GitHub
parent 195dac90fa
commit 5ce45c440b
90 changed files with 1249 additions and 430 deletions

View File

@@ -0,0 +1,247 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hudi-spark-datasource</artifactId>
<groupId>org.apache.hudi</groupId>
<version>0.11.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>hudi-spark3-common</artifactId>
<properties>
<main.basedir>${project.parent.parent.basedir}</main.basedir>
<maven.compiler.source>8</maven.compiler.source>
<maven.compiler.target>8</maven.compiler.target>
</properties>
<build>
<resources>
<resource>
<directory>src/main/resources</directory>
</resource>
</resources>
<pluginManagement>
<plugins>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<version>${scala-maven-plugin.version}</version>
<configuration>
<args>
<arg>-nobootcp</arg>
</args>
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>
</plugins>
</pluginManagement>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<executions>
<execution>
<id>copy-dependencies</id>
<phase>prepare-package</phase>
<goals>
<goal>copy-dependencies</goal>
</goals>
<configuration>
<outputDirectory>${project.build.directory}/lib</outputDirectory>
<overWriteReleases>true</overWriteReleases>
<overWriteSnapshots>true</overWriteSnapshots>
<overWriteIfNewer>true</overWriteIfNewer>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<executions>
<execution>
<id>scala-compile-first</id>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
</goals>
</execution>
<execution>
<id>scala-test-compile</id>
<phase>process-test-resources</phase>
<goals>
<goal>testCompile</goal>
</goals>
</execution>
</executions>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<executions>
<execution>
<phase>compile</phase>
<goals>
<goal>compile</goal>
</goals>
</execution>
</executions>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
<phase>test-compile</phase>
</execution>
</executions>
<configuration>
<skip>false</skip>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<skipTests>${skip.hudi-spark3.unit.tests}</skipTests>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.scalastyle</groupId>
<artifactId>scalastyle-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
<dependencies>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala12.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.12</artifactId>
<version>${spark3.version}</version>
<optional>true</optional>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>${fasterxml.spark3.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
<version>${fasterxml.spark3.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<version>${fasterxml.spark3.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark-client</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<!-- Hoodie - Test -->
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-client-common</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark-client</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-common</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@@ -0,0 +1,68 @@
/*
* 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.common.table.HoodieTableConfig;
import org.apache.hudi.config.HoodieInternalConfig;
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.HashMap;
import java.util.Map;
import static org.apache.hudi.DataSourceUtils.mayBeOverwriteParquetWriteLegacyFormatProp;
/**
* 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(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key()));
}
@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.TBL_NAME.key());
boolean populateMetaFields = Boolean.parseBoolean(properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(),
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
boolean arePartitionRecordsSorted = Boolean.parseBoolean(properties.getOrDefault(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED,
Boolean.toString(HoodieInternalConfig.DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED)));
// Create a new map as the properties is an unmodifiableMap on Spark 3.2.0
Map<String, String> newProps = new HashMap<>(properties);
// Auto set the value of "hoodie.parquet.writeLegacyFormat.enabled"
mayBeOverwriteParquetWriteLegacyFormatProp(newProps, schema);
// 1st arg to createHoodieConfig is not really required to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(newProps.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()), path, tblName, newProps);
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
getConfiguration(), newProps, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -0,0 +1,65 @@
/*
* 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, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
writeConfig, instantTime, taskPartitionId, taskId, 0, structType, populateMetaFields, arePartitionRecordsSorted);
}
@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();
}
}

View File

@@ -0,0 +1,57 @@
/*
* 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;
private final boolean populateMetaFields;
private final boolean arePartitionRecordsSorted;
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, StructType structType, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@Override
public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId,
structType, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -0,0 +1,100 @@
/*
* 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.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.BatchWrite;
import org.apache.spark.sql.connector.write.DataWriterFactory;
import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
import org.apache.spark.sql.connector.write.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;
/**
* 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 boolean arePartitionRecordsSorted;
private final boolean populateMetaFields;
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
private Map<String, String> extraMetadata = new HashMap<>();
public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean populateMetaFields, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.extraMetadata = DataSourceUtils.getExtraMetadata(properties);
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
jss, hadoopConfiguration, extraMetadata);
}
@Override
public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
dataSourceInternalWriterHelper.createInflightCommit();
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted);
} 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();
}
}

View File

@@ -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.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;
import java.util.Map;
/**
* 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;
private final Map<String, String> properties;
private final boolean populateMetaFields;
private final boolean arePartitionRecordsSorted;
public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.jss = jss;
this.hadoopConfiguration = hadoopConfiguration;
this.properties = properties;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@Override
public BatchWrite buildForBatch() {
return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss,
hadoopConfiguration, properties, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -0,0 +1,87 @@
/*
* 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.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.Map;
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;
private final boolean arePartitionRecordsSorted;
private final Map<String, String> properties;
private final boolean populateMetaFields;
public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config,
StructType schema, SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties,
boolean populateMetaFields, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = config;
this.structType = schema;
this.jss = jss;
this.hadoopConfiguration = hadoopConfiguration;
this.properties = properties;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@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, properties, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -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);
}
}

View File

@@ -0,0 +1,70 @@
/*
* 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.HoodieSparkUtils;
import org.apache.spark.sql.catalyst.plans.logical.InsertIntoStatement;
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan;
import org.apache.spark.sql.catalyst.util.DateFormatter;
import scala.Option;
import scala.collection.Seq;
import scala.collection.immutable.Map;
import java.lang.reflect.Constructor;
import java.lang.reflect.Method;
import java.time.ZoneId;
public class ReflectUtil {
public static InsertIntoStatement createInsertInto(LogicalPlan table, Map<String, Option<String>> partition, Seq<String> userSpecifiedCols,
LogicalPlan query, boolean overwrite, boolean ifPartitionNotExists) {
try {
if (HoodieSparkUtils.isSpark3_0()) {
Constructor<InsertIntoStatement> constructor = InsertIntoStatement.class.getConstructor(
LogicalPlan.class, Map.class, LogicalPlan.class, boolean.class, boolean.class);
return constructor.newInstance(table, partition, query, overwrite, ifPartitionNotExists);
} else {
Constructor<InsertIntoStatement> constructor = InsertIntoStatement.class.getConstructor(
LogicalPlan.class, Map.class, Seq.class, LogicalPlan.class, boolean.class, boolean.class);
return constructor.newInstance(table, partition, userSpecifiedCols, query, overwrite, ifPartitionNotExists);
}
} catch (Exception e) {
throw new RuntimeException("Error in create InsertIntoStatement", e);
}
}
public static DateFormatter getDateFormatter(ZoneId zoneId) {
try {
ClassLoader loader = Thread.currentThread().getContextClassLoader();
if (HoodieSparkUtils.isSpark3_2()) {
Class clazz = loader.loadClass(DateFormatter.class.getName());
Method applyMethod = clazz.getDeclaredMethod("apply");
applyMethod.setAccessible(true);
return (DateFormatter)applyMethod.invoke(null);
} else {
Class clazz = loader.loadClass(DateFormatter.class.getName());
Method applyMethod = clazz.getDeclaredMethod("apply", ZoneId.class);
applyMethod.setAccessible(true);
return (DateFormatter)applyMethod.invoke(null, zoneId);
}
} catch (Exception e) {
throw new RuntimeException("Error in apply DateFormatter", e);
}
}
}

View File

@@ -0,0 +1,37 @@
/*
* 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.SparkRowSerDe
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
class Spark3RowSerDe(val encoder: ExpressionEncoder[Row]) extends SparkRowSerDe {
private val deserializer: ExpressionEncoder.Deserializer[Row] = encoder.createDeserializer()
private val serializer: ExpressionEncoder.Serializer[Row] = encoder.createSerializer()
def deserializeRow(internalRow: InternalRow): Row = {
deserializer.apply(internalRow)
}
override def serializeRow(row: Row): InternalRow = {
serializer.apply(row)
}
}

View File

@@ -0,0 +1,97 @@
/*
* 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.spark.sql.adapter
import org.apache.hudi.Spark3RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.spark3.internal.ReflectUtil
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.execution.datasources.{LogicalRelation, Spark3ParsePartitionUtil, SparkParsePartitionUtil}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.internal.SQLConf
/**
* The adapter for spark3.
*/
class Spark3Adapter extends SparkAdapter {
override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = {
new Spark3RowSerDe(encoder)
}
override def toTableIdentifier(aliasId: AliasIdentifier): TableIdentifier = {
aliasId match {
case AliasIdentifier(name, Seq(database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq(_, database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq()) =>
TableIdentifier(name, None)
case _=> throw new IllegalArgumentException(s"Cannot cast $aliasId to TableIdentifier")
}
}
override def toTableIdentifier(relation: UnresolvedRelation): TableIdentifier = {
relation.multipartIdentifier.asTableIdentifier
}
override def createJoin(left: LogicalPlan, right: LogicalPlan, joinType: JoinType): Join = {
Join(left, right, joinType, None, JoinHint.NONE)
}
override def isInsertInto(plan: LogicalPlan): Boolean = {
plan.isInstanceOf[InsertIntoStatement]
}
override def getInsertIntoChildren(plan: LogicalPlan):
Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)] = {
plan match {
case insert: InsertIntoStatement =>
Some((insert.table, insert.partitionSpec, insert.query, insert.overwrite, insert.ifPartitionNotExists))
case _ =>
None
}
}
override def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]],
query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan = {
ReflectUtil.createInsertInto(table, partition, Seq.empty[String], query, overwrite, ifPartitionNotExists)
}
override def createSparkParsePartitionUtil(conf: SQLConf): SparkParsePartitionUtil = {
new Spark3ParsePartitionUtil(conf)
}
override def createLike(left: Expression, right: Expression): Expression = {
new Like(left, right)
}
override def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String] = {
parser.parseMultipartIdentifier(sqlText)
}
}

View File

@@ -0,0 +1,274 @@
/*
* 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.spark.sql.execution.datasources
import java.lang.{Double => JDouble, Long => JLong}
import java.math.{BigDecimal => JBigDecimal}
import java.time.ZoneId
import java.util.{Locale, TimeZone}
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH
import org.apache.hudi.spark3.internal.ReflectUtil
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName
import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter}
import org.apache.spark.sql.execution.datasources.PartitioningUtils.timestampPartitionPattern
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import scala.collection.mutable.ArrayBuffer
import scala.util.Try
import scala.util.control.NonFatal
class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
/**
* The definition of PartitionValues has been changed by SPARK-34314 in Spark3.2.
* To solve the compatibility between 3.1 and 3.2, copy some codes from PartitioningUtils in Spark3.2 here.
* And this method will generate and return `InternalRow` directly instead of `PartitionValues`.
*/
override def parsePartition(
path: Path,
typeInference: Boolean,
basePaths: Set[Path],
userSpecifiedDataTypes: Map[String, DataType],
timeZone: TimeZone): InternalRow = {
val dateFormatter = ReflectUtil.getDateFormatter(timeZone.toZoneId)
val timestampFormatter = TimestampFormatter(timestampPartitionPattern,
timeZone.toZoneId, isParsing = true)
val (partitionValues, _) = parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes,
conf.validatePartitionColumns, timeZone.toZoneId, dateFormatter, timestampFormatter)
partitionValues.map {
case PartitionValues(columnNames: Seq[String], typedValues: Seq[TypedPartValue]) =>
val rowValues = columnNames.zip(typedValues).map { case (columnName, typedValue) =>
try {
castPartValueToDesiredType(typedValue.dataType, typedValue.value, timeZone.toZoneId)
} catch {
case NonFatal(_) =>
if (conf.validatePartitionColumns) {
throw new RuntimeException(s"Failed to cast value `${typedValue.value}` to " +
s"`${typedValue.dataType}` for partition column `$columnName`")
} else null
}
}
InternalRow.fromSeq(rowValues)
}.getOrElse(InternalRow.empty)
}
case class TypedPartValue(value: String, dataType: DataType)
case class PartitionValues(columnNames: Seq[String], typedValues: Seq[TypedPartValue])
{
require(columnNames.size == typedValues.size)
}
private def parsePartition(
path: Path,
typeInference: Boolean,
basePaths: Set[Path],
userSpecifiedDataTypes: Map[String, DataType],
validatePartitionColumns: Boolean,
zoneId: ZoneId,
dateFormatter: DateFormatter,
timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = {
val columns = ArrayBuffer.empty[(String, TypedPartValue)]
// Old Hadoop versions don't have `Path.isRoot`
var finished = path.getParent == null
// currentPath is the current path that we will use to parse partition column value.
var currentPath: Path = path
while (!finished) {
// Sometimes (e.g., when speculative task is enabled), temporary directories may be left
// uncleaned. Here we simply ignore them.
if (currentPath.getName.toLowerCase(Locale.ROOT) == "_temporary") {
// scalastyle:off return
return (None, None)
// scalastyle:on return
}
if (basePaths.contains(currentPath)) {
// If the currentPath is one of base paths. We should stop.
finished = true
} else {
// Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1.
// Once we get the string, we try to parse it and find the partition column and value.
val maybeColumn =
parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes,
validatePartitionColumns, zoneId, dateFormatter, timestampFormatter)
maybeColumn.foreach(columns += _)
// Now, we determine if we should stop.
// When we hit any of the following cases, we will stop:
// - In this iteration, we could not parse the value of partition column and value,
// i.e. maybeColumn is None, and columns is not empty. At here we check if columns is
// empty to handle cases like /table/a=1/_temporary/something (we need to find a=1 in
// this case).
// - After we get the new currentPath, this new currentPath represent the top level dir
// i.e. currentPath.getParent == null. For the example of "/table/a=1/",
// the top level dir is "/table".
finished =
(maybeColumn.isEmpty && !columns.isEmpty) || currentPath.getParent == null
if (!finished) {
// For the above example, currentPath will be "/table/".
currentPath = currentPath.getParent
}
}
}
if (columns.isEmpty) {
(None, Some(path))
} else {
val (columnNames, values) = columns.reverse.unzip
(Some(PartitionValues(columnNames.toSeq, values.toSeq)), Some(currentPath))
}
}
private def parsePartitionColumn(
columnSpec: String,
typeInference: Boolean,
userSpecifiedDataTypes: Map[String, DataType],
validatePartitionColumns: Boolean,
zoneId: ZoneId,
dateFormatter: DateFormatter,
timestampFormatter: TimestampFormatter): Option[(String, TypedPartValue)] = {
val equalSignIndex = columnSpec.indexOf('=')
if (equalSignIndex == -1) {
None
} else {
val columnName = unescapePathName(columnSpec.take(equalSignIndex))
assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'")
val rawColumnValue = columnSpec.drop(equalSignIndex + 1)
assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'")
val dataType = if (userSpecifiedDataTypes.contains(columnName)) {
// SPARK-26188: if user provides corresponding column schema, get the column value without
// inference, and then cast it as user specified data type.
userSpecifiedDataTypes(columnName)
} else {
inferPartitionColumnValue(
rawColumnValue,
typeInference,
zoneId,
dateFormatter,
timestampFormatter)
}
Some(columnName -> TypedPartValue(rawColumnValue, dataType))
}
}
private def inferPartitionColumnValue(
raw: String,
typeInference: Boolean,
zoneId: ZoneId,
dateFormatter: DateFormatter,
timestampFormatter: TimestampFormatter): DataType = {
val decimalTry = Try {
// `BigDecimal` conversion can fail when the `field` is not a form of number.
val bigDecimal = new JBigDecimal(raw)
// It reduces the cases for decimals by disallowing values having scale (e.g. `1.1`).
require(bigDecimal.scale <= 0)
// `DecimalType` conversion can fail when
// 1. The precision is bigger than 38.
// 2. scale is bigger than precision.
fromDecimal(Decimal(bigDecimal))
}
val dateTry = Try {
// try and parse the date, if no exception occurs this is a candidate to be resolved as
// DateType
dateFormatter.parse(raw)
// SPARK-23436: Casting the string to date may still return null if a bad Date is provided.
// This can happen since DateFormat.parse may not use the entire text of the given string:
// so if there are extra-characters after the date, it returns correctly.
// We need to check that we can cast the raw string since we later can use Cast to get
// the partition values with the right DataType (see
// org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning)
val dateValue = Cast(Literal(raw), DateType, Some(zoneId.getId)).eval()
// Disallow DateType if the cast returned null
require(dateValue != null)
DateType
}
val timestampTry = Try {
val unescapedRaw = unescapePathName(raw)
// the inferred data type is consistent with the default timestamp type
val timestampType = TimestampType
// try and parse the date, if no exception occurs this is a candidate to be resolved as TimestampType
timestampFormatter.parse(unescapedRaw)
// SPARK-23436: see comment for date
val timestampValue = Cast(Literal(unescapedRaw), timestampType, Some(zoneId.getId)).eval()
// Disallow TimestampType if the cast returned null
require(timestampValue != null)
timestampType
}
if (typeInference) {
// First tries integral types
Try({ Integer.parseInt(raw); IntegerType })
.orElse(Try { JLong.parseLong(raw); LongType })
.orElse(decimalTry)
// Then falls back to fractional types
.orElse(Try { JDouble.parseDouble(raw); DoubleType })
// Then falls back to date/timestamp types
.orElse(timestampTry)
.orElse(dateTry)
// Then falls back to string
.getOrElse {
if (raw == DEFAULT_PARTITION_PATH) NullType else StringType
}
} else {
if (raw == DEFAULT_PARTITION_PATH) NullType else StringType
}
}
def castPartValueToDesiredType(
desiredType: DataType,
value: String,
zoneId: ZoneId): Any = desiredType match {
case _ if value == DEFAULT_PARTITION_PATH => null
case NullType => null
case StringType => UTF8String.fromString(unescapePathName(value))
case IntegerType => Integer.parseInt(value)
case LongType => JLong.parseLong(value)
case DoubleType => JDouble.parseDouble(value)
case _: DecimalType => Literal(new JBigDecimal(value)).value
case DateType =>
Cast(Literal(value), DateType, Some(zoneId.getId)).eval()
// Timestamp types
case dt: TimestampType =>
Try {
Cast(Literal(unescapePathName(value)), dt, Some(zoneId.getId)).eval()
}.getOrElse {
Cast(Cast(Literal(value), DateType, Some(zoneId.getId)), dt).eval()
}
case dt => throw new IllegalArgumentException(s"Unexpected type $dt")
}
private def fromDecimal(d: Decimal): DecimalType = DecimalType(d.precision, d.scale)
}