Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1065,6 +1065,10 @@ public boolean isHiveStylePartitioningEnabled() {
return getBooleanOrDefault(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE);
}

public boolean shouldURLEncodePartitionPath() {
return getBooleanOrDefault(KeyGeneratorOptions.URL_ENCODE_PARTITIONING);
}

public int getMarkersTimelineServerBasedBatchNumThreads() {
return getInt(MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,16 +146,25 @@ public static String getRecordKey(GenericRecord record, String recordKeyField, b
public static String getPartitionPath(GenericRecord record, String partitionPathField,
boolean hiveStylePartitioning, boolean encodePartitionPath, boolean consistentLogicalTimestampEnabled) {
String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true, consistentLogicalTimestampEnabled);
if (partitionPath == null || partitionPath.isEmpty()) {
partitionPath = HUDI_DEFAULT_PARTITION_PATH;
return handlePartitionPathDecoration(partitionPathField, partitionPath, encodePartitionPath, hiveStylePartitioning);
}

public static String handlePartitionPathDecoration(String partitionPathField,
String partitionPathValue,
boolean encodePartitionPath,
boolean hiveStylePartitioning) {
String decoratedPartitionPath = partitionPathValue;
if (StringUtils.isNullOrEmpty(decoratedPartitionPath)) {
decoratedPartitionPath = HUDI_DEFAULT_PARTITION_PATH;
}
if (encodePartitionPath) {
partitionPath = PartitionPathEncodeUtils.escapePathName(partitionPath);
decoratedPartitionPath = PartitionPathEncodeUtils.escapePathName(decoratedPartitionPath);
}
if (hiveStylePartitioning) {
partitionPath = partitionPathField + "=" + partitionPath;
decoratedPartitionPath = partitionPathField + "=" + decoratedPartitionPath;
}
return partitionPath;

return decoratedPartitionPath;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,8 +165,10 @@ object HoodieSparkUtils extends SparkAdapterSupport {
} else {
val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr)
val transform: GenericRecord => GenericRecord =
if (sameSchema) identity
else {
if (sameSchema) {
identity
} else {
// NOTE: Avro schema parsing is performed outside of the transforming lambda
HoodieAvroUtils.rewriteRecordDeep(_, readerAvroSchema)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public class FileSystemTestUtils {
public static final String FORWARD_SLASH = "/";
public static final String FILE_SCHEME = "file";
public static final String COLON = ":";
public static final Random RANDOM = new Random();
public static final Random RANDOM = new Random(0xDEED);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1


public static Path getRandomOuterInMemPath() {
String randomFileName = UUID.randomUUID().toString();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,11 @@
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.BuiltinKeyGenerator;
import org.apache.hudi.keygen.ComplexKeyGenerator;
import org.apache.hudi.keygen.KeyGenUtils;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.table.BulkInsertPartitioner;

import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.Column;
Expand All @@ -38,15 +38,14 @@
import org.apache.spark.sql.api.java.UDF1;
import org.apache.spark.sql.functions;
import org.apache.spark.sql.types.DataTypes;
import scala.collection.JavaConverters;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import scala.collection.JavaConverters;

import static org.apache.spark.sql.functions.callUDF;

/**
Expand Down Expand Up @@ -97,8 +96,31 @@ public static Dataset<Row> prepareHoodieDatasetForBulkInsert(SQLContext sqlConte
// simple fields for both record key and partition path: can directly use withColumn
String partitionPathField = keyGeneratorClass.equals(SimpleKeyGenerator.class.getName()) ? partitionPathFields :
partitionPathFields.substring(partitionPathFields.indexOf(":") + 1);
rowDatasetWithRecordKeysAndPartitionPath = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, functions.col(recordKeyFields).cast(DataTypes.StringType))
.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD, functions.col(partitionPathField).cast(DataTypes.StringType));

// TODO(HUDI-3993) cleanup duplication
String tableName = properties.getString(HoodieWriteConfig.TBL_NAME.key());
String partitionPathDecorationUDFName = PARTITION_PATH_UDF_FN + tableName;

boolean shouldURLEncodePartitionPath = config.shouldURLEncodePartitionPath();
boolean isHiveStylePartitioned = config.isHiveStylePartitioningEnabled();

if (shouldURLEncodePartitionPath || isHiveStylePartitioned) {
sqlContext.udf().register(
partitionPathDecorationUDFName,
(UDF1<String, String>) partitionPathValue ->
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So, I assume this UDF registration would be gone after HUDI-3993?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Correct

KeyGenUtils.handlePartitionPathDecoration(partitionPathField, partitionPathValue,
shouldURLEncodePartitionPath, isHiveStylePartitioned),
DataTypes.StringType);

rowDatasetWithRecordKeysAndPartitionPath =
rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, functions.col(recordKeyFields).cast(DataTypes.StringType))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In ComplexKeyGenerator.getRecordKey(Row row), we setup prefixFieldName as true in method RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true), so the record key will have a prefix, which is record key field name, when we use ComplexKeyGenerator.

As I understand here, we use withColumn here for recordKeyFields, then we will get the same value in RECORD_KEY_METADATA_FIELD as the original recordKeyFields, so no prefix when key generator is ComplexKeyGenerator. Will it cause problem?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@TengHuo That's a good point. It can be a problem if you mix the write operation type or switch row-writing config for a table. I would suggest filing another JIRA ticket to keep it consistent across. I don't deem it to be a blocker but would be good to keep it consistent.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

okay, got it

then think it will have duplicate data issue if user upgrade from 0.10 or older version when they only setup one column as record key and use ComplexKeyGenerator . The same issue as upgrading from 0.10 to 0.11.

.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
callUDF(partitionPathDecorationUDFName, functions.col(partitionPathField).cast(DataTypes.StringType)));
} else {
rowDatasetWithRecordKeysAndPartitionPath =
rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, functions.col(recordKeyFields).cast(DataTypes.StringType))
.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD, functions.col(partitionPathField).cast(DataTypes.StringType));
}
} else {
// use udf
String tableName = properties.getString(HoodieWriteConfig.TBL_NAME.key());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,17 +23,16 @@
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.row.HoodieRowCreateHandle;
import org.apache.hudi.io.storage.row.HoodieRowCreateHandleWithoutMetaFields;
import org.apache.hudi.keygen.BuiltinKeyGenerator;
import org.apache.hudi.keygen.KeyGenUtils;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.hudi.table.HoodieTable;

import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.catalyst.InternalRow;
Expand Down Expand Up @@ -126,14 +125,17 @@ public void write(InternalRow record) throws IOException {
if (populateMetaFields) { // usual path where meta fields are pre populated in prep step.
partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS));
} else { // if meta columns are disabled.
// TODO(HUDI-3993) remove duplication, unify with HoodieDatasetBulkInsertHelper
if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
partitionPath = "";
} else if (simpleKeyGen) { // SimpleKeyGen
Object parititionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
partitionPath = parititionPathValue != null ? parititionPathValue.toString() : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
if (writeConfig.isHiveStylePartitioningEnabled()) {
partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath;
}
Object partitionPathValue = record.get(simplePartitionFieldIndex, simplePartitionFieldDataType);
String partitionPathField = keyGeneratorOpt.get().getPartitionPathFields().get(0);
boolean shouldURLEncodePartitionPath = writeConfig.shouldURLEncodePartitionPath();
boolean hiveStylePartitioningEnabled = writeConfig.isHiveStylePartitioningEnabled();

partitionPath = KeyGenUtils.handlePartitionPathDecoration(partitionPathField,
partitionPathValue == null ? null : partitionPathValue.toString(), shouldURLEncodePartitionPath, hiveStylePartitioningEnabled);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider passing Option.of(partitionPathValue) instead of null

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, done that initially but then decided to optimize it out given that this is a hot-path

} else {
// only BuiltIn key generators are supported if meta fields are disabled.
partitionPath = keyGeneratorOpt.get().getPartitionPath(record, structType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@
*/
public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarness {

protected static final Random RANDOM = new Random();
protected static final Random RANDOM = new Random(0xDEED);

@BeforeEach
public void setUp() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public void testDataInternalWriter(boolean sorted, boolean populateMetaFields) t
}

@Test
public void testDataInternalWriterHiveStylePartitioning() throws Exception {
public void testDataInternalWriterPartitioningHandling() throws Exception {
boolean sorted = true;
boolean populateMetaFields = false;
// init config and table
Expand Down