From d491a32d85239b30b4321023a5001979b12c6abb Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Sun, 5 Jul 2020 14:11:20 -0700 Subject: [PATCH 1/5] Bulk Insert Dataset Based Implementation using Datasource to improve performance --- .../org/apache/hudi/cli/SparkHelpers.scala | 4 +- .../client/AbstractHoodieWriteClient.java | 12 +- .../client/HoodieInternalWriteStatus.java | 150 ++++++++ .../hudi/client/model/HoodieInternalRow.java | 243 +++++++++++++ .../apache/hudi/config/HoodieWriteConfig.java | 4 +- .../apache/hudi/io/HoodieRowCreateHandle.java | 203 +++++++++++ .../io/storage/HoodieAvroParquetConfig.java | 36 ++ ...nfig.java => HoodieBaseParquetConfig.java} | 23 +- .../io/storage/HoodieFileWriterFactory.java | 2 +- .../storage/HoodieInternalRowFileWriter.java | 46 +++ .../HoodieInternalRowFileWriterFactory.java | 79 +++++ .../HoodieInternalRowParquetWriter.java | 72 ++++ .../hudi/io/storage/HoodieParquetWriter.java | 2 +- .../io/storage/HoodieRowParquetConfig.java | 34 ++ .../storage/HoodieRowParquetWriteSupport.java | 89 +++++ .../hudi/keygen/BuiltinKeyGenerator.java | 88 ----- .../org/apache/hudi/keygen/KeyGenerator.java | 30 ++ .../client/TestHoodieInternalWriteStatus.java | 87 +++++ .../client/model/TestHoodieInternalRow.java | 239 +++++++++++++ .../hudi/io/TestHoodieRowCreateHandle.java | 231 +++++++++++++ .../TestHoodieInternalRowParquetWriter.java | 117 +++++++ .../hudi/testutils/HoodieClientTestUtils.java | 4 +- .../testutils/HoodieDatasetTestUtils.java | 175 ++++++++++ .../hudi/common/model/HoodieRecord.java | 8 + .../hudi/common/model/WriteOperationType.java | 1 + .../resources/timestamp-test-evolved.avsc | 2 +- .../java/org/apache/hudi/DataSourceUtils.java | 49 ++- .../hudi/HoodieDatasetBulkInsertHelper.java | 110 ++++++ .../apache/hudi/internal/DefaultSource.java | 99 ++++++ .../HoodieBulkInsertDataInternalWriter.java | 119 +++++++ ...ieBulkInsertDataInternalWriterFactory.java | 52 +++ .../HoodieDataSourceInternalWriter.java | 119 +++++++ .../internal/HoodieWriterCommitMessage.java | 45 +++ .../hudi/keygen/BuiltinKeyGenerator.java | 163 +++++++++ .../hudi/keygen/ComplexKeyGenerator.java | 25 +- .../hudi/keygen/CustomKeyGenerator.java | 44 +-- .../hudi/keygen/GlobalDeleteKeyGenerator.java | 26 +- .../org/apache/hudi/keygen/KeyGenUtils.java | 0 .../keygen/NonpartitionedKeyGenerator.java | 17 +- .../hudi/keygen/RowKeyGeneratorHelper.java | 203 +++++++++++ .../hudi/keygen/SimpleKeyGenerator.java | 22 +- .../keygen/TimestampBasedKeyGenerator.java | 102 ++++-- .../org/apache/hudi/DataSourceOptions.scala | 4 +- .../scala/org/apache/hudi/DefaultSource.scala | 4 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 74 ++-- .../org/apache/hudi/HoodieWriterUtils.scala | 73 ++++ .../TestHoodieDatasetBulkInsertHelper.java | 156 +++++++++ ...estHoodieBulkInsertDataInternalWriter.java | 213 ++++++++++++ .../TestHoodieDataSourceInternalWriter.java | 322 ++++++++++++++++++ .../TestTimestampBasedKeyGenerator.java | 179 ++++++---- .../hudi/testutils/DataSourceTestUtils.java | 106 ++++++ .../src/test/resources/exampleSchema.txt | 36 ++ .../apache/hudi/TestDataSourceDefaults.scala | 314 +++++++++++++++-- .../HoodieSparkSqlWriterSuite.scala | 138 +++++++- style/checkstyle-suppressions.xml | 2 + 55 files changed, 4435 insertions(+), 362 deletions(-) create mode 100644 hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java rename hudi-client/src/main/java/org/apache/hudi/io/storage/{HoodieParquetConfig.java => HoodieBaseParquetConfig.java} (81%) create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java create mode 100644 hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java delete mode 100644 hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java create mode 100644 hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java create mode 100644 hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java create mode 100644 hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java create mode 100644 hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java create mode 100644 hudi-client/src/test/java/org/apache/hudi/testutils/HoodieDatasetTestUtils.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java create mode 100644 hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java rename {hudi-client => hudi-spark}/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java (100%) create mode 100644 hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java create mode 100644 hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala create mode 100644 hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java create mode 100644 hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java create mode 100644 hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java create mode 100644 hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java create mode 100644 hudi-spark/src/test/resources/exampleSchema.txt diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala index 7de5f42f51bde..6859f7038c731 100644 --- a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala +++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala @@ -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) diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 81b0161213c0e..cbb32711132e9 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -95,20 +95,20 @@ public boolean commit(String instantTime, JavaRDD writeStatuses) { */ public boolean commit(String instantTime, JavaRDD writeStatuses, Option> extraMetadata) { - HoodieTableMetaClient metaClient = createMetaClient(false); - return commit(instantTime, writeStatuses, extraMetadata, metaClient.getCommitActionType()); + List stats = writeStatuses.map(WriteStatus::getStat).collect(); + return commitStat(instantTime, stats, extraMetadata); } - private boolean commit(String instantTime, JavaRDD writeStatuses, - Option> extraMetadata, String actionType) { - + // fixme(bulkinsertv2) this name is ughh + public boolean commitStat(String instantTime, List stats, Option> 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 table = HoodieTable.create(config, hadoopConf); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - List stats = writeStatuses.map(WriteStatus::getStat).collect(); stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); // Finalize write diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java b/hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java new file mode 100644 index 0000000000000..87a117bb595e9 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java @@ -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 successRecordKeys = new ArrayList<>(); + private List> 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 getSuccessRecordKeys() { + return successRecordKeys; + } + + public long getFailedRowsSize() { + return failedRecordKeys.size(); + } + + public List> getFailedRecordKeys() { + return failedRecordKeys; + } + + public void setFailedRecordKeys(List> 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 successRecordKeys) { + this.successRecordKeys = successRecordKeys; + } + + @Override + public String toString() { + return "PartitionPath " + partitionPath + ", FileID " + fileId + ", Success records " + + totalRecords + ", errored Rows " + totalErrorRecords + + ", global error " + (globalError != null); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java new file mode 100644 index 0000000000000..241ed35205bcf --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java @@ -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 getHoodieColumnVal(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 == getHoodieColumnVal(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(getHoodieColumnVal(ordinal).getBytes()); + } + return row.getUTF8String(ordinal); + } + + @Override + public String getString(int ordinal) { + if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { + return new String(getHoodieColumnVal(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(getHoodieColumnVal(ordinal).getBytes()); + } + return row.get(ordinal, dataType); + } + + @Override + public InternalRow copy() { + return new HoodieInternalRow(commitTime, commitSeqNumber, recordKey, partitionPath, fileName, row.copy()); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 89efc4e477055..65598e7badf4e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -776,7 +776,9 @@ public Builder withPath(String basePath) { } public Builder withSchema(String schemaStr) { - props.setProperty(AVRO_SCHEMA, schemaStr); + if (null != schemaStr) { + props.setProperty(AVRO_SCHEMA, schemaStr); + } return this; } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java new file mode 100644 index 0000000000000..723d9f9ad622f --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java @@ -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); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java new file mode 100644 index 0000000000000..f934a8a83784f --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java @@ -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 { + + public HoodieAvroParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName, + int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, + double compressionRatio) { + super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetConfig.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java similarity index 81% rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetConfig.java rename to hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java index ca63ee2a6d25c..6e6f66c5eac6d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetConfig.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java @@ -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 + */ +public class HoodieBaseParquetConfig { + 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 HoodieParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodec this.compressionRatio = compressionRatio; } - public HoodieAvroWriteSupport getWriteSupport() { - return writeSupport; - } - public CompressionCodecName getCompressionCodecName() { return compressionCodecName; } @@ -71,4 +68,8 @@ public Configuration getHadoopConf() { public double getCompressionRatio() { return compressionRatio; } + + public T getWriteSupport() { + return writeSupport; + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 90566fba0bd57..0fab31e42433d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -58,7 +58,7 @@ private static HoodieFi 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()); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java new file mode 100644 index 0000000000000..6ab80b6987ba3 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java @@ -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; +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java new file mode 100644 index 0000000000000..cb238bb31cada --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java @@ -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())); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java new file mode 100644 index 0000000000000..7d0516300f751 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java @@ -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 + 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(); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java index 8c4c7e63f1b26..5c8f99ffad89d 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java @@ -52,7 +52,7 @@ public class HoodieParquetWriter { + + public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName, + int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, + double compressionRatio) { + super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java new file mode 100644 index 0000000000000..f6cef204aae6f --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java @@ -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 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; + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java deleted file mode 100644 index 7c3edf747b557..0000000000000 --- a/hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ /dev/null @@ -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 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 getPartitionPathFields() { - throw new IllegalStateException("This method is expected to be overridden by subclasses"); - } - - @Override - public final List 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()); - } -} diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java index 1a798af7485e4..351584156c177 100644 --- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java +++ b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java @@ -22,6 +22,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; import java.io.Serializable; import java.util.List; @@ -51,4 +53,32 @@ public List getRecordKeyFieldNames() { throw new UnsupportedOperationException("Bootstrap not supported for key generator. " + "Please override this method in your custom key generator."); } + + /** + * Initializes {@link KeyGenerator} for {@link Row} based operations. + * @param structType structype of the dataset. + * @param structName struct name of the dataset. + * @param recordNamespace record namespace of the dataset. + */ + public void initializeRowKeyGenerator(StructType structType, String structName, String recordNamespace) { + throw new UnsupportedOperationException("This method is invoked only for operation BULK_INSERT_DATASET. Expected to be overridden by sub classes"); + } + + /** + * 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 getRecordKeyFromRow(Row row) { + throw new UnsupportedOperationException("This method is invoked only for operation BULK_INSERT_DATASET. Expected to be overridden by sub classes"); + } + + /** + * 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 getPartitionPathFromRow(Row row) { + throw new UnsupportedOperationException("This method is invoked only for operation BULK_INSERT_DATASET. Expected to be overridden by sub classes"); + } } diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java b/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java new file mode 100644 index 0000000000000..3f69c65b660cc --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java @@ -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()); + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java b/hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java new file mode 100644 index 0000000000000..bfcb012c3748c --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java @@ -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 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 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 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)); + } + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java new file mode 100644 index 0000000000000..6f949fd724365 --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java @@ -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.HoodieDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getInternalRowWithError; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.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 fileNames = new ArrayList<>(); + List fileAbsPaths = new ArrayList<>(); + + Dataset 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 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 inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false); + List 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 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 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 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 inputRows, HoodieRowCreateHandle handle) throws IOException { + List 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 inputRows, List filenames, + List 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 result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0])); + assertRows(inputRows, result, instantTime, filenames); + } + + private void assertRows(Dataset expectedRows, Dataset actualRows, String instantTime, List 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 trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + assertEquals(0, trimmedActual.except(trimmedExpected).count()); + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java b/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java new file mode 100644 index 0000000000000..c74a83cdf9f1b --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java @@ -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.HoodieDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.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 inputRows = getRandomRows(sqlContext, size, partitionPath, false); + List internalRows = toInternalRows(inputRows, ENCODER); + + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + + // close the writer + writer.close(); + + // verify rows + Dataset 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); + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 6db6529f2929a..4aaf58514a696 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -39,7 +39,7 @@ 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 static String writeParquetFile(String basePath, String partitionPath, Str 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 = diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieDatasetTestUtils.java b/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieDatasetTestUtils.java new file mode 100644 index 0000000000000..4d571782717ac --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieDatasetTestUtils.java @@ -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 HoodieDatasetTestUtils { + + 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 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 Datasets thus generated. + */ + public static Dataset getRandomRows(SQLContext sqlContext, int count, String partitionPath, boolean isError) { + List 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 Datasets to List of {@link InternalRow}s. + * + * @param rows Datasets to be converted + * @return the List of {@link InternalRow}s thus converted. + */ + public static List toInternalRows(Dataset rows, ExpressionEncoder encoder) { + List toReturn = new ArrayList<>(); + List 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); + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index df7843f2ba628..ce2562ddc554d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -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 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 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. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index d1a53d5da2ad6..db212d116e747 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -35,6 +35,7 @@ public enum WriteOperationType { // bulk insert BULK_INSERT("bulk_insert"), BULK_INSERT_PREPPED("bulk_insert_prepped"), + BULK_INSERT_DATASET("bulk_insert_dataset"), // delete DELETE("delete"), BOOTSTRAP("bootstrap"), diff --git a/hudi-common/src/test/resources/timestamp-test-evolved.avsc b/hudi-common/src/test/resources/timestamp-test-evolved.avsc index 421c6722cd842..beb36329eabac 100644 --- a/hudi-common/src/test/resources/timestamp-test-evolved.avsc +++ b/hudi-common/src/test/resources/timestamp-test-evolved.avsc @@ -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} ] } \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 5647e65d3f129..2c2dfbe7a7997 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -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.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 static Object getNestedFieldVal(GenericRecord record, String fieldName, b 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 tablePath = TablePathUtils.getTablePath(fs, path); if (tablePath.isPresent()) { @@ -123,8 +123,7 @@ public static String getTablePath(FileSystem fs, Path[] userProvidedPaths) throw /** * 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 @@ private static boolean isLogicalTypeDate(Schema fieldSchema) { /** * Create a key generator class via reflection, passing in any configs needed. *

- * 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 static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx /** * 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 static HoodieDateTimeParser createDateTimeParser(TypedProperties props, S * Create a UserDefinedBulkInsertPartitioner class via reflection, *
* if the class name of UserDefinedBulkInsertPartitioner is configured through the HoodieWriteConfig. + * * @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass() */ private static Option createUserDefinedBulkInsertPartitioner(HoodieWriteConfig config) @@ -225,22 +220,15 @@ public static void checkRequiredProperties(TypedProperties props, List c }); } - public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath, + public static HoodieWriteConfig createHoodieConfig(String schemaStr, String basePath, String tblName, Map parameters) { boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY())); - // inline compaction is on by default for MOR 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 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 writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath).withAutoCommit(false) + return HoodieWriteConfig.newBuilder().withPath(basePath).withAutoCommit(false) .combineInput(combineInserts, true).withSchema(schemaStr).forTable(tblName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() @@ -248,12 +236,15 @@ public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String .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 parameters) { + return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true); } public static JavaRDD doWriteOperation(HoodieWriteClient client, JavaRDD hoodieRecords, - String instantTime, String operation) throws HoodieException { + String instantTime, String operation) throws HoodieException { if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) { Option userDefinedBulkInsertPartitioner = createUserDefinedBulkInsertPartitioner(client.getConfig()); @@ -267,12 +258,12 @@ public static JavaRDD doWriteOperation(HoodieWriteClient client, Ja } public static JavaRDD doDeleteOperation(HoodieWriteClient client, JavaRDD 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 +271,13 @@ public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable order /** * 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 dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords, - HoodieWriteConfig writeConfig) { + HoodieWriteConfig writeConfig) { try { HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig); return client.tagLocation(incomingHoodieRecords) @@ -300,7 +291,7 @@ public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRD @SuppressWarnings("unchecked") public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords, - Map parameters) { + Map parameters) { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build(); return dropDuplicates(jssc, incomingHoodieRecords, writeConfig); diff --git a/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java new file mode 100644 index 0000000000000..47e01d143002e --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java @@ -0,0 +1,110 @@ +/* + * 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}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 prepareHoodieDatasetForBulkInsert(SQLContext sqlContext, + HoodieWriteConfig config, Dataset rows, String structName, String recordNamespace) { + List 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(); + + keyGenerator.initializeRowKeyGenerator(structTypeForUDF, structName, recordNamespace); + + sqlContext.udf().register(RECORD_KEY_UDF_FN, (UDF1) keyGenerator::getRecordKeyFromRow, DataTypes.StringType); + + sqlContext.udf().register(PARTITION_PATH_UDF_FN, (UDF1) keyGenerator::getPartitionPathFromRow, DataTypes.StringType); + + final Dataset 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 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 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 orderedFields = Stream.concat(HoodieRecord.HOODIE_META_COLUMNS.stream().map(Column::new), + originalFields.stream()).collect(Collectors.toList()); + Dataset 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()); + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java new file mode 100644 index 0000000000000..3cdf227f458ff --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -0,0 +1,99 @@ +/* + * 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.DataSourceWriteOptions; +import org.apache.hudi.HoodieWriterUtils; +import org.apache.hudi.common.model.WriteOperationType; +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.Map; +import java.util.Optional; +import java.util.Properties; + +/** + * 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 createWriter(String writeUUID, StructType schema, SaveMode mode, + DataSourceOptions options) { + String instantTime = options.get(DataSourceWriteOptions.INSTANT_TIME()).get(); + Map paramsWithDefaults = HoodieWriterUtils.javaParametersWithWriteDefaults(options.asMap()); + Properties props = new Properties(); + props.putAll(paramsWithDefaults); + 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(), WriteOperationType.BULK_INSERT_DATASET)); + } + + 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; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..7aa0fc6a3846f --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java @@ -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}. This is used in data source implementation for bulk insert. + */ +public class HoodieBulkInsertDataInternalWriter implements DataWriter { + + 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 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++); + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java new file mode 100644 index 0000000000000..1dd0aa382cdd8 --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java @@ -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 { + + 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 createDataWriter(int partitionId, long taskId, long epochId) { + return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId, + structType); + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java new file mode 100644 index 0000000000000..6487fc184dff8 --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -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); + + 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, + WriteOperationType operationType) { + this.instantTime = instantTime; + this.writeConfig = writeConfig; + this.structType = structType; + this.operationType = operationType; + 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 createWriterFactory() { + metaClient.getActiveTimeline().transitionRequestedToInflight( + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty()); + if (WriteOperationType.BULK_INSERT_DATASET == 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 writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList()); + + try { + writeClient.commitStat(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(); + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java new file mode 100644 index 0000000000000..757000c57c1dd --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java @@ -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 writeStatuses = new ArrayList<>(); + + public HoodieWriterCommitMessage(List writeStatuses) { + this.writeStatuses = writeStatuses; + } + + public List getWriteStatuses() { + return writeStatuses; + } + + @Override + public String toString() { + return "HoodieWriterCommitMessage{" + "writeStatuses=" + writeStatuses + '}'; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java new file mode 100644 index 0000000000000..649aa03353b6e --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -0,0 +1,163 @@ +/* + * 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.AvroConversionHelper; +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.Row; +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; + +import scala.Function1; + +/** + * Base class for all the built-in key generators. Contains methods structured for + * code reuse amongst them. + */ +public abstract class BuiltinKeyGenerator extends KeyGenerator { + + private List recordKeyFields; + private List partitionPathFields; + private Map> recordKeyPositions = new HashMap<>(); + + private Map> partitionPathPositions = new HashMap<>(); + private transient Function1 converterFn = null; + protected StructType structType; + private String structName; + private String recordNamespace; + + 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)); + } + + @Override + public final List 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()); + } + + @Override + public void initializeRowKeyGenerator(StructType structType, String structName, String recordNamespace) { + // parse simple feilds + getRecordKeyFields().stream() + .filter(f -> !(f.contains("."))) + .forEach(f -> recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get())))); + // 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 -> partitionPathPositions.put(f, + Collections.singletonList((Integer) (structType.getFieldIndex(f).get())))); + // parse nested fields + getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains(".")) + .forEach(f -> partitionPathPositions.put(f, + RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false))); + } + this.structName = structName; + this.structType = structType; + this.recordNamespace = recordNamespace; + } + + /** + * 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}. + */ + @Override + public String getRecordKeyFromRow(Row row) { + if (null != converterFn) { + converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); + } + 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}. + */ + @Override + public String getPartitionPathFromRow(Row row) { + if (null != converterFn) { + converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); + } + GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); + return getKey(genericRecord).getPartitionPath(); + } + + public List getRecordKeyFields() { + return recordKeyFields; + } + + public List getPartitionPathFields() { + return partitionPathFields; + } + + protected void setRecordKeyFields(List recordKeyFields) { + this.recordKeyFields = recordKeyFields; + } + + protected void setPartitionPathFields(List partitionPathFields) { + this.partitionPathFields = partitionPathFields; + } + + protected Map> getRecordKeyPositions() { + return recordKeyPositions; + } + + protected Map> getPartitionPathPositions() { + return partitionPathPositions; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index e8996424d3cc7..7fb7ffd46e9bd 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -24,8 +24,8 @@ 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. @@ -33,17 +33,15 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator { public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":"; - - protected final List recordKeyFields; - protected final List 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.setRecordKeyFields(Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()) + .split(",")).map(String::trim).collect(Collectors.toList())); + this.setPartitionPathFields(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(), @@ -52,21 +50,22 @@ public ComplexKeyGenerator(TypedProperties props) { @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 getRecordKeyFields() { - return recordKeyFields; + public String getRecordKeyFromRow(Row row) { + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), true); } @Override - public List getPartitionPathFields() { - return partitionPathFields; + public String getPartitionPathFromRow(Row row) { + return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), + hiveStylePartitioning, getPartitionPathPositions()); } } \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java index e5cc61c0a3cce..e4eec126a8c52 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java @@ -20,36 +20,30 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; - -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.exception.HoodieDeltaStreamerException; import org.apache.hudi.exception.HoodieKeyException; +import org.apache.avro.generic.GenericRecord; + 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 / 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 recordKeyFields; - protected final List partitionPathFields; protected final TypedProperties properties; private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; private static final String SPLIT_REGEX = ":"; @@ -64,14 +58,14 @@ public enum PartitionKeyType { 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.setRecordKeyFields(Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList())); + this.setPartitionPathFields( + Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList())); } @Override public String getPartitionPath(GenericRecord record) { - if (partitionPathFields == null) { + if (getPartitionPathFields() == null) { throw new HoodieKeyException("Unable to find field names for partition path in cfg"); } @@ -79,10 +73,10 @@ public String getPartitionPath(GenericRecord record) { 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"); @@ -114,20 +108,10 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(GenericRecord record) { - if (recordKeyFields == null || recordKeyFields.isEmpty()) { + 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 getRecordKeyFields() { - return recordKeyFields; - } - - @Override - public List getPartitionPathFields() { - return partitionPathFields; + return getRecordKeyFields().size() == 1 ? new SimpleKeyGenerator(properties).getRecordKey(record) : new ComplexKeyGenerator(properties).getRecordKey(record); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index 5851a9dc7c28a..193bdc5f6104d 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -22,30 +22,27 @@ 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 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.setRecordKeyFields(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 @@ -54,12 +51,17 @@ public String getPartitionPath(GenericRecord record) { } @Override - public List getRecordKeyFields() { - return recordKeyFields; + public List getPartitionPathFields() { + return new ArrayList<>(); } @Override - public List getPartitionPathFields() { - return new ArrayList<>(); + public String getRecordKeyFromRow(Row row) { + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), true); + } + + @Override + public String getPartitionPathFromRow(Row row) { + return EMPTY_PARTITION; } -} \ No newline at end of file +} diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java similarity index 100% rename from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java rename to hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index de4f50bf93df7..253bb64855b36 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -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; @@ -33,11 +33,8 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator { private static final String EMPTY_PARTITION = ""; - protected final String recordKeyField; - public NonpartitionedKeyGenerator(TypedProperties props) { super(props); - this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); } @Override @@ -49,4 +46,14 @@ public String getPartitionPath(GenericRecord record) { public List getPartitionPathFields() { return new ArrayList<>(); } -} \ No newline at end of file + + @Override + public String getRecordKeyFromRow(Row row) { + return super.getRecordKeyFromRow(row); + } + + @Override + public String getPartitionPathFromRow(Row row) { + return EMPTY_PARTITION; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java new file mode 100644 index 0000000000000..740a89cdeb52b --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java @@ -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.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 recordKeyFields, Map> recordKeyPositions, boolean prefixFieldName) { + AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true); + String toReturn = IntStream.range(0, recordKeyFields.size()).mapToObj(idx -> { + String field = recordKeyFields.get(idx); + String val = null; + List 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 partitionPathFields, boolean hiveStylePartitioning, Map> partitionPathPositions) { + return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> { + String field = partitionPathFields.get(idx); + String val = null; + List 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 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 getNestedFieldIndices(StructType structType, String field, boolean isRecordKey) { + String[] slices = field.split("\\."); + List positions = new ArrayList<>(); + int index = 0; + int totalCount = slices.length; + while (index < totalCount) { + String slice = slices[index]; + Option 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; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index ea460b5ff980d..b81fcaed89a61 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -22,17 +22,19 @@ 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; /** * Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs. */ public class SimpleKeyGenerator extends BuiltinKeyGenerator { + @Deprecated protected final String recordKeyField; + @Deprecated protected final String partitionPathField; protected final boolean hiveStylePartitioning; @@ -45,12 +47,15 @@ public SimpleKeyGenerator(TypedProperties props) { public SimpleKeyGenerator(TypedProperties props, String partitionPathField) { super(props); - this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); - this.partitionPathField = partitionPathField; + this.setRecordKeyFields(Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()))); + this.setPartitionPathFields(Arrays.asList(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())); + // Retaining this for compatibility + this.recordKeyField = getRecordKeyFields().get(0); + this.partitionPathField = getPartitionPathFields().get(0); } @Override @@ -64,12 +69,13 @@ public String getPartitionPath(GenericRecord record) { } @Override - public List getRecordKeyFields() { - return Arrays.asList(recordKeyField); + public String getRecordKeyFromRow(Row row) { + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), false); } @Override - public List getPartitionPathFields() { - return Arrays.asList(partitionPathField); + public String getPartitionPathFromRow(Row row) { + return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), + hiveStylePartitioning, getPartitionPathPositions()); } -} \ No newline at end of file +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index b5e6fe8ec0df9..ee6ec78155d5b 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -25,10 +25,11 @@ 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.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. @@ -129,45 +134,54 @@ public String getPartitionPath(GenericRecord record) { 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); - } + + 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 ? partitionPathField + "=" + partitionPath : partitionPath; - } catch (Exception e) { - throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e); } + return hiveStylePartitioning ? partitionPathField + "=" + partitionPath : partitionPath; } private long convertLongTimeToMillis(Long partitionVal) { @@ -177,4 +191,26 @@ private long convertLongTimeToMillis(Long partitionVal) { } return MILLISECONDS.convert(partitionVal, timeUnit); } + + @Override + public String getRecordKeyFromRow(Row row) { + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), false); + } + + @Override + public String getPartitionPathFromRow(Row row) { + Object fieldVal = null; + Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, getPartitionPathPositions().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 (ParseException e) { + throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + fieldVal, e); + } + } } diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala index ec51cb6db1911..3f41b7e2396c2 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -141,6 +141,7 @@ object DataSourceWriteOptions { val UPSERT_OPERATION_OPT_VAL = "upsert" val DELETE_OPERATION_OPT_VAL = "delete" val BOOTSTRAP_OPERATION_OPT_VAL = "bootstrap" + val BULK_INSERT_DATASET_OPERATION_OPT_VAL = "bulk_insert_dataset" val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL /** @@ -297,8 +298,9 @@ object DataSourceWriteOptions { val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false" val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" 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" + // Internal configs + val INSTANT_TIME = "hoodie.instant.time" } diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index 6f42169282bcc..e8d2f6c64a8c7 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -119,13 +119,11 @@ class DefaultSource extends RelationProvider optParams: Map[String, String], df: DataFrame): BaseRelation = { val parameters = HoodieSparkSqlWriter.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, diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index d66981ebd1fd4..c3cbeb0bf22ee 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -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 @@ -62,7 +61,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 +104,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 (operation.equalsIgnoreCase(BULK_INSERT_DATASET_OPERATION_OPT_VAL)) { + val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName, + basePath, path, instantTime) + return (success, commitTime, common.util.Option.of(""), hoodieWriteClient.orNull, tableConfig) + } + // scalastyle:on val (writeStatuses, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) = if (!operation.equalsIgnoreCase(DELETE_OPERATION_OPT_VAL)) { @@ -128,14 +143,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,12 +257,37 @@ private[hudi] object HoodieSparkSqlWriter { metaSyncSuccess } + 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(INSTANT_TIME, 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)) + } + /** - * Add default options for unspecified write options keys. - * - * @param parameters - * @return - */ + * 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, @@ -298,7 +330,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 +443,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()) diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala new file mode 100644 index 0000000000000..3f6cd90347fd6 --- /dev/null +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -0,0 +1,73 @@ +/* + * 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, + HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_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 + ) ++ translateStorageTypeToTableType(parameters) + } + + def toProperties(params: Map[String, String]): TypedProperties = { + val props = new TypedProperties() + params.foreach(kv => props.setProperty(kv._1, kv._2)) + props + } +} diff --git a/hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java b/hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java new file mode 100644 index 0000000000000..12a7d20038d38 --- /dev/null +++ b/hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java @@ -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 rows = DataSourceTestUtils.generateRandomRows(10); + Dataset dataset = sqlContext.createDataFrame(rows, structType); + Dataset 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 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 getPropsAllSet() { + return getProps(true, true, true, true); + } + + private Map getProps(boolean setAll, boolean setKeyGen, boolean setRecordKey, boolean setPartitionPath) { + Map 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 rows = DataSourceTestUtils.generateRandomRows(10); + Dataset 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 + } + } +} diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..c17530fe6d56f --- /dev/null +++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java @@ -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.HoodieDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getInternalRowWithError; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.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 totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset 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 fileAbsPaths = new ArrayList<>(); + List fileNames = new ArrayList<>(); + + // verify write statuses + assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames); + + // verify rows + Dataset 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 inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false); + List 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 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 fileAbsPaths = new ArrayList<>(); + List fileNames = new ArrayList<>(); + // verify write statuses + assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames); + + // verify rows + Dataset result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0])); + assertOutput(inputRows, result, instantTime, fileNames); + } + + private void writeRows(Dataset inputRows, HoodieBulkInsertDataInternalWriter writer) throws IOException { + List internalRows = toInternalRows(inputRows, ENCODER); + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + } + + private void assertWriteStatuses(List writeStatuses, int batches, int size, List fileAbsPaths, List 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 expectedRows, Dataset actualRows, String instantTime, List 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 trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + assertEquals(0, trimmedActual.except(trimmedExpected).count()); + } +} diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java new file mode 100644 index 0000000000000..0d474ecadb788 --- /dev/null +++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java @@ -0,0 +1,322 @@ +/* + * 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.model.WriteOperationType; +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.HoodieDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.HoodieDatasetTestUtils.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, WriteOperationType.BULK_INSERT_DATASET); + DataWriter writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong()); + + List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); + List partitionPathsAbs = new ArrayList<>(); + for (String partitionPath : partitionPaths) { + partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); + } + + int size = 10 + RANDOM.nextInt(1000); + int batches = 5; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset 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 commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + Dataset 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, WriteOperationType.BULK_INSERT_DATASET); + + List commitMessages = new ArrayList<>(); + Dataset totalInputRows = null; + DataWriter 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 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 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, WriteOperationType.BULK_INSERT_DATASET); + + List commitMessages = new ArrayList<>(); + Dataset totalInputRows = null; + DataWriter 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 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 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, WriteOperationType.BULK_INSERT_DATASET); + DataWriter writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong()); + + List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); + List partitionPathsAbs = new ArrayList<>(); + for (String partitionPath : partitionPaths) { + partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); + } + + int size = 10 + RANDOM.nextInt(100); + int batches = 1; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset 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 commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + // commit 1st batch + dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + Dataset 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, WriteOperationType.BULK_INSERT_DATASET); + 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 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 inputRows, DataWriter writer) throws IOException { + List internalRows = toInternalRows(inputRows, ENCODER); + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + } + + private void assertWriteStatuses(List 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 expectedRows, Dataset 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 trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + assertEquals(0, trimmedActual.except(trimmedExpected).count()); + } +} diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index b8e0d2921e83e..fab81ce65e5f1 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -18,31 +18,47 @@ 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; + private String testStructName = "testStructName"; + private String testNamespace = "testNamespace"; + @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 +77,17 @@ private TypedProperties getBaseKeyConfig(String timestampType, String dateFormat return properties; } + private Row genericRecordToRow(GenericRecord baseRecord) { + Function1 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 +115,47 @@ public void testTimestampBasedKeyGenerator() throws IOException { // 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); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020-01-06 12", keyGen.getPartitionPathFromRow(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 + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020-01-06 04", keyGen.getPartitionPathFromRow(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); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020-01-06 12", keyGen.getPartitionPathFromRow(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 + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020-01-06 12", keyGen.getPartitionPathFromRow(baseRow)); } @Test @@ -116,20 +165,26 @@ public void testScalar() throws IOException { // 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); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2024-10-04 12", keyGen.getPartitionPathFromRow(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"); + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "GMT"); HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); @@ -139,12 +194,12 @@ public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsU 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", - ""); + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + ""); HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); @@ -154,12 +209,12 @@ public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAs 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"); + "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); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); @@ -169,12 +224,12 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone 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"); + "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); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); @@ -184,12 +239,12 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAs 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"); + "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); Assertions.assertEquals("2020040118", hk1.getPartitionPath()); @@ -199,12 +254,12 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputT 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"); + "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); Assertions.assertEquals("2020040118", hk1.getPartitionPath()); @@ -214,12 +269,12 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_Outpu 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"); + "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); Assertions.assertEquals("2020040109", hk1.getPartitionPath()); @@ -229,12 +284,12 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() { 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"); + "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)); } @@ -242,12 +297,12 @@ public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() { 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"); + "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); Assertions.assertEquals("04/01/2020", hk1.getPartitionPath()); diff --git a/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java b/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java new file mode 100644 index 0000000000000..4c707f588ac55 --- /dev/null +++ b/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java @@ -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 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 convertToStringList(List records) { + return records.stream().map(DataSourceTestUtils::convertToString).filter(Option::isPresent).map(Option::get) + .collect(Collectors.toList()); + } + + public static List convertKeysToStringList(List keys) { + return keys.stream() + .map(hr -> "{\"_row_key\":\"" + hr.getRecordKey() + "\",\"partition\":\"" + hr.getPartitionPath() + "\"}") + .collect(Collectors.toList()); + } + + public static class NoOpBulkInsertPartitioner + implements BulkInsertPartitioner { + + @Override + public JavaRDD> repartitionRecords(JavaRDD> 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 generateRandomRows(int count) { + Random random = new Random(); + List toReturn = new ArrayList<>(); + List 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; + } +} diff --git a/hudi-spark/src/test/resources/exampleSchema.txt b/hudi-spark/src/test/resources/exampleSchema.txt new file mode 100644 index 0000000000000..c7c0ff73792ec --- /dev/null +++ b/hudi-spark/src/test/resources/exampleSchema.txt @@ -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"] + } + ] +} diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index a1229fbf5b0ed..1d7d242451c93 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -24,6 +24,8 @@ 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.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{BeforeEach, Test} import org.scalatest.Assertions.fail @@ -34,13 +36,28 @@ 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 = genericRecordToRow(baseRecord) } + private def genericRecordToRow(baseRecord: GenericRecord): Row = { + val convertor = AvroConversionHelper.createConverterToRow(schema, structType) + val row = convertor.apply(baseRecord).asInstanceOf[Row] + val fieldCount = structType.fieldNames.length + val values = new Array[Any](fieldCount) + for (i <- 0 until fieldCount) { + values(i) = row.get(i) + } + new GenericRowWithSchema(values, structType) + } private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = { val props = new TypedProperties() @@ -51,11 +68,17 @@ 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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("name1", keyGen.getPartitionPathFromRow(baseRow)) + // partition path field not specified try { val props = new TypedProperties() @@ -64,7 +87,20 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: IllegalArgumentException => + // do nothing } // recordkey field not specified @@ -75,7 +111,20 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getPartitionPathFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: IllegalArgumentException => + // do nothing } // nested field as record key and partition path @@ -91,7 +140,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 +148,48 @@ 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")) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + val hk3_row = keyGen.getPartitionPathFromRow(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")) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + val hk4_row = keyGen.getPartitionPathFromRow(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")) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = genericRecordToRow(baseRecord) + val hk5_row = keyGen.getPartitionPathFromRow(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")) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = genericRecordToRow(baseRecord) + val hk6_row = keyGen.getPartitionPathFromRow(baseRow) + assertEquals("default", hk6_row) + // if record key is empty, throw error try { baseRecord.put("field1", "") @@ -125,7 +200,22 @@ 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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = genericRecordToRow(baseRecord) + keyGen.getRecordKeyFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: HoodieKeyException => + // do nothing } // if record key is null, throw error @@ -138,16 +228,37 @@ 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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = genericRecordToRow(baseRecord) + keyGen.getRecordKeyFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: HoodieKeyException => + // do nothing } } @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 + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("field1/name1", keyGen.getPartitionPathFromRow(baseRow)) + // partition path field not specified try { val props = new TypedProperties() @@ -156,7 +267,20 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: IllegalArgumentException => + // do nothing } // recordkey field not specified @@ -167,15 +291,32 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + 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 + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("UserId1@001/false", keyGen.getPartitionPathFromRow(baseRow)) + // Nested record key not found try { new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) @@ -183,31 +324,62 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("default", keyGen.getPartitionPathFromRow(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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("field1=field1/name=name1", keyGen.getPartitionPathFromRow(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 = genericRecordToRow(baseRecord) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:__empty__", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("field1/default", keyGen.getPartitionPathFromRow(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 = genericRecordToRow(baseRecord) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:__null__", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("field1/default", keyGen.getPartitionPathFromRow(baseRow)) + // if all parts of the composite record key are null/empty, throw error try { baseRecord.put("name", "") @@ -219,46 +391,96 @@ 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 = genericRecordToRow(baseRecord) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(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 = genericRecordToRow(baseRecord) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("field1/name1", keyGen.getPartitionPathFromRow(baseRow)) + + keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false")) + val hk8 = keyGen.getKey(baseRecord) assertEquals("field1:field1", hk8.getRecordKey) assertEquals("field1", hk8.getPartitionPath) + + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("field1", keyGen.getPartitionPathFromRow(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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("", keyGen.getPartitionPathFromRow(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) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("", keyGen.getPartitionPathFromRow(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 = genericRecordToRow(baseRecord) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:__empty__", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("", keyGen.getPartitionPathFromRow(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 = genericRecordToRow(baseRecord) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1:field1,name:__null__", keyGen.getRecordKeyFromRow(baseRow)) + assertEquals("", keyGen.getPartitionPathFromRow(baseRow)) + // recordkey field not specified try { val props = new TypedProperties() @@ -267,7 +489,20 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: IllegalArgumentException => + // do nothing } // Nested record key not found @@ -277,7 +512,18 @@ 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(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 +538,22 @@ 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 = genericRecordToRow(baseRecord) + val props = new TypedProperties() + props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name") + val keyGen = new GlobalDeleteKeyGenerator(props) + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getRecordKeyFromRow(baseRow) + fail("Should have errored out") + } catch { + case e: HoodieKeyException => + // do nothing + } } @Test def testOverwriteWithLatestAvroPayload() = { diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala index feeec96e0d9d4..e5b3ecd34786f 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala @@ -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,128 @@ 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_DATASET_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) + + // 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_DATASET_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 + } diff --git a/style/checkstyle-suppressions.xml b/style/checkstyle-suppressions.xml index 30dc51274e3a3..a0a7680db4f10 100644 --- a/style/checkstyle-suppressions.xml +++ b/style/checkstyle-suppressions.xml @@ -26,4 +26,6 @@ + + From 06c1370f3805ba3667ea0f61f414c21787762772 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Tue, 11 Aug 2020 06:42:12 -0700 Subject: [PATCH 2/5] Code review comments, cleanup, fixes, restructuring - Clean up KeyGenerator classes and fix test failures --- .../client/AbstractHoodieWriteClient.java | 5 +- .../hudi/client/model/HoodieInternalRow.java | 10 +-- .../apache/hudi/config/HoodieWriteConfig.java | 4 +- .../org/apache/hudi/keygen/KeyGenerator.java | 10 +-- .../hudi/common/model/WriteOperationType.java | 1 - .../java/org/apache/hudi/DataSourceUtils.java | 8 +- .../hudi/HoodieDatasetBulkInsertHelper.java | 11 ++- .../apache/hudi/internal/DefaultSource.java | 2 +- .../HoodieDataSourceInternalWriter.java | 9 +-- .../hudi/keygen/BuiltinKeyGenerator.java | 19 ++--- .../hudi/keygen/ComplexKeyGenerator.java | 12 +-- .../hudi/keygen/CustomKeyGenerator.java | 16 ++-- .../hudi/keygen/GlobalDeleteKeyGenerator.java | 6 +- .../keygen/NonpartitionedKeyGenerator.java | 6 +- .../hudi/keygen/SimpleKeyGenerator.java | 21 ++--- .../keygen/TimestampBasedKeyGenerator.java | 8 +- .../scala/org/apache/hudi/DefaultSource.scala | 2 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 39 +--------- .../org/apache/hudi/HoodieWriterUtils.scala | 5 +- .../TestHoodieDataSourceInternalWriter.java | 10 +-- .../TestTimestampBasedKeyGenerator.java | 10 +-- .../apache/hudi/TestDataSourceDefaults.scala | 78 +++++++++---------- 22 files changed, 119 insertions(+), 173 deletions(-) diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index cbb32711132e9..d0526b0236026 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -96,11 +96,10 @@ public boolean commit(String instantTime, JavaRDD writeStatuses) { public boolean commit(String instantTime, JavaRDD writeStatuses, Option> extraMetadata) { List stats = writeStatuses.map(WriteStatus::getStat).collect(); - return commitStat(instantTime, stats, extraMetadata); + return commitStats(instantTime, stats, extraMetadata); } - // fixme(bulkinsertv2) this name is ughh - public boolean commitStat(String instantTime, List stats, Option> extraMetadata) { + public boolean commitStats(String instantTime, List stats, Option> extraMetadata) { LOG.info("Committing " + instantTime); HoodieTableMetaClient metaClient = createMetaClient(false); String actionType = metaClient.getCommitActionType(); diff --git a/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java index 241ed35205bcf..5fcd1dcbfb9b2 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java @@ -118,7 +118,7 @@ public void update(int i, Object value) { } } - private String getHoodieColumnVal(int ordinal) { + private String getMetaColumnVal(int ordinal) { switch (ordinal) { case 0: { return commitTime; @@ -142,7 +142,7 @@ private String getHoodieColumnVal(int ordinal) { @Override public boolean isNullAt(int ordinal) { if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { - return null == getHoodieColumnVal(ordinal); + return null == getMetaColumnVal(ordinal); } return row.isNullAt(ordinal); } @@ -190,7 +190,7 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { @Override public UTF8String getUTF8String(int ordinal) { if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { - return UTF8String.fromBytes(getHoodieColumnVal(ordinal).getBytes()); + return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes()); } return row.getUTF8String(ordinal); } @@ -198,7 +198,7 @@ public UTF8String getUTF8String(int ordinal) { @Override public String getString(int ordinal) { if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { - return new String(getHoodieColumnVal(ordinal).getBytes()); + return new String(getMetaColumnVal(ordinal).getBytes()); } return row.getString(ordinal); } @@ -231,7 +231,7 @@ public MapData getMap(int ordinal) { @Override public Object get(int ordinal, DataType dataType) { if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { - return UTF8String.fromBytes(getHoodieColumnVal(ordinal).getBytes()); + return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes()); } return row.get(ordinal, dataType); } diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 65598e7badf4e..89efc4e477055 100644 --- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -776,9 +776,7 @@ public Builder withPath(String basePath) { } public Builder withSchema(String schemaStr) { - if (null != schemaStr) { - props.setProperty(AVRO_SCHEMA, schemaStr); - } + props.setProperty(AVRO_SCHEMA, schemaStr); return this; } diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java index 351584156c177..cdd6f99ec0e89 100644 --- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java +++ b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java @@ -61,7 +61,7 @@ public List getRecordKeyFieldNames() { * @param recordNamespace record namespace of the dataset. */ public void initializeRowKeyGenerator(StructType structType, String structName, String recordNamespace) { - throw new UnsupportedOperationException("This method is invoked only for operation BULK_INSERT_DATASET. Expected to be overridden by sub classes"); + throw new UnsupportedOperationException("Expected to be overridden by sub classes, to improve performance for spark datasource writes "); } /** @@ -69,8 +69,8 @@ public void initializeRowKeyGenerator(StructType structType, String structName, * @param row instance of {@link Row} from which record key is requested. * @return the record key of interest from {@link Row}. */ - public String getRecordKeyFromRow(Row row) { - throw new UnsupportedOperationException("This method is invoked only for operation BULK_INSERT_DATASET. Expected to be overridden by sub classes"); + public String getRecordKey(Row row) { + throw new UnsupportedOperationException("Expected to be overridden by sub classes, to improve performance for spark datasource writes "); } /** @@ -78,7 +78,7 @@ public String getRecordKeyFromRow(Row row) { * @param row instance of {@link Row} from which partition path is requested * @return the partition path of interest from {@link Row}. */ - public String getPartitionPathFromRow(Row row) { - throw new UnsupportedOperationException("This method is invoked only for operation BULK_INSERT_DATASET. Expected to be overridden by sub classes"); + public String getPartitionPath(Row row) { + throw new UnsupportedOperationException("Expected to be overridden by sub classes, to improve performance for spark datasource writes "); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index db212d116e747..d1a53d5da2ad6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -35,7 +35,6 @@ public enum WriteOperationType { // bulk insert BULK_INSERT("bulk_insert"), BULK_INSERT_PREPPED("bulk_insert_prepped"), - BULK_INSERT_DATASET("bulk_insert_dataset"), // delete DELETE("delete"), BOOTSTRAP("bootstrap"), diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 2c2dfbe7a7997..7cc6e9a8e1bcf 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -227,9 +227,13 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base .equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL()); // 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); + } - return 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())) diff --git a/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java index 47e01d143002e..65a0523d12ef6 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java +++ b/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java @@ -58,6 +58,7 @@ public class HoodieDatasetBulkInsertHelper { * 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 @@ -75,10 +76,8 @@ public static Dataset prepareHoodieDatasetForBulkInsert(SQLContext sqlConte StructType structTypeForUDF = rows.schema(); keyGenerator.initializeRowKeyGenerator(structTypeForUDF, structName, recordNamespace); - - sqlContext.udf().register(RECORD_KEY_UDF_FN, (UDF1) keyGenerator::getRecordKeyFromRow, DataTypes.StringType); - - sqlContext.udf().register(PARTITION_PATH_UDF_FN, (UDF1) keyGenerator::getPartitionPathFromRow, DataTypes.StringType); + sqlContext.udf().register(RECORD_KEY_UDF_FN, (UDF1) keyGenerator::getRecordKey, DataTypes.StringType); + sqlContext.udf().register(PARTITION_PATH_UDF_FN, (UDF1) keyGenerator::getPartitionPath, DataTypes.StringType); final Dataset rowDatasetWithRecordKeys = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, callUDF(RECORD_KEY_UDF_FN, org.apache.spark.sql.functions.struct( @@ -103,8 +102,8 @@ public static Dataset prepareHoodieDatasetForBulkInsert(SQLContext sqlConte Dataset 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)) + return colOrderedDataset + .sort(functions.col(HoodieRecord.PARTITION_PATH_METADATA_FIELD), functions.col(HoodieRecord.RECORD_KEY_METADATA_FIELD)) .coalesce(config.getBulkInsertShuffleParallelism()); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java index 3cdf227f458ff..d8fc13c494c35 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -80,7 +80,7 @@ public Optional createWriter(String writeUUID, StructType sche 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(), WriteOperationType.BULK_INSERT_DATASET)); + getConfiguration())); } private SparkSession getSparkSession() { diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java index 6487fc184dff8..fc9ee9a1809c1 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -62,12 +62,11 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter { private final WriteOperationType operationType; public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType, - SparkSession sparkSession, Configuration configuration, - WriteOperationType operationType) { + SparkSession sparkSession, Configuration configuration) { this.instantTime = instantTime; this.writeConfig = writeConfig; this.structType = structType; - this.operationType = operationType; + this.operationType = WriteOperationType.BULK_INSERT; this.writeClient = new HoodieWriteClient<>(new JavaSparkContext(sparkSession.sparkContext()), writeConfig, true); writeClient.setOperationType(operationType); writeClient.startCommitWithTime(instantTime); @@ -79,7 +78,7 @@ public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writ public DataWriterFactory createWriterFactory() { metaClient.getActiveTimeline().transitionRequestedToInflight( new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty()); - if (WriteOperationType.BULK_INSERT_DATASET == operationType) { + if (WriteOperationType.BULK_INSERT == operationType) { return new HoodieBulkInsertDataInternalWriterFactory(hoodieTable, writeConfig, instantTime, structType); } else { throw new IllegalArgumentException("Write Operation Type + " + operationType + " not supported "); @@ -102,7 +101,7 @@ public void commit(WriterCommitMessage[] messages) { .flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList()); try { - writeClient.commitStat(instantTime, writeStatList, Option.empty()); + writeClient.commitStats(instantTime, writeStatList, Option.empty()); } catch (Exception ioe) { throw new HoodieException(ioe.getMessage(), ioe); } finally { diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 649aa03353b6e..6872a8b9d798e 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -41,11 +41,12 @@ */ public abstract class BuiltinKeyGenerator extends KeyGenerator { - private List recordKeyFields; - private List partitionPathFields; - private Map> recordKeyPositions = new HashMap<>(); + protected List recordKeyFields; + protected List partitionPathFields; + private Map> recordKeyPositions = new HashMap<>(); private Map> partitionPathPositions = new HashMap<>(); + private transient Function1 converterFn = null; protected StructType structType; private String structName; @@ -115,7 +116,7 @@ public void initializeRowKeyGenerator(StructType structType, String structName, * @return the record key of interest from {@link Row}. */ @Override - public String getRecordKeyFromRow(Row row) { + public String getRecordKey(Row row) { if (null != converterFn) { converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); } @@ -129,7 +130,7 @@ public String getRecordKeyFromRow(Row row) { * @return the partition path of interest from {@link Row}. */ @Override - public String getPartitionPathFromRow(Row row) { + public String getPartitionPath(Row row) { if (null != converterFn) { converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); } @@ -145,14 +146,6 @@ public List getPartitionPathFields() { return partitionPathFields; } - protected void setRecordKeyFields(List recordKeyFields) { - this.recordKeyFields = recordKeyFields; - } - - protected void setPartitionPathFields(List partitionPathFields) { - this.partitionPathFields = partitionPathFields; - } - protected Map> getRecordKeyPositions() { return recordKeyPositions; } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index 7fb7ffd46e9bd..f0cfc52732cf6 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -38,10 +38,10 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator { public ComplexKeyGenerator(TypedProperties props) { super(props); - this.setRecordKeyFields(Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()) - .split(",")).map(String::trim).collect(Collectors.toList())); - this.setPartitionPathFields(Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()) - .split(",")).map(String::trim).collect(Collectors.toList())); + 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(), @@ -59,12 +59,12 @@ public String getPartitionPath(GenericRecord record) { } @Override - public String getRecordKeyFromRow(Row row) { + public String getRecordKey(Row row) { return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), true); } @Override - public String getPartitionPathFromRow(Row row) { + public String getPartitionPath(Row row) { return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), hiveStylePartitioning, getPartitionPathPositions()); } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java index e4eec126a8c52..b6c57eaa2855c 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java @@ -44,7 +44,6 @@ */ public class CustomKeyGenerator extends BuiltinKeyGenerator { - protected final TypedProperties properties; private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; private static final String SPLIT_REGEX = ":"; @@ -57,10 +56,8 @@ public enum PartitionKeyType { public CustomKeyGenerator(TypedProperties props) { super(props); - this.properties = props; - this.setRecordKeyFields(Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList())); - this.setPartitionPathFields( - Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList())); + 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 @@ -86,11 +83,11 @@ public String getPartitionPath(GenericRecord record) { PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase()); switch (keyType) { case SIMPLE: - partitionPath.append(new SimpleKeyGenerator(properties, partitionPathField).getPartitionPath(record)); + partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(record)); break; case TIMESTAMP: try { - partitionPath.append(new TimestampBasedKeyGenerator(properties, partitionPathField).getPartitionPath(record)); + partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(record)); } catch (IOException ioe) { throw new HoodieDeltaStreamerException("Unable to initialise TimestampBasedKeyGenerator class"); } @@ -111,7 +108,8 @@ public String getRecordKey(GenericRecord record) { if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) { throw new HoodieKeyException("Unable to find field names for record key in cfg"); } - - return getRecordKeyFields().size() == 1 ? new SimpleKeyGenerator(properties).getRecordKey(record) : new ComplexKeyGenerator(properties).getRecordKey(record); + return getRecordKeyFields().size() == 1 + ? new SimpleKeyGenerator(config).getRecordKey(record) + : new ComplexKeyGenerator(config).getRecordKey(record); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index 193bdc5f6104d..ea187a2c4e16a 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -37,7 +37,7 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator { public GlobalDeleteKeyGenerator(TypedProperties config) { super(config); - this.setRecordKeyFields(Arrays.asList(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","))); + this.recordKeyFields = Arrays.asList(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")); } @Override @@ -56,12 +56,12 @@ public List getPartitionPathFields() { } @Override - public String getRecordKeyFromRow(Row row) { + public String getRecordKey(Row row) { return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), true); } @Override - public String getPartitionPathFromRow(Row row) { + public String getPartitionPath(Row row) { return EMPTY_PARTITION; } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index 253bb64855b36..9418925ca579c 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -48,12 +48,12 @@ public List getPartitionPathFields() { } @Override - public String getRecordKeyFromRow(Row row) { - return super.getRecordKeyFromRow(row); + public String getRecordKey(Row row) { + return super.getRecordKey(row); } @Override - public String getPartitionPathFromRow(Row row) { + public String getPartitionPath(Row row) { return EMPTY_PARTITION; } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index b81fcaed89a61..68f4c6d541db8 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -31,12 +31,6 @@ */ public class SimpleKeyGenerator extends BuiltinKeyGenerator { - @Deprecated - protected final String recordKeyField; - - @Deprecated - protected final String partitionPathField; - protected final boolean hiveStylePartitioning; protected final boolean encodePartitionPath; @@ -47,34 +41,31 @@ public SimpleKeyGenerator(TypedProperties props) { public SimpleKeyGenerator(TypedProperties props, String partitionPathField) { super(props); - this.setRecordKeyFields(Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()))); - this.setPartitionPathFields(Arrays.asList(partitionPathField)); + this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY())); + this.partitionPathFields = Arrays.asList(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())); - // Retaining this for compatibility - this.recordKeyField = getRecordKeyFields().get(0); - this.partitionPathField = getPartitionPathFields().get(0); } @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 String getRecordKeyFromRow(Row row) { + public String getRecordKey(Row row) { return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), false); } @Override - public String getPartitionPathFromRow(Row row) { + public String getPartitionPath(Row row) { return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), hiveStylePartitioning, getPartitionPathPositions()); } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index ee6ec78155d5b..99115b09f7ea5 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -130,7 +130,7 @@ public TimestampBasedKeyGenerator(TypedProperties config, String partitionPathFi @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; } @@ -181,7 +181,7 @@ private String getPartitionPath(Object partitionVal) throws ParseException { throw new HoodieException(uoe.getMessage(), uoe); } } - return hiveStylePartitioning ? partitionPathField + "=" + partitionPath : partitionPath; + return hiveStylePartitioning ? getPartitionPathFields().get(0) + "=" + partitionPath : partitionPath; } private long convertLongTimeToMillis(Long partitionVal) { @@ -193,12 +193,12 @@ private long convertLongTimeToMillis(Long partitionVal) { } @Override - public String getRecordKeyFromRow(Row row) { + public String getRecordKey(Row row) { return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), false); } @Override - public String getPartitionPathFromRow(Row row) { + public String getPartitionPath(Row row) { Object fieldVal = null; Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, getPartitionPathPositions().get(getPartitionPathFields().get(0))); try { diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index e8d2f6c64a8c7..1cf9bdb4952a4 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -118,7 +118,7 @@ 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 { diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index c3cbeb0bf22ee..766cdd0e6bb32 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -117,7 +117,7 @@ private[hudi] object HoodieSparkSqlWriter { if (operation.equalsIgnoreCase(BULK_INSERT_DATASET_OPERATION_OPT_VAL)) { val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName, basePath, path, instantTime) - return (success, commitTime, common.util.Option.of(""), hoodieWriteClient.orNull, tableConfig) + return (success, commitTime, common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig) } // scalastyle:on @@ -282,43 +282,6 @@ private[hudi] object HoodieSparkSqlWriter { (syncHiveSucess, common.util.Option.ofNullable(instantTime)) } - /** - * 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 toProperties(params: Map[String, String]): TypedProperties = { val props = new TypedProperties() params.foreach(kv => props.setProperty(kv._1, kv._2)) diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 3f6cd90347fd6..79d68fba14dc5 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -51,7 +51,9 @@ object HoodieWriterUtils { 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, @@ -61,7 +63,8 @@ object HoodieWriterUtils { 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 + HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL, + ASYNC_COMPACT_ENABLE_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_VAL ) ++ translateStorageTypeToTableType(parameters) } diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java index 0d474ecadb788..21091f7059e43 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java +++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java @@ -79,7 +79,7 @@ public void testDataSourceWriter() throws IOException { String instantTime = "001"; // init writer HoodieDataSourceInternalWriter dataSourceInternalWriter = - new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, WriteOperationType.BULK_INSERT_DATASET); + new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); DataWriter writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong()); List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); @@ -125,7 +125,7 @@ public void testMultipleDataSourceWrites() throws IOException { String instantTime = "00" + i; // init writer HoodieDataSourceInternalWriter dataSourceInternalWriter = - new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, WriteOperationType.BULK_INSERT_DATASET); + new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); List commitMessages = new ArrayList<>(); Dataset totalInputRows = null; @@ -169,7 +169,7 @@ public void testLargeWrites() throws IOException { String instantTime = "00" + i; // init writer HoodieDataSourceInternalWriter dataSourceInternalWriter = - new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, WriteOperationType.BULK_INSERT_DATASET); + new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); List commitMessages = new ArrayList<>(); Dataset totalInputRows = null; @@ -216,7 +216,7 @@ public void testAbort() throws IOException { String instantTime0 = "00" + 0; // init writer HoodieDataSourceInternalWriter dataSourceInternalWriter = - new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, WriteOperationType.BULK_INSERT_DATASET); + new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); DataWriter writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong()); List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); @@ -254,7 +254,7 @@ public void testAbort() throws IOException { // 2nd batch. abort in the end String instantTime1 = "00" + 1; dataSourceInternalWriter = - new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, WriteOperationType.BULK_INSERT_DATASET); + 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++) { diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index fab81ce65e5f1..bea7c615701dc 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -122,7 +122,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { // test w/ Row baseRow = genericRecordToRow(baseRecord); keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); - assertEquals("2020-01-06 12", keyGen.getPartitionPathFromRow(baseRow)); + assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null); @@ -132,7 +132,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { // test w/ Row keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); - assertEquals("2020-01-06 04", keyGen.getPartitionPathFromRow(baseRow)); + 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"); @@ -145,7 +145,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { // test w/ Row baseRow = genericRecordToRow(baseRecord); keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); - assertEquals("2020-01-06 12", keyGen.getPartitionPathFromRow(baseRow)); + assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT", null); @@ -155,7 +155,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { // test w/ Row keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); - assertEquals("2020-01-06 12", keyGen.getPartitionPathFromRow(baseRow)); + assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); } @Test @@ -172,7 +172,7 @@ public void testScalar() throws IOException { // test w/ Row baseRow = genericRecordToRow(baseRecord); keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); - assertEquals("2024-10-04 12", keyGen.getPartitionPathFromRow(baseRow)); + assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow)); } @Test diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index 1d7d242451c93..2ac0ee4223854 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -76,8 +76,8 @@ class TestDataSourceDefaults { assertEquals("name1", hk1.getPartitionPath) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("name1", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("field1", keyGen.getRecordKey(baseRow)) + assertEquals("name1", keyGen.getPartitionPath(baseRow)) // partition path field not specified try { @@ -96,7 +96,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") val keyGen = new SimpleKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: IllegalArgumentException => @@ -120,7 +120,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField") val keyGen = new SimpleKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getPartitionPathFromRow(baseRow) + keyGen.getPartitionPath(baseRow) fail("Should have errored out") } catch { case e: IllegalArgumentException => @@ -151,7 +151,7 @@ class TestDataSourceDefaults { // if partition path can't be found, return default partition path using row keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - val hk3_row = keyGen.getPartitionPathFromRow(baseRow) + val hk3_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk3_row) // if enable hive style partitioning @@ -161,7 +161,7 @@ class TestDataSourceDefaults { // if enable hive style partitioning using row keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - val hk4_row = keyGen.getPartitionPathFromRow(baseRow) + val hk4_row = keyGen.getPartitionPath(baseRow) assertEquals("name=name1", hk4_row) // if partition is null, return default partition path @@ -174,7 +174,7 @@ class TestDataSourceDefaults { keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) baseRow = genericRecordToRow(baseRecord) - val hk5_row = keyGen.getPartitionPathFromRow(baseRow) + val hk5_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk5_row) // if partition is empty, return default partition path @@ -187,7 +187,7 @@ class TestDataSourceDefaults { keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) baseRow = genericRecordToRow(baseRecord) - val hk6_row = keyGen.getPartitionPathFromRow(baseRow) + val hk6_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk6_row) // if record key is empty, throw error @@ -211,7 +211,7 @@ class TestDataSourceDefaults { keyGen = new SimpleKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) baseRow = genericRecordToRow(baseRecord) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: HoodieKeyException => @@ -239,7 +239,7 @@ class TestDataSourceDefaults { keyGen = new SimpleKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) baseRow = genericRecordToRow(baseRecord) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: HoodieKeyException => @@ -256,8 +256,8 @@ class TestDataSourceDefaults { // top level, valid fields with Row keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("field1/name1", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) + assertEquals("field1/name1", keyGen.getPartitionPath(baseRow)) // partition path field not specified try { @@ -276,7 +276,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") val keyGen = new ComplexKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: IllegalArgumentException => @@ -314,8 +314,8 @@ class TestDataSourceDefaults { // nested field as record key and partition path keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("UserId1@001/false", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow)) + assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow)) // Nested record key not found try { @@ -331,7 +331,7 @@ class TestDataSourceDefaults { try { val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: HoodieException => @@ -344,7 +344,7 @@ class TestDataSourceDefaults { assertEquals("default", hk3.getPartitionPath) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("default", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("default", keyGen.getPartitionPath(baseRow)) // if enable hive style partitioning keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true")) @@ -353,8 +353,8 @@ class TestDataSourceDefaults { assertEquals("field1=field1/name=name1", hk4.getPartitionPath) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("field1=field1/name=name1", keyGen.getPartitionPathFromRow(baseRow)) + 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", "") @@ -365,8 +365,8 @@ class TestDataSourceDefaults { baseRow = genericRecordToRow(baseRecord) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:__empty__", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("field1/default", keyGen.getPartitionPathFromRow(baseRow)) + 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) @@ -377,8 +377,8 @@ class TestDataSourceDefaults { baseRow = genericRecordToRow(baseRecord) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:__null__", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("field1/default", keyGen.getPartitionPathFromRow(baseRow)) + 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 { @@ -404,7 +404,7 @@ class TestDataSourceDefaults { keyGen = new ComplexKeyGenerator(props) baseRow = genericRecordToRow(baseRecord) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: HoodieKeyException => @@ -421,8 +421,8 @@ class TestDataSourceDefaults { baseRow = genericRecordToRow(baseRecord) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("field1/name1", keyGen.getPartitionPathFromRow(baseRow)) + 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) @@ -430,8 +430,8 @@ class TestDataSourceDefaults { assertEquals("field1", hk8.getPartitionPath) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("field1", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("field1:field1", keyGen.getRecordKey(baseRow)) + assertEquals("field1", keyGen.getPartitionPath(baseRow)) } @Test def testGlobalDeleteKeyGenerator() = { @@ -442,8 +442,8 @@ class TestDataSourceDefaults { assertEquals("", hk1.getPartitionPath) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) + assertEquals("", keyGen.getPartitionPath(baseRow)) // top level, partition value not included val props = new TypedProperties() @@ -454,8 +454,8 @@ class TestDataSourceDefaults { assertEquals("", hk2.getPartitionPath) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:name1", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("", keyGen.getPartitionPathFromRow(baseRow)) + 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", "") @@ -466,8 +466,8 @@ class TestDataSourceDefaults { baseRow = genericRecordToRow(baseRecord) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:__empty__", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("", keyGen.getPartitionPathFromRow(baseRow)) + 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) @@ -478,8 +478,8 @@ class TestDataSourceDefaults { baseRow = genericRecordToRow(baseRecord) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - assertEquals("field1:field1,name:__null__", keyGen.getRecordKeyFromRow(baseRow)) - assertEquals("", keyGen.getPartitionPathFromRow(baseRow)) + assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) + assertEquals("", keyGen.getPartitionPath(baseRow)) // recordkey field not specified try { @@ -498,7 +498,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField") val keyGen = new GlobalDeleteKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: IllegalArgumentException => @@ -519,7 +519,7 @@ class TestDataSourceDefaults { try { val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: HoodieException => @@ -548,7 +548,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name") val keyGen = new GlobalDeleteKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - keyGen.getRecordKeyFromRow(baseRow) + keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { case e: HoodieKeyException => From 383a74578defdcb82a5dddde079e65899f6d60f0 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Wed, 12 Aug 2020 12:06:11 -0400 Subject: [PATCH 3/5] Some fixes to key generators and adding more tests for Row apis --- .../apache/hudi/internal/DefaultSource.java | 1 - .../hudi/keygen/BuiltinKeyGenerator.java | 24 ++++-- .../keygen/TimestampBasedKeyGenerator.java | 2 +- .../TestHoodieDataSourceInternalWriter.java | 1 - .../hudi/keygen/TestComplexKeyGenerator.java | 12 ++- .../hudi/keygen/TestCustomKeyGenerator.java | 81 +++++++++++++++++-- .../keygen/TestGlobalDeleteKeyGenerator.java | 12 ++- .../keygen/TestKeyGeneratorUtilities.java | 30 ++++++- .../hudi/keygen/TestSimpleKeyGenerator.java | 30 +++++++ .../TestTimestampBasedKeyGenerator.java | 73 ++++++++++++----- .../apache/hudi/TestDataSourceDefaults.scala | 73 +++++++++++------ 11 files changed, 280 insertions(+), 59 deletions(-) diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java index d8fc13c494c35..a89757b0ba4a8 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -21,7 +21,6 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.HoodieWriterUtils; -import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hadoop.conf.Configuration; diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 6872a8b9d798e..2dd69c95086ac 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -90,7 +90,13 @@ public void initializeRowKeyGenerator(StructType structType, String structName, // parse simple feilds getRecordKeyFields().stream() .filter(f -> !(f.contains("."))) - .forEach(f -> recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get())))); + .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(".")) @@ -98,8 +104,14 @@ public void initializeRowKeyGenerator(StructType structType, String structName, // parse simple fields if (getPartitionPathFields() != null) { getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains("."))) - .forEach(f -> partitionPathPositions.put(f, - Collections.singletonList((Integer) (structType.getFieldIndex(f).get())))); + .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, @@ -112,12 +124,13 @@ public void initializeRowKeyGenerator(StructType structType, String structName, /** * 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}. */ @Override public String getRecordKey(Row row) { - if (null != converterFn) { + if (null == converterFn) { converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); } GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); @@ -126,12 +139,13 @@ public String getRecordKey(Row row) { /** * 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}. */ @Override public String getPartitionPath(Row row) { - if (null != converterFn) { + if (null == converterFn) { converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); } GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index 99115b09f7ea5..752accf034a7d 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -209,7 +209,7 @@ public String getPartitionPath(Row row) { fieldVal = partitionPathFieldVal; } return getPartitionPath(fieldVal); - } catch (ParseException e) { + } catch (Exception e) { throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + fieldVal, e); } } diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java index 21091f7059e43..01d78933b0aa3 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java +++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java @@ -21,7 +21,6 @@ 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.model.WriteOperationType; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.HoodieClientTestHarness; diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java index bb94c25388c0e..1b35a7d5cb287 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieKeyException; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -76,13 +78,21 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); } @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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); + Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=2020-03-21"); } } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java index 699bf43a4a80b..45d3eff5c8f1a 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java @@ -22,6 +22,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.config.TypedProperties; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -97,25 +99,43 @@ private TypedProperties getPropertiesForNonPartitionedKeyGen() { @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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + 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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + 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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); + Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty()); } @Test @@ -127,6 +147,19 @@ public void testInvalidPartitionKeyType() { } 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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + keyGenerator.getRecordKey(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 +171,18 @@ public void testNoRecordKeyFieldProp() { } 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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + 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 +194,47 @@ public void testPartitionFieldsInImproperFormat() { } 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 = TestKeyGeneratorUtilities.getRow(record); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + 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"); + + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = TestKeyGeneratorUtilities.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"); + + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = TestKeyGeneratorUtilities.getRow(record); + Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); + Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=20200321"); } } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java index e46c783b313f4..111034d3cfd94 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieKeyException; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -66,13 +68,21 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); } @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.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, + TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = TestKeyGeneratorUtilities.getRow(record); + Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); + Assertions.assertEquals(keyGenerator.getPartitionPath(row), ""); } } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java index c0d027e38429e..e756d1f71a83d 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java @@ -18,17 +18,30 @@ package org.apache.hudi.keygen; +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 TestKeyGeneratorUtilities { - public String exampleSchema = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ " + 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); @@ -37,4 +50,19 @@ public GenericRecord getRecord() { 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 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); + } } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java index f36331a19efa6..3bdfbc41907f5 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.exception.HoodieKeyException; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -52,6 +54,13 @@ private TypedProperties getWrongRecordKeyFieldProps() { 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 +88,40 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); + Assertions.assertThrows(HoodieKeyException.class, + () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); + } + + @Test + public void testWrongPartitionPathField() { + SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps()); + GenericRecord record = getRecord(); + Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.DEFAULT_PARTITION_PATH); + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Assertions.assertEquals(keyGenerator.getPartitionPath(TestKeyGeneratorUtilities.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.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); } @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"); + + keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = TestKeyGeneratorUtilities.getRow(record); + Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); + Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); } + } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index bea7c615701dc..97ed73760c58c 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -185,9 +185,13 @@ public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsU "", "yyyyMMddHH", "GMT"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test @@ -200,9 +204,13 @@ public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAs "", "yyyyMMddHH", ""); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test @@ -215,9 +223,13 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone "", "yyyyMMddHH", "UTC"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test @@ -230,9 +242,13 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAs "", "yyyyMMddHH", "UTC"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040113", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @Test @@ -245,9 +261,13 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputT "", "yyyyMMddHH", "UTC"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040118", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020040118", keyGen.getPartitionPath(baseRow)); } @Test @@ -260,9 +280,13 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_Outpu "", "yyyyMMddHH", "UTC"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040118", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("2020040118", keyGen.getPartitionPath(baseRow)); } @Test @@ -275,13 +299,17 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone "", "yyyyMMddHH", "EST"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("2020040109", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + 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", @@ -290,7 +318,12 @@ public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() { "", "yyyyMMddHH", "UTC"); - Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> new TimestampBasedKeyGenerator(properties).getKey(baseRecord)); + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getKey(baseRecord)); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getPartitionPath(baseRow)); } @Test @@ -303,8 +336,12 @@ public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() "UTC", "MM/dd/yyyy", "UTC"); - HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord); - + KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey hk1 = keyGen.getKey(baseRecord); Assertions.assertEquals("04/01/2020", hk1.getPartitionPath()); + + baseRow = genericRecordToRow(baseRecord); + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); + assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow)); } } \ No newline at end of file diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index 2ac0ee4223854..76a45ba9b5eb8 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -17,15 +17,17 @@ 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.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.spark.sql.Row -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{BeforeEach, Test} import org.scalatest.Assertions.fail @@ -45,18 +47,7 @@ class TestDataSourceDefaults { @BeforeEach def initialize(): Unit = { baseRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 1, "001", "f1") - baseRow = genericRecordToRow(baseRecord) - } - - private def genericRecordToRow(baseRecord: GenericRecord): Row = { - val convertor = AvroConversionHelper.createConverterToRow(schema, structType) - val row = convertor.apply(baseRecord).asInstanceOf[Row] - val fieldCount = structType.fieldNames.length - val values = new Array[Any](fieldCount) - for (i <- 0 until fieldCount) { - values(i) = row.get(i) - } - new GenericRowWithSchema(values, structType) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) } private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = { @@ -173,7 +164,7 @@ class TestDataSourceDefaults { // if partition is null, return default partition path using Row keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) val hk5_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk5_row) @@ -186,7 +177,7 @@ class TestDataSourceDefaults { // if partition is empty, return default partition path using Row keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) val hk6_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk6_row) @@ -210,7 +201,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name") keyGen = new SimpleKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -238,7 +229,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name") keyGen = new SimpleKeyGenerator(props) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -247,6 +238,38 @@ class TestDataSourceDefaults { } } + @Test def testUserDefinedKeyGenerator(): Unit = { + var keyGen = new UserDefinedKeyGenerator(getKeyConfig("field1", "name", "false")) + val hk1 = keyGen.getKey(baseRecord) + assertEquals("field1", hk1.getRecordKey) + assertEquals("name1", hk1.getPartitionPath) + + keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + assertEquals("field1", keyGen.getRecordKey(baseRow)) + assertEquals("name1", keyGen.getPartitionPath(baseRow)) + } + + class UserDefinedKeyGenerator(props: TypedProperties) extends BuiltinKeyGenerator(props) { + val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY) + val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY) + this.recordKeyFields = util.Arrays.asList(recordKeyProp) + this.partitionPathFields = util.Arrays.asList(partitionPathProp) + + /** + * Generate a record Key out of provided generic record. + */ + override def getRecordKey(record: GenericRecord): String = { + HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true) + } + + /** + * Generate a partition path out of provided generic record. + */ + override def getPartitionPath(record: GenericRecord): String = { + HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true) + } + } + @Test def testComplexKeyGenerator() = { // top level, valid fields var keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) @@ -363,7 +386,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__empty__", hk5.getRecordKey) assertEquals("field1/default", hk5.getPartitionPath) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow)) assertEquals("field1/default", keyGen.getPartitionPath(baseRow)) @@ -375,7 +398,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__null__", hk6.getRecordKey) assertEquals("field1/default", hk6.getPartitionPath) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) assertEquals("field1/default", keyGen.getPartitionPath(baseRow)) @@ -402,7 +425,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name") props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name") keyGen = new ComplexKeyGenerator(props) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") @@ -419,7 +442,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:name1", hk7.getRecordKey) assertEquals("field1/name1", hk7.getPartitionPath) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) assertEquals("field1/name1", keyGen.getPartitionPath(baseRow)) @@ -464,7 +487,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__empty__", hk3.getRecordKey) assertEquals("", hk3.getPartitionPath) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow)) assertEquals("", keyGen.getPartitionPath(baseRow)) @@ -476,7 +499,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__null__", hk4.getRecordKey) assertEquals("", hk4.getPartitionPath) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) assertEquals("", keyGen.getPartitionPath(baseRow)) @@ -543,7 +566,7 @@ class TestDataSourceDefaults { try { baseRecord.put("name", "") baseRecord.put("field1", null) - baseRow = genericRecordToRow(baseRecord) + baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name") val keyGen = new GlobalDeleteKeyGenerator(props) From fefd4b7618aa47b8f2c3e9a28b898b5a713d88e6 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 12 Aug 2020 13:39:33 -0700 Subject: [PATCH 4/5] Cleaning up config placements, naming --- .../java/org/apache/hudi/DataSourceUtils.java | 2 +- .../org/apache/hudi/internal/DefaultSource.java | 5 +---- .../internal/HoodieDataSourceInternalWriter.java | 1 + .../scala/org/apache/hudi/DataSourceOptions.scala | 15 ++++++++++----- .../org/apache/hudi/HoodieSparkSqlWriter.scala | 7 ++++--- .../scala/org/apache/hudi/HoodieWriterUtils.scala | 3 ++- hudi-spark/src/test/java/HoodieJavaApp.java | 6 +++--- .../src/test/java/HoodieJavaStreamingApp.java | 2 +- .../functional/HoodieSparkSqlWriterSuite.scala | 5 +++-- 9 files changed, 26 insertions(+), 20 deletions(-) diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 7cc6e9a8e1bcf..ea2cc5c0ed1ac 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -222,7 +222,7 @@ public static void checkRequiredProperties(TypedProperties props, List c public static HoodieWriteConfig createHoodieConfig(String schemaStr, String basePath, String tblName, Map parameters) { - boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY())); + 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()); // insert/bulk-insert combining to be true, if filtering for duplicates diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java index a89757b0ba4a8..8f785320819d9 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -71,10 +71,7 @@ public DataSourceReader createReader(DataSourceOptions options) { @Override public Optional createWriter(String writeUUID, StructType schema, SaveMode mode, DataSourceOptions options) { - String instantTime = options.get(DataSourceWriteOptions.INSTANT_TIME()).get(); - Map paramsWithDefaults = HoodieWriterUtils.javaParametersWithWriteDefaults(options.asMap()); - Properties props = new Properties(); - props.putAll(paramsWithDefaults); + 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()); diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java index fc9ee9a1809c1..6e67dc9f2ac85 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -52,6 +52,7 @@ 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; diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 3f41b7e2396c2..ba6bc879cccb8 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -141,7 +141,6 @@ object DataSourceWriteOptions { val UPSERT_OPERATION_OPT_VAL = "upsert" val DELETE_OPERATION_OPT_VAL = "delete" val BOOTSTRAP_OPERATION_OPT_VAL = "bootstrap" - val BULK_INSERT_DATASET_OPERATION_OPT_VAL = "bulk_insert_dataset" val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL /** @@ -229,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 @@ -298,9 +304,8 @@ object DataSourceWriteOptions { val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false" val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" 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" - // Internal configs - val INSTANT_TIME = "hoodie.instant.time" + val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable" + val DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL = "true" } diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 766cdd0e6bb32..36886cdace809 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -37,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 @@ -114,7 +115,7 @@ private[hudi] object HoodieSparkSqlWriter { // short-circuit if bulk_insert via row is enabled. // scalastyle:off - if (operation.equalsIgnoreCase(BULK_INSERT_DATASET_OPERATION_OPT_VAL)) { + 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) @@ -269,7 +270,7 @@ private[hudi] object HoodieSparkSqlWriter { 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(INSTANT_TIME, instantTime) + .option(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY, instantTime) .options(parameters) .save() val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean) @@ -443,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 diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 79d68fba14dc5..484605c768d3d 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -64,7 +64,8 @@ object HoodieWriterUtils { 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 + ASYNC_COMPACT_ENABLE_OPT_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL, + ENABLE_ROW_WRITER_OPT_KEY -> DEFAULT_ENABLE_ROW_WRITER_OPT_VAL ) ++ translateStorageTypeToTableType(parameters) } diff --git a/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark/src/test/java/HoodieJavaApp.java index 6eda0517d461c..594d813980930 100644 --- a/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark/src/test/java/HoodieJavaApp.java @@ -151,7 +151,7 @@ public void run() throws Exception { .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 void run() throws Exception { 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 void run() throws Exception { 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); diff --git a/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 500189d1cf211..e93784e75dfae 100644 --- a/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -358,7 +358,7 @@ public void stream(Dataset streamingInput, String operationType, String che .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()); diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala index e5b3ecd34786f..8995b7cfe1f68 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala @@ -122,7 +122,8 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { val fooTableModifier = Map("path" -> path.toAbsolutePath.toString, HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName, "hoodie.bulkinsert.shuffle.parallelism" -> "4", - DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_DATASET_OPERATION_OPT_VAL, + 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") @@ -178,7 +179,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { val fooTableModifier = Map("path" -> path.toAbsolutePath.toString, HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName, "hoodie.bulkinsert.shuffle.parallelism" -> "4", - DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_DATASET_OPERATION_OPT_VAL, + 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") From 865d8d6c6dbded3f4e5338ca9c9849f5369a6075 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 12 Aug 2020 19:34:11 -0700 Subject: [PATCH 5/5] Clean up changes made on the public KeyGenerator class - 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 --- .../hudi/keygen/KeyGeneratorInterface.java | 41 +++++++ .../BootstrapCommitActionExecutor.java | 6 +- .../hudi/io/TestHoodieRowCreateHandle.java | 12 +- .../TestHoodieInternalRowParquetWriter.java | 10 +- ...tUtils.java => SparkDatasetTestUtils.java} | 2 +- .../hudi/HoodieDatasetBulkInsertHelper.java | 1 - .../apache/hudi/internal/DefaultSource.java | 4 - .../hudi/keygen/BuiltinKeyGenerator.java | 112 ++++++------------ .../hudi/keygen/ComplexKeyGenerator.java | 12 +- .../hudi/keygen/CustomKeyGenerator.java | 41 ++++++- .../hudi/keygen/GlobalDeleteKeyGenerator.java | 6 +- .../org/apache/hudi/keygen/KeyGenerator.java | 31 ++--- .../keygen/NonpartitionedKeyGenerator.java | 8 +- .../hudi/keygen/RowKeyGeneratorHelper.java | 3 +- .../hudi/keygen/SimpleKeyGenerator.java | 31 ++--- .../keygen/TimestampBasedKeyGenerator.java | 17 ++- ...estHoodieBulkInsertDataInternalWriter.java | 12 +- .../TestHoodieDataSourceInternalWriter.java | 10 +- .../hudi/keygen/TestComplexKeyGenerator.java | 10 +- .../hudi/keygen/TestCustomKeyGenerator.java | 38 ++---- .../keygen/TestGlobalDeleteKeyGenerator.java | 11 +- .../hudi/keygen/TestSimpleKeyGenerator.java | 15 +-- .../TestTimestampBasedKeyGenerator.java | 16 --- .../KeyGeneratorTestUtilities.java} | 4 +- .../apache/hudi/TestDataSourceDefaults.scala | 84 ++++--------- 25 files changed, 244 insertions(+), 293 deletions(-) create mode 100644 hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java rename hudi-client/src/test/java/org/apache/hudi/testutils/{HoodieDatasetTestUtils.java => SparkDatasetTestUtils.java} (99%) rename {hudi-client => hudi-spark}/src/main/java/org/apache/hudi/keygen/KeyGenerator.java (72%) rename hudi-spark/src/test/java/org/apache/hudi/{keygen/TestKeyGeneratorUtilities.java => testutils/KeyGeneratorTestUtilities.java} (97%) diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java new file mode 100644 index 0000000000000..6412a2f81fc31 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java @@ -0,0 +1,41 @@ +/* + * 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.avro.generic.GenericRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.spark.sql.Row; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents the interface key generators need to adhere to. + */ +public interface KeyGeneratorInterface extends Serializable { + + HoodieKey getKey(GenericRecord record); + + List getRecordKeyFieldNames(); + + String getRecordKey(Row row); + + String getPartitionPath(Row row); + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java index 3b7c501122f44..47791d994bea4 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java @@ -53,7 +53,7 @@ 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 @@ protected CommitActionExecutor getBulkInsertActionExecutor(JavaRDD runMetadataBootstrap(List prepareHoodieDatasetForBulkInsert(SQLContext sqlConte KeyGenerator keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties); StructType structTypeForUDF = rows.schema(); - keyGenerator.initializeRowKeyGenerator(structTypeForUDF, structName, recordNamespace); sqlContext.udf().register(RECORD_KEY_UDF_FN, (UDF1) keyGenerator::getRecordKey, DataTypes.StringType); sqlContext.udf().register(PARTITION_PATH_UDF_FN, (UDF1) keyGenerator::getPartitionPath, DataTypes.StringType); diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java index 8f785320819d9..5fb71df777df4 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -19,8 +19,6 @@ package org.apache.hudi.internal; import org.apache.hudi.DataSourceUtils; -import org.apache.hudi.DataSourceWriteOptions; -import org.apache.hudi.HoodieWriterUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hadoop.conf.Configuration; @@ -37,9 +35,7 @@ import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; import org.apache.spark.sql.types.StructType; -import java.util.Map; import java.util.Optional; -import java.util.Properties; /** * DataSource V2 implementation for managing internal write logic. Only called internally. diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 2dd69c95086ac..8c973a6ba8284 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -18,13 +18,12 @@ package org.apache.hudi.keygen; -import org.apache.hudi.AvroConversionHelper; +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.Row; import org.apache.spark.sql.types.StructType; import java.util.Collections; @@ -33,8 +32,6 @@ import java.util.Map; import java.util.stream.Collectors; -import scala.Function1; - /** * Base class for all the built-in key generators. Contains methods structured for * code reuse amongst them. @@ -43,17 +40,19 @@ public abstract class BuiltinKeyGenerator extends KeyGenerator { protected List recordKeyFields; protected List partitionPathFields; + protected final boolean encodePartitionPath; + protected final boolean hiveStylePartitioning; - private Map> recordKeyPositions = new HashMap<>(); - private Map> partitionPathPositions = new HashMap<>(); - - private transient Function1 converterFn = null; + protected Map> recordKeyPositions = new HashMap<>(); + protected Map> partitionPathPositions = new HashMap<>(); protected StructType structType; - private String structName; - private String recordNamespace; 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())); } /** @@ -85,71 +84,40 @@ public final List getRecordKeyFieldNames() { }).collect(Collectors.toList()); } - @Override - public void initializeRowKeyGenerator(StructType structType, String structName, String recordNamespace) { - // parse simple feilds - 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("."))) + void buildFieldPositionMapIfNeeded(StructType structType) { + if (this.structType == null) { + // parse simple fields + getRecordKeyFields().stream() + .filter(f -> !(f.contains("."))) .forEach(f -> { if (structType.getFieldIndex(f).isDefined()) { - partitionPathPositions.put(f, - Collections.singletonList((Integer) (structType.getFieldIndex(f).get()))); + recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get()))); } else { - partitionPathPositions.put(f, Collections.singletonList(-1)); + throw new HoodieKeyException("recordKey value not found for field: \"" + f + "\""); } }); // parse nested fields - getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains(".")) - .forEach(f -> partitionPathPositions.put(f, - RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false))); - } - this.structName = structName; - this.structType = structType; - this.recordNamespace = recordNamespace; - } - - /** - * 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}. - */ - @Override - public String getRecordKey(Row row) { - if (null == converterFn) { - converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); + 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; } - 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}. - */ - @Override - public String getPartitionPath(Row row) { - if (null == converterFn) { - converterFn = AvroConversionHelper.createConverterToAvro(structType, structName, recordNamespace); - } - GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); - return getKey(genericRecord).getPartitionPath(); } public List getRecordKeyFields() { @@ -159,12 +127,4 @@ public List getRecordKeyFields() { public List getPartitionPathFields() { return partitionPathFields; } - - protected Map> getRecordKeyPositions() { - return recordKeyPositions; - } - - protected Map> getPartitionPathPositions() { - return partitionPathPositions; - } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index f0cfc52732cf6..664824c319a6f 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -33,8 +33,6 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator { public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":"; - protected final boolean hiveStylePartitioning; - protected final boolean encodePartitionPath; public ComplexKeyGenerator(TypedProperties props) { super(props); @@ -42,10 +40,6 @@ public ComplexKeyGenerator(TypedProperties props) { .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())); } @Override @@ -60,12 +54,14 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), true); + buildFieldPositionMapIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true); } @Override public String getPartitionPath(Row row) { + buildFieldPositionMapIfNeeded(row.schema()); return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), - hiveStylePartitioning, getPartitionPathPositions()); + hiveStylePartitioning, partitionPathPositions); } } \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java index b6c57eaa2855c..2ac6b77b28a8d 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java @@ -20,10 +20,12 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; +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; @@ -60,8 +62,17 @@ public CustomKeyGenerator(TypedProperties props) { 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) { + return getPartitionPath(Option.of(record), Option.empty()); + } + + private String getPartitionPath(Option record, Option row) { if (getPartitionPathFields() == null) { throw new HoodieKeyException("Unable to find field names for partition path in cfg"); } @@ -83,11 +94,19 @@ public String getPartitionPath(GenericRecord record) { PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase()); switch (keyType) { case SIMPLE: - partitionPath.append(new SimpleKeyGenerator(config, 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(config, 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"); } @@ -105,11 +124,23 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(GenericRecord record) { - if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) { - throw new HoodieKeyException("Unable to find field names for record key in cfg"); - } + 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"); + } + } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index ea187a2c4e16a..243493b45291a 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -29,7 +29,8 @@ import java.util.List; /** - * 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 { @@ -57,7 +58,8 @@ public List getPartitionPathFields() { @Override public String getRecordKey(Row row) { - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), true); + buildFieldPositionMapIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true); } @Override diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java similarity index 72% rename from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java rename to hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java index cdd6f99ec0e89..a11bc84922f41 100644 --- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java @@ -18,12 +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 org.apache.spark.sql.types.StructType; +import scala.Function1; import java.io.Serializable; import java.util.List; @@ -31,9 +32,13 @@ /** * 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 converterFn = null; protected KeyGenerator(TypedProperties config) { this.config = config; @@ -54,23 +59,17 @@ public List getRecordKeyFieldNames() { + "Please override this method in your custom key generator."); } - /** - * Initializes {@link KeyGenerator} for {@link Row} based operations. - * @param structType structype of the dataset. - * @param structName struct name of the dataset. - * @param recordNamespace record namespace of the dataset. - */ - public void initializeRowKeyGenerator(StructType structType, String structName, String recordNamespace) { - throw new UnsupportedOperationException("Expected to be overridden by sub classes, to improve performance for spark datasource writes "); - } - /** * 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) { - throw new UnsupportedOperationException("Expected to be overridden by sub classes, to improve performance for spark datasource writes "); + if (null == converterFn) { + converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); + } + GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); + return getKey(genericRecord).getRecordKey(); } /** @@ -79,6 +78,10 @@ public String getRecordKey(Row row) { * @return the partition path of interest from {@link Row}. */ public String getPartitionPath(Row row) { - throw new UnsupportedOperationException("Expected to be overridden by sub classes, to improve performance for spark datasource writes "); + if (null == converterFn) { + converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); + } + GenericRecord genericRecord = (GenericRecord) converterFn.apply(row); + return getKey(genericRecord).getPartitionPath(); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index 9418925ca579c..db51024017d12 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -32,6 +32,7 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator { private static final String EMPTY_PARTITION = ""; + private static final List EMPTY_PARTITION_FIELD_LIST = new ArrayList<>(); public NonpartitionedKeyGenerator(TypedProperties props) { super(props); @@ -44,12 +45,7 @@ public String getPartitionPath(GenericRecord record) { @Override public List getPartitionPathFields() { - return new ArrayList<>(); - } - - @Override - public String getRecordKey(Row row) { - return super.getRecordKey(row); + return EMPTY_PARTITION_FIELD_LIST; } @Override diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java index 740a89cdeb52b..02b8492f1e7e5 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java @@ -55,8 +55,7 @@ public class RowKeyGeneratorHelper { */ public static String getRecordKeyFromRow(Row row, List recordKeyFields, Map> recordKeyPositions, boolean prefixFieldName) { AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true); - String toReturn = IntStream.range(0, recordKeyFields.size()).mapToObj(idx -> { - String field = recordKeyFields.get(idx); + String toReturn = recordKeyFields.stream().map(field -> { String val = null; List fieldPositions = recordKeyPositions.get(field); if (fieldPositions.size() == 1) { // simple field diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index 68f4c6d541db8..c2b8b12f93cad 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -24,29 +24,28 @@ import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; -import java.util.Arrays; +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 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())); + } + + SimpleKeyGenerator(TypedProperties props, String partitionPathField) { + this(props, null, partitionPathField); } - public SimpleKeyGenerator(TypedProperties props, String partitionPathField) { + SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) { super(props); - this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY())); - this.partitionPathFields = Arrays.asList(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 @@ -61,12 +60,14 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), false); + buildFieldPositionMapIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false); } @Override public String getPartitionPath(Row row) { + buildFieldPositionMapIfNeeded(row.schema()); return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), - hiveStylePartitioning, getPartitionPathPositions()); + hiveStylePartitioning, partitionPathPositions); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index 752accf034a7d..0209fe8b370ad 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -94,11 +94,16 @@ public static class Config { } 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(); @@ -194,13 +199,15 @@ private long convertLongTimeToMillis(Long partitionVal) { @Override public String getRecordKey(Row row) { - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), getRecordKeyPositions(), false); + buildFieldPositionMapIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false); } @Override public String getPartitionPath(Row row) { Object fieldVal = null; - Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, getPartitionPathPositions().get(getPartitionPathFields().get(0))); + 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)) { diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java index c17530fe6d56f..884e11ca8af33 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java @@ -38,12 +38,12 @@ import java.util.List; import java.util.Random; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.ENCODER; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.STRUCT_TYPE; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getConfigBuilder; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getInternalRowWithError; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getRandomRows; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.toInternalRows; +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; diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java index 01d78933b0aa3..89d748f671aac 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java +++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java @@ -40,11 +40,11 @@ import java.util.List; import java.util.Random; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.ENCODER; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.STRUCT_TYPE; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getConfigBuilder; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.getRandomRows; -import static org.apache.hudi.testutils.HoodieDatasetTestUtils.toInternalRows; +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; diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java index 1b35a7d5cb287..4c5ded3089a72 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -24,11 +24,12 @@ 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(); @@ -78,8 +79,7 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -89,9 +89,7 @@ public void testHappyFlow() { 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 = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + 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"); } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java index 45d3eff5c8f1a..add2547e7e4c8 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java @@ -23,11 +23,12 @@ 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(); @@ -103,9 +104,7 @@ public void testSimpleKeyGenerator() { HoodieKey key = keyGenerator.getKey(record); Assertions.assertEquals(key.getRecordKey(), "key1"); Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686"); - Row row = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); } @@ -117,9 +116,7 @@ public void testTimestampBasedKeyGenerator() { HoodieKey key = keyGenerator.getKey(record); Assertions.assertEquals(key.getRecordKey(), "key1"); Assertions.assertEquals(key.getPartitionPath(), "ts_ms=20200321"); - Row row = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321"); } @@ -131,9 +128,7 @@ public void testNonPartitionedKeyGenerator() { HoodieKey key = keyGenerator.getKey(record); Assertions.assertEquals(key.getRecordKey(), "key1"); Assertions.assertTrue(key.getPartitionPath().isEmpty()); - Row row = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty()); } @@ -151,15 +146,12 @@ public void testInvalidPartitionKeyType() { try { KeyGenerator keyGenerator = new CustomKeyGenerator(getInvalidPartitionKeyTypeProps()); GenericRecord record = getRecord(); - Row row = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); - keyGenerator.getRecordKey(row); + 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 @@ -175,9 +167,7 @@ public void testNoRecordKeyFieldProp() { try { KeyGenerator keyGenerator = new CustomKeyGenerator(getPropsWithoutRecordKeyFieldProps()); GenericRecord record = getRecord(); - Row row = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = KeyGeneratorTestUtilities.getRow(record); keyGenerator.getRecordKey(row); Assertions.fail("should fail when record key field is not provided!"); } catch (Exception e) { @@ -198,9 +188,7 @@ public void testPartitionFieldsInImproperFormat() { try { KeyGenerator keyGenerator = new CustomKeyGenerator(getImproperPartitionFieldFormatProp()); GenericRecord record = getRecord(); - Row row = TestKeyGeneratorUtilities.getRow(record); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); + Row row = KeyGeneratorTestUtilities.getRow(record); keyGenerator.getPartitionPath(row); Assertions.fail("should fail when partition key field is provided in improper format!"); } catch (Exception e) { @@ -216,9 +204,7 @@ public void testComplexRecordKeyWithSimplePartitionPath() { Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686"); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); - Row row = TestKeyGeneratorUtilities.getRow(record); + Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); } @@ -231,9 +217,7 @@ public void testComplexRecordKeysWithComplexPartitionPath() { Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=20200321"); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); - Row row = TestKeyGeneratorUtilities.getRow(record); + 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"); } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java index 111034d3cfd94..96d607af8cae1 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java @@ -24,11 +24,12 @@ 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(); @@ -68,8 +69,7 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -79,9 +79,8 @@ public void testHappyFlow() { HoodieKey key = keyGenerator.getKey(record); Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(key.getPartitionPath(), ""); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, - TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); - Row row = TestKeyGeneratorUtilities.getRow(record); + 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), ""); } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java index 3bdfbc41907f5..4eb184e26f499 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java @@ -24,11 +24,12 @@ 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(); @@ -88,8 +89,7 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, - () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -97,8 +97,7 @@ public void testWrongPartitionPathField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps()); GenericRecord record = getRecord(); Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.DEFAULT_PARTITION_PATH); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); - Assertions.assertEquals(keyGenerator.getPartitionPath(TestKeyGeneratorUtilities.getRow(record)), + Assertions.assertEquals(keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)), KeyGenUtils.DEFAULT_PARTITION_PATH); } @@ -106,8 +105,7 @@ public void testWrongPartitionPathField() { public void testComplexRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, - () -> keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -118,8 +116,7 @@ public void testHappyFlow() { Assertions.assertEquals(key.getRecordKey(), "key1"); Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686"); - keyGenerator.initializeRowKeyGenerator(TestKeyGeneratorUtilities.structType, TestKeyGeneratorUtilities.TEST_STRUCTNAME, TestKeyGeneratorUtilities.TEST_RECORD_NAMESPACE); - Row row = TestKeyGeneratorUtilities.getRow(record); + Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); } diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index 97ed73760c58c..6afc6ebb00544 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -49,8 +49,6 @@ public class TestTimestampBasedKeyGenerator { private Schema schema; private StructType structType; private Row baseRow; - private String testStructName = "testStructName"; - private String testNamespace = "testNamespace"; @BeforeEach public void initialize() throws IOException { @@ -121,7 +119,6 @@ public void testTimestampBasedKeyGenerator() throws IOException { // test w/ Row baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT @@ -131,7 +128,6 @@ public void testTimestampBasedKeyGenerator() throws IOException { assertEquals("2020-01-06 04", hk2.getPartitionPath()); // test w/ Row - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow)); // timestamp is DATE_STRING, timezone is GMT+8:00 @@ -144,7 +140,6 @@ public void testTimestampBasedKeyGenerator() throws IOException { // test w/ Row baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); // timezone is GMT @@ -154,7 +149,6 @@ public void testTimestampBasedKeyGenerator() throws IOException { assertEquals("2020-01-06 12", hk4.getPartitionPath()); // test w/ Row - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); } @@ -171,7 +165,6 @@ public void testScalar() throws IOException { // test w/ Row baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow)); } @@ -190,7 +183,6 @@ public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsU Assertions.assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @@ -209,7 +201,6 @@ public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAs Assertions.assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @@ -228,7 +219,6 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone Assertions.assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @@ -247,7 +237,6 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAs Assertions.assertEquals("2020040113", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040113", keyGen.getPartitionPath(baseRow)); } @@ -266,7 +255,6 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputT Assertions.assertEquals("2020040118", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040118", keyGen.getPartitionPath(baseRow)); } @@ -285,7 +273,6 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_Outpu Assertions.assertEquals("2020040118", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040118", keyGen.getPartitionPath(baseRow)); } @@ -304,7 +291,6 @@ public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezone Assertions.assertEquals("2020040109", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("2020040109", keyGen.getPartitionPath(baseRow)); } @@ -322,7 +308,6 @@ public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throw Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getKey(baseRecord)); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getPartitionPath(baseRow)); } @@ -341,7 +326,6 @@ public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() Assertions.assertEquals("04/01/2020", hk1.getPartitionPath()); baseRow = genericRecordToRow(baseRecord); - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace); assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow)); } } \ No newline at end of file diff --git a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java b/hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java similarity index 97% rename from hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java rename to hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java index e756d1f71a83d..53b2abfd9b6ea 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/keygen/TestKeyGeneratorUtilities.java +++ b/hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.keygen; +package org.apache.hudi.testutils; import org.apache.hudi.AvroConversionHelper; import org.apache.hudi.AvroConversionUtils; @@ -30,7 +30,7 @@ import scala.Function1; -public class TestKeyGeneratorUtilities { +public class KeyGeneratorTestUtilities { public static String exampleSchema = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"}," diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index 76a45ba9b5eb8..73e1f5df63dea 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -22,11 +22,12 @@ 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._ +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} @@ -47,7 +48,7 @@ class TestDataSourceDefaults { @BeforeEach def initialize(): Unit = { baseRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 1, "001", "f1") - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) } private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = { @@ -66,7 +67,6 @@ class TestDataSourceDefaults { assertEquals("field1", hk1.getRecordKey) assertEquals("name1", hk1.getPartitionPath) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1", keyGen.getRecordKey(baseRow)) assertEquals("name1", keyGen.getPartitionPath(baseRow)) @@ -86,7 +86,6 @@ class TestDataSourceDefaults { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") val keyGen = new SimpleKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -110,7 +109,6 @@ class TestDataSourceDefaults { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField") val keyGen = new SimpleKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getPartitionPath(baseRow) fail("Should have errored out") } catch { @@ -141,7 +139,6 @@ class TestDataSourceDefaults { // if partition path can't be found, return default partition path using row keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) val hk3_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk3_row) @@ -151,7 +148,6 @@ class TestDataSourceDefaults { // if enable hive style partitioning using row keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) val hk4_row = keyGen.getPartitionPath(baseRow) assertEquals("name=name1", hk4_row) @@ -163,8 +159,7 @@ class TestDataSourceDefaults { // if partition is null, return default partition path using Row keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) val hk5_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk5_row) @@ -176,8 +171,7 @@ class TestDataSourceDefaults { // if partition is empty, return default partition path using Row keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) val hk6_row = keyGen.getPartitionPath(baseRow) assertEquals("default", hk6_row) @@ -200,8 +194,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name") keyGen = new SimpleKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -228,8 +221,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name") keyGen = new SimpleKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -238,35 +230,19 @@ class TestDataSourceDefaults { } } - @Test def testUserDefinedKeyGenerator(): Unit = { - var keyGen = new UserDefinedKeyGenerator(getKeyConfig("field1", "name", "false")) - val hk1 = keyGen.getKey(baseRecord) - assertEquals("field1", hk1.getRecordKey) - assertEquals("name1", hk1.getPartitionPath) - - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + @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 BuiltinKeyGenerator(props) { + 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) - this.recordKeyFields = util.Arrays.asList(recordKeyProp) - this.partitionPathFields = util.Arrays.asList(partitionPathProp) - - /** - * Generate a record Key out of provided generic record. - */ - override def getRecordKey(record: GenericRecord): String = { - HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true) - } - /** - * Generate a partition path out of provided generic record. - */ - override def getPartitionPath(record: GenericRecord): String = { - HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true) + override def getKey(record: GenericRecord): HoodieKey = { + new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true), + HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true)) } } @@ -278,7 +254,6 @@ class TestDataSourceDefaults { assertEquals("field1/name1", hk1.getPartitionPath) // top level, valid fields with Row - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) assertEquals("field1/name1", keyGen.getPartitionPath(baseRow)) @@ -298,7 +273,6 @@ class TestDataSourceDefaults { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1") val keyGen = new ComplexKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -322,7 +296,7 @@ class TestDataSourceDefaults { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField") val keyGen = new ComplexKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + keyGen.getPartitionPath(baseRow) fail("Should have errored out") } catch { case e: IllegalArgumentException => @@ -336,7 +310,6 @@ class TestDataSourceDefaults { assertEquals("UserId1@001/false", hk2.getPartitionPath) // nested field as record key and partition path - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow)) assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow)) @@ -353,7 +326,6 @@ class TestDataSourceDefaults { // Nested record key not found try { val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -366,7 +338,6 @@ class TestDataSourceDefaults { val hk3 = keyGen.getKey(baseRecord) assertEquals("default", hk3.getPartitionPath) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("default", keyGen.getPartitionPath(baseRow)) // if enable hive style partitioning @@ -375,7 +346,6 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:name1", hk4.getRecordKey) assertEquals("field1=field1/name=name1", hk4.getPartitionPath) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow)) @@ -386,8 +356,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__empty__", hk5.getRecordKey) assertEquals("field1/default", hk5.getPartitionPath) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow)) assertEquals("field1/default", keyGen.getPartitionPath(baseRow)) @@ -398,8 +367,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__null__", hk6.getRecordKey) assertEquals("field1/default", hk6.getPartitionPath) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) assertEquals("field1/default", keyGen.getPartitionPath(baseRow)) @@ -425,8 +393,7 @@ class TestDataSourceDefaults { props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name") props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name") keyGen = new ComplexKeyGenerator(props) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -442,8 +409,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:name1", hk7.getRecordKey) assertEquals("field1/name1", hk7.getPartitionPath) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) assertEquals("field1/name1", keyGen.getPartitionPath(baseRow)) @@ -452,7 +418,6 @@ class TestDataSourceDefaults { assertEquals("field1:field1", hk8.getRecordKey) assertEquals("field1", hk8.getPartitionPath) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1", keyGen.getRecordKey(baseRow)) assertEquals("field1", keyGen.getPartitionPath(baseRow)) } @@ -464,7 +429,6 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:name1", hk1.getRecordKey) assertEquals("", hk1.getPartitionPath) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) assertEquals("", keyGen.getPartitionPath(baseRow)) @@ -476,7 +440,6 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:name1", hk2.getRecordKey) assertEquals("", hk2.getPartitionPath) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) assertEquals("", keyGen.getPartitionPath(baseRow)) @@ -487,8 +450,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__empty__", hk3.getRecordKey) assertEquals("", hk3.getPartitionPath) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow)) assertEquals("", keyGen.getPartitionPath(baseRow)) @@ -499,8 +461,7 @@ class TestDataSourceDefaults { assertEquals("field1:field1,name:__null__", hk4.getRecordKey) assertEquals("", hk4.getPartitionPath) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) assertEquals("", keyGen.getPartitionPath(baseRow)) @@ -520,7 +481,6 @@ class TestDataSourceDefaults { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField") val keyGen = new GlobalDeleteKeyGenerator(props) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -541,7 +501,6 @@ class TestDataSourceDefaults { // Nested record key not found try { val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - keyGen.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch { @@ -566,11 +525,10 @@ class TestDataSourceDefaults { try { baseRecord.put("name", "") baseRecord.put("field1", null) - baseRow = TestKeyGeneratorUtilities.getRow(baseRecord, schema, structType) + 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.initializeRowKeyGenerator(structType, testStructName, testNamespace) keyGen.getRecordKey(baseRow) fail("Should have errored out") } catch {