[HUDI-2161] Adding support to disable meta columns with bulk insert operation (#3247)
This commit is contained in:
committed by
GitHub
parent
2099bf41db
commit
d5026e9a24
@@ -403,6 +403,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
upgradeDowngrade.run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
|
||||
}
|
||||
}
|
||||
metaClient.validateTableProperties(config.getProps(), operationType);
|
||||
return getTableAndInitCtx(metaClient, operationType, instantTime);
|
||||
}
|
||||
|
||||
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
|
||||
@@ -33,12 +33,19 @@ public interface HoodieInternalRowFileWriter {
|
||||
boolean canWrite();
|
||||
|
||||
/**
|
||||
* Writes an {@link InternalRow} to the HoodieInternalRowFileWriter.
|
||||
* Writes an {@link InternalRow} to the HoodieInternalRowFileWriter. Also takes in associated record key to be added to bloom filter if required.
|
||||
*
|
||||
* @throws IOException on any exception while writing.
|
||||
*/
|
||||
void writeRow(String key, InternalRow row) throws IOException;
|
||||
|
||||
/**
|
||||
* Writes an {@link InternalRow} to the HoodieInternalRowFileWriter.
|
||||
*
|
||||
* @throws IOException on any exception while writing.
|
||||
*/
|
||||
void writeRow(InternalRow row) throws IOException;
|
||||
|
||||
/**
|
||||
* Closes the {@link HoodieInternalRowFileWriter} and may not take in any more writes.
|
||||
*/
|
||||
@@ -16,21 +16,22 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
|
||||
/**
|
||||
* Factory to assist in instantiating a new {@link HoodieInternalRowFileWriter}.
|
||||
*/
|
||||
@@ -76,4 +77,29 @@ public class HoodieInternalRowFileWriterFactory {
|
||||
writeSupport.getHadoopConf(),
|
||||
writeConfig.getParquetCompressionRatio()));
|
||||
}
|
||||
|
||||
public static HoodieInternalRowFileWriter getInternalRowFileWriterWithoutMetaFields(
|
||||
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
|
||||
throws IOException {
|
||||
if (PARQUET.getFileExtension().equals(hoodieTable.getBaseFileExtension())) {
|
||||
return newParquetInternalRowFileWriterWithoutMetaFields(path, config, schema, hoodieTable);
|
||||
}
|
||||
throw new HoodieIOException(hoodieTable.getBaseFileExtension() + " format not supported yet in row writer path");
|
||||
}
|
||||
|
||||
private static HoodieInternalRowFileWriter newParquetInternalRowFileWriterWithoutMetaFields(
|
||||
Path path, HoodieWriteConfig writeConfig, StructType structType, HoodieTable table)
|
||||
throws IOException {
|
||||
HoodieRowParquetWriteSupport writeSupport =
|
||||
new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, null);
|
||||
return new HoodieInternalRowParquetWriter(
|
||||
path, new HoodieRowParquetConfig(
|
||||
writeSupport,
|
||||
writeConfig.getParquetCompressionCodec(),
|
||||
writeConfig.getParquetBlockSize(),
|
||||
writeConfig.getParquetPageSize(),
|
||||
writeConfig.getParquetMaxFileSize(),
|
||||
writeSupport.getHadoopConf(),
|
||||
writeConfig.getParquetCompressionRatio()));
|
||||
}
|
||||
}
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -65,6 +65,11 @@ public class HoodieInternalRowParquetWriter extends ParquetWriter<InternalRow>
|
||||
writeSupport.add(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeRow(InternalRow row) throws IOException {
|
||||
super.write(row);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.client.model.HoodieInternalRow;
|
||||
@@ -30,8 +30,6 @@ import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.io.storage.HoodieInternalRowFileWriter;
|
||||
import org.apache.hudi.io.storage.HoodieInternalRowFileWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
@@ -61,12 +59,12 @@ public class HoodieRowCreateHandle implements Serializable {
|
||||
private final long taskEpochId;
|
||||
private final HoodieTable table;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final HoodieInternalRowFileWriter fileWriter;
|
||||
protected final HoodieInternalRowFileWriter fileWriter;
|
||||
private final String partitionPath;
|
||||
private final Path path;
|
||||
private final String fileId;
|
||||
private final FileSystem fs;
|
||||
private final HoodieInternalWriteStatus writeStatus;
|
||||
protected final HoodieInternalWriteStatus writeStatus;
|
||||
private final HoodieTimer currTimer;
|
||||
|
||||
public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId,
|
||||
@@ -197,7 +195,7 @@ public class HoodieRowCreateHandle implements Serializable {
|
||||
return taskPartitionId + "-" + taskId + "-" + taskEpochId;
|
||||
}
|
||||
|
||||
private HoodieInternalRowFileWriter createNewFileWriter(
|
||||
protected HoodieInternalRowFileWriter createNewFileWriter(
|
||||
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
|
||||
throws IOException {
|
||||
return HoodieInternalRowFileWriterFactory.getInternalRowFileWriter(
|
||||
@@ -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.io.storage.row;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* RowCreateHandle to be used when meta fields are disabled.
|
||||
*/
|
||||
public class HoodieRowCreateHandleWithoutMetaFields extends HoodieRowCreateHandle {
|
||||
|
||||
public HoodieRowCreateHandleWithoutMetaFields(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId, String instantTime,
|
||||
int taskPartitionId, long taskId, long taskEpochId, StructType structType) {
|
||||
super(table, writeConfig, partitionPath, fileId, instantTime, taskPartitionId, taskId, taskEpochId, structType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write the incoming InternalRow as is.
|
||||
*
|
||||
* @param record instance of {@link InternalRow} that needs to be written to the fileWriter.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void write(InternalRow record) throws IOException {
|
||||
try {
|
||||
fileWriter.writeRow(record);
|
||||
writeStatus.markSuccess();
|
||||
} catch (Throwable ge) {
|
||||
writeStatus.setGlobalError(ge);
|
||||
throw new HoodieException("Exception thrown while writing spark InternalRows to file ", ge);
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieInternalRowFileWriter createNewFileWriter(
|
||||
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
|
||||
throws IOException {
|
||||
return HoodieInternalRowFileWriterFactory.getInternalRowFileWriterWithoutMetaFields(
|
||||
path, hoodieTable, config, schema);
|
||||
}
|
||||
}
|
||||
@@ -16,7 +16,9 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.io.storage.HoodieBaseParquetConfig;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
@@ -18,21 +18,28 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.ApiMaturityLevel;
|
||||
import org.apache.hudi.AvroConversionHelper;
|
||||
import org.apache.hudi.HoodieSparkUtils;
|
||||
import org.apache.hudi.PublicAPIMethod;
|
||||
import org.apache.hudi.client.utils.SparkRowSerDe;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import scala.Function1;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import scala.Function1;
|
||||
|
||||
/**
|
||||
* Base class for the built-in key generators. Contains methods structured for
|
||||
* code reuse amongst them.
|
||||
@@ -42,10 +49,12 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
|
||||
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
|
||||
private static final String NAMESPACE = "hoodieRow";
|
||||
private transient Function1<Object, Object> converterFn = null;
|
||||
private SparkRowSerDe sparkRowSerDe;
|
||||
protected StructType structType;
|
||||
|
||||
protected Map<String, List<Integer>> recordKeyPositions = new HashMap<>();
|
||||
protected Map<String, List<Integer>> partitionPathPositions = new HashMap<>();
|
||||
protected Map<String, List<DataType>> partitionPathDataTypes = null;
|
||||
|
||||
protected BuiltinKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
@@ -81,6 +90,29 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
|
||||
return getKey(genericRecord).getPartitionPath();
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch partition path from {@link InternalRow}.
|
||||
*
|
||||
* @param internalRow {@link InternalRow} instance from which partition path needs to be fetched from.
|
||||
* @param structType schema of the internalRow.
|
||||
* @return the partition path.
|
||||
*/
|
||||
public String getPartitionPath(InternalRow internalRow, StructType structType) {
|
||||
try {
|
||||
initDeserializer(structType);
|
||||
Row row = sparkRowSerDe.deserializeRow(internalRow);
|
||||
return getPartitionPath(row);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIOException("Conversion of InternalRow to Row failed with exception " + e);
|
||||
}
|
||||
}
|
||||
|
||||
private void initDeserializer(StructType structType) {
|
||||
if (sparkRowSerDe == null) {
|
||||
sparkRowSerDe = HoodieSparkUtils.getDeserializer(structType);
|
||||
}
|
||||
}
|
||||
|
||||
void buildFieldPositionMapIfNeeded(StructType structType) {
|
||||
if (this.structType == null) {
|
||||
// parse simple fields
|
||||
@@ -116,5 +148,39 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
|
||||
this.structType = structType;
|
||||
}
|
||||
}
|
||||
|
||||
protected String getPartitionPathInternal(InternalRow row, StructType structType) {
|
||||
buildFieldDataTypesMapIfNeeded(structType);
|
||||
validatePartitionFieldsForInternalRow();
|
||||
return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(row, getPartitionPathFields(),
|
||||
hiveStylePartitioning, partitionPathPositions, partitionPathDataTypes);
|
||||
}
|
||||
|
||||
protected void validatePartitionFieldsForInternalRow() {
|
||||
partitionPathPositions.entrySet().forEach(entry -> {
|
||||
if (entry.getValue().size() > 1) {
|
||||
throw new IllegalArgumentException("Nested column for partitioning is not supported with disabling meta columns");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void buildFieldDataTypesMapIfNeeded(StructType structType) {
|
||||
buildFieldPositionMapIfNeeded(structType);
|
||||
if (this.partitionPathDataTypes == null) {
|
||||
this.partitionPathDataTypes = new HashMap<>();
|
||||
if (getPartitionPathFields() != null) {
|
||||
// populating simple fields are good enough
|
||||
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains(".")))
|
||||
.forEach(f -> {
|
||||
if (structType.getFieldIndex(f).isDefined()) {
|
||||
partitionPathDataTypes.put(f,
|
||||
Collections.singletonList((structType.fields()[structType.fieldIndex(f)].dataType())));
|
||||
} else {
|
||||
partitionPathDataTypes.put(f, Collections.singletonList(null));
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -17,10 +17,13 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -64,4 +67,9 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
|
||||
hiveStylePartitioning, partitionPathPositions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(InternalRow row, StructType structType) {
|
||||
return getPartitionPathInternal(row, structType);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -18,13 +18,17 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
import org.apache.hudi.exception.HoodieKeyGeneratorException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
@@ -74,10 +78,15 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(Row row) {
|
||||
return getPartitionPath(Option.empty(), Option.of(row));
|
||||
return getPartitionPath(Option.empty(), Option.of(row), Option.empty());
|
||||
}
|
||||
|
||||
private String getPartitionPath(Option<GenericRecord> record, Option<Row> row) {
|
||||
@Override
|
||||
public String getPartitionPath(InternalRow row, StructType structType) {
|
||||
return getPartitionPath(Option.empty(), Option.empty(), Option.of(Pair.of(row, structType)));
|
||||
}
|
||||
|
||||
private String getPartitionPath(Option<GenericRecord> record, Option<Row> row, Option<Pair<InternalRow, StructType>> internalRowStructTypePair) {
|
||||
if (getPartitionPathFields() == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for partition path in cfg");
|
||||
}
|
||||
@@ -101,16 +110,22 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
case SIMPLE:
|
||||
if (record.isPresent()) {
|
||||
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(record.get()));
|
||||
} else {
|
||||
} else if (row.isPresent()) {
|
||||
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(row.get()));
|
||||
} else {
|
||||
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(internalRowStructTypePair.get().getKey(),
|
||||
internalRowStructTypePair.get().getValue()));
|
||||
}
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
try {
|
||||
if (record.isPresent()) {
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(record.get()));
|
||||
} else {
|
||||
} else if (row.isPresent()) {
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(row.get()));
|
||||
} else {
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(internalRowStructTypePair.get().getKey(),
|
||||
internalRowStructTypePair.get().getValue()));
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class", ioe);
|
||||
|
||||
@@ -18,10 +18,13 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
@@ -65,5 +68,10 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
|
||||
public String getPartitionPath(Row row) {
|
||||
return globalAvroDeleteKeyGenerator.getEmptyPartition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(InternalRow row, StructType structType) {
|
||||
return globalAvroDeleteKeyGenerator.getEmptyPartition();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -22,6 +22,8 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
@@ -63,5 +65,9 @@ public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator {
|
||||
return nonpartitionedAvroKeyGenerator.getEmptyPartition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(InternalRow internalRow, StructType structType) {
|
||||
return nonpartitionedAvroKeyGenerator.getEmptyPartition();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -19,10 +19,13 @@
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.sql.types.StructField;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import scala.Option;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
@@ -33,6 +36,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import scala.Option;
|
||||
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
|
||||
@@ -121,6 +126,54 @@ public class RowKeyGeneratorHelper {
|
||||
}).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR));
|
||||
}
|
||||
|
||||
public static String getPartitionPathFromInternalRow(InternalRow row, List<String> partitionPathFields, boolean hiveStylePartitioning,
|
||||
Map<String, List<Integer>> partitionPathPositions,
|
||||
Map<String, List<DataType>> partitionPathDataTypes) {
|
||||
return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> {
|
||||
String field = partitionPathFields.get(idx);
|
||||
String val = null;
|
||||
List<Integer> fieldPositions = partitionPathPositions.get(field);
|
||||
if (fieldPositions.size() == 1) { // simple
|
||||
Integer fieldPos = fieldPositions.get(0);
|
||||
// for partition path, if field is not found, index will be set to -1
|
||||
if (fieldPos == -1 || row.isNullAt(fieldPos)) {
|
||||
val = DEFAULT_PARTITION_PATH;
|
||||
} else {
|
||||
Object value = row.get(fieldPos, partitionPathDataTypes.get(field).get(0));
|
||||
if (value == null || value.toString().isEmpty()) {
|
||||
val = DEFAULT_PARTITION_PATH;
|
||||
} else {
|
||||
val = value.toString();
|
||||
}
|
||||
}
|
||||
if (hiveStylePartitioning) {
|
||||
val = field + "=" + val;
|
||||
}
|
||||
} else { // nested
|
||||
throw new IllegalArgumentException("Nested partitioning is not supported with disabling meta columns.");
|
||||
}
|
||||
return val;
|
||||
}).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR));
|
||||
}
|
||||
|
||||
public static Object getFieldValFromInternalRow(InternalRow internalRow,
|
||||
Integer partitionPathPosition,
|
||||
DataType partitionPathDataType) {
|
||||
Object val = null;
|
||||
if (internalRow.isNullAt(partitionPathPosition)) {
|
||||
return DEFAULT_PARTITION_PATH;
|
||||
} else {
|
||||
Object value = partitionPathDataType == DataTypes.StringType ? internalRow.getString(partitionPathPosition) : internalRow.get(partitionPathPosition, partitionPathDataType);
|
||||
if (value == null || value.toString().isEmpty()) {
|
||||
val = DEFAULT_PARTITION_PATH;
|
||||
} else {
|
||||
val = value;
|
||||
}
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Fetch the field value located at the positions requested for.
|
||||
*
|
||||
|
||||
@@ -18,10 +18,13 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
@@ -72,4 +75,9 @@ public class SimpleKeyGenerator extends BuiltinKeyGenerator {
|
||||
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
|
||||
hiveStylePartitioning, partitionPathPositions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(InternalRow row, StructType structType) {
|
||||
return getPartitionPathInternal(row, structType);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,11 +18,14 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieKeyGeneratorException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@@ -64,9 +67,23 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(Row row) {
|
||||
Object fieldVal = null;
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathPositions.get(getPartitionPathFields().get(0)));
|
||||
return getTimestampBasedPartitionPath(partitionPathFieldVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(InternalRow internalRow, StructType structType) {
|
||||
buildFieldDataTypesMapIfNeeded(structType);
|
||||
validatePartitionFieldsForInternalRow();
|
||||
Object partitionPathFieldVal = RowKeyGeneratorHelper.getFieldValFromInternalRow(internalRow,
|
||||
partitionPathPositions.get(getPartitionPathFields().get(0)).get(0),
|
||||
partitionPathDataTypes.get(getPartitionPathFields().get(0)).get(0));
|
||||
return getTimestampBasedPartitionPath(partitionPathFieldVal);
|
||||
}
|
||||
|
||||
private String getTimestampBasedPartitionPath(Object partitionPathFieldVal) {
|
||||
Object fieldVal = null;
|
||||
try {
|
||||
if (partitionPathFieldVal == null || partitionPathFieldVal.toString().contains(DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER)
|
||||
|| partitionPathFieldVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
|
||||
|
||||
@@ -0,0 +1,214 @@
|
||||
/*
|
||||
* 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.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.client.utils.SparkRowSerDe
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal}
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex}
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
|
||||
import scala.collection.JavaConverters.asScalaBufferConverter
|
||||
|
||||
object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
|
||||
def isSpark3: Boolean = SPARK_VERSION.startsWith("3.")
|
||||
|
||||
def getMetaSchema: StructType = {
|
||||
StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => {
|
||||
StructField(col, StringType, nullable = true)
|
||||
}))
|
||||
}
|
||||
|
||||
/**
|
||||
* This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]].
|
||||
* [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally.
|
||||
*/
|
||||
def isGlobPath(pattern: Path): Boolean = {
|
||||
pattern.toString.exists("{}[]*?\\".toSet.contains)
|
||||
}
|
||||
|
||||
/**
|
||||
* This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]].
|
||||
* [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally.
|
||||
*/
|
||||
def globPath(fs: FileSystem, pattern: Path): Seq[Path] = {
|
||||
Option(fs.globStatus(pattern)).map { statuses =>
|
||||
statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq
|
||||
}.getOrElse(Seq.empty[Path])
|
||||
}
|
||||
|
||||
/**
|
||||
* This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]].
|
||||
* [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally.
|
||||
*/
|
||||
def globPathIfNecessary(fs: FileSystem, pattern: Path): Seq[Path] = {
|
||||
if (isGlobPath(pattern)) globPath(fs, pattern) else Seq(pattern)
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks to see whether input path contains a glob pattern and if yes, maps it to a list of absolute paths
|
||||
* which match the glob pattern. Otherwise, returns original path
|
||||
*
|
||||
* @param paths List of absolute or globbed paths
|
||||
* @param fs File system
|
||||
* @return list of absolute file paths
|
||||
*/
|
||||
def checkAndGlobPathIfNecessary(paths: Seq[String], fs: FileSystem): Seq[Path] = {
|
||||
paths.flatMap(path => {
|
||||
val qualified = new Path(path).makeQualified(fs.getUri, fs.getWorkingDirectory)
|
||||
val globPaths = globPathIfNecessary(fs, qualified)
|
||||
globPaths
|
||||
})
|
||||
}
|
||||
|
||||
def createInMemoryFileIndex(sparkSession: SparkSession, globbedPaths: Seq[Path]): InMemoryFileIndex = {
|
||||
val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
|
||||
new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache)
|
||||
}
|
||||
|
||||
def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = {
|
||||
val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, recordNamespace)
|
||||
createRdd(df, avroSchema, structName, recordNamespace)
|
||||
}
|
||||
|
||||
def createRdd(df: DataFrame, avroSchema: Schema, structName: String, recordNamespace: String)
|
||||
: RDD[GenericRecord] = {
|
||||
// Use the Avro schema to derive the StructType which has the correct nullability information
|
||||
val dataType = SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType]
|
||||
val encoder = RowEncoder.apply(dataType).resolveAndBind()
|
||||
val deserializer = sparkAdapter.createSparkRowSerDe(encoder)
|
||||
df.queryExecution.toRdd.map(row => deserializer.deserializeRow(row))
|
||||
.mapPartitions { records =>
|
||||
if (records.isEmpty) Iterator.empty
|
||||
else {
|
||||
val convertor = AvroConversionHelper.createConverterToAvro(dataType, structName, recordNamespace)
|
||||
records.map { x => convertor(x).asInstanceOf[GenericRecord] }
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getDeserializer(structType: StructType) : SparkRowSerDe = {
|
||||
val encoder = RowEncoder.apply(structType).resolveAndBind()
|
||||
sparkAdapter.createSparkRowSerDe(encoder)
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert Filters to Catalyst Expressions and joined by And. If convert success return an
|
||||
* Non-Empty Option[Expression],or else return None.
|
||||
*/
|
||||
def convertToCatalystExpressions(filters: Array[Filter],
|
||||
tableSchema: StructType): Option[Expression] = {
|
||||
val expressions = filters.map(convertToCatalystExpression(_, tableSchema))
|
||||
if (expressions.forall(p => p.isDefined)) {
|
||||
if (expressions.isEmpty) {
|
||||
None
|
||||
} else if (expressions.length == 1) {
|
||||
expressions(0)
|
||||
} else {
|
||||
Some(expressions.map(_.get).reduce(org.apache.spark.sql.catalyst.expressions.And))
|
||||
}
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert Filter to Catalyst Expression. If convert success return an Non-Empty
|
||||
* Option[Expression],or else return None.
|
||||
*/
|
||||
def convertToCatalystExpression(filter: Filter, tableSchema: StructType): Option[Expression] = {
|
||||
Option(
|
||||
filter match {
|
||||
case EqualTo(attribute, value) =>
|
||||
org.apache.spark.sql.catalyst.expressions.EqualTo(toAttribute(attribute, tableSchema), Literal.create(value))
|
||||
case EqualNullSafe(attribute, value) =>
|
||||
org.apache.spark.sql.catalyst.expressions.EqualNullSafe(toAttribute(attribute, tableSchema), Literal.create(value))
|
||||
case GreaterThan(attribute, value) =>
|
||||
org.apache.spark.sql.catalyst.expressions.GreaterThan(toAttribute(attribute, tableSchema), Literal.create(value))
|
||||
case GreaterThanOrEqual(attribute, value) =>
|
||||
org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual(toAttribute(attribute, tableSchema), Literal.create(value))
|
||||
case LessThan(attribute, value) =>
|
||||
org.apache.spark.sql.catalyst.expressions.LessThan(toAttribute(attribute, tableSchema), Literal.create(value))
|
||||
case LessThanOrEqual(attribute, value) =>
|
||||
org.apache.spark.sql.catalyst.expressions.LessThanOrEqual(toAttribute(attribute, tableSchema), Literal.create(value))
|
||||
case In(attribute, values) =>
|
||||
val attrExp = toAttribute(attribute, tableSchema)
|
||||
val valuesExp = values.map(v => Literal.create(v))
|
||||
org.apache.spark.sql.catalyst.expressions.In(attrExp, valuesExp)
|
||||
case IsNull(attribute) =>
|
||||
org.apache.spark.sql.catalyst.expressions.IsNull(toAttribute(attribute, tableSchema))
|
||||
case IsNotNull(attribute) =>
|
||||
org.apache.spark.sql.catalyst.expressions.IsNotNull(toAttribute(attribute, tableSchema))
|
||||
case And(left, right) =>
|
||||
val leftExp = convertToCatalystExpression(left, tableSchema)
|
||||
val rightExp = convertToCatalystExpression(right, tableSchema)
|
||||
if (leftExp.isEmpty || rightExp.isEmpty) {
|
||||
null
|
||||
} else {
|
||||
org.apache.spark.sql.catalyst.expressions.And(leftExp.get, rightExp.get)
|
||||
}
|
||||
case Or(left, right) =>
|
||||
val leftExp = convertToCatalystExpression(left, tableSchema)
|
||||
val rightExp = convertToCatalystExpression(right, tableSchema)
|
||||
if (leftExp.isEmpty || rightExp.isEmpty) {
|
||||
null
|
||||
} else {
|
||||
org.apache.spark.sql.catalyst.expressions.Or(leftExp.get, rightExp.get)
|
||||
}
|
||||
case Not(child) =>
|
||||
val childExp = convertToCatalystExpression(child, tableSchema)
|
||||
if (childExp.isEmpty) {
|
||||
null
|
||||
} else {
|
||||
org.apache.spark.sql.catalyst.expressions.Not(childExp.get)
|
||||
}
|
||||
case StringStartsWith(attribute, value) =>
|
||||
val leftExp = toAttribute(attribute, tableSchema)
|
||||
val rightExp = Literal.create(s"$value%")
|
||||
sparkAdapter.createLike(leftExp, rightExp)
|
||||
case StringEndsWith(attribute, value) =>
|
||||
val leftExp = toAttribute(attribute, tableSchema)
|
||||
val rightExp = Literal.create(s"%$value")
|
||||
sparkAdapter.createLike(leftExp, rightExp)
|
||||
case StringContains(attribute, value) =>
|
||||
val leftExp = toAttribute(attribute, tableSchema)
|
||||
val rightExp = Literal.create(s"%$value%")
|
||||
sparkAdapter.createLike(leftExp, rightExp)
|
||||
case _=> null
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
private def toAttribute(columnName: String, tableSchema: StructType): AttributeReference = {
|
||||
val field = tableSchema.find(p => p.name == columnName)
|
||||
assert(field.isDefined, s"Cannot find column: $columnName, Table Columns are: " +
|
||||
s"${tableSchema.fieldNames.mkString(",")}")
|
||||
AttributeReference(columnName, field.get.dataType, field.get.nullable)()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi.SparkAdapter
|
||||
|
||||
/**
|
||||
* Use the SparkAdapterSupport trait to get the SparkAdapter when we
|
||||
* need to adapt the difference between spark2 and spark3.
|
||||
*/
|
||||
trait SparkAdapterSupport {
|
||||
|
||||
lazy val sparkAdapter: SparkAdapter = {
|
||||
val adapterClass = if (HoodieSparkUtils.isSpark3) {
|
||||
"org.apache.spark.sql.adapter.Spark3Adapter"
|
||||
} else {
|
||||
"org.apache.spark.sql.adapter.Spark2Adapter"
|
||||
}
|
||||
getClass.getClassLoader.loadClass(adapterClass)
|
||||
.newInstance().asInstanceOf[SparkAdapter]
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,90 @@
|
||||
/*
|
||||
* 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.hudi
|
||||
|
||||
import org.apache.hudi.client.utils.SparkRowSerDe
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.execution.datasources.SparkParsePartitionUtil
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
|
||||
/**
|
||||
* An interface to adapter the difference between spark2 and spark3
|
||||
* in some spark related class.
|
||||
*/
|
||||
trait SparkAdapter extends Serializable {
|
||||
|
||||
/**
|
||||
* Create the SparkRowSerDe.
|
||||
*/
|
||||
def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe
|
||||
|
||||
/**
|
||||
* Convert a AliasIdentifier to TableIdentifier.
|
||||
*/
|
||||
def toTableIdentify(aliasId: AliasIdentifier): TableIdentifier
|
||||
|
||||
/**
|
||||
* Convert a UnresolvedRelation to TableIdentifier.
|
||||
*/
|
||||
def toTableIdentify(relation: UnresolvedRelation): TableIdentifier
|
||||
|
||||
/**
|
||||
* Create Join logical plan.
|
||||
*/
|
||||
def createJoin(left: LogicalPlan, right: LogicalPlan, joinType: JoinType): Join
|
||||
|
||||
/**
|
||||
* Test if the logical plan is a Insert Into LogicalPlan.
|
||||
*/
|
||||
def isInsertInto(plan: LogicalPlan): Boolean
|
||||
|
||||
/**
|
||||
* Get the member of the Insert Into LogicalPlan.
|
||||
*/
|
||||
def getInsertIntoChildren(plan: LogicalPlan):
|
||||
Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)]
|
||||
|
||||
/**
|
||||
* Create a Insert Into LogicalPlan.
|
||||
*/
|
||||
def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]],
|
||||
query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan
|
||||
|
||||
/**
|
||||
* Create the hoodie's extended spark sql parser.
|
||||
*/
|
||||
def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = None
|
||||
|
||||
/**
|
||||
* Create the SparkParsePartitionUtil.
|
||||
*/
|
||||
def createSparkParsePartitionUtil(conf: SQLConf): SparkParsePartitionUtil
|
||||
|
||||
/**
|
||||
* Create Like expression.
|
||||
*/
|
||||
def createLike(left: Expression, right: Expression): Expression
|
||||
}
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
@@ -27,6 +28,7 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
@@ -95,6 +97,9 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=2020-03-21");
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=2020-03-21");
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -112,6 +117,12 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
|
||||
assertEquals("_row_key:" + rowKey, hoodieKey.getRecordKey());
|
||||
assertEquals(partitionPath, hoodieKey.getPartitionPath());
|
||||
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -131,6 +142,13 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
|
||||
assertEquals(rowKey, hoodieKey.getRecordKey());
|
||||
assertEquals(partitionPath, hoodieKey.getPartitionPath());
|
||||
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -150,5 +168,12 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
|
||||
assertEquals(rowKey, hoodieKey.getRecordKey());
|
||||
assertEquals(partitionPath, hoodieKey.getPartitionPath());
|
||||
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,6 +28,7 @@ import org.apache.hudi.keygen.constant.KeyGeneratorType;
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
@@ -141,6 +142,8 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -164,6 +167,8 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321");
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "ts_ms=20200321");
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -187,6 +192,9 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
|
||||
Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty());
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertTrue(keyGenerator.getPartitionPath(internalRow, row.schema()).isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -335,6 +343,9 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -359,5 +370,8 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=20200321");
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=20200321");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,11 +21,12 @@ package org.apache.hudi.keygen;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
@@ -130,6 +131,9 @@ public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
Row row = KeyGeneratorTestUtilities.getRow(record);
|
||||
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
|
||||
|
||||
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
|
||||
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
|
||||
}
|
||||
|
||||
private static Stream<GenericRecord> nestedColTestRecords() {
|
||||
|
||||
@@ -28,7 +28,10 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.exception.HoodieKeyGeneratorException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
|
||||
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
@@ -50,6 +53,7 @@ public class TestTimestampBasedKeyGenerator {
|
||||
private Schema schema;
|
||||
private StructType structType;
|
||||
private Row baseRow;
|
||||
private InternalRow internalRow;
|
||||
|
||||
@BeforeEach
|
||||
public void initialize() throws IOException {
|
||||
@@ -58,6 +62,7 @@ public class TestTimestampBasedKeyGenerator {
|
||||
baseRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 1, "001", "f1");
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
|
||||
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "field1");
|
||||
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "createTime");
|
||||
@@ -77,7 +82,7 @@ public class TestTimestampBasedKeyGenerator {
|
||||
}
|
||||
|
||||
private Row genericRecordToRow(GenericRecord baseRecord) {
|
||||
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(schema, structType);
|
||||
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(baseRecord.getSchema(), structType);
|
||||
Row row = (Row) convertor.apply(baseRecord);
|
||||
int fieldCount = structType.fieldNames().length;
|
||||
Object[] values = new Object[fieldCount];
|
||||
@@ -117,6 +122,10 @@ public class TestTimestampBasedKeyGenerator {
|
||||
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
|
||||
HoodieKey hk1 = keyGen.getKey(baseRecord);
|
||||
assertEquals("2020-01-06 12", hk1.getPartitionPath());
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
assertEquals("2020-01-06 12", keyGen.getPartitionPath(internalRow, baseRow.schema()));
|
||||
|
||||
// timezone is GMT+8:00, createTime is BigDecimal
|
||||
baseRecord.put("createTime", new BigDecimal(1578283932000.00001));
|
||||
@@ -169,6 +178,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// test w/ Row
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
assertEquals("1970-01-01 08", keyGen.getPartitionPath(baseRow));
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
assertEquals("1970-01-01 08", keyGen.getPartitionPath(internalRow, baseRow.schema()));
|
||||
|
||||
// timestamp is DATE_STRING, timezone is GMT, createTime is null
|
||||
baseRecord.put("createTime", null);
|
||||
@@ -181,6 +192,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// test w/ Row
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(baseRow));
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(internalRow, baseRow.schema()));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -197,6 +210,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// test w/ Row
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow));
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
assertEquals("2024-10-04 12", keyGen.getPartitionPath(internalRow, baseRow.schema()));
|
||||
|
||||
// timezone is GMT, createTime is null
|
||||
baseRecord.put("createTime", null);
|
||||
@@ -208,6 +223,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// test w/ Row
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
assertEquals("1970-01-02 12", keyGen.getPartitionPath(baseRow));
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
assertEquals("1970-01-02 12", keyGen.getPartitionPath(internalRow, baseRow.schema()));
|
||||
|
||||
// timezone is GMT. number of days store integer in mysql
|
||||
baseRecord.put("createTime", 18736);
|
||||
@@ -216,6 +233,9 @@ public class TestTimestampBasedKeyGenerator {
|
||||
HoodieKey scalarSecondsKey = keyGen.getKey(baseRecord);
|
||||
assertEquals("2021-04-19", scalarSecondsKey.getPartitionPath());
|
||||
|
||||
// test w/ Row
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
assertEquals("2021-04-19", keyGen.getPartitionPath(baseRow));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -107,6 +107,11 @@ public class HoodieClientTestUtils {
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String instantTime) {
|
||||
return readCommit(basePath, sqlContext, commitTimeline, instantTime, true);
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String instantTime, boolean filterByCommitTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
throw new HoodieException("No commit exists at " + instantTime);
|
||||
@@ -115,14 +120,21 @@ public class HoodieClientTestUtils {
|
||||
HashMap<String, String> paths =
|
||||
getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
|
||||
LOG.info("Path :" + paths.values());
|
||||
Dataset<Row> unFilteredRows = null;
|
||||
if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.PARQUET)) {
|
||||
return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
unFilteredRows = sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]));
|
||||
} else if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.ORC)) {
|
||||
return sqlContext.read().orc(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
unFilteredRows = sqlContext.read().orc(paths.values().toArray(new String[paths.size()]));
|
||||
}
|
||||
if (unFilteredRows != null) {
|
||||
if (filterByCommitTime) {
|
||||
return unFilteredRows.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
} else {
|
||||
return unFilteredRows;
|
||||
}
|
||||
} else {
|
||||
return sqlContext.emptyDataFrame();
|
||||
}
|
||||
return sqlContext.emptyDataFrame();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading commit " + instantTime, e);
|
||||
}
|
||||
|
||||
@@ -24,11 +24,24 @@ import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.package$;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
|
||||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
||||
import org.apache.spark.sql.catalyst.expressions.Attribute;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Function1;
|
||||
import scala.collection.JavaConversions;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
public class KeyGeneratorTestUtilities {
|
||||
|
||||
@@ -60,7 +73,7 @@ public class KeyGeneratorTestUtilities {
|
||||
|
||||
public static GenericRecord getRecord(GenericRecord nestedColRecord) {
|
||||
GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA));
|
||||
record.put("timestamp", 4357686);
|
||||
record.put("timestamp", 4357686L);
|
||||
record.put("_row_key", "key1");
|
||||
record.put("ts_ms", "2020-03-21");
|
||||
record.put("pii_col", "pi");
|
||||
@@ -82,4 +95,39 @@ public class KeyGeneratorTestUtilities {
|
||||
}
|
||||
return new GenericRowWithSchema(values, structType);
|
||||
}
|
||||
|
||||
public static InternalRow getInternalRow(Row row) {
|
||||
try {
|
||||
return getInternalRow(row, getEncoder(row.schema()));
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Exception thrown while converting Row to InternalRow", e);
|
||||
}
|
||||
}
|
||||
|
||||
private static ExpressionEncoder getEncoder(StructType schema) {
|
||||
List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
|
||||
.map(Attribute::toAttribute).collect(Collectors.toList());
|
||||
return RowEncoder.apply(schema)
|
||||
.resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(),
|
||||
SimpleAnalyzer$.MODULE$);
|
||||
}
|
||||
|
||||
public static InternalRow getInternalRow(Row row, ExpressionEncoder<Row> encoder) throws ClassNotFoundException, InvocationTargetException, IllegalAccessException, NoSuchMethodException {
|
||||
return serializeRow(encoder, row);
|
||||
}
|
||||
|
||||
private static InternalRow serializeRow(ExpressionEncoder encoder, Row row)
|
||||
throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
|
||||
// TODO remove reflection if Spark 2.x support is dropped
|
||||
if (package$.MODULE$.SPARK_VERSION().startsWith("2.")) {
|
||||
Method spark2method = encoder.getClass().getMethod("toRow", Object.class);
|
||||
return (InternalRow) spark2method.invoke(encoder, row);
|
||||
} else {
|
||||
Class<?> serializerClass = Class.forName("org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer");
|
||||
Object serializer = encoder.getClass().getMethod("createSerializer").invoke(encoder);
|
||||
Method aboveSpark2method = serializerClass.getMethod("apply", Object.class);
|
||||
return (InternalRow) aboveSpark2method.invoke(serializer, row);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -59,12 +59,17 @@ import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
|
||||
*/
|
||||
public class SparkDatasetTestUtils {
|
||||
|
||||
public static final String RECORD_KEY_FIELD_NAME = "record_key";
|
||||
public static final String PARTITION_PATH_FIELD_NAME = "partition_path";
|
||||
|
||||
public static final StructType STRUCT_TYPE = new StructType(new StructField[] {
|
||||
new StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.FILENAME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(RECORD_KEY_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(PARTITION_PATH_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("randomLong", DataTypes.LongType, false, Metadata.empty())});
|
||||
|
||||
@@ -74,6 +79,8 @@ public class SparkDatasetTestUtils {
|
||||
new StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.FILENAME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(RECORD_KEY_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(PARTITION_PATH_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("randomStr", DataTypes.StringType, false, Metadata.empty())});
|
||||
|
||||
@@ -117,7 +124,7 @@ public class SparkDatasetTestUtils {
|
||||
*/
|
||||
public static Row getRandomValue(String partitionPath, boolean isError) {
|
||||
// order commit time, seq no, record key, partition path, file name
|
||||
Object[] values = new Object[7];
|
||||
Object[] values = new Object[9];
|
||||
values[0] = ""; //commit time
|
||||
if (!isError) {
|
||||
values[1] = ""; // commit seq no
|
||||
@@ -127,11 +134,13 @@ public class SparkDatasetTestUtils {
|
||||
values[2] = UUID.randomUUID().toString();
|
||||
values[3] = partitionPath;
|
||||
values[4] = ""; // filename
|
||||
values[5] = RANDOM.nextInt();
|
||||
values[5] = UUID.randomUUID().toString();
|
||||
values[6] = partitionPath;
|
||||
values[7] = RANDOM.nextInt();
|
||||
if (!isError) {
|
||||
values[6] = RANDOM.nextLong();
|
||||
values[8] = RANDOM.nextLong();
|
||||
} else {
|
||||
values[6] = UUID.randomUUID().toString();
|
||||
values[8] = UUID.randomUUID().toString();
|
||||
}
|
||||
return new GenericRow(values);
|
||||
}
|
||||
@@ -154,14 +163,16 @@ public class SparkDatasetTestUtils {
|
||||
public static InternalRow getInternalRowWithError(String partitionPath) {
|
||||
// order commit time, seq no, record key, partition path, file name
|
||||
String recordKey = UUID.randomUUID().toString();
|
||||
Object[] values = new Object[7];
|
||||
Object[] values = new Object[9];
|
||||
values[0] = "";
|
||||
values[1] = "";
|
||||
values[2] = recordKey;
|
||||
values[3] = partitionPath;
|
||||
values[4] = "";
|
||||
values[5] = RANDOM.nextInt();
|
||||
values[6] = RANDOM.nextBoolean();
|
||||
values[5] = recordKey;
|
||||
values[6] = partitionPath;
|
||||
values[7] = RANDOM.nextInt();
|
||||
values[8] = RANDOM.nextBoolean();
|
||||
return new GenericInternalRow(values);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user