1
0

[HUDI-1013] Adding Bulk Insert V2 implementation (#1834)

- Adding ability to use native spark row writing for bulk_insert
 - Controlled by `ENABLE_ROW_WRITER_OPT_KEY` datasource write option
 - Introduced KeyGeneratorInterface in hudi-client, moved KeyGenerator back to hudi-spark
 - Simplified the new API additions to just two new methods : getRecordKey(row), getPartitionPath(row)
 - Fixed all built-in key generators with new APIs
 - Made the field position map lazily created upon the first call to row based apis
 - Implemented native row based key generators for CustomKeyGenerator
 - Fixed all the tests, with these new APIs

Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Sivabalan Narayanan
2020-08-13 03:33:39 -04:00
committed by GitHub
parent 8d04268264
commit 379cf0786f
62 changed files with 4682 additions and 485 deletions

View File

@@ -28,7 +28,7 @@ import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.util.ParquetUtils
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
import org.apache.hudi.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter}
import org.apache.parquet.avro.AvroSchemaConverter
import org.apache.parquet.hadoop.metadata.CompressionCodecName
import org.apache.spark.sql.{DataFrame, SQLContext}
@@ -45,7 +45,7 @@ object SparkHelpers {
val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble,
HoodieIndexConfig.DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_INDEX_FILTER_TYPE);
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
// Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'.
parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader)

View File

@@ -95,20 +95,19 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
*/
public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
Option<Map<String, String>> extraMetadata) {
HoodieTableMetaClient metaClient = createMetaClient(false);
return commit(instantTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
List<HoodieWriteStat> stats = writeStatuses.map(WriteStatus::getStat).collect();
return commitStats(instantTime, stats, extraMetadata);
}
private boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
Option<Map<String, String>> extraMetadata, String actionType) {
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata) {
LOG.info("Committing " + instantTime);
HoodieTableMetaClient metaClient = createMetaClient(false);
String actionType = metaClient.getCommitActionType();
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
List<HoodieWriteStat> stats = writeStatuses.map(WriteStatus::getStat).collect();
stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat));
// Finalize write

View File

@@ -0,0 +1,150 @@
/*
* 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.client;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
/**
* Hoodie's internal write status used in datasource implementation of bulk insert.
*/
public class HoodieInternalWriteStatus implements Serializable {
private static final long serialVersionUID = 1L;
private static final long RANDOM_SEED = 9038412832L;
private String fileId;
private String partitionPath;
private List<String> successRecordKeys = new ArrayList<>();
private List<Pair<String, Throwable>> failedRecordKeys = new ArrayList<>();
private HoodieWriteStat stat;
private long totalRecords = 0;
private long totalErrorRecords = 0;
private Throwable globalError = null;
private final double failureFraction;
private final boolean trackSuccessRecords;
private final transient Random random;
public HoodieInternalWriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
this.trackSuccessRecords = trackSuccessRecords;
this.failureFraction = failureFraction;
this.random = new Random(RANDOM_SEED);
}
public void markSuccess(String recordKey) {
if (trackSuccessRecords) {
this.successRecordKeys.add(recordKey);
}
totalRecords++;
}
public void markFailure(String recordKey, Throwable t) {
if (failedRecordKeys.isEmpty() || (random.nextDouble() <= failureFraction)) {
failedRecordKeys.add(Pair.of(recordKey, t));
}
totalRecords++;
}
public boolean hasErrors() {
return failedRecordKeys.size() != 0;
}
public HoodieWriteStat getStat() {
return stat;
}
public void setStat(HoodieWriteStat stat) {
this.stat = stat;
}
public String getFileId() {
return fileId;
}
public void setFileId(String fileId) {
this.fileId = fileId;
}
public String getPartitionPath() {
return partitionPath;
}
public void setPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
}
public List<String> getSuccessRecordKeys() {
return successRecordKeys;
}
public long getFailedRowsSize() {
return failedRecordKeys.size();
}
public List<Pair<String, Throwable>> getFailedRecordKeys() {
return failedRecordKeys;
}
public void setFailedRecordKeys(List<Pair<String, Throwable>> failedRecordKeys) {
this.failedRecordKeys = failedRecordKeys;
}
public long getTotalRecords() {
return totalRecords;
}
public void setTotalRecords(long totalRecords) {
this.totalRecords = totalRecords;
}
public long getTotalErrorRecords() {
return totalErrorRecords;
}
public void setTotalErrorRecords(long totalErrorRecords) {
this.totalErrorRecords = totalErrorRecords;
}
public Throwable getGlobalError() {
return globalError;
}
public void setGlobalError(Throwable globalError) {
this.globalError = globalError;
}
public void setSuccessRecordKeys(List<String> successRecordKeys) {
this.successRecordKeys = successRecordKeys;
}
@Override
public String toString() {
return "PartitionPath " + partitionPath + ", FileID " + fileId + ", Success records "
+ totalRecords + ", errored Rows " + totalErrorRecords
+ ", global error " + (globalError != null);
}
}

View File

@@ -0,0 +1,243 @@
/*
* 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.client.model;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.util.ArrayData;
import org.apache.spark.sql.catalyst.util.MapData;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
* Internal Row implementation for Hoodie Row. It wraps an {@link InternalRow} and keeps meta columns locally. But the {@link InternalRow}
* does include the meta columns as well just that {@link HoodieInternalRow} will intercept queries for meta columns and serve from its
* copy rather than fetching from {@link InternalRow}.
*/
public class HoodieInternalRow extends InternalRow {
private String commitTime;
private String commitSeqNumber;
private String recordKey;
private String partitionPath;
private String fileName;
private InternalRow row;
public HoodieInternalRow(String commitTime, String commitSeqNumber, String recordKey, String partitionPath,
String fileName, InternalRow row) {
this.commitTime = commitTime;
this.commitSeqNumber = commitSeqNumber;
this.recordKey = recordKey;
this.partitionPath = partitionPath;
this.fileName = fileName;
this.row = row;
}
@Override
public int numFields() {
return row.numFields();
}
@Override
public void setNullAt(int i) {
if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) {
switch (i) {
case 0: {
this.commitTime = null;
break;
}
case 1: {
this.commitSeqNumber = null;
break;
}
case 2: {
this.recordKey = null;
break;
}
case 3: {
this.partitionPath = null;
break;
}
case 4: {
this.fileName = null;
break;
}
default: throw new IllegalArgumentException("Not expected");
}
} else {
row.setNullAt(i);
}
}
@Override
public void update(int i, Object value) {
if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) {
switch (i) {
case 0: {
this.commitTime = value.toString();
break;
}
case 1: {
this.commitSeqNumber = value.toString();
break;
}
case 2: {
this.recordKey = value.toString();
break;
}
case 3: {
this.partitionPath = value.toString();
break;
}
case 4: {
this.fileName = value.toString();
break;
}
default: throw new IllegalArgumentException("Not expected");
}
} else {
row.update(i, value);
}
}
private String getMetaColumnVal(int ordinal) {
switch (ordinal) {
case 0: {
return commitTime;
}
case 1: {
return commitSeqNumber;
}
case 2: {
return recordKey;
}
case 3: {
return partitionPath;
}
case 4: {
return fileName;
}
default: throw new IllegalArgumentException("Not expected");
}
}
@Override
public boolean isNullAt(int ordinal) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return null == getMetaColumnVal(ordinal);
}
return row.isNullAt(ordinal);
}
@Override
public boolean getBoolean(int ordinal) {
return row.getBoolean(ordinal);
}
@Override
public byte getByte(int ordinal) {
return row.getByte(ordinal);
}
@Override
public short getShort(int ordinal) {
return row.getShort(ordinal);
}
@Override
public int getInt(int ordinal) {
return row.getInt(ordinal);
}
@Override
public long getLong(int ordinal) {
return row.getLong(ordinal);
}
@Override
public float getFloat(int ordinal) {
return row.getFloat(ordinal);
}
@Override
public double getDouble(int ordinal) {
return row.getDouble(ordinal);
}
@Override
public Decimal getDecimal(int ordinal, int precision, int scale) {
return row.getDecimal(ordinal, precision, scale);
}
@Override
public UTF8String getUTF8String(int ordinal) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes());
}
return row.getUTF8String(ordinal);
}
@Override
public String getString(int ordinal) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return new String(getMetaColumnVal(ordinal).getBytes());
}
return row.getString(ordinal);
}
@Override
public byte[] getBinary(int ordinal) {
return row.getBinary(ordinal);
}
@Override
public CalendarInterval getInterval(int ordinal) {
return row.getInterval(ordinal);
}
@Override
public InternalRow getStruct(int ordinal, int numFields) {
return row.getStruct(ordinal, numFields);
}
@Override
public ArrayData getArray(int ordinal) {
return row.getArray(ordinal);
}
@Override
public MapData getMap(int ordinal) {
return row.getMap(ordinal);
}
@Override
public Object get(int ordinal, DataType dataType) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes());
}
return row.get(ordinal, dataType);
}
@Override
public InternalRow copy() {
return new HoodieInternalRow(commitTime, commitSeqNumber, recordKey, partitionPath, fileName, row.copy());
}
}

View File

@@ -0,0 +1,203 @@
/*
* 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;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.client.model.HoodieInternalRow;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableConfig;
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;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import java.io.Serializable;
import java.util.concurrent.atomic.AtomicLong;
/**
* Create handle with InternalRow for datasource implemention of bulk insert.
*/
public class HoodieRowCreateHandle implements Serializable {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieRowCreateHandle.class);
private static final AtomicLong SEQGEN = new AtomicLong(1);
private final String instantTime;
private final int taskPartitionId;
private final long taskId;
private final long taskEpochId;
private final HoodieTable table;
private final HoodieWriteConfig writeConfig;
private final HoodieInternalRowFileWriter fileWriter;
private final String partitionPath;
private final Path path;
private final String fileId;
private final FileSystem fs;
private final HoodieInternalWriteStatus writeStatus;
private final HoodieTimer currTimer;
public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId,
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
StructType structType) {
this.partitionPath = partitionPath;
this.table = table;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.taskPartitionId = taskPartitionId;
this.taskId = taskId;
this.taskEpochId = taskEpochId;
this.fileId = fileId;
this.currTimer = new HoodieTimer();
this.currTimer.startTimer();
this.fs = table.getMetaClient().getFs();
this.path = makeNewPath(partitionPath);
this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(),
writeConfig.getWriteStatusFailureFraction());
writeStatus.setPartitionPath(partitionPath);
writeStatus.setFileId(fileId);
try {
HoodiePartitionMetadata partitionMetadata =
new HoodiePartitionMetadata(
fs,
instantTime,
new Path(writeConfig.getBasePath()),
FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath));
partitionMetadata.trySave(taskPartitionId);
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
this.fileWriter = createNewFileWriter(path, table, writeConfig, structType);
} catch (IOException e) {
throw new HoodieInsertException("Failed to initialize file writer for path " + path, e);
}
LOG.info("New handle created for partition :" + partitionPath + " with fileId " + fileId);
}
/**
* Writes an {@link InternalRow} to the underlying HoodieInternalRowFileWriter. Before writing, value for meta columns are computed as required
* and wrapped in {@link HoodieInternalRow}. {@link HoodieInternalRow} is what gets written to HoodieInternalRowFileWriter.
* @param record instance of {@link InternalRow} that needs to be written to the fileWriter.
* @throws IOException
*/
public void write(InternalRow record) throws IOException {
try {
String partitionPath = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(
HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement());
String recordKey = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(
HoodieRecord.RECORD_KEY_METADATA_FIELD)).toString();
HoodieInternalRow internalRow = new HoodieInternalRow(instantTime, seqId, recordKey, partitionPath, path.getName(),
record);
try {
fileWriter.writeRow(recordKey, internalRow);
writeStatus.markSuccess(recordKey);
} catch (Throwable t) {
writeStatus.markFailure(recordKey, t);
}
} catch (Throwable ge) {
writeStatus.setGlobalError(ge);
throw ge;
}
}
/**
* @returns {@code true} if this handle can take in more writes. else {@code false}.
*/
public boolean canWrite() {
return fileWriter.canWrite();
}
/**
* Closes the {@link HoodieRowCreateHandle} and returns an instance of {@link HoodieInternalWriteStatus} containing the stats and
* status of the writes to this handle.
* @return the {@link HoodieInternalWriteStatus} containing the stats and status of the writes to this handle.
* @throws IOException
*/
public HoodieInternalWriteStatus close() throws IOException {
fileWriter.close();
HoodieWriteStat stat = new HoodieWriteStat();
stat.setPartitionPath(partitionPath);
stat.setNumWrites(writeStatus.getTotalRecords());
stat.setNumDeletes(0);
stat.setNumInserts(writeStatus.getTotalRecords());
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
stat.setFileId(fileId);
stat.setPath(new Path(writeConfig.getBasePath()), path);
long fileSizeInBytes = FSUtils.getFileSize(table.getMetaClient().getFs(), path);
stat.setTotalWriteBytes(fileSizeInBytes);
stat.setFileSizeInBytes(fileSizeInBytes);
stat.setTotalWriteErrors(writeStatus.getFailedRowsSize());
HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats();
runtimeStats.setTotalCreateTime(currTimer.endTimer());
stat.setRuntimeStats(runtimeStats);
writeStatus.setStat(stat);
return writeStatus;
}
public String getFileName() {
return path.getName();
}
private Path makeNewPath(String partitionPath) {
Path path = FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath);
try {
fs.mkdirs(path); // create a new partition as needed.
} catch (IOException e) {
throw new HoodieIOException("Failed to make dir " + path, e);
}
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId,
tableConfig.getBaseFileFormat().getFileExtension()));
}
/**
* Creates an empty marker file corresponding to storage writer path.
*
* @param partitionPath Partition path
*/
private void createMarkerFile(String partitionPath, String dataFileName) {
MarkerFiles markerFiles = new MarkerFiles(table, instantTime);
markerFiles.create(partitionPath, dataFileName, IOType.CREATE);
}
private String getWriteToken() {
return taskPartitionId + "-" + taskId + "-" + taskEpochId;
}
private HoodieInternalRowFileWriter createNewFileWriter(
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
throws IOException {
return HoodieInternalRowFileWriterFactory.getInternalRowFileWriter(
path, hoodieTable, config, schema);
}
}

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.io.storage;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
/**
* ParquetConfig for writing avro records in Parquet files.
*/
public class HoodieAvroParquetConfig extends HoodieBaseParquetConfig<HoodieAvroWriteSupport> {
public HoodieAvroParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
double compressionRatio) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
}
}

View File

@@ -18,14 +18,15 @@
package org.apache.hudi.io.storage;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
public class HoodieParquetConfig {
private HoodieAvroWriteSupport writeSupport;
/**
* Base ParquetConfig to hold config params for writing to Parquet.
* @param <T>
*/
public class HoodieBaseParquetConfig<T> {
private final T writeSupport;
private CompressionCodecName compressionCodecName;
private int blockSize;
private int pageSize;
@@ -33,8 +34,8 @@ public class HoodieParquetConfig {
private Configuration hadoopConf;
private double compressionRatio;
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio) {
public HoodieBaseParquetConfig(T writeSupport, CompressionCodecName compressionCodecName, int blockSize,
int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio) {
this.writeSupport = writeSupport;
this.compressionCodecName = compressionCodecName;
this.blockSize = blockSize;
@@ -44,10 +45,6 @@ public class HoodieParquetConfig {
this.compressionRatio = compressionRatio;
}
public HoodieAvroWriteSupport getWriteSupport() {
return writeSupport;
}
public CompressionCodecName getCompressionCodecName() {
return compressionCodecName;
}
@@ -71,4 +68,8 @@ public class HoodieParquetConfig {
public double getCompressionRatio() {
return compressionRatio;
}
public T getWriteSupport() {
return writeSupport;
}
}

View File

@@ -58,7 +58,7 @@ public class HoodieFileWriterFactory {
HoodieAvroWriteSupport writeSupport =
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, config.getParquetCompressionCodec(),
HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(),
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio());

View File

@@ -0,0 +1,46 @@
/*
* 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;
import org.apache.spark.sql.catalyst.InternalRow;
import java.io.IOException;
/**
* Abstraction to assist in writing {@link InternalRow}s to be used in datasource implementation.
*/
public interface HoodieInternalRowFileWriter {
/**
* @returns {@code true} if this RowFileWriter can take in more writes. else {@code false}.
*/
boolean canWrite();
/**
* Writes an {@link InternalRow} to the HoodieInternalRowFileWriter.
*
* @throws IOException on any exception while writing.
*/
void writeRow(String key, InternalRow row) throws IOException;
/**
* Closes the {@link HoodieInternalRowFileWriter} and may not take in any more writes.
*/
void close() throws IOException;
}

View File

@@ -0,0 +1,79 @@
/*
* 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;
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.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;
/**
* Factory to assist in instantiating a new {@link HoodieInternalRowFileWriter}.
*/
public class HoodieInternalRowFileWriterFactory {
/**
* Factory method to assist in instantiating an instance of {@link HoodieInternalRowFileWriter}.
* @param path path of the RowFileWriter.
* @param hoodieTable instance of {@link HoodieTable} in use.
* @param config instance of {@link HoodieWriteConfig} to use.
* @param schema schema of the dataset in use.
* @return the instantiated {@link HoodieInternalRowFileWriter}.
* @throws IOException if format is not supported or if any exception during instantiating the RowFileWriter.
*
*/
public static HoodieInternalRowFileWriter getInternalRowFileWriter(
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
throws IOException {
final String extension = FSUtils.getFileExtension(path.getName());
if (PARQUET.getFileExtension().equals(extension)) {
return newParquetInternalRowFileWriter(path, config, schema, hoodieTable);
}
throw new UnsupportedOperationException(extension + " format not supported yet.");
}
private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(
Path path, HoodieWriteConfig writeConfig, StructType structType, HoodieTable table)
throws IOException {
BloomFilter filter = BloomFilterFactory.createBloomFilter(
writeConfig.getBloomFilterNumEntries(),
writeConfig.getBloomFilterFPP(),
writeConfig.getDynamicBloomFilterMaxNumEntries(),
writeConfig.getBloomFilterType());
HoodieRowParquetWriteSupport writeSupport =
new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, filter);
return new HoodieInternalRowParquetWriter(
path, new HoodieRowParquetConfig(
writeSupport,
writeConfig.getParquetCompressionCodec(),
writeConfig.getParquetBlockSize(),
writeConfig.getParquetPageSize(),
writeConfig.getParquetMaxFileSize(),
writeSupport.getHadoopConf(),
writeConfig.getParquetCompressionRatio()));
}
}

View File

@@ -0,0 +1,72 @@
/*
* 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;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.spark.sql.catalyst.InternalRow;
import java.io.IOException;
/**
* Parquet's impl of {@link HoodieInternalRowFileWriter} to write {@link InternalRow}s.
*/
public class HoodieInternalRowParquetWriter extends ParquetWriter<InternalRow>
implements HoodieInternalRowFileWriter {
private final Path file;
private final HoodieWrapperFileSystem fs;
private final long maxFileSize;
private final HoodieRowParquetWriteSupport writeSupport;
public HoodieInternalRowParquetWriter(Path file, HoodieRowParquetConfig parquetConfig)
throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
ParquetWriter.DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file,
parquetConfig.getHadoopConf()));
this.maxFileSize = parquetConfig.getMaxFileSize()
+ Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
this.writeSupport = parquetConfig.getWriteSupport();
}
@Override
public boolean canWrite() {
return fs.getBytesWritten(file) < maxFileSize;
}
@Override
public void writeRow(String key, InternalRow row) throws IOException {
super.write(row);
writeSupport.add(key);
}
@Override
public void close() throws IOException {
super.close();
}
}

View File

@@ -52,7 +52,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final String instantTime;
private final SparkTaskContextSupplier sparkTaskContextSupplier;
public HoodieParquetWriter(String instantTime, Path file, HoodieParquetConfig parquetConfig,
public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig,
Schema schema, SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),

View File

@@ -0,0 +1,34 @@
/*
* 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;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
/**
* ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}.
*/
public class HoodieRowParquetConfig extends HoodieBaseParquetConfig<HoodieRowParquetWriteSupport> {
public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
double compressionRatio) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
}
}

View File

@@ -0,0 +1,89 @@
/*
* 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;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
import org.apache.spark.sql.types.StructType;
import java.util.HashMap;
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE;
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
/**
* Hoodie Write Support for directly writing Row to Parquet.
*/
public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
private Configuration hadoopConf;
private BloomFilter bloomFilter;
private String minRecordKey;
private String maxRecordKey;
public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, BloomFilter bloomFilter) {
super();
Configuration hadoopConf = new Configuration(conf);
hadoopConf.set("spark.sql.parquet.writeLegacyFormat", "false");
hadoopConf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MILLIS");
this.hadoopConf = hadoopConf;
setSchema(structType, hadoopConf);
this.bloomFilter = bloomFilter;
}
public Configuration getHadoopConf() {
return hadoopConf;
}
@Override
public WriteSupport.FinalizedWriteContext finalizeWrite() {
HashMap<String, String> extraMetaData = new HashMap<>();
if (bloomFilter != null) {
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
if (minRecordKey != null && maxRecordKey != null) {
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
}
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name());
}
}
return new WriteSupport.FinalizedWriteContext(extraMetaData);
}
public void add(String recordKey) {
this.bloomFilter.add(recordKey);
if (minRecordKey != null) {
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
} else {
minRecordKey = recordKey;
}
if (maxRecordKey != null) {
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
} else {
maxRecordKey = recordKey;
}
}
}

View File

@@ -1,88 +0,0 @@
/*
* 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.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 java.util.List;
import java.util.stream.Collectors;
/**
* Base class for all the built-in key generators. Contains methods structured for
* code reuse amongst them.
*/
public abstract class BuiltinKeyGenerator extends KeyGenerator {
protected BuiltinKeyGenerator(TypedProperties config) {
super(config);
}
/**
* Generate a record Key out of provided generic record.
*/
public abstract String getRecordKey(GenericRecord record);
/**
* Generate a partition path out of provided generic record.
*/
public abstract String getPartitionPath(GenericRecord record);
/**
* Generate a Hoodie Key out of provided generic record.
*/
public final HoodieKey getKey(GenericRecord record) {
if (getRecordKeyFields() == null || getPartitionPathFields() == null) {
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
}
return new HoodieKey(getRecordKey(record), getPartitionPath(record));
}
/**
* Return fields that constitute record key. Used by Metadata bootstrap.
* Have a base implementation inorder to prevent forcing custom KeyGenerator implementation
* to implement this method
* @return list of record key fields
*/
public List<String> getRecordKeyFields() {
throw new IllegalStateException("This method is expected to be overridden by subclasses");
}
/**
* Return fields that constiture partition path. Used by Metadata bootstrap.
* Have a base implementation inorder to prevent forcing custom KeyGenerator implementation
* to implement this method
* @return list of partition path fields
*/
public List<String> getPartitionPathFields() {
throw new IllegalStateException("This method is expected to be overridden by subclasses");
}
@Override
public final List<String> getRecordKeyFieldNames() {
// For nested columns, pick top level column name
return getRecordKeyFields().stream().map(k -> {
int idx = k.indexOf('.');
return idx > 0 ? k.substring(0, idx) : k;
}).collect(Collectors.toList());
}
}

View File

@@ -18,23 +18,24 @@
package org.apache.hudi.keygen;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.spark.sql.Row;
public class TestKeyGeneratorUtilities {
import java.io.Serializable;
import java.util.List;
public String exampleSchema = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ "{\"name\": \"pii_col\", \"type\": \"string\"}]}";
/**
* Represents the interface key generators need to adhere to.
*/
public interface KeyGeneratorInterface extends Serializable {
HoodieKey getKey(GenericRecord record);
List<String> getRecordKeyFieldNames();
String getRecordKey(Row row);
String getPartitionPath(Row row);
public GenericRecord getRecord() {
GenericRecord record = new GenericData.Record(new Schema.Parser().parse(exampleSchema));
record.put("timestamp", 4357686);
record.put("_row_key", "key1");
record.put("ts_ms", "2020-03-21");
record.put("pii_col", "pi");
return record;
}
}

View File

@@ -53,7 +53,7 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
import org.apache.hudi.io.HoodieBootstrapHandle;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.KeyGeneratorInterface;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.action.HoodieWriteMetadata;
@@ -225,7 +225,7 @@ public class BootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>>
}
private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, String partitionPath,
HoodieFileStatus srcFileStatus, KeyGenerator keyGenerator) {
HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) {
Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath());
HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS,
@@ -311,7 +311,7 @@ public class BootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>>
TypedProperties properties = new TypedProperties();
properties.putAll(config.getProps());
KeyGenerator keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(config.getBootstrapKeyGeneratorClass(),
KeyGeneratorInterface keyGenerator = (KeyGeneratorInterface) ReflectionUtils.loadClass(config.getBootstrapKeyGeneratorClass(),
properties);
BootstrapPartitionPathTranslator translator = (BootstrapPartitionPathTranslator) ReflectionUtils.loadClass(
config.getBootstrapPartitionPathTranslatorClass(), properties);

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.client;
import org.junit.jupiter.api.Test;
import java.util.UUID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Unit tests {@link HoodieInternalWriteStatus}.
*/
public class TestHoodieInternalWriteStatus {
@Test
public void testFailureFraction() {
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(true, 0.1);
String fileId = UUID.randomUUID().toString();
String partitionPath = UUID.randomUUID().toString();
status.setFileId(fileId);
status.setPartitionPath(partitionPath);
Throwable t = new Exception("some error in writing");
for (int i = 0; i < 1000; i++) {
status.markFailure(UUID.randomUUID().toString(), t);
}
// verification
assertEquals(fileId, status.getFileId());
assertEquals(partitionPath, status.getPartitionPath());
assertTrue(status.getFailedRecordKeys().size() > 0);
assertTrue(status.getFailedRecordKeys().size() < 150); // 150 instead of 100, to prevent flaky test
assertTrue(status.hasErrors());
}
@Test
public void testSuccessRecordTracking() {
boolean[] vals = {true, false};
for (boolean trackSuccess : vals) {
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(trackSuccess, 1.0);
String fileId = UUID.randomUUID().toString();
status.setFileId(fileId);
String partitionPath = UUID.randomUUID().toString();
status.setPartitionPath(partitionPath);
Throwable t = new Exception("some error in writing");
for (int i = 0; i < 1000; i++) {
status.markSuccess(UUID.randomUUID().toString());
status.markFailure(UUID.randomUUID().toString(), t);
}
// verification
assertEquals(fileId, status.getFileId());
assertEquals(partitionPath, status.getPartitionPath());
assertEquals(1000, status.getFailedRecordKeys().size());
assertTrue(status.hasErrors());
if (trackSuccess) {
assertEquals(1000, status.getSuccessRecordKeys().size());
} else {
assertTrue(status.getSuccessRecordKeys().isEmpty());
}
assertEquals(2000, status.getTotalRecords());
}
}
@Test
public void testGlobalError() {
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(true, 0.1);
Throwable t = new Exception("some error in writing");
status.setGlobalError(t);
assertEquals(t, status.getGlobalError());
}
}

View File

@@ -0,0 +1,239 @@
/*
* 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.client.model;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.types.DataTypes;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Unit tests {@link HoodieInternalRow}.
*/
public class TestHoodieInternalRow {
private static final Random RANDOM = new Random();
private static final int INTEGER_INDEX = 5;
private static final int STRING_INDEX = 6;
private static final int BOOLEAN_INDEX = 7;
private static final int SHORT_INDEX = 8;
private static final int BYTE_INDEX = 9;
private static final int LONG_INDEX = 10;
private static final int FLOAT_INDEX = 11;
private static final int DOUBLE_INDEX = 12;
private static final int DECIMAL_INDEX = 13;
private static final int BINARY_INDEX = 14;
private static final int STRUCT_INDEX = 15;
// to do array and map
private static final int ARRAY_INDEX = 16;
private static final int MAP_INDEX = 17;
private List<Integer> nullIndices;
public TestHoodieInternalRow() {
this.nullIndices = new ArrayList<>();
}
@Test
public void testGet() {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
"fileName", values, nullIndices);
}
@Test
public void testUpdate() {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
hoodieInternalRow.update(0, "commitTime_updated");
hoodieInternalRow.update(1, "commitSeqNo_updated");
hoodieInternalRow.update(2, "recordKey_updated");
hoodieInternalRow.update(3, "partitionPath_updated");
hoodieInternalRow.update(4, "fileName_updated");
values = getRandomValue(true);
hoodieInternalRow.update(INTEGER_INDEX, values[INTEGER_INDEX]);
hoodieInternalRow.update(BOOLEAN_INDEX, values[BOOLEAN_INDEX]);
hoodieInternalRow.update(SHORT_INDEX, values[SHORT_INDEX]);
hoodieInternalRow.update(BYTE_INDEX, values[BYTE_INDEX]);
hoodieInternalRow.update(LONG_INDEX, values[LONG_INDEX]);
hoodieInternalRow.update(FLOAT_INDEX, values[FLOAT_INDEX]);
hoodieInternalRow.update(DOUBLE_INDEX, values[DOUBLE_INDEX]);
//hoodieInternalRow.update(decimalIndex, values[decimalIndex]);
hoodieInternalRow.update(BINARY_INDEX, values[BINARY_INDEX]);
hoodieInternalRow.update(STRUCT_INDEX, values[STRUCT_INDEX]);
hoodieInternalRow.update(STRING_INDEX, values[STRING_INDEX].toString());
assertValues(hoodieInternalRow, "commitTime_updated", "commitSeqNo_updated", "recordKey_updated", "partitionPath_updated",
"fileName_updated", values, nullIndices);
}
@Test
public void testIsNullCheck() {
for (int i = 0; i < 16; i++) {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
hoodieInternalRow.setNullAt(i);
nullIndices.clear();
nullIndices.add(i);
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
"fileName", values, nullIndices);
}
// try setting multiple values as null
// run it for 5 rounds
for (int i = 0; i < 5; i++) {
int numNullValues = 1 + RANDOM.nextInt(4);
List<Integer> nullsSoFar = new ArrayList<>();
while (nullsSoFar.size() < numNullValues) {
int randomIndex = RANDOM.nextInt(16);
if (!nullsSoFar.contains(randomIndex)) {
nullsSoFar.add(randomIndex);
}
}
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
nullIndices.clear();
for (Integer index : nullsSoFar) {
hoodieInternalRow.setNullAt(index);
nullIndices.add(index);
}
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
"fileName", values, nullIndices);
}
}
/**
* Fetches a random Object[] of values for testing.
*
* @param withStructType true if structType need to be added as one of the elements in the Object[]
* @return the random Object[] thus generated
*/
private Object[] getRandomValue(boolean withStructType) {
Object[] values = new Object[16];
values[INTEGER_INDEX] = RANDOM.nextInt();
values[STRING_INDEX] = UUID.randomUUID().toString();
values[BOOLEAN_INDEX] = RANDOM.nextBoolean();
values[SHORT_INDEX] = (short) RANDOM.nextInt(2);
byte[] bytes = new byte[1];
RANDOM.nextBytes(bytes);
values[BYTE_INDEX] = bytes[0];
values[LONG_INDEX] = RANDOM.nextLong();
values[FLOAT_INDEX] = RANDOM.nextFloat();
values[DOUBLE_INDEX] = RANDOM.nextDouble();
// TODO fix decimal type.
values[DECIMAL_INDEX] = RANDOM.nextFloat();
bytes = new byte[20];
RANDOM.nextBytes(bytes);
values[BINARY_INDEX] = bytes;
if (withStructType) {
Object[] structField = getRandomValue(false);
values[STRUCT_INDEX] = new GenericInternalRow(structField);
}
return values;
}
private void assertValues(HoodieInternalRow hoodieInternalRow, String commitTime, String commitSeqNo, String recordKey, String partitionPath, String filename, Object[] values,
List<Integer> nullIndexes) {
for (Integer index : nullIndexes) {
assertTrue(hoodieInternalRow.isNullAt(index));
}
for (int i = 0; i < 16; i++) {
if (!nullIndexes.contains(i)) {
assertFalse(hoodieInternalRow.isNullAt(i));
}
}
if (!nullIndexes.contains(0)) {
assertEquals(commitTime, hoodieInternalRow.get(0, DataTypes.StringType).toString());
}
if (!nullIndexes.contains(1)) {
assertEquals(commitSeqNo, hoodieInternalRow.get(1, DataTypes.StringType).toString());
}
if (!nullIndexes.contains(2)) {
assertEquals(recordKey, hoodieInternalRow.get(2, DataTypes.StringType).toString());
}
if (!nullIndexes.contains(3)) {
assertEquals(partitionPath, hoodieInternalRow.get(3, DataTypes.StringType).toString());
}
if (!nullIndexes.contains(4)) {
assertEquals(filename, hoodieInternalRow.get(4, DataTypes.StringType).toString());
}
if (!nullIndexes.contains(INTEGER_INDEX)) {
assertEquals(values[INTEGER_INDEX], hoodieInternalRow.getInt(INTEGER_INDEX));
assertEquals(values[INTEGER_INDEX], hoodieInternalRow.get(INTEGER_INDEX, DataTypes.IntegerType));
}
if (!nullIndexes.contains(STRING_INDEX)) {
assertEquals(values[STRING_INDEX].toString(), hoodieInternalRow.get(STRING_INDEX, DataTypes.StringType));
}
if (!nullIndexes.contains(BOOLEAN_INDEX)) {
assertEquals(values[BOOLEAN_INDEX], hoodieInternalRow.getBoolean(BOOLEAN_INDEX));
assertEquals(values[BOOLEAN_INDEX], hoodieInternalRow.get(BOOLEAN_INDEX, DataTypes.BooleanType));
}
if (!nullIndexes.contains(SHORT_INDEX)) {
assertEquals(values[SHORT_INDEX], hoodieInternalRow.getShort(SHORT_INDEX));
assertEquals(values[SHORT_INDEX], hoodieInternalRow.get(SHORT_INDEX, DataTypes.ShortType));
}
if (!nullIndexes.contains(BYTE_INDEX)) {
assertEquals(values[BYTE_INDEX], hoodieInternalRow.getByte(BYTE_INDEX));
assertEquals(values[BYTE_INDEX], hoodieInternalRow.get(BYTE_INDEX, DataTypes.ByteType));
}
if (!nullIndexes.contains(LONG_INDEX)) {
assertEquals(values[LONG_INDEX], hoodieInternalRow.getLong(LONG_INDEX));
assertEquals(values[LONG_INDEX], hoodieInternalRow.get(LONG_INDEX, DataTypes.LongType));
}
if (!nullIndexes.contains(FLOAT_INDEX)) {
assertEquals(values[FLOAT_INDEX], hoodieInternalRow.getFloat(FLOAT_INDEX));
assertEquals(values[FLOAT_INDEX], hoodieInternalRow.get(FLOAT_INDEX, DataTypes.FloatType));
}
if (!nullIndexes.contains(DOUBLE_INDEX)) {
assertEquals(values[DOUBLE_INDEX], hoodieInternalRow.getDouble(DOUBLE_INDEX));
assertEquals(values[DOUBLE_INDEX], hoodieInternalRow.get(DOUBLE_INDEX, DataTypes.DoubleType));
}
if (!nullIndexes.contains(BINARY_INDEX)) {
assertEquals(values[BINARY_INDEX], hoodieInternalRow.getBinary(BINARY_INDEX));
assertEquals(values[BINARY_INDEX], hoodieInternalRow.get(BINARY_INDEX, DataTypes.BinaryType));
}
if (!nullIndexes.contains(STRUCT_INDEX)) {
assertEquals(values[STRUCT_INDEX], hoodieInternalRow.getStruct(STRUCT_INDEX, 18));
}
}
}

View File

@@ -0,0 +1,231 @@
/*
* 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;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
/**
* Unit tests {@link HoodieRowCreateHandle}.
*/
public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestHoodieRowCreateHandle");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testRowCreateHandle() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
List<String> fileNames = new ArrayList<>();
List<String> fileAbsPaths = new ArrayList<>();
Dataset<Row> totalInputRows = null;
// one round per partition
for (int i = 0; i < 5; i++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[i % 3];
// init some args
String fileId = UUID.randomUUID().toString();
String instantTime = "000";
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(1000);
// Generate inputs
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
// issue writes
HoodieInternalWriteStatus writeStatus = writeAndGetWriteStatus(inputRows, handle);
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
fileNames.add(handle.getFileName());
// verify output
assertOutput(writeStatus, size, fileId, partitionPath, instantTime, totalInputRows, fileNames, fileAbsPaths);
}
}
/**
* Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch 2 of invalid records Global Error
* should be thrown.
*/
@Test
public void testGlobalFailure() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
// init some args
String fileId = UUID.randomUUID().toString();
String instantTime = "000";
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(1000);
int totalFailures = 5;
// Generate first batch of valid rows
Dataset<Row> inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false);
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// generate some failures rows
for (int i = 0; i < totalFailures; i++) {
internalRows.add(getInternalRowWithError(partitionPath));
}
// generate 2nd batch of valid rows
Dataset<Row> inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false);
internalRows.addAll(toInternalRows(inputRows2, ENCODER));
// issue writes
try {
for (InternalRow internalRow : internalRows) {
handle.write(internalRow);
}
fail("Should have failed");
} catch (Throwable e) {
// expected
}
// close the create handle
HoodieInternalWriteStatus writeStatus = handle.close();
List<String> fileNames = new ArrayList<>();
fileNames.add(handle.getFileName());
// verify write status
assertNotNull(writeStatus.getGlobalError());
assertTrue(writeStatus.getGlobalError().getMessage().contains("java.lang.String cannot be cast to org.apache.spark.unsafe.types.UTF8String"));
assertEquals(writeStatus.getFileId(), fileId);
assertEquals(writeStatus.getPartitionPath(), partitionPath);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(basePath + "/" + partitionPath);
// passing only first batch of inputRows since after first batch global error would have been thrown
assertRows(inputRows, result, instantTime, fileNames);
}
@Test
public void testInstantiationFailure() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).withPath("/dummypath/abc/").build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
try {
new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
fail("Should have thrown exception");
} catch (HoodieInsertException ioe) {
// expected
}
}
private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset<Row> inputRows, HoodieRowCreateHandle handle) throws IOException {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
handle.write(internalRow);
}
// close the create handle
return handle.close();
}
private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, String fileId, String partitionPath, String instantTime, Dataset<Row> inputRows, List<String> filenames,
List<String> fileAbsPaths) {
assertEquals(writeStatus.getPartitionPath(), partitionPath);
assertEquals(writeStatus.getTotalRecords(), size);
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertEquals(writeStatus.getTotalErrorRecords(), 0);
assertFalse(writeStatus.hasErrors());
assertNull(writeStatus.getGlobalError());
assertEquals(writeStatus.getFileId(), fileId);
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(partitionPath, writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertRows(inputRows, result, instantTime, filenames);
}
private void assertRows(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> filenames) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertTrue(filenames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)).toString()));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}

View File

@@ -0,0 +1,117 @@
/*
* 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;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* Unit tests {@link HoodieInternalRowParquetWriter}.
*/
public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestHoodieInternalRowParquetWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void endToEndTest() throws IOException {
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
for (int i = 0; i < 5; i++) {
// init write support and parquet config
HoodieRowParquetWriteSupport writeSupport = getWriteSupport(cfg, hadoopConf);
HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio());
// prepare path
String fileId = UUID.randomUUID().toString();
Path filePath = new Path(basePath + "/" + fileId);
String partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
metaClient.getFs().mkdirs(new Path(basePath));
// init writer
HoodieInternalRowParquetWriter writer = new HoodieInternalRowParquetWriter(filePath, parquetConfig);
// generate input
int size = 10 + RANDOM.nextInt(100);
// Generate inputs
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
// close the writer
writer.close();
// verify rows
Dataset<Row> result = sqlContext.read().parquet(basePath);
assertEquals(0, inputRows.except(result).count());
}
}
private HoodieRowParquetWriteSupport getWriteSupport(HoodieWriteConfig writeConfig, Configuration hadoopConf) {
BloomFilter filter = BloomFilterFactory.createBloomFilter(
writeConfig.getBloomFilterNumEntries(),
writeConfig.getBloomFilterFPP(),
writeConfig.getDynamicBloomFilterMaxNumEntries(),
writeConfig.getBloomFilterType());
return new HoodieRowParquetWriteSupport(hadoopConf, STRUCT_TYPE, filter);
}
}

View File

@@ -39,7 +39,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.IOType;
import org.apache.hudi.io.storage.HoodieParquetConfig;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetWriter;
import org.apache.avro.Schema;
@@ -255,7 +255,7 @@ public class HoodieClientTestUtils {
HoodieAvroWriteSupport writeSupport =
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
String instantTime = FSUtils.getCommitTime(filename);
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
HoodieTestUtils.getDefaultHadoopConf(), Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
HoodieParquetWriter writer =

View File

@@ -0,0 +1,175 @@
/*
* 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.testutils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
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.GenericInternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericRow;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.Metadata;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.stream.Collectors;
import scala.collection.JavaConversions;
import scala.collection.JavaConverters;
import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
/**
* Dataset test utils.
*/
public class SparkDatasetTestUtils {
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("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
new StructField("randomLong", DataTypes.LongType, false, Metadata.empty())});
public static final StructType ERROR_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.LongType, 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("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
new StructField("randomStr", DataTypes.StringType, false, Metadata.empty())});
public static final ExpressionEncoder ENCODER = getEncoder(STRUCT_TYPE);
public static final ExpressionEncoder ERROR_ENCODER = getEncoder(ERROR_STRUCT_TYPE);
/**
* Generate Encode for the passed in {@link StructType}.
*
* @param schema instance of {@link StructType} for which encoder is requested.
* @return the encoder thus generated.
*/
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$);
}
/**
* Generate random Rows.
*
* @param count total number of Rows to be generated.
* @param partitionPath partition path to be set
* @return the Dataset<Row>s thus generated.
*/
public static Dataset<Row> getRandomRows(SQLContext sqlContext, int count, String partitionPath, boolean isError) {
List<Row> records = new ArrayList<>();
for (long recordNum = 0; recordNum < count; recordNum++) {
records.add(getRandomValue(partitionPath, isError));
}
return sqlContext.createDataFrame(records, isError ? ERROR_STRUCT_TYPE : STRUCT_TYPE);
}
/**
* Generate random Row.
*
* @param partitionPath partition path to be set in the Row.
* @return the Row thus generated.
*/
public static Row getRandomValue(String partitionPath, boolean isError) {
// order commit time, seq no, record key, partition path, file name
Object[] values = new Object[7];
values[0] = ""; //commit time
if (!isError) {
values[1] = ""; // commit seq no
} else {
values[1] = RANDOM.nextLong();
}
values[2] = UUID.randomUUID().toString();
values[3] = partitionPath;
values[4] = ""; // filename
values[5] = RANDOM.nextInt();
if (!isError) {
values[6] = RANDOM.nextLong();
} else {
values[6] = UUID.randomUUID().toString();
}
return new GenericRow(values);
}
/**
* Convert Dataset<Row>s to List of {@link InternalRow}s.
*
* @param rows Dataset<Row>s to be converted
* @return the List of {@link InternalRow}s thus converted.
*/
public static List<InternalRow> toInternalRows(Dataset<Row> rows, ExpressionEncoder encoder) {
List<InternalRow> toReturn = new ArrayList<>();
List<Row> rowList = rows.collectAsList();
for (Row row : rowList) {
toReturn.add(encoder.toRow(row).copy());
}
return toReturn;
}
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];
values[0] = "";
values[1] = "";
values[2] = recordKey;
values[3] = partitionPath;
values[4] = "";
values[5] = RANDOM.nextInt();
values[6] = RANDOM.nextBoolean();
return new GenericInternalRow(values);
}
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withBulkInsertParallelism(2);
}
}

View File

@@ -18,12 +18,16 @@
package org.apache.hudi.common.model;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import java.io.Serializable;
import java.util.List;
import java.util.Objects;
import org.apache.hudi.common.util.collection.Pair;
/**
* A Single Record managed by Hoodie.
@@ -40,6 +44,10 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD,
RECORD_KEY_METADATA_FIELD, PARTITION_PATH_METADATA_FIELD, FILENAME_METADATA_FIELD);
public static final Map<String, Integer> HOODIE_META_COLUMNS_NAME_TO_POS =
IntStream.range(0, HOODIE_META_COLUMNS.size()).mapToObj(idx -> Pair.of(HOODIE_META_COLUMNS.get(idx), idx))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
/**
* Identifies the record across the table.
*/

View File

@@ -21,6 +21,6 @@
"name": "User",
"fields": [
{"name": "field1", "type": ["null", "string"], "default": null},
{"name": "createTime", "type": ["null", "string"], "default": null}
{"name": "createTime", "type": ["null", "long"], "default": null}
]
}

View File

@@ -18,8 +18,6 @@
package org.apache.hudi;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.WriteStatus;
@@ -48,6 +46,8 @@ import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -106,7 +106,7 @@ public class DataSourceUtils {
public static String getTablePath(FileSystem fs, Path[] userProvidedPaths) throws IOException {
LOG.info("Getting table path..");
for (Path path: userProvidedPaths) {
for (Path path : userProvidedPaths) {
try {
Option<Path> tablePath = TablePathUtils.getTablePath(fs, path);
if (tablePath.isPresent()) {
@@ -123,8 +123,7 @@ public class DataSourceUtils {
/**
* This method converts values for fields with certain Avro/Parquet data types that require special handling.
*
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is
* represented/stored in parquet.
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is represented/stored in parquet.
*
* @param fieldSchema avro field schema
* @param fieldValue avro field value
@@ -157,9 +156,8 @@ public class DataSourceUtils {
/**
* Create a key generator class via reflection, passing in any configs needed.
* <p>
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the
* corresponding key generator class; otherwise, use the default key generator class specified in {@code
* DataSourceWriteOptions}.
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the corresponding key generator class; otherwise, use the default key generator class
* specified in {@code DataSourceWriteOptions}.
*/
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
String keyGeneratorClass = props.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
@@ -173,10 +171,6 @@ public class DataSourceUtils {
/**
* Create a date time parser class for TimestampBasedKeyGenerator, passing in any configs needed.
* @param props
* @param parserClass
* @return
* @throws IOException
*/
public static HoodieDateTimeParser createDateTimeParser(TypedProperties props, String parserClass) throws IOException {
try {
@@ -190,6 +184,7 @@ public class DataSourceUtils {
* Create a UserDefinedBulkInsertPartitioner class via reflection,
* <br>
* if the class name of UserDefinedBulkInsertPartitioner is configured through the HoodieWriteConfig.
*
* @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass()
*/
private static Option<BulkInsertPartitioner> createUserDefinedBulkInsertPartitioner(HoodieWriteConfig config)
@@ -225,35 +220,35 @@ public class DataSourceUtils {
});
}
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
public static HoodieWriteConfig createHoodieConfig(String schemaStr, String basePath,
String tblName, Map<String, String> parameters) {
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY()));
// inline compaction is on by default for MOR
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY()));
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY())
.equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
return createHoodieClient(jssc, schemaStr, basePath, tblName, parameters, inlineCompact);
}
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
String tblName, Map<String, String> parameters, boolean inlineCompact) {
// insert/bulk-insert combining to be true, if filtering for duplicates
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY()));
HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder()
.withPath(basePath).withAutoCommit(false).combineInput(combineInserts, true);
if (schemaStr != null) {
builder = builder.withSchema(schemaStr);
}
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath).withAutoCommit(false)
.combineInput(combineInserts, true).withSchema(schemaStr).forTable(tblName)
return builder.forTable(tblName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY()))
.withInlineCompaction(inlineCompact).build())
// override above with Hoodie configs specified as options.
.withProps(parameters).build();
}
return new HoodieWriteClient<>(jssc, writeConfig, true);
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
String tblName, Map<String, String> parameters) {
return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
}
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
String instantTime, String operation) throws HoodieException {
String instantTime, String operation) throws HoodieException {
if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) {
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner =
createUserDefinedBulkInsertPartitioner(client.getConfig());
@@ -267,12 +262,12 @@ public class DataSourceUtils {
}
public static JavaRDD<WriteStatus> doDeleteOperation(HoodieWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
String instantTime) {
String instantTime) {
return client.delete(hoodieKeys, instantTime);
}
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey,
String payloadClass) throws IOException {
String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
return new HoodieRecord<>(hKey, payload);
}
@@ -280,13 +275,13 @@ public class DataSourceUtils {
/**
* Drop records already present in the dataset.
*
* @param jssc JavaSparkContext
* @param jssc JavaSparkContext
* @param incomingHoodieRecords HoodieRecords to deduplicate
* @param writeConfig HoodieWriteConfig
* @param writeConfig HoodieWriteConfig
*/
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
HoodieWriteConfig writeConfig) {
HoodieWriteConfig writeConfig) {
try {
HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig);
return client.tagLocation(incomingHoodieRecords)
@@ -300,7 +295,7 @@ public class DataSourceUtils {
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
Map<String, String> parameters) {
Map<String, String> parameters) {
HoodieWriteConfig writeConfig =
HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build();
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);

View File

@@ -0,0 +1,108 @@
/*
* 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 static org.apache.spark.sql.functions.callUDF;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.Column;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.api.java.UDF1;
import org.apache.spark.sql.functions;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructType;
import scala.collection.JavaConverters;
/**
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
*/
public class HoodieDatasetBulkInsertHelper {
private static final Logger LOG = LogManager.getLogger(HoodieDatasetBulkInsertHelper.class);
private static final String RECORD_KEY_UDF_FN = "hudi_recordkey_gen_function";
private static final String PARTITION_PATH_UDF_FN = "hudi_partition_gen_function";
/**
* Prepares input hoodie spark dataset for bulk insert. It does the following steps.
* 1. Uses KeyGenerator to generate hoodie record keys and partition path.
* 2. Add hoodie columns to input spark dataset.
* 3. Reorders input dataset columns so that hoodie columns appear in the beginning.
* 4. Sorts input dataset by hoodie partition path and record key
*
* @param sqlContext SQL Context
* @param config Hoodie Write Config
* @param rows Spark Input dataset
* @return hoodie dataset which is ready for bulk insert.
*/
public static Dataset<Row> prepareHoodieDatasetForBulkInsert(SQLContext sqlContext,
HoodieWriteConfig config, Dataset<Row> rows, String structName, String recordNamespace) {
List<Column> originalFields =
Arrays.stream(rows.schema().fields()).map(f -> new Column(f.name())).collect(Collectors.toList());
TypedProperties properties = new TypedProperties();
properties.putAll(config.getProps());
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY());
KeyGenerator keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
StructType structTypeForUDF = rows.schema();
sqlContext.udf().register(RECORD_KEY_UDF_FN, (UDF1<Row, String>) keyGenerator::getRecordKey, DataTypes.StringType);
sqlContext.udf().register(PARTITION_PATH_UDF_FN, (UDF1<Row, String>) keyGenerator::getPartitionPath, DataTypes.StringType);
final Dataset<Row> rowDatasetWithRecordKeys = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD,
callUDF(RECORD_KEY_UDF_FN, org.apache.spark.sql.functions.struct(
JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq())));
final Dataset<Row> rowDatasetWithRecordKeysAndPartitionPath =
rowDatasetWithRecordKeys.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
callUDF(PARTITION_PATH_UDF_FN,
org.apache.spark.sql.functions.struct(
JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq())));
// Add other empty hoodie fields which will be populated before writing to parquet.
Dataset<Row> rowDatasetWithHoodieColumns =
rowDatasetWithRecordKeysAndPartitionPath.withColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType))
.withColumn(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType))
.withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType));
List<Column> orderedFields = Stream.concat(HoodieRecord.HOODIE_META_COLUMNS.stream().map(Column::new),
originalFields.stream()).collect(Collectors.toList());
Dataset<Row> colOrderedDataset = rowDatasetWithHoodieColumns.select(
JavaConverters.collectionAsScalaIterableConverter(orderedFields).asScala().toSeq());
return colOrderedDataset
.sort(functions.col(HoodieRecord.PARTITION_PATH_METADATA_FIELD), functions.col(HoodieRecord.RECORD_KEY_METADATA_FIELD))
.coalesce(config.getBulkInsertShuffleParallelism());
}
}

View File

@@ -0,0 +1,91 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.sources.DataSourceRegister;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
import org.apache.spark.sql.sources.v2.DataSourceV2;
import org.apache.spark.sql.sources.v2.ReadSupport;
import org.apache.spark.sql.sources.v2.WriteSupport;
import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
import org.apache.spark.sql.types.StructType;
import java.util.Optional;
/**
* DataSource V2 implementation for managing internal write logic. Only called internally.
*/
public class DefaultSource implements DataSourceV2, ReadSupport, WriteSupport,
DataSourceRegister {
private static final Logger LOG = LogManager
.getLogger(DefaultSource.class);
private SparkSession sparkSession = null;
private Configuration configuration = null;
@Override
public String shortName() {
return "hudi_internal";
}
@Override
public DataSourceReader createReader(StructType schema, DataSourceOptions options) {
return null;
}
@Override
public DataSourceReader createReader(DataSourceOptions options) {
return null;
}
@Override
public Optional<DataSourceWriter> createWriter(String writeUUID, StructType schema, SaveMode mode,
DataSourceOptions options) {
String instantTime = options.get(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY).get();
String path = options.get("path").get();
String tblName = options.get(HoodieWriteConfig.TABLE_NAME).get();
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, options.asMap());
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
getConfiguration()));
}
private SparkSession getSparkSession() {
if (sparkSession == null) {
sparkSession = SparkSession.builder().getOrCreate();
}
return sparkSession;
}
private Configuration getConfiguration() {
if (configuration == null) {
this.configuration = getSparkSession().sparkContext().hadoopConfiguration();
}
return configuration;
}
}

View File

@@ -0,0 +1,119 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieRowCreateHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
/**
* Hoodie's Implementation of {@link DataWriter<InternalRow>}. This is used in data source implementation for bulk insert.
*/
public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRow> {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieBulkInsertDataInternalWriter.class);
private final String instantTime;
private final int taskPartitionId;
private final long taskId;
private final long taskEpochId;
private final HoodieTable hoodieTable;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
private HoodieRowCreateHandle handle;
private String lastKnownPartitionPath = null;
private String fileIdPrefix = null;
private int numFilesWritten = 0;
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
StructType structType) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.taskPartitionId = taskPartitionId;
this.taskId = taskId;
this.taskEpochId = taskEpochId;
this.structType = structType;
this.fileIdPrefix = UUID.randomUUID().toString();
}
@Override
public void write(InternalRow record) throws IOException {
try {
String partitionPath = record.getUTF8String(
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
LOG.info("Creating new file for partition path " + partitionPath);
createNewHandle(partitionPath);
lastKnownPartitionPath = partitionPath;
}
handle.write(record);
} catch (Throwable t) {
LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t);
throw t;
}
}
@Override
public WriterCommitMessage commit() throws IOException {
close();
return new HoodieWriterCommitMessage(writeStatusList);
}
@Override
public void abort() throws IOException {
}
private void createNewHandle(String partitionPath) throws IOException {
if (null != handle) {
close();
}
handle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType);
}
public void close() throws IOException {
if (null != handle) {
writeStatusList.add(handle.close());
}
}
protected String getNextFileId() {
return String.format("%s-%d", fileIdPrefix, numFilesWritten++);
}
}

View File

@@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
import org.apache.spark.sql.types.StructType;
/**
* Factory to assist in instantiating {@link HoodieBulkInsertDataInternalWriter}.
*/
public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFactory<InternalRow> {
private final String instantTime;
private final HoodieTable hoodieTable;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, StructType structType) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.structType = structType;
}
@Override
public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId,
structType);
}
}

View File

@@ -0,0 +1,119 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
import org.apache.spark.sql.types.StructType;
/**
* Implementation of {@link DataSourceWriter} for datasource "hudi.internal" to be used in datasource implementation
* of bulk insert.
*/
public class HoodieDataSourceInternalWriter implements DataSourceWriter {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieDataSourceInternalWriter.class);
public static final String INSTANT_TIME_OPT_KEY = "hoodie.instant.time";
private final String instantTime;
private final HoodieTableMetaClient metaClient;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final HoodieWriteClient writeClient;
private final HoodieTable hoodieTable;
private final WriteOperationType operationType;
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession sparkSession, Configuration configuration) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.operationType = WriteOperationType.BULK_INSERT;
this.writeClient = new HoodieWriteClient<>(new JavaSparkContext(sparkSession.sparkContext()), writeConfig, true);
writeClient.setOperationType(operationType);
writeClient.startCommitWithTime(instantTime);
this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath());
this.hoodieTable = HoodieTable.create(metaClient, writeConfig, metaClient.getHadoopConf());
}
@Override
public DataWriterFactory<InternalRow> createWriterFactory() {
metaClient.getActiveTimeline().transitionRequestedToInflight(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty());
if (WriteOperationType.BULK_INSERT == operationType) {
return new HoodieBulkInsertDataInternalWriterFactory(hoodieTable, writeConfig, instantTime, structType);
} else {
throw new IllegalArgumentException("Write Operation Type + " + operationType + " not supported ");
}
}
@Override
public boolean useCommitCoordinator() {
return true;
}
@Override
public void onDataWriterCommit(WriterCommitMessage message) {
LOG.info("Received commit of a data writer =" + message);
}
@Override
public void commit(WriterCommitMessage[] messages) {
List<HoodieWriteStat> writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m)
.flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList());
try {
writeClient.commitStats(instantTime, writeStatList, Option.empty());
} catch (Exception ioe) {
throw new HoodieException(ioe.getMessage(), ioe);
} finally {
writeClient.close();
}
}
@Override
public void abort(WriterCommitMessage[] messages) {
LOG.error("Commit " + instantTime + " aborted ");
writeClient.rollback(instantTime);
writeClient.close();
}
}

View File

@@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import java.util.ArrayList;
import java.util.List;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
/**
* Hoodie's {@link WriterCommitMessage} used in datasource implementation.
*/
public class HoodieWriterCommitMessage implements WriterCommitMessage {
private List<HoodieInternalWriteStatus> writeStatuses = new ArrayList<>();
public HoodieWriterCommitMessage(List<HoodieInternalWriteStatus> writeStatuses) {
this.writeStatuses = writeStatuses;
}
public List<HoodieInternalWriteStatus> getWriteStatuses() {
return writeStatuses;
}
@Override
public String toString() {
return "HoodieWriterCommitMessage{" + "writeStatuses=" + writeStatuses + '}';
}
}

View File

@@ -0,0 +1,130 @@
/*
* 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.keygen;
import org.apache.hudi.DataSourceWriteOptions;
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.spark.sql.types.StructType;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Base class for all the built-in key generators. Contains methods structured for
* code reuse amongst them.
*/
public abstract class BuiltinKeyGenerator extends KeyGenerator {
protected List<String> recordKeyFields;
protected List<String> partitionPathFields;
protected final boolean encodePartitionPath;
protected final boolean hiveStylePartitioning;
protected Map<String, List<Integer>> recordKeyPositions = new HashMap<>();
protected Map<String, List<Integer>> partitionPathPositions = new HashMap<>();
protected StructType structType;
protected BuiltinKeyGenerator(TypedProperties config) {
super(config);
this.encodePartitionPath = config.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
this.hiveStylePartitioning = config.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
}
/**
* Generate a record Key out of provided generic record.
*/
public abstract String getRecordKey(GenericRecord record);
/**
* Generate a partition path out of provided generic record.
*/
public abstract String getPartitionPath(GenericRecord record);
/**
* Generate a Hoodie Key out of provided generic record.
*/
public final HoodieKey getKey(GenericRecord record) {
if (getRecordKeyFields() == null || getPartitionPathFields() == null) {
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
}
return new HoodieKey(getRecordKey(record), getPartitionPath(record));
}
@Override
public final List<String> getRecordKeyFieldNames() {
// For nested columns, pick top level column name
return getRecordKeyFields().stream().map(k -> {
int idx = k.indexOf('.');
return idx > 0 ? k.substring(0, idx) : k;
}).collect(Collectors.toList());
}
void buildFieldPositionMapIfNeeded(StructType structType) {
if (this.structType == null) {
// parse simple fields
getRecordKeyFields().stream()
.filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
} else {
throw new HoodieKeyException("recordKey value not found for field: \"" + f + "\"");
}
});
// parse nested fields
getRecordKeyFields().stream()
.filter(f -> f.contains("."))
.forEach(f -> recordKeyPositions.put(f, RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, true)));
// parse simple fields
if (getPartitionPathFields() != null) {
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
partitionPathPositions.put(f,
Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
} else {
partitionPathPositions.put(f, Collections.singletonList(-1));
}
});
// parse nested fields
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains("."))
.forEach(f -> partitionPathPositions.put(f,
RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false)));
}
this.structType = structType;
}
}
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
public List<String> getPartitionPathFields() {
return partitionPathFields;
}
}

View File

@@ -24,8 +24,8 @@ import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.spark.sql.Row;
/**
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
@@ -34,39 +34,34 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":";
protected final List<String> recordKeyFields;
protected final List<String> partitionPathFields;
protected final boolean hiveStylePartitioning;
protected final boolean encodePartitionPath;
public ComplexKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields =
Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY())
.split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY())
.split(",")).map(String::trim).collect(Collectors.toList());
}
@Override
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, recordKeyFields);
return KeyGenUtils.getRecordKey(record, getRecordKeyFields());
}
@Override
public String getPartitionPath(GenericRecord record) {
return KeyGenUtils.getRecordPartitionPath(record, partitionPathFields, hiveStylePartitioning, encodePartitionPath);
return KeyGenUtils.getRecordPartitionPath(record, getPartitionPathFields(), hiveStylePartitioning, encodePartitionPath);
}
@Override
public List<String> getRecordKeyFields() {
return recordKeyFields;
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
}
@Override
public List<String> getPartitionPathFields() {
return partitionPathFields;
public String getPartitionPath(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
hiveStylePartitioning, partitionPathPositions);
}
}

View File

@@ -20,37 +20,32 @@ package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
/**
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
* Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
* "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields. Similarly partition path can be configured to have multiple
* fields or only one field. This class expects value for prop "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
*
* properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
*
* The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
*
* Few points to consider:
* 1. If you want to customize some partition path field on a timestamp basis, you can use field1:timestampBased
* 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
* 3. If you want your table to be non partitioned, simply leave it as blank.
* Few points to consider: 1. If you want to customize some partition path field on a timestamp basis, you can use field1:timestampBased 2. If you simply want to have the value of your configured
* field in the partition path, use field1:simple 3. If you want your table to be non partitioned, simply leave it as blank.
*
* RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
*/
public class CustomKeyGenerator extends BuiltinKeyGenerator {
protected final List<String> recordKeyFields;
protected final List<String> partitionPathFields;
protected final TypedProperties properties;
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
private static final String SPLIT_REGEX = ":";
@@ -63,15 +58,22 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
public CustomKeyGenerator(TypedProperties props) {
super(props);
this.properties = props;
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields =
Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
}
@Override
public String getPartitionPath(Row row) {
return getPartitionPath(Option.empty(), Option.of(row));
}
@Override
public String getPartitionPath(GenericRecord record) {
if (partitionPathFields == null) {
return getPartitionPath(Option.of(record), Option.empty());
}
private String getPartitionPath(Option<GenericRecord> record, Option<Row> row) {
if (getPartitionPathFields() == null) {
throw new HoodieKeyException("Unable to find field names for partition path in cfg");
}
@@ -79,10 +81,10 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
StringBuilder partitionPath = new StringBuilder();
//Corresponds to no partition case
if (partitionPathFields.size() == 1 && partitionPathFields.get(0).isEmpty()) {
if (getPartitionPathFields().size() == 1 && getPartitionPathFields().get(0).isEmpty()) {
return "";
}
for (String field : partitionPathFields) {
for (String field : getPartitionPathFields()) {
String[] fieldWithType = field.split(SPLIT_REGEX);
if (fieldWithType.length != 2) {
throw new HoodieKeyException("Unable to find field names for partition path in proper format");
@@ -92,11 +94,19 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
switch (keyType) {
case SIMPLE:
partitionPath.append(new SimpleKeyGenerator(properties, partitionPathField).getPartitionPath(record));
if (record.isPresent()) {
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(record.get()));
} else {
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(row.get()));
}
break;
case TIMESTAMP:
try {
partitionPath.append(new TimestampBasedKeyGenerator(properties, partitionPathField).getPartitionPath(record));
if (record.isPresent()) {
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(record.get()));
} else {
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(row.get()));
}
} catch (IOException ioe) {
throw new HoodieDeltaStreamerException("Unable to initialise TimestampBasedKeyGenerator class");
}
@@ -114,20 +124,23 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
@Override
public String getRecordKey(GenericRecord record) {
if (recordKeyFields == null || recordKeyFields.isEmpty()) {
validateRecordKeyFields();
return getRecordKeyFields().size() == 1
? new SimpleKeyGenerator(config).getRecordKey(record)
: new ComplexKeyGenerator(config).getRecordKey(record);
}
@Override
public String getRecordKey(Row row) {
validateRecordKeyFields();
return getRecordKeyFields().size() == 1
? new SimpleKeyGenerator(config).getRecordKey(row)
: new ComplexKeyGenerator(config).getRecordKey(row);
}
private void validateRecordKeyFields() {
if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) {
throw new HoodieKeyException("Unable to find field names for record key in cfg");
}
return recordKeyFields.size() == 1 ? new SimpleKeyGenerator(properties).getRecordKey(record) : new ComplexKeyGenerator(properties).getRecordKey(record);
}
@Override
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
@Override
public List<String> getPartitionPathFields() {
return partitionPathFields;
}
}

View File

@@ -22,30 +22,28 @@ import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
/**
* Key generator for deletes using global indices. Global index deletes do not require partition value
* so this key generator avoids using partition value for generating HoodieKey.
* Key generator for deletes using global indices. Global index deletes do not require partition value so this key generator
* avoids using partition value for generating HoodieKey.
*/
public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
private static final String EMPTY_PARTITION = "";
protected final List<String> recordKeyFields;
public GlobalDeleteKeyGenerator(TypedProperties config) {
super(config);
this.recordKeyFields = Arrays.stream(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
this.recordKeyFields = Arrays.asList(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","));
}
@Override
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, recordKeyFields);
return KeyGenUtils.getRecordKey(record, getRecordKeyFields());
}
@Override
@@ -53,13 +51,19 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
return EMPTY_PARTITION;
}
@Override
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
@Override
public List<String> getPartitionPathFields() {
return new ArrayList<>();
}
}
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
}
@Override
public String getPartitionPath(Row row) {
return EMPTY_PARTITION;
}
}

View File

@@ -18,10 +18,13 @@
package org.apache.hudi.keygen;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import scala.Function1;
import java.io.Serializable;
import java.util.List;
@@ -29,9 +32,13 @@ import java.util.List;
/**
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
*/
public abstract class KeyGenerator implements Serializable {
public abstract class KeyGenerator implements Serializable, KeyGeneratorInterface {
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
private static final String NAMESPACE = "hoodieRow";
protected transient TypedProperties config;
private transient Function1<Object, Object> converterFn = null;
protected KeyGenerator(TypedProperties config) {
this.config = config;
@@ -51,4 +58,30 @@ public abstract class KeyGenerator implements Serializable {
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
+ "Please override this method in your custom key generator.");
}
/**
* Fetch record key from {@link Row}.
* @param row instance of {@link Row} from which record key is requested.
* @return the record key of interest from {@link Row}.
*/
public String getRecordKey(Row row) {
if (null == converterFn) {
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
return getKey(genericRecord).getRecordKey();
}
/**
* Fetch partition path from {@link Row}.
* @param row instance of {@link Row} from which partition path is requested
* @return the partition path of interest from {@link Row}.
*/
public String getPartitionPath(Row row) {
if (null == converterFn) {
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
return getKey(genericRecord).getPartitionPath();
}
}

View File

@@ -18,10 +18,10 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import java.util.ArrayList;
import java.util.List;
@@ -32,12 +32,10 @@ import java.util.List;
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
private static final String EMPTY_PARTITION = "";
protected final String recordKeyField;
private static final List<String> EMPTY_PARTITION_FIELD_LIST = new ArrayList<>();
public NonpartitionedKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
}
@Override
@@ -47,6 +45,11 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
@Override
public List<String> getPartitionPathFields() {
return new ArrayList<>();
return EMPTY_PARTITION_FIELD_LIST;
}
}
@Override
public String getPartitionPath(Row row) {
return EMPTY_PARTITION;
}
}

View File

@@ -0,0 +1,202 @@
/*
* 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.keygen;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
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;
import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
/**
* Helper class to fetch fields from Row.
*/
public class RowKeyGeneratorHelper {
/**
* Generates record key for the corresponding {@link Row}.
* @param row instance of {@link Row} of interest
* @param recordKeyFields record key fields as a list
* @param recordKeyPositions record key positions for the corresponding record keys in {@code recordKeyFields}
* @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise.
* @return the record key thus generated
*/
public static String getRecordKeyFromRow(Row row, List<String> recordKeyFields, Map<String, List<Integer>> recordKeyPositions, boolean prefixFieldName) {
AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true);
String toReturn = recordKeyFields.stream().map(field -> {
String val = null;
List<Integer> fieldPositions = recordKeyPositions.get(field);
if (fieldPositions.size() == 1) { // simple field
Integer fieldPos = fieldPositions.get(0);
if (row.isNullAt(fieldPos)) {
val = NULL_RECORDKEY_PLACEHOLDER;
} else {
val = row.getAs(field).toString();
if (val.isEmpty()) {
val = EMPTY_RECORDKEY_PLACEHOLDER;
} else {
keyIsNullOrEmpty.set(false);
}
}
} else { // nested fields
val = getNestedFieldVal(row, recordKeyPositions.get(field)).toString();
if (!val.contains(NULL_RECORDKEY_PLACEHOLDER) && !val.contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
keyIsNullOrEmpty.set(false);
}
}
return prefixFieldName ? (field + ":" + val) : val;
}).collect(Collectors.joining(","));
if (keyIsNullOrEmpty.get()) {
throw new HoodieKeyException("recordKey value: \"" + toReturn + "\" for fields: \"" + Arrays.toString(recordKeyFields.toArray()) + "\" cannot be null or empty.");
}
return toReturn;
}
/**
* Generates partition path for the corresponding {@link Row}.
* @param row instance of {@link Row} of interest
* @param partitionPathFields partition path fields as a list
* @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise
* @param partitionPathPositions partition path positions for the corresponding fields in {@code partitionPathFields}
* @return the generated partition path for the row
*/
public static String getPartitionPathFromRow(Row row, List<String> partitionPathFields, boolean hiveStylePartitioning, Map<String, List<Integer>> partitionPathPositions) {
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 {
val = row.getAs(field).toString();
if (val.isEmpty()) {
val = DEFAULT_PARTITION_PATH;
}
}
if (hiveStylePartitioning) {
val = field + "=" + val;
}
} else { // nested
Object nestedVal = getNestedFieldVal(row, partitionPathPositions.get(field));
if (nestedVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER) || nestedVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
val = hiveStylePartitioning ? field + "=" + DEFAULT_PARTITION_PATH : DEFAULT_PARTITION_PATH;
} else {
val = hiveStylePartitioning ? field + "=" + nestedVal.toString() : nestedVal.toString();
}
}
return val;
}).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR));
}
/**
* Fetch the field value located at the positions requested for.
* @param row instance of {@link Row} of interest
* @param positions tree style positions where the leaf node need to be fetched and returned
* @return the field value as per the positions requested for.
*/
public static Object getNestedFieldVal(Row row, List<Integer> positions) {
if (positions.size() == 1 && positions.get(0) == -1) {
return DEFAULT_PARTITION_PATH;
}
int index = 0;
int totalCount = positions.size();
Row valueToProcess = row;
Object toReturn = null;
while (index < totalCount) {
if (index < totalCount - 1) {
if (valueToProcess.isNullAt(positions.get(index))) {
toReturn = NULL_RECORDKEY_PLACEHOLDER;
break;
}
valueToProcess = (Row) valueToProcess.get(positions.get(index));
} else { // last index
if (valueToProcess.getAs(positions.get(index)).toString().isEmpty()) {
toReturn = EMPTY_RECORDKEY_PLACEHOLDER;
break;
}
toReturn = valueToProcess.getAs(positions.get(index));
}
index++;
}
return toReturn;
}
/**
* Generate the tree style positions for the field requested for as per the defined struct type.
* @param structType schema of interest
* @param field field of interest for which the positions are requested for
* @param isRecordKey {@code true} if the field requested for is a record key. {@code false} incase of a partition path.
* @return the positions of the field as per the struct type.
*/
public static List<Integer> getNestedFieldIndices(StructType structType, String field, boolean isRecordKey) {
String[] slices = field.split("\\.");
List<Integer> positions = new ArrayList<>();
int index = 0;
int totalCount = slices.length;
while (index < totalCount) {
String slice = slices[index];
Option<Object> curIndexOpt = structType.getFieldIndex(slice);
if (curIndexOpt.isDefined()) {
int curIndex = (int) curIndexOpt.get();
positions.add(curIndex);
final StructField nestedField = structType.fields()[curIndex];
if (index < totalCount - 1) {
if (!(nestedField.dataType() instanceof StructType)) {
if (isRecordKey) {
throw new HoodieKeyException("Nested field should be of type StructType " + nestedField);
} else {
positions = Collections.singletonList(-1);
break;
}
}
structType = (StructType) nestedField.dataType();
}
} else {
if (isRecordKey) {
throw new HoodieKeyException("Can't find " + slice + " in StructType for the field " + field);
} else {
positions = Collections.singletonList(-1);
break;
}
}
index++;
}
return positions;
}
}

View File

@@ -22,54 +22,52 @@ import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import java.util.Arrays;
import java.util.List;
import java.util.Collections;
/**
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
*/
public class SimpleKeyGenerator extends BuiltinKeyGenerator {
protected final String recordKeyField;
protected final String partitionPathField;
protected final boolean hiveStylePartitioning;
protected final boolean encodePartitionPath;
public SimpleKeyGenerator(TypedProperties props) {
this(props, props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
this(props, props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()),
props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
}
public SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
this(props, null, partitionPathField);
}
SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
super(props);
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
this.partitionPathField = partitionPathField;
this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
this.recordKeyFields = recordKeyField == null
? Collections.emptyList()
: Collections.singletonList(recordKeyField);
this.partitionPathFields = Collections.singletonList(partitionPathField);
}
@Override
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, recordKeyField);
return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0));
}
@Override
public String getPartitionPath(GenericRecord record) {
return KeyGenUtils.getPartitionPath(record, partitionPathField, hiveStylePartitioning, encodePartitionPath);
return KeyGenUtils.getPartitionPath(record, getPartitionPathFields().get(0), hiveStylePartitioning, encodePartitionPath);
}
@Override
public List<String> getRecordKeyFields() {
return Arrays.asList(recordKeyField);
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
}
@Override
public List<String> getPartitionPathFields() {
return Arrays.asList(partitionPathField);
public String getPartitionPath(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
hiveStylePartitioning, partitionPathPositions);
}
}
}

View File

@@ -25,10 +25,11 @@ import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.keygen.parser.HoodieDateTimeParser;
import org.apache.hudi.keygen.parser.HoodieDateTimeParserImpl;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
@@ -39,10 +40,14 @@ import java.io.Serializable;
import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.concurrent.TimeUnit;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
/**
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
@@ -89,11 +94,16 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
}
public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
this(config, config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
this(config, config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()),
config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
}
public TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
super(config, partitionPathField);
TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
this(config, null, partitionPathField);
}
TimestampBasedKeyGenerator(TypedProperties config, String recordKeyField, String partitionPathField) throws IOException {
super(config, recordKeyField, partitionPathField);
String dateTimeParserClass = config.getString(Config.DATE_TIME_PARSER_PROP, HoodieDateTimeParserImpl.class.getName());
this.parser = DataSourceUtils.createDateTimeParser(config, dateTimeParserClass);
this.outputDateTimeZone = parser.getOutputDateTimeZone();
@@ -125,49 +135,58 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
@Override
public String getPartitionPath(GenericRecord record) {
Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, partitionPathField, true);
Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, getPartitionPathFields().get(0), true);
if (partitionVal == null) {
partitionVal = 1L;
}
try {
return getPartitionPath(partitionVal);
} catch (Exception e) {
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e);
}
}
/**
* Parse and fetch partition path based on data type.
*
* @param partitionVal partition path object value fetched from record/row
* @return the parsed partition path based on data type
* @throws ParseException on any parse exception
*/
private String getPartitionPath(Object partitionVal) throws ParseException {
DateTimeFormatter partitionFormatter = DateTimeFormat.forPattern(outputDateFormat);
if (this.outputDateTimeZone != null) {
partitionFormatter = partitionFormatter.withZone(outputDateTimeZone);
}
try {
long timeMs;
if (partitionVal instanceof Double) {
timeMs = convertLongTimeToMillis(((Double) partitionVal).longValue());
} else if (partitionVal instanceof Float) {
timeMs = convertLongTimeToMillis(((Float) partitionVal).longValue());
} else if (partitionVal instanceof Long) {
timeMs = convertLongTimeToMillis((Long) partitionVal);
} else if (partitionVal instanceof CharSequence) {
DateTime parsedDateTime = inputFormatter.parseDateTime(partitionVal.toString());
if (this.outputDateTimeZone == null) {
// Use the timezone that came off the date that was passed in, if it had one
partitionFormatter = partitionFormatter.withZone(parsedDateTime.getZone());
}
timeMs = inputFormatter.parseDateTime(partitionVal.toString()).getMillis();
} else {
throw new HoodieNotSupportedException(
"Unexpected type for partition field: " + partitionVal.getClass().getName());
long timeMs;
if (partitionVal instanceof Double) {
timeMs = convertLongTimeToMillis(((Double) partitionVal).longValue());
} else if (partitionVal instanceof Float) {
timeMs = convertLongTimeToMillis(((Float) partitionVal).longValue());
} else if (partitionVal instanceof Long) {
timeMs = convertLongTimeToMillis((Long) partitionVal);
} else if (partitionVal instanceof CharSequence) {
DateTime parsedDateTime = inputFormatter.parseDateTime(partitionVal.toString());
if (this.outputDateTimeZone == null) {
// Use the timezone that came off the date that was passed in, if it had one
partitionFormatter = partitionFormatter.withZone(parsedDateTime.getZone());
}
DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
String partitionPath = timestamp.toString(partitionFormatter);
if (encodePartitionPath) {
try {
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
return hiveStylePartitioning ? partitionPathField + "=" + partitionPath : partitionPath;
} catch (Exception e) {
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e);
timeMs = inputFormatter.parseDateTime(partitionVal.toString()).getMillis();
} else {
throw new HoodieNotSupportedException(
"Unexpected type for partition field: " + partitionVal.getClass().getName());
}
DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
String partitionPath = timestamp.toString(partitionFormatter);
if (encodePartitionPath) {
try {
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
return hiveStylePartitioning ? getPartitionPathFields().get(0) + "=" + partitionPath : partitionPath;
}
private long convertLongTimeToMillis(Long partitionVal) {
@@ -177,4 +196,28 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
}
return MILLISECONDS.convert(partitionVal, timeUnit);
}
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
}
@Override
public String getPartitionPath(Row row) {
Object fieldVal = null;
buildFieldPositionMapIfNeeded(row.schema());
Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathPositions.get(getPartitionPathFields().get(0)));
try {
if (partitionPathFieldVal.toString().contains(DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER)
|| partitionPathFieldVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
fieldVal = 1L;
} else {
fieldVal = partitionPathFieldVal;
}
return getPartitionPath(fieldVal);
} catch (Exception e) {
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + fieldVal, e);
}
}
}

View File

@@ -228,6 +228,13 @@ object DataSourceWriteOptions {
val KEYGENERATOR_CLASS_OPT_KEY = "hoodie.datasource.write.keygenerator.class"
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName
/**
* When set to true, will perform write operations directly using the spark native `Row` representation.
* By default, false (will be enabled as default in a future release)
*/
val ENABLE_ROW_WRITER_OPT_KEY = "hoodie.datasource.write.row.writer.enable"
val DEFAULT_ENABLE_ROW_WRITER_OPT_VAL = "false"
/**
* Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata.
* This is useful to store checkpointing information, in a consistent way with the hoodie timeline
@@ -299,6 +306,6 @@ object DataSourceWriteOptions {
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = "true"
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE_KEY = "hoodie.datasource.compaction.async.enable"
val DEFAULT_ASYNC_COMPACT_ENABLE_VAL = "true"
val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable"
val DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL = "true"
}

View File

@@ -118,14 +118,12 @@ class DefaultSource extends RelationProvider
mode: SaveMode,
optParams: Map[String, String],
df: DataFrame): BaseRelation = {
val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams)
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
if (parameters(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, df)
} else {
HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df)
}
new HoodieEmptyRelation(sqlContext, df.schema)
}
@@ -133,7 +131,7 @@ class DefaultSource extends RelationProvider
optParams: Map[String, String],
partitionColumns: Seq[String],
outputMode: OutputMode): Sink = {
val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams)
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
new HoodieStreamingSink(
sqlContext,
parameters,

View File

@@ -29,7 +29,6 @@ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.client.{HoodieWriteClient, WriteStatus}
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
@@ -38,6 +37,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, B
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.internal.HoodieDataSourceInternalWriter
import org.apache.hudi.sync.common.AbstractSyncTool
import org.apache.log4j.LogManager
import org.apache.spark.SparkContext
@@ -62,7 +62,7 @@ private[hudi] object HoodieSparkSqlWriter {
asyncCompactionTriggerFn: Option[Function1[HoodieWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
)
: (Boolean, common.util.Option[String], common.util.Option[String],
HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
val sparkContext = sqlContext.sparkContext
val path = parameters.get("path")
@@ -105,6 +105,22 @@ private[hudi] object HoodieSparkSqlWriter {
} else {
// Handle various save modes
handleSaveModes(mode, basePath, tableConfig, tblName, operation, fs)
// Create the table if not present
if (!tableExists) {
val tableMetaClient = HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get,
HoodieTableType.valueOf(tableType), tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY),
null.asInstanceOf[String])
tableConfig = tableMetaClient.getTableConfig
}
// short-circuit if bulk_insert via row is enabled.
// scalastyle:off
if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean) {
val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName,
basePath, path, instantTime)
return (success, commitTime, common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig)
}
// scalastyle:on
val (writeStatuses, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) =
if (!operation.equalsIgnoreCase(DELETE_OPERATION_OPT_VAL)) {
@@ -128,14 +144,6 @@ private[hudi] object HoodieSparkSqlWriter {
parameters(PAYLOAD_CLASS_OPT_KEY))
}).toJavaRDD()
// Create the table if not present
if (!tableExists) {
val tableMetaClient = HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get,
HoodieTableType.valueOf(tableType), tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY),
null.asInstanceOf[String])
tableConfig = tableMetaClient.getTableConfig
}
// Create a HoodieWriteClient & issue the write.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
tblName, mapAsJavaMap(parameters)
@@ -250,41 +258,29 @@ private[hudi] object HoodieSparkSqlWriter {
metaSyncSuccess
}
/**
* Add default options for unspecified write options keys.
*
* @param parameters
* @return
*/
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL,
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL,
INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL,
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
//just for backwards compatiblity
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL,
HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL,
HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL,
HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL,
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL,
HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL,
HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL,
ASYNC_COMPACT_ENABLE_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_VAL
) ++ translateStorageTypeToTableType(parameters)
def bulkInsertAsRow(sqlContext: SQLContext,
parameters: Map[String, String],
df: DataFrame,
tblName: String,
basePath: Path,
path: Option[String],
instantTime: String): (Boolean, common.util.Option[String]) = {
val structName = s"${tblName}_record"
val nameSpace = s"hoodie.${tblName}"
val writeConfig = DataSourceUtils.createHoodieConfig(null, path.get, tblName, mapAsJavaMap(parameters))
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
hoodieDF.write.format("org.apache.hudi.internal")
.option(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY, instantTime)
.options(parameters)
.save()
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
val metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
val syncHiveSucess = if (hiveSyncEnabled || metaSyncEnabled) {
metaSync(parameters, basePath, sqlContext.sparkContext.hadoopConfiguration)
} else {
true
}
(syncHiveSucess, common.util.Option.ofNullable(instantTime))
}
def toProperties(params: Map[String, String]): TypedProperties = {
@@ -298,7 +294,7 @@ private[hudi] object HoodieSparkSqlWriter {
if (mode == SaveMode.Append && tableExists) {
val existingTableName = tableConfig.getTableName
if (!existingTableName.equals(tableName)) {
throw new HoodieException(s"hoodie table with name $existingTableName already exist at $tablePath")
throw new HoodieException(s"hoodie table with name $existingTableName already exists at $tablePath")
}
}
@@ -411,11 +407,11 @@ private[hudi] object HoodieSparkSqlWriter {
val asyncCompactionEnabled = isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())
val compactionInstant : common.util.Option[java.lang.String] =
if (asyncCompactionEnabled) {
client.scheduleCompaction(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
} else {
common.util.Option.empty()
}
if (asyncCompactionEnabled) {
client.scheduleCompaction(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
} else {
common.util.Option.empty()
}
log.info(s"Compaction Scheduled is $compactionInstant")
val metaSyncSuccess = metaSync(parameters, basePath, jsc.hadoopConfiguration())
@@ -448,7 +444,7 @@ private[hudi] object HoodieSparkSqlWriter {
parameters: Map[String, String], configuration: Configuration) : Boolean = {
log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}")
if (!client.getConfig.isInlineCompaction
&& parameters.get(ASYNC_COMPACT_ENABLE_KEY).exists(r => r.toBoolean)) {
&& parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY).exists(r => r.toBoolean)) {
tableConfig.getTableType == HoodieTableType.MERGE_ON_READ
} else {
false

View File

@@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.TypedProperties
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
/**
* WriterUtils to assist in write path in Datasource and tests.
*/
object HoodieWriterUtils {
def javaParametersWithWriteDefaults(parameters: java.util.Map[String, String]): java.util.Map[String, String] = {
mapAsJavaMap(parametersWithWriteDefaults(parameters.asScala.toMap))
}
/**
* Add default options for unspecified write options keys.
*
* @param parameters
* @return
*/
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL,
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL,
INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL,
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL,
HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL,
HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL,
HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL,
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL,
HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL,
HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL,
ASYNC_COMPACT_ENABLE_OPT_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL,
ENABLE_ROW_WRITER_OPT_KEY -> DEFAULT_ENABLE_ROW_WRITER_OPT_VAL
) ++ translateStorageTypeToTableType(parameters)
}
def toProperties(params: Map[String, String]): TypedProperties = {
val props = new TypedProperties()
params.foreach(kv => props.setProperty(kv._1, kv._2))
props
}
}

View File

@@ -151,7 +151,7 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
// This will remove any existing data at path below, and create a
.mode(SaveMode.Overwrite);
@@ -178,7 +178,7 @@ public class HoodieJavaApp {
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
@@ -204,7 +204,7 @@ public class HoodieJavaApp {
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);

View File

@@ -358,7 +358,7 @@ public class HoodieJavaStreamingApp {
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "true")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "true")
.option(HoodieWriteConfig.TABLE_NAME, tableName).option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());

View File

@@ -0,0 +1,156 @@
/*
* 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.common.model.HoodieRecord;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.DataSourceTestUtils;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.avro.Schema;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.types.StructType;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
/**
* Tests {@link HoodieDatasetBulkInsertHelper}.
*/
public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
private String schemaStr;
private Schema schema;
private StructType structType;
public TestHoodieDatasetBulkInsertHelper() throws IOException {
init();
}
private void init() throws IOException {
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
schema = DataSourceTestUtils.getStructTypeExampleSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
}
@Test
public void testBulkInsertHelper() throws IOException {
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
StructType resultSchema = result.schema();
assertEquals(result.count(), 10);
assertEquals(resultSchema.fieldNames().length, structType.fieldNames().length + HoodieRecord.HOODIE_META_COLUMNS.size());
for (Map.Entry<String, Integer> entry : HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.entrySet()) {
assertTrue(resultSchema.fieldIndex(entry.getKey()) == entry.getValue());
}
int metadataRecordKeyIndex = resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD);
int metadataParitionPathIndex = resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
int metadataCommitTimeIndex = resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
int metadataCommitSeqNoIndex = resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD);
int metadataFilenameIndex = resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD);
result.toJavaRDD().foreach(entry -> {
assertTrue(entry.get(metadataRecordKeyIndex).equals(entry.getAs("_row_key")));
assertTrue(entry.get(metadataParitionPathIndex).equals(entry.getAs("partition")));
assertTrue(entry.get(metadataCommitSeqNoIndex).equals(""));
assertTrue(entry.get(metadataCommitTimeIndex).equals(""));
assertTrue(entry.get(metadataFilenameIndex).equals(""));
});
}
private Map<String, String> getPropsAllSet() {
return getProps(true, true, true, true);
}
private Map<String, String> getProps(boolean setAll, boolean setKeyGen, boolean setRecordKey, boolean setPartitionPath) {
Map<String, String> props = new HashMap<>();
if (setAll) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
} else {
if (setKeyGen) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
}
if (setRecordKey) {
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
}
if (setPartitionPath) {
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
}
}
return props;
}
@Test
public void testNoPropsSet() {
HoodieWriteConfig config = getConfigBuilder(schemaStr).build();
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
config = getConfigBuilder(schemaStr).withProps(getProps(false, false, true, true)).build();
rows = DataSourceTestUtils.generateRandomRows(10);
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
config = getConfigBuilder(schemaStr).withProps(getProps(false, true, false, true)).build();
rows = DataSourceTestUtils.generateRandomRows(10);
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
config = getConfigBuilder(schemaStr).withProps(getProps(false, true, true, false)).build();
rows = DataSourceTestUtils.generateRandomRows(10);
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
}
}

View File

@@ -0,0 +1,213 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
/**
* Unit tests {@link HoodieBulkInsertDataInternalWriter}.
*/
public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestHoodieBulkInsertDataInternalWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testDataInternalWriter() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
// execute N rounds
for (int i = 0; i < 5; i++) {
String instantTime = "00" + i;
// init writer
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(1000);
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
int batches = 5;
Dataset<Row> totalInputRows = null;
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<String> fileAbsPaths = new ArrayList<>();
List<String> fileNames = new ArrayList<>();
// verify write statuses
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertOutput(totalInputRows, result, instantTime, fileNames);
}
}
/**
* Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch2 of invalid records which is expected
* to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk.
*/
@Test
public void testGlobalFailure() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
String instantTime = "001";
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(100);
int totalFailures = 5;
// Generate first batch of valid rows
Dataset<Row> inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false);
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// generate some failures rows
for (int i = 0; i < totalFailures; i++) {
internalRows.add(getInternalRowWithError(partitionPath));
}
// generate 2nd batch of valid rows
Dataset<Row> inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false);
internalRows.addAll(toInternalRows(inputRows2, ENCODER));
// issue writes
try {
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
fail("Should have failed");
} catch (Throwable e) {
// expected
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<String> fileAbsPaths = new ArrayList<>();
List<String> fileNames = new ArrayList<>();
// verify write statuses
assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertOutput(inputRows, result, instantTime, fileNames);
}
private void writeRows(Dataset<Row> inputRows, HoodieBulkInsertDataInternalWriter writer) throws IOException {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
}
private void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size, List<String> fileAbsPaths, List<String> fileNames) {
assertEquals(batches, writeStatuses.size());
int counter = 0;
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
// verify write status
assertEquals(writeStatus.getTotalRecords(), size);
assertNull(writeStatus.getGlobalError());
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertNotNull(writeStatus.getFileId());
String fileId = writeStatus.getFileId();
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath());
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
fileNames.add(writeStatus.getStat().getPath().substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1));
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
}
}
private void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> fileNames) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
assertTrue(fileNames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}

View File

@@ -0,0 +1,321 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
/**
* Unit tests {@link HoodieDataSourceInternalWriter}.
*/
public class TestHoodieDataSourceInternalWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestHoodieDataSourceInternalWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testDataSourceWriter() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
String instantTime = "001";
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
List<String> partitionPathsAbs = new ArrayList<>();
for (String partitionPath : partitionPaths) {
partitionPathsAbs.add(basePath + "/" + partitionPath + "/*");
}
int size = 10 + RANDOM.nextInt(1000);
int batches = 5;
Dataset<Row> totalInputRows = null;
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
@Test
public void testMultipleDataSourceWrites() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
int partitionCounter = 0;
// execute N rounds
for (int i = 0; i < 5; i++) {
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
int size = 10 + RANDOM.nextInt(1000);
int batches = 5; // one batch per partition
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
}
@Test
public void testLargeWrites() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
int partitionCounter = 0;
// execute N rounds
for (int i = 0; i < 3; i++) {
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
int size = 10000 + RANDOM.nextInt(10000);
int batches = 3; // one batch per partition
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
}
/**
* Tests that DataSourceWriter.abort() will abort the written records of interest write and commit batch1 write and abort batch2 Read of entire dataset should show only records from batch1.
* commit batch1
* abort batch2
* verify only records from batch1 is available to read
*/
@Test
public void testAbort() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
String instantTime0 = "00" + 0;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
List<String> partitionPathsAbs = new ArrayList<>();
for (String partitionPath : partitionPaths) {
partitionPathsAbs.add(basePath + "/" + partitionPath + "/*");
}
int size = 10 + RANDOM.nextInt(100);
int batches = 1;
Dataset<Row> totalInputRows = null;
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
// commit 1st batch
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
assertOutput(totalInputRows, result, instantTime0);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
// 2nd batch. abort in the end
String instantTime1 = "00" + 1;
dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
}
commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
// commit 1st batch
dataSourceInternalWriter.abort(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
// only rows from first batch should be present
assertOutput(totalInputRows, result, instantTime0);
}
private void writeRows(Dataset<Row> inputRows, DataWriter<InternalRow> writer) throws IOException {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
}
private void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size) {
assertEquals(batches, writeStatuses.size());
int counter = 0;
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
assertEquals(writeStatus.getPartitionPath(), HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]);
assertEquals(writeStatus.getTotalRecords(), size);
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertEquals(writeStatus.getTotalErrorRecords(), 0);
assertFalse(writeStatus.hasErrors());
assertNull(writeStatus.getGlobalError());
assertNotNull(writeStatus.getFileId());
String fileId = writeStatus.getFileId();
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
}
}
private void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}

View File

@@ -23,10 +23,13 @@ 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.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestComplexKeyGenerator extends TestKeyGeneratorUtilities {
public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
@@ -76,13 +79,18 @@ public class TestComplexKeyGenerator extends TestKeyGeneratorUtilities {
public void testWrongRecordKeyField() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=2020-03-21");
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");
}
}

View File

@@ -22,10 +22,13 @@ import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
@@ -97,25 +100,37 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
@Test
public void testSimpleKeyGenerator() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropertiesForSimpleKeyGen());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
}
@Test
public void testTimestampBasedKeyGenerator() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropertiesForTimestampBasedKeyGen());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "ts_ms=20200321");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321");
}
@Test
public void testNonPartitionedKeyGenerator() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropertiesForNonPartitionedKeyGen());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertTrue(key.getPartitionPath().isEmpty());
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty());
}
@Test
@@ -127,6 +142,16 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomKeyGenerator.PartitionKeyType.DUMMY"));
}
try {
KeyGenerator keyGenerator = new CustomKeyGenerator(getInvalidPartitionKeyTypeProps());
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when invalid PartitionKeyType is provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomKeyGenerator.PartitionKeyType.DUMMY"));
}
}
@Test
@@ -138,6 +163,16 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found"));
}
try {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropsWithoutRecordKeyFieldProps());
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getRecordKey(row);
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
@Test
@@ -149,21 +184,41 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
try {
KeyGenerator keyGenerator = new CustomKeyGenerator(getImproperPartitionFieldFormatProp());
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when partition key field is provided in improper format!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
}
@Test
public void testComplexRecordKeyWithSimplePartitionPath() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getComplexRecordKeyWithSimplePartitionProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
}
@Test
public void testComplexRecordKeysWithComplexPartitionPath() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getComplexRecordKeyAndPartitionPathProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=20200321");
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");
}
}

View File

@@ -23,10 +23,13 @@ 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.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestGlobalDeleteKeyGenerator extends TestKeyGeneratorUtilities {
public class TestGlobalDeleteKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
@@ -66,13 +69,19 @@ public class TestGlobalDeleteKeyGenerator extends TestKeyGeneratorUtilities {
public void testWrongRecordKeyField() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "");
keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType);
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
}

View File

@@ -23,10 +23,13 @@ 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.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps() {
TypedProperties properties = new TypedProperties();
@@ -52,6 +55,13 @@ public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
return properties;
}
private TypedProperties getWrongPartitionPathFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "_wrong_partition_path");
properties.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
return properties;
}
private TypedProperties getComplexRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "timestamp");
@@ -79,19 +89,36 @@ public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
public void testWrongRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testWrongPartitionPathField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps());
GenericRecord record = getRecord();
Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.DEFAULT_PARTITION_PATH);
Assertions.assertEquals(keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)),
KeyGenUtils.DEFAULT_PARTITION_PATH);
}
@Test
public void testComplexRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getProps());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(getRecord());
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
}
}

View File

@@ -18,31 +18,45 @@
package org.apache.hudi.keygen;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import scala.Function1;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestTimestampBasedKeyGenerator {
private GenericRecord baseRecord;
private TypedProperties properties = new TypedProperties();
private Schema schema;
private StructType structType;
private Row baseRow;
@BeforeEach
public void initialize() throws IOException {
Schema schema = SchemaTestUtil.getTimestampEvolvedSchema();
schema = SchemaTestUtil.getTimestampEvolvedSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1");
.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRow = genericRecordToRow(baseRecord);
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "field1");
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "createTime");
@@ -61,6 +75,17 @@ public class TestTimestampBasedKeyGenerator {
return properties;
}
private Row genericRecordToRow(GenericRecord baseRecord) {
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(schema, structType);
Row row = (Row) convertor.apply(baseRecord);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {
values[i] = row.get(i);
}
return new GenericRowWithSchema(values, structType);
}
private TypedProperties getBaseKeyConfig(String timestampType, String inputFormatList, String inputFormatDelimiterRegex, String inputTimezone, String outputFormat, String outputTimezone) {
if (timestampType != null) {
properties.setProperty(TimestampBasedKeyGenerator.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType);
@@ -88,25 +113,43 @@ public class TestTimestampBasedKeyGenerator {
// timezone is GMT+8:00
baseRecord.put("createTime", 1578283932000L);
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk1.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null);
HoodieKey hk2 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 04", hk2.getPartitionPath());
// test w/ Row
assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow));
// timestamp is DATE_STRING, timezone is GMT+8:00
baseRecord.put("createTime", "2020-01-06 12:12:12");
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT+8:00", null);
properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss");
HoodieKey hk3 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk3 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk3.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT", null);
HoodieKey hk4 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk4 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk4.getPartitionPath());
// test w/ Row
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
}
@Test
@@ -116,140 +159,173 @@ public class TestTimestampBasedKeyGenerator {
// timezone is GMT
properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd hh", "GMT", "days");
HoodieKey hk5 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk5 = keyGen.getKey(baseRecord);
assertEquals(hk5.getPartitionPath(), "2024-10-04 12");
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"GMT");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"GMT");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.123Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"EST");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"EST");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040109", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040109", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() {
public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException {
baseRecord.put("createTime", "2020-04-01 13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> new TimestampBasedKeyGenerator(properties).getKey(baseRecord));
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getKey(baseRecord));
baseRow = genericRecordToRow(baseRecord);
Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException {
baseRecord.put("createTime", "20200401");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd",
"",
"UTC",
"MM/dd/yyyy",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd",
"",
"UTC",
"MM/dd/yyyy",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("04/01/2020", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow));
}
}

View File

@@ -0,0 +1,106 @@
/*
* 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.testutils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH;
/**
* Test utils for data source tests.
*/
public class DataSourceTestUtils {
public static Option<String> convertToString(HoodieRecord record) {
try {
String str = ((RawTripTestPayload) record.getData()).getJsonData();
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
// Remove the last } bracket
str = str.substring(0, str.length() - 1);
return Option.of(str + ", \"partition\": \"" + record.getPartitionPath() + "\"}");
} catch (IOException e) {
return Option.empty();
}
}
public static List<String> convertToStringList(List<HoodieRecord> records) {
return records.stream().map(DataSourceTestUtils::convertToString).filter(Option::isPresent).map(Option::get)
.collect(Collectors.toList());
}
public static List<String> convertKeysToStringList(List<HoodieKey> keys) {
return keys.stream()
.map(hr -> "{\"_row_key\":\"" + hr.getRecordKey() + "\",\"partition\":\"" + hr.getPartitionPath() + "\"}")
.collect(Collectors.toList());
}
public static class NoOpBulkInsertPartitioner<T extends HoodieRecordPayload>
implements BulkInsertPartitioner<T> {
@Override
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
return records;
}
@Override
public boolean arePartitionRecordsSorted() {
return false;
}
}
public static Schema getStructTypeExampleSchema() throws IOException {
return new Schema.Parser().parse(FileIOUtils.readAsUTFString(DataSourceTestUtils.class.getResourceAsStream("/exampleSchema.txt")));
}
public static List<Row> generateRandomRows(int count) {
Random random = new Random();
List<Row> toReturn = new ArrayList<>();
List<String> partitions = Arrays.asList(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH});
for (int i = 0; i < count; i++) {
Object[] values = new Object[3];
values[0] = UUID.randomUUID().toString();
values[1] = partitions.get(random.nextInt(3));
values[2] = new Date().getTime();
toReturn.add(RowFactory.create(values));
}
return toReturn;
}
}

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.testutils;
import org.apache.hudi.AvroConversionHelper;
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.sql.Row;
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
public class KeyGeneratorTestUtilities {
public static String exampleSchema = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ "{\"name\": \"pii_col\", \"type\": \"string\"}]}";
public static final String TEST_STRUCTNAME = "test_struct_name";
public static final String TEST_RECORD_NAMESPACE = "test_record_namespace";
public static Schema schema = new Schema.Parser().parse(exampleSchema);
public static StructType structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
public GenericRecord getRecord() {
GenericRecord record = new GenericData.Record(new Schema.Parser().parse(exampleSchema));
record.put("timestamp", 4357686);
record.put("_row_key", "key1");
record.put("ts_ms", "2020-03-21");
record.put("pii_col", "pi");
return record;
}
public static Row getRow(GenericRecord record) {
return getRow(record, schema, structType);
}
public static Row getRow(GenericRecord record, Schema schema, StructType structType) {
Function1<Object, Object> converterFn = AvroConversionHelper.createConverterToRow(schema, structType);
Row row = (Row) converterFn.apply(record);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {
values[i] = row.get(i);
}
return new GenericRowWithSchema(values, structType);
}
}

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.
*/
{
"namespace": "example.schema",
"type": "record",
"name": "trip",
"fields": [
{
"name": "_row_key",
"type": "string"
},
{
"name": "partition",
"type": "string"
},
{
"name": "ts",
"type": ["long", "null"]
}
]
}

View File

@@ -17,13 +17,18 @@
package org.apache.hudi
import java.util
import org.apache.avro.generic.GenericRecord
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, OverwriteWithLatestAvroPayload}
import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, HoodieKey, OverwriteWithLatestAvroPayload}
import org.apache.hudi.common.testutils.SchemaTestUtil
import org.apache.hudi.common.util.Option
import org.apache.hudi.exception.{HoodieException, HoodieKeyException}
import org.apache.hudi.keygen.{ComplexKeyGenerator, GlobalDeleteKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.keygen._
import org.apache.hudi.testutils.KeyGeneratorTestUtilities
import org.apache.spark.sql.Row
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{BeforeEach, Test}
import org.scalatest.Assertions.fail
@@ -34,14 +39,18 @@ import org.scalatest.Assertions.fail
class TestDataSourceDefaults {
val schema = SchemaTestUtil.getComplexEvolvedSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
var baseRecord: GenericRecord = _
var baseRow: Row = _
val testStructName = "testStructName"
val testNamespace = "testNamespace"
@BeforeEach def initialize(): Unit = {
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1")
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
}
private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
@@ -51,11 +60,16 @@ class TestDataSourceDefaults {
}
@Test def testSimpleKeyGenerator() = {
// top level, valid fields
val hk1 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")).getKey(baseRecord)
var keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
val hk1 = keyGen.getKey(baseRecord)
assertEquals("field1", hk1.getRecordKey)
assertEquals("name1", hk1.getPartitionPath)
assertEquals("field1", keyGen.getRecordKey(baseRow))
assertEquals("name1", keyGen.getPartitionPath(baseRow))
// partition path field not specified
try {
val props = new TypedProperties()
@@ -64,7 +78,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// partition path field not specified using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
val keyGen = new SimpleKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// recordkey field not specified
@@ -75,7 +101,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// recordkey field not specified using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
val keyGen = new SimpleKeyGenerator(props)
keyGen.getPartitionPath(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// nested field as record key and partition path
@@ -91,7 +129,7 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
// do nothing
}
// if partition path can't be found, return default partition path
@@ -99,22 +137,44 @@ class TestDataSourceDefaults {
.getKey(baseRecord)
assertEquals("default", hk3.getPartitionPath)
// if partition path can't be found, return default partition path using row
keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
val hk3_row = keyGen.getPartitionPath(baseRow)
assertEquals("default", hk3_row)
// if enable hive style partitioning
val hk4 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")).getKey(baseRecord)
assertEquals("name=name1", hk4.getPartitionPath)
// if enable hive style partitioning using row
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true"))
val hk4_row = keyGen.getPartitionPath(baseRow)
assertEquals("name=name1", hk4_row)
// if partition is null, return default partition path
baseRecord.put("name", "")
val hk5 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
.getKey(baseRecord)
assertEquals("default", hk5.getPartitionPath)
// if partition is null, return default partition path using Row
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
val hk5_row = keyGen.getPartitionPath(baseRow)
assertEquals("default", hk5_row)
// if partition is empty, return default partition path
baseRecord.put("name", null)
val hk6 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
.getKey(baseRecord)
assertEquals("default", hk6.getPartitionPath)
// if partition is empty, return default partition path using Row
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
val hk6_row = keyGen.getPartitionPath(baseRow)
assertEquals("default", hk6_row)
// if record key is empty, throw error
try {
baseRecord.put("field1", "")
@@ -125,7 +185,21 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// do nothing
}
// if record key is empty, throw error. Using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
keyGen = new SimpleKeyGenerator(props)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
// if record key is null, throw error
@@ -138,16 +212,51 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// do nothing
}
// if record key is null, throw error. Using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
keyGen = new SimpleKeyGenerator(props)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
}
@Test def testUserDefinedKeyGeneratorWorksWithRows(): Unit = {
val keyGen = new UserDefinedKeyGenerator(getKeyConfig("field1", "name", "false"))
assertEquals("field1", keyGen.getRecordKey(baseRow))
assertEquals("name1", keyGen.getPartitionPath(baseRow))
}
class UserDefinedKeyGenerator(props: TypedProperties) extends KeyGenerator(props) {
val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY)
override def getKey(record: GenericRecord): HoodieKey = {
new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true),
HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true))
}
}
@Test def testComplexKeyGenerator() = {
// top level, valid fields
val hk1 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
var keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk1 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
assertEquals("field1/name1", hk1.getPartitionPath)
// top level, valid fields with Row
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
// partition path field not specified
try {
val props = new TypedProperties()
@@ -156,7 +265,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// partition path field not specified using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
val keyGen = new ComplexKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// recordkey field not specified
@@ -167,15 +288,31 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// recordkey field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
val keyGen = new ComplexKeyGenerator(props)
keyGen.getPartitionPath(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// nested field as record key and partition path
val hk2 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
.getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
val hk2 = keyGen.getKey(baseRecord)
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", hk2.getRecordKey)
assertEquals("UserId1@001/false", hk2.getPartitionPath)
// nested field as record key and partition path
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow))
assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow))
// Nested record key not found
try {
new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
@@ -183,31 +320,57 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
// do nothing
}
// Nested record key not found
try {
val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
}
// if partition path can't be found, return default partition path
val hk3 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
.getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
val hk3 = keyGen.getKey(baseRecord)
assertEquals("default", hk3.getPartitionPath)
assertEquals("default", keyGen.getPartitionPath(baseRow))
// if enable hive style partitioning
val hk4 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true"))
val hk4 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk4.getRecordKey)
assertEquals("field1=field1/name=name1", hk4.getPartitionPath)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow))
// if one part of the record key is empty, replace with "__empty__"
baseRecord.put("name", "")
val hk5 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk5 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__empty__", hk5.getRecordKey)
assertEquals("field1/default", hk5.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
// if one part of the record key is null, replace with "__null__"
baseRecord.put("name", null)
val hk6 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk6 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__null__", hk6.getRecordKey)
assertEquals("field1/default", hk6.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
// if all parts of the composite record key are null/empty, throw error
try {
baseRecord.put("name", "")
@@ -219,46 +382,89 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// do nothing
}
// if all parts of the composite record key are null/empty, throw error
try {
baseRecord.put("name", "")
baseRecord.put("field1", null)
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
keyGen = new ComplexKeyGenerator(props)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
// reset name and field1 values.
baseRecord.put("name", "name1")
baseRecord.put("field1", "field1")
val hk7 = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false"))
val hk7 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk7.getRecordKey)
assertEquals("field1/name1", hk7.getPartitionPath)
val hk8 = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false")).getKey(baseRecord)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false"))
val hk8 = keyGen.getKey(baseRecord)
assertEquals("field1:field1", hk8.getRecordKey)
assertEquals("field1", hk8.getPartitionPath)
assertEquals("field1:field1", keyGen.getRecordKey(baseRow))
assertEquals("field1", keyGen.getPartitionPath(baseRow))
}
@Test def testGlobalDeleteKeyGenerator() = {
// top level, partition value included but not actually used
val hk1 = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
var keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk1 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
assertEquals("", hk1.getPartitionPath)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// top level, partition value not included
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
val hk2 = new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
keyGen = new GlobalDeleteKeyGenerator(props)
val hk2 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk2.getRecordKey)
assertEquals("", hk2.getPartitionPath)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// if one part of the record key is empty, replace with "__empty__"
baseRecord.put("name", "")
val hk3 = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk3 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__empty__", hk3.getRecordKey)
assertEquals("", hk3.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// if one part of the record key is null, replace with "__null__"
baseRecord.put("name", null)
val hk4 = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk4 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__null__", hk4.getRecordKey)
assertEquals("", hk4.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// recordkey field not specified
try {
val props = new TypedProperties()
@@ -267,7 +473,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// recordkey field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
val keyGen = new GlobalDeleteKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// Nested record key not found
@@ -277,7 +495,17 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
// do nothing
}
// Nested record key not found
try {
val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
}
// if all parts of the composite record key are null/empty, throw error
@@ -292,6 +520,21 @@ class TestDataSourceDefaults {
case e: HoodieKeyException =>
// do nothing
}
// if all parts of the composite record key are null/empty, throw error
try {
baseRecord.put("name", "")
baseRecord.put("field1", null)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
val keyGen = new GlobalDeleteKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
}
@Test def testOverwriteWithLatestAvroPayload() = {

View File

@@ -17,24 +17,28 @@
package org.apache.hudi.functional
import java.util
import java.util.{Date, UUID}
import org.apache.commons.io.FileUtils
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.HoodieSparkSqlWriter
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils}
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.scalatest.{FunSuite, Matchers}
class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
test("Parameters With Write Defaults") {
val originals = HoodieSparkSqlWriter.parametersWithWriteDefaults(Map.empty)
val originals = HoodieWriterUtils.parametersWithWriteDefaults(Map.empty)
val rhsKey = "hoodie.right.hand.side.key"
val rhsVal = "hoodie.right.hand.side.val"
val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
val modified = HoodieSparkSqlWriter.parametersWithWriteDefaults(modifier)
val modified = HoodieWriterUtils.parametersWithWriteDefaults(modifier)
val matcher = (k: String, v: String) => modified(k) should be(v)
originals foreach {
@@ -77,7 +81,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4")
val fooTableParams = HoodieSparkSqlWriter.parametersWithWriteDefaults(fooTableModifier)
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val dataFrame = session.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame)
@@ -86,7 +90,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
HoodieWriteConfig.TABLE_NAME -> "hoodie_bar_tbl",
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4")
val barTableParams = HoodieSparkSqlWriter.parametersWithWriteDefaults(barTableModifier)
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
val dataFrame2 = session.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableParams, dataFrame2))
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
@@ -101,6 +105,129 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
}
}
test("test bulk insert dataset with datasource impl") {
val session = SparkSession.builder()
.appName("test_bulk_insert_datasource")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate()
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
try {
val sqlContext = session.sqlContext
val sc = session.sparkContext
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val records = DataSourceTestUtils.generateRandomRows(100)
val recordsSeq = convertRowListToSeq(records)
val df = session.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
// collect all parition paths to issue read of parquet files
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
// Check the entire dataset has all records still
val fullPartitionPaths = new Array[String](3)
for (i <- 0 until fullPartitionPaths.length) {
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
}
// fetch all records from parquet files generated from write to hudi
val actualDf = session.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
// remove metadata columns so that expected and actual DFs can be compared as is
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
assert(df.except(trimmedDf).count() == 0)
} finally {
session.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
test("test bulk insert dataset with datasource impl multiple rounds") {
val session = SparkSession.builder()
.appName("test_bulk_insert_datasource")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate()
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
try {
val sqlContext = session.sqlContext
val sc = session.sparkContext
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
val fullPartitionPaths = new Array[String](3)
for (i <- 0 to 2) {
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
}
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
var totalExpectedDf = session.createDataFrame(sc.emptyRDD[Row], structType)
for (_ <- 0 to 2) {
// generate the inserts
val records = DataSourceTestUtils.generateRandomRows(200)
val recordsSeq = convertRowListToSeq(records)
val df = session.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
// Fetch records from entire dataset
val actualDf = session.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
// remove metadata columns so that expected and actual DFs can be compared as is
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
// find total df (union from multiple rounds)
totalExpectedDf = totalExpectedDf.union(df)
// find mismatch between actual and expected df
assert(totalExpectedDf.except(trimmedDf).count() == 0)
}
} finally {
session.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
case class Test(uuid: String, ts: Long)
import scala.collection.JavaConverters
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
}

View File

@@ -26,4 +26,6 @@
<!-- Member Names expected to start with "_" -->
<suppress checks="naming" files="TestRecord.java" lines="1-9999"/>
<suppress checks="IllegalImport" files="Option.java" />
<!-- java.util.Optional part of DataSource V2 API signature -->
<suppress checks="IllegalImport" files="DefaultSource.java" />
</suppressions>