1
0

[HUDI-1659] Basic Implement Of Spark Sql Support For Hoodie (#2645)

Main functions:
Support create table for hoodie.
Support CTAS.
Support Insert for hoodie. Including dynamic partition and static partition insert.
Support MergeInto for hoodie.
Support DELETE
Support UPDATE
Both support spark2 & spark3 based on DataSourceV1.

Main changes:
Add sql parser for spark2.
Add HoodieAnalysis for sql resolve and logical plan rewrite.
Add commands implementation for CREATE TABLE、INSERT、MERGE INTO & CTAS.
In order to push down the update&insert logical to the HoodieRecordPayload for MergeInto, I make same change to the
HoodieWriteHandler and other related classes.
1、Add the inputSchema for parser the incoming record. This is because the inputSchema for MergeInto is different from writeSchema as there are some transforms in the update& insert expression.
2、Add WRITE_SCHEMA to HoodieWriteConfig to pass the write schema for merge into.
3、Pass properties to HoodieRecordPayload#getInsertValue to pass the insert expression and table schema.


Verify this pull request
Add TestCreateTable for test create hoodie tables and CTAS.
Add TestInsertTable for test insert hoodie tables.
Add TestMergeIntoTable for test merge hoodie tables.
Add TestUpdateTable for test update hoodie tables.
Add TestDeleteTable for test delete hoodie tables.
Add TestSqlStatement for test supported ddl/dml currently.
This commit is contained in:
pengzhiwei
2021-06-08 14:24:32 +08:00
committed by GitHub
parent cf83f10f5b
commit f760ec543e
86 changed files with 7346 additions and 255 deletions

View File

@@ -56,21 +56,10 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
return Option.empty();
}
GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
/*
* Combining strategy here returns currentValue on disk if incoming record is older.
* The incoming record can be either a delete (sent as an upsert with _hoodie_is_deleted set to true)
* or an insert/update record. In any case, if it is older than the record in disk, the currentValue
* in disk is returned (to be rewritten with new commit time).
*
* NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path
* and need to be dealt with separately.
*/
Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), true);
Comparable incomingOrderingVal = (Comparable) getNestedFieldVal(incomingRecord, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), false);
// Null check is needed here to support schema evolution. The record in storage may be from old schema where
// the new ordering column might not be present and hence returns null.
if (persistedOrderingVal != null && ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) > 0) {
if (!needUpdatingPersistedRecord(currentValue, incomingRecord, properties)) {
return Option.of(currentValue);
}
@@ -97,4 +86,22 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
}
return metadata.isEmpty() ? Option.empty() : Option.of(metadata);
}
protected boolean needUpdatingPersistedRecord(IndexedRecord currentValue,
IndexedRecord incomingRecord, Properties properties) {
/*
* Combining strategy here returns currentValue on disk if incoming record is older.
* The incoming record can be either a delete (sent as an upsert with _hoodie_is_deleted set to true)
* or an insert/update record. In any case, if it is older than the record in disk, the currentValue
* in disk is returned (to be rewritten with new commit time).
*
* NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path
* and need to be dealt with separately.
*/
Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue,
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), true);
Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord,
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), false);
return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0;
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.table;
import java.util.Arrays;
import org.apache.avro.Schema;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex;
import org.apache.hudi.common.model.HoodieFileFormat;
@@ -60,6 +61,8 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_TABLE_VERSION_PROP_NAME = "hoodie.table.version";
public static final String HOODIE_TABLE_PRECOMBINE_FIELD = "hoodie.table.precombine.field";
public static final String HOODIE_TABLE_PARTITION_COLUMNS = "hoodie.table.partition.columns";
public static final String HOODIE_TABLE_RECORDKEY_FIELDS = "hoodie.table.recordkey.fields";
public static final String HOODIE_TABLE_CREATE_SCHEMA = "hoodie.table.create.schema";
@Deprecated
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
@@ -227,6 +230,14 @@ public class HoodieTableConfig implements Serializable {
return Option.ofNullable(props.getProperty(HOODIE_BOOTSTRAP_BASE_PATH));
}
public Option<Schema> getTableCreateSchema() {
if (props.containsKey(HOODIE_TABLE_CREATE_SCHEMA)) {
return Option.of(new Schema.Parser().parse(props.getProperty(HOODIE_TABLE_CREATE_SCHEMA)));
} else {
return Option.empty();
}
}
/**
* Read the table name.
*/

View File

@@ -592,6 +592,8 @@ public class HoodieTableMetaClient implements Serializable {
private HoodieTableType tableType;
private String tableName;
private String tableCreateSchema;
private String recordKeyFields;
private String archiveLogFolder;
private String payloadClassName;
private Integer timelineLayoutVersion;
@@ -619,6 +621,16 @@ public class HoodieTableMetaClient implements Serializable {
return this;
}
public PropertyBuilder setTableCreateSchema(String tableCreateSchema) {
this.tableCreateSchema = tableCreateSchema;
return this;
}
public PropertyBuilder setRecordKeyFields(String recordKeyFields) {
this.recordKeyFields = recordKeyFields;
return this;
}
public PropertyBuilder setArchiveLogFolder(String archiveLogFolder) {
this.archiveLogFolder = archiveLogFolder;
return this;
@@ -704,7 +716,14 @@ public class HoodieTableMetaClient implements Serializable {
setPreCombineField(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS)) {
setPartitionColumns(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS));
setPartitionColumns(
properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
setRecordKeyFields(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
setTableCreateSchema(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
}
return this;
}
@@ -722,6 +741,10 @@ public class HoodieTableMetaClient implements Serializable {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
if (null != tableCreateSchema) {
properties.put(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
}
if (!StringUtils.isNullOrEmpty(archiveLogFolder)) {
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
} else {
@@ -754,6 +777,9 @@ public class HoodieTableMetaClient implements Serializable {
if (null != partitionColumns) {
properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS, partitionColumns);
}
if (null != recordKeyFields) {
properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
}
return properties;
}

View File

@@ -164,7 +164,22 @@ public class TableSchemaResolver {
*/
public Schema getTableAvroSchema(boolean includeMetadataFields) throws Exception {
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(includeMetadataFields);
return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() : getTableAvroSchemaFromDataFile();
if (schemaFromCommitMetadata.isPresent()) {
return schemaFromCommitMetadata.get();
}
Option<Schema> schemaFromTableConfig = metaClient.getTableConfig().getTableCreateSchema();
if (schemaFromTableConfig.isPresent()) {
if (includeMetadataFields) {
return HoodieAvroUtils.addMetadataFields(schemaFromTableConfig.get());
} else {
return schemaFromTableConfig.get();
}
}
if (includeMetadataFields) {
return getTableAvroSchemaFromDataFile();
} else {
return HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
}
}
/**
@@ -186,10 +201,7 @@ public class TableSchemaResolver {
* @throws Exception
*/
public Schema getTableAvroSchemaWithoutMetadataFields() throws Exception {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(timeline.lastInstant().get(), false);
return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() :
HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
return getTableAvroSchema(false);
}
/**
@@ -201,8 +213,14 @@ public class TableSchemaResolver {
*/
public Schema getTableAvroSchemaWithoutMetadataFields(HoodieInstant instant) throws Exception {
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(instant, false);
return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() :
HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
if (schemaFromCommitMetadata.isPresent()) {
return schemaFromCommitMetadata.get();
}
Option<Schema> schemaFromTableConfig = metaClient.getTableConfig().getTableCreateSchema();
if (schemaFromTableConfig.isPresent()) {
return schemaFromTableConfig.get();
}
return HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
}
/**
@@ -212,7 +230,11 @@ public class TableSchemaResolver {
*/
private Option<Schema> getTableSchemaFromCommitMetadata(boolean includeMetadataFields) {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
return getTableSchemaFromCommitMetadata(timeline.lastInstant().get(), includeMetadataFields);
if (timeline.lastInstant().isPresent()) {
return getTableSchemaFromCommitMetadata(timeline.lastInstant().get(), includeMetadataFields);
} else {
return Option.empty();
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.util;
import java.util.Properties;
import org.apache.hudi.common.util.collection.Pair;
import java.util.Collections;
@@ -31,6 +32,9 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
public class CollectionUtils {
public static final Properties EMPTY_PROPERTIES = new Properties();
/**
* Determines whether two iterators contain equal elements in the same order. More specifically,
* this method returns {@code true} if {@code iterator1} and {@code iterator2} contain the same

View File

@@ -0,0 +1,114 @@
/*
* 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.common.util;
import java.util.BitSet;
/**
* Utils to encode/decode the partition path.
* This code is mainly copy from Hive (org.apache.hadoop.hive.common.FileUtils).
*/
public class PartitionPathEncodeUtils {
static BitSet charToEscape = new BitSet(128);
static {
for (char c = 0; c < ' '; c++) {
charToEscape.set(c);
}
/**
* ASCII 01-1F are HTTP control characters that need to be escaped.
* \u000A and \u000D are \n and \r, respectively.
*/
char[] clist = new char[] {'\u0001', '\u0002', '\u0003', '\u0004',
'\u0005', '\u0006', '\u0007', '\u0008', '\u0009', '\n', '\u000B',
'\u000C', '\r', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012',
'\u0013', '\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019',
'\u001A', '\u001B', '\u001C', '\u001D', '\u001E', '\u001F',
'"', '#', '%', '\'', '*', '/', ':', '=', '?', '\\', '\u007F', '{',
'[', ']', '^'};
for (char c : clist) {
charToEscape.set(c);
}
}
static boolean needsEscaping(char c) {
return c >= 0 && c < charToEscape.size() && charToEscape.get(c);
}
public static String escapePathName(String path) {
return escapePathName(path, null);
}
/**
* Escapes a path name.
* @param path The path to escape.
* @param defaultPath
* The default name for the path, if the given path is empty or null.
* @return An escaped path name.
*/
public static String escapePathName(String path, String defaultPath) {
// __HIVE_DEFAULT_NULL__ is the system default value for null and empty string.
// TODO: we should allow user to specify default partition or HDFS file location.
if (path == null || path.length() == 0) {
if (defaultPath == null) {
//previously, when path is empty or null and no default path is specified,
// __HIVE_DEFAULT_PARTITION__ was the return value for escapePathName
return "__HIVE_DEFAULT_PARTITION__";
} else {
return defaultPath;
}
}
StringBuilder sb = new StringBuilder();
for (int i = 0; i < path.length(); i++) {
char c = path.charAt(i);
if (needsEscaping(c)) {
sb.append('%');
sb.append(String.format("%1$02X", (int) c));
} else {
sb.append(c);
}
}
return sb.toString();
}
public static String unescapePathName(String path) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < path.length(); i++) {
char c = path.charAt(i);
if (c == '%' && i + 2 < path.length()) {
int code = -1;
try {
code = Integer.parseInt(path.substring(i + 1, i + 3), 16);
} catch (Exception e) {
code = -1;
}
if (code >= 0) {
sb.append((char) code);
i += 2;
continue;
}
}
sb.append(c);
}
return sb.toString();
}
}

View File

@@ -0,0 +1,29 @@
/*
* 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.exception;
/**
* Exception throws when insert a duplicate key to the table using sql insert statement.
*/
public class HoodieDuplicateKeyException extends HoodieException {
public HoodieDuplicateKeyException(String duplicateKey) {
super("Duplicate key found for insert statement, key is: " + duplicateKey);
}
}