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

@@ -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

@@ -1,110 +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 java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.util.List;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieKeyException;
public class KeyGenUtils {
protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
protected static final String DEFAULT_PARTITION_PATH = "default";
protected static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
public static String getRecordKey(GenericRecord record, List<String> recordKeyFields) {
boolean keyIsNullEmpty = true;
StringBuilder recordKey = new StringBuilder();
for (String recordKeyField : recordKeyFields) {
String recordKeyValue = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true);
if (recordKeyValue == null) {
recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ",");
} else if (recordKeyValue.isEmpty()) {
recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ",");
} else {
recordKey.append(recordKeyField + ":" + recordKeyValue + ",");
keyIsNullEmpty = false;
}
}
recordKey.deleteCharAt(recordKey.length() - 1);
if (keyIsNullEmpty) {
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
}
return recordKey.toString();
}
public static String getRecordPartitionPath(GenericRecord record, List<String> partitionPathFields,
boolean hiveStylePartitioning, boolean encodePartitionPath) {
StringBuilder partitionPath = new StringBuilder();
for (String partitionPathField : partitionPathFields) {
String fieldVal = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
if (fieldVal == null || fieldVal.isEmpty()) {
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH
: DEFAULT_PARTITION_PATH);
} else {
if (encodePartitionPath) {
try {
fieldVal = URLEncoder.encode(fieldVal, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal);
}
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
}
partitionPath.deleteCharAt(partitionPath.length() - 1);
return partitionPath.toString();
}
public static String getRecordKey(GenericRecord record, String recordKeyField) {
String recordKey = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true);
if (recordKey == null || recordKey.isEmpty()) {
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
}
return recordKey;
}
public static String getPartitionPath(GenericRecord record, String partitionPathField,
boolean hiveStylePartitioning, boolean encodePartitionPath) {
String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
if (partitionPath == null || partitionPath.isEmpty()) {
partitionPath = DEFAULT_PARTITION_PATH;
}
if (encodePartitionPath) {
try {
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
if (hiveStylePartitioning) {
partitionPath = partitionPathField + "=" + partitionPath;
}
return partitionPath;
}
}

View File

@@ -18,37 +18,24 @@
package org.apache.hudi.keygen;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.spark.sql.Row;
import java.io.Serializable;
import java.util.List;
/**
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
* Represents the interface key generators need to adhere to.
*/
public abstract class KeyGenerator implements Serializable {
public interface KeyGeneratorInterface extends Serializable {
protected transient TypedProperties config;
HoodieKey getKey(GenericRecord record);
protected KeyGenerator(TypedProperties config) {
this.config = config;
}
List<String> getRecordKeyFieldNames();
/**
* Generate a Hoodie Key out of provided generic record.
*/
public abstract HoodieKey getKey(GenericRecord record);
String getRecordKey(Row row);
String getPartitionPath(Row row);
/**
* Used during bootstrap, to project out only the record key fields from bootstrap source dataset.
*
* @return list of field names, when concatenated make up the record key.
*/
public List<String> getRecordKeyFieldNames() {
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
+ "Please override this method in your custom key generator.");
}
}

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