[HUDI-2176, 2178, 2179] Adding virtual key support to COW table (#3306)
This commit is contained in:
committed by
GitHub
parent
5353243449
commit
61148c1c43
@@ -244,6 +244,22 @@ public class HoodieAvroUtils {
|
||||
return recordSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch schema for record key and partition path.
|
||||
*/
|
||||
public static Schema getSchemaForFields(Schema fileSchema, List<String> fields) {
|
||||
List<Schema.Field> toBeAddedFields = new ArrayList<>();
|
||||
Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false);
|
||||
|
||||
for (Schema.Field schemaField: fileSchema.getFields()) {
|
||||
if (fields.contains(schemaField.name())) {
|
||||
toBeAddedFields.add(new Schema.Field(schemaField.name(), schemaField.schema(), schemaField.doc(), schemaField.defaultValue()));
|
||||
}
|
||||
}
|
||||
recordSchema.setFields(toBeAddedFields);
|
||||
return recordSchema;
|
||||
}
|
||||
|
||||
public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath,
|
||||
String fileName) {
|
||||
record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName);
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.avro;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.parquet.avro.AvroWriteSupport;
|
||||
@@ -33,7 +34,7 @@ import java.util.HashMap;
|
||||
*/
|
||||
public class HoodieAvroWriteSupport extends AvroWriteSupport {
|
||||
|
||||
private BloomFilter bloomFilter;
|
||||
private Option<BloomFilter> bloomFilterOpt;
|
||||
private String minRecordKey;
|
||||
private String maxRecordKey;
|
||||
|
||||
@@ -44,39 +45,41 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
|
||||
public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = "hoodie_bloom_filter_type_code";
|
||||
public static final String HOODIE_AVRO_SCHEMA_METADATA_KEY = "orc.avro.schema";
|
||||
|
||||
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {
|
||||
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, Option<BloomFilter> bloomFilterOpt) {
|
||||
super(schema, avroSchema);
|
||||
this.bloomFilter = bloomFilter;
|
||||
this.bloomFilterOpt = bloomFilterOpt;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteSupport.FinalizedWriteContext finalizeWrite() {
|
||||
HashMap<String, String> extraMetaData = new HashMap<>();
|
||||
if (bloomFilter != null) {
|
||||
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
|
||||
if (bloomFilterOpt.isPresent()) {
|
||||
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilterOpt.get().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());
|
||||
if (bloomFilterOpt.get().getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
|
||||
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilterOpt.get().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 (bloomFilterOpt.isPresent()) {
|
||||
this.bloomFilterOpt.get().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;
|
||||
if (maxRecordKey != null) {
|
||||
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
|
||||
} else {
|
||||
maxRecordKey = recordKey;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -319,12 +319,6 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
* @param operationType operation type to be executed.
|
||||
*/
|
||||
public void validateTableProperties(Properties properties, WriteOperationType operationType) {
|
||||
// disabling meta fields are allowed only for bulk_insert operation
|
||||
if (!Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()))
|
||||
&& operationType != WriteOperationType.BULK_INSERT) {
|
||||
throw new HoodieException(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() + " can only be disabled for " + WriteOperationType.BULK_INSERT
|
||||
+ " operation");
|
||||
}
|
||||
// once meta fields are disabled, it cant be re-enabled for a given table.
|
||||
if (!getTableConfig().populateMetaFields()
|
||||
&& Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()))) {
|
||||
|
||||
@@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
|
||||
public abstract class BaseFileUtils {
|
||||
|
||||
@@ -170,6 +171,15 @@ public abstract class BaseFileUtils {
|
||||
*/
|
||||
public abstract List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath);
|
||||
|
||||
/**
|
||||
* Fetch {@link HoodieKey}s from the given data file.
|
||||
* @param configuration configuration to build fs object
|
||||
* @param filePath The data file path
|
||||
* @param keyGeneratorOpt instance of KeyGenerator.
|
||||
* @return {@link List} of {@link HoodieKey}s fetched from the parquet file
|
||||
*/
|
||||
public abstract List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath, Option<BaseKeyGenerator> keyGeneratorOpt);
|
||||
|
||||
/**
|
||||
* Read the Avro schema of the data file.
|
||||
* @param configuration Configuration
|
||||
|
||||
@@ -41,6 +41,8 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
|
||||
import org.apache.orc.OrcFile;
|
||||
import org.apache.orc.OrcProto.UserMetadataItem;
|
||||
import org.apache.orc.Reader;
|
||||
@@ -109,6 +111,11 @@ public class OrcUtils extends BaseFileUtils {
|
||||
return hoodieKeys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
throw new HoodieIOException("UnsupportedOperation : Disabling meta fields not yet supported for Orc");
|
||||
}
|
||||
|
||||
/**
|
||||
* NOTE: This literally reads the entire file contents, thus should be used with caution.
|
||||
*/
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -115,23 +116,36 @@ public class ParquetUtils extends BaseFileUtils {
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath) {
|
||||
return fetchRecordKeyPartitionPathInternal(configuration, filePath, Option.empty());
|
||||
}
|
||||
|
||||
private List<HoodieKey> fetchRecordKeyPartitionPathInternal(Configuration configuration, Path filePath, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
List<HoodieKey> hoodieKeys = new ArrayList<>();
|
||||
try {
|
||||
if (!filePath.getFileSystem(configuration).exists(filePath)) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
Configuration conf = new Configuration(configuration);
|
||||
conf.addResource(FSUtils.getFs(filePath.toString(), conf).getConf());
|
||||
Schema readSchema = HoodieAvroUtils.getRecordKeyPartitionPathSchema();
|
||||
Schema readSchema = keyGeneratorOpt.map(keyGenerator -> {
|
||||
List<String> fields = new ArrayList<>();
|
||||
fields.addAll(keyGenerator.getRecordKeyFields());
|
||||
fields.addAll(keyGenerator.getPartitionPathFields());
|
||||
return HoodieAvroUtils.getSchemaForFields(readAvroSchema(conf, filePath), fields);
|
||||
})
|
||||
.orElse(HoodieAvroUtils.getRecordKeyPartitionPathSchema());
|
||||
AvroReadSupport.setAvroReadSchema(conf, readSchema);
|
||||
AvroReadSupport.setRequestedProjection(conf, readSchema);
|
||||
ParquetReader reader = AvroParquetReader.builder(filePath).withConf(conf).build();
|
||||
Object obj = reader.read();
|
||||
while (obj != null) {
|
||||
if (obj instanceof GenericRecord) {
|
||||
String recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String partitionPath = ((GenericRecord) obj).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
String recordKey = null;
|
||||
String partitionPath = null;
|
||||
if (keyGeneratorOpt.isPresent()) {
|
||||
recordKey = keyGeneratorOpt.get().getRecordKey((GenericRecord) obj);
|
||||
partitionPath = keyGeneratorOpt.get().getPartitionPath((GenericRecord) obj);
|
||||
} else {
|
||||
recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
partitionPath = ((GenericRecord) obj).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
}
|
||||
hoodieKeys.add(new HoodieKey(recordKey, partitionPath));
|
||||
obj = reader.read();
|
||||
}
|
||||
@@ -142,6 +156,19 @@ public class ParquetUtils extends BaseFileUtils {
|
||||
return hoodieKeys;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch {@link HoodieKey}s from the given parquet file.
|
||||
*
|
||||
* @param configuration configuration to build fs object
|
||||
* @param filePath The parquet file path.
|
||||
* @param keyGeneratorOpt
|
||||
* @return {@link List} of {@link HoodieKey}s fetched from the parquet file
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
return fetchRecordKeyPartitionPathInternal(configuration, filePath, keyGeneratorOpt);
|
||||
}
|
||||
|
||||
public ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
|
||||
ParquetMetadata footer;
|
||||
try {
|
||||
@@ -223,7 +250,7 @@ public class ParquetUtils extends BaseFileUtils {
|
||||
/**
|
||||
* Returns the number of records in the parquet file.
|
||||
*
|
||||
* @param conf Configuration
|
||||
* @param conf Configuration
|
||||
* @param parquetFilePath path of the file
|
||||
*/
|
||||
@Override
|
||||
|
||||
@@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class BaseKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected List<String> recordKeyFields;
|
||||
protected List<String> partitionPathFields;
|
||||
protected final boolean encodePartitionPath;
|
||||
protected final boolean hiveStylePartitioning;
|
||||
|
||||
protected BaseKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
@Override
|
||||
public final HoodieKey getKey(GenericRecord record) {
|
||||
if (getRecordKeyFields() == null || getPartitionPathFields() == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
return new HoodieKey(getRecordKey(record), getPartitionPath(record));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final List<String> getRecordKeyFieldNames() {
|
||||
// For nested columns, pick top level column name
|
||||
return getRecordKeyFields().stream().map(k -> {
|
||||
int idx = k.indexOf('.');
|
||||
return idx > 0 ? k.substring(0, idx) : k;
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,59 @@
|
||||
/*
|
||||
* 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.ApiMaturityLevel;
|
||||
import org.apache.hudi.PublicAPIClass;
|
||||
import org.apache.hudi.PublicAPIMethod;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract class KeyGenerator implements KeyGeneratorInterface {
|
||||
|
||||
protected TypedProperties config;
|
||||
|
||||
protected KeyGenerator(TypedProperties config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a Hoodie Key out of provided generic record.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract HoodieKey getKey(GenericRecord record);
|
||||
|
||||
/**
|
||||
* Used during bootstrap, to project out only the record key fields from bootstrap source dataset.
|
||||
*
|
||||
* @return list of field names, when concatenated make up the record key.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public List<String> getRecordKeyFieldNames() {
|
||||
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
|
||||
+ "Please override this method in your custom key generator.");
|
||||
}
|
||||
}
|
||||
@@ -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.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
|
||||
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<String> getRecordKeyFieldNames();
|
||||
|
||||
}
|
||||
@@ -0,0 +1,59 @@
|
||||
/*
|
||||
* 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.constant;
|
||||
|
||||
import org.apache.hudi.common.config.ConfigClassProperty;
|
||||
import org.apache.hudi.common.config.ConfigGroups;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
@ConfigClassProperty(name = "Key Generator Options",
|
||||
groupName = ConfigGroups.Names.WRITE_CLIENT,
|
||||
description = "Hudi maintains keys (record key + partition path) "
|
||||
+ "for uniquely identifying a particular record. "
|
||||
+ "This config allows developers to setup the Key generator class that "
|
||||
+ "will extract these out of incoming records.")
|
||||
public class KeyGeneratorOptions extends HoodieConfig {
|
||||
|
||||
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.partitionpath.urlencode")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Should we url encode the partition path value, before creating the folder structure.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.hive_style_partitioning")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Flag to indicate whether to use Hive style partitioning.\n"
|
||||
+ "If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.\n"
|
||||
+ "By default false (the names of partition folders are only partition values)");
|
||||
|
||||
public static final ConfigProperty<String> RECORDKEY_FIELD_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.recordkey.field")
|
||||
.defaultValue("uuid")
|
||||
.withDocumentation("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
|
||||
+ "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using\n"
|
||||
+ "the dot notation eg: `a.b.c`");
|
||||
|
||||
public static final ConfigProperty<String> PARTITIONPATH_FIELD_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.partitionpath.field")
|
||||
.defaultValue("partitionpath")
|
||||
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
|
||||
+ "Actual value ontained by invoking .toString()");
|
||||
}
|
||||
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.constant;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Types of {@link org.apache.hudi.keygen.KeyGenerator}.
|
||||
*/
|
||||
public enum KeyGeneratorType {
|
||||
/**
|
||||
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
*/
|
||||
SIMPLE,
|
||||
|
||||
/**
|
||||
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
*/
|
||||
COMPLEX,
|
||||
|
||||
/**
|
||||
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
|
||||
*/
|
||||
TIMESTAMP,
|
||||
|
||||
/**
|
||||
* This is a generic implementation type 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.
|
||||
* <p>
|
||||
* This KeyGenerator 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").
|
||||
*/
|
||||
CUSTOM,
|
||||
|
||||
/**
|
||||
* Simple Key generator for unpartitioned Hive Tables.
|
||||
*/
|
||||
NON_PARTITION,
|
||||
|
||||
/**
|
||||
* Key generator for deletes using global indices.
|
||||
*/
|
||||
GLOBAL_DELETE;
|
||||
|
||||
public static List<String> getNames() {
|
||||
List<String> names = new ArrayList<>(KeyGeneratorType.values().length);
|
||||
Arrays.stream(KeyGeneratorType.values())
|
||||
.forEach(x -> names.add(x.name()));
|
||||
return names;
|
||||
}
|
||||
}
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
@@ -52,7 +53,7 @@ public class TestHoodieAvroWriteSupport {
|
||||
1000, 0.0001, 10000,
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
|
||||
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,
|
||||
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
|
||||
for (String rowKey : rowKeys) {
|
||||
|
||||
@@ -90,6 +90,7 @@ public class HoodieTestDataGenerator {
|
||||
public static final int DEFAULT_PARTITION_DEPTH = 3;
|
||||
public static final String TRIP_SCHEMA_PREFIX = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"timestamp\",\"type\": \"long\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"partition_path\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"rider\", \"type\": \"string\"}," + "{\"name\": \"driver\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"begin_lat\", \"type\": \"double\"}," + "{\"name\": \"begin_lon\", \"type\": \"double\"},"
|
||||
+ "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"},";
|
||||
@@ -123,7 +124,7 @@ public class HoodieTestDataGenerator {
|
||||
+ "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
|
||||
|
||||
public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
|
||||
public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6),"
|
||||
public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6),"
|
||||
+ "map<string,string>,struct<amount:double,currency:string>,array<struct<amount:double,currency:string>>,boolean";
|
||||
|
||||
|
||||
@@ -208,7 +209,7 @@ public class HoodieTestDataGenerator {
|
||||
public static RawTripTestPayload generateRandomValue(
|
||||
HoodieKey key, String instantTime, boolean isFlattened) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(
|
||||
key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
false, isFlattened);
|
||||
return new RawTripTestPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
@@ -230,7 +231,7 @@ public class HoodieTestDataGenerator {
|
||||
* Generates a new avro record of the above schema format for a delete.
|
||||
*/
|
||||
public static RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
true, false);
|
||||
return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true);
|
||||
}
|
||||
@@ -239,21 +240,22 @@ public class HoodieTestDataGenerator {
|
||||
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
|
||||
*/
|
||||
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String instantTime) {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0);
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0);
|
||||
return new HoodieAvroPayload(Option.of(rec));
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
|
||||
long timestamp) {
|
||||
return generateGenericRecord(rowKey, riderName, driverName, timestamp, false, false);
|
||||
return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false);
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
|
||||
long timestamp, boolean isDeleteRecord,
|
||||
boolean isFlattened) {
|
||||
GenericRecord rec = new GenericData.Record(isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA);
|
||||
rec.put("_row_key", rowKey);
|
||||
rec.put("timestamp", timestamp);
|
||||
rec.put("partition_path", partitionPath);
|
||||
rec.put("rider", riderName);
|
||||
rec.put("driver", driverName);
|
||||
rec.put("begin_lat", RAND.nextDouble());
|
||||
@@ -807,7 +809,7 @@ public class HoodieTestDataGenerator {
|
||||
public List<GenericRecord> generateGenericRecords(int numRecords) {
|
||||
List<GenericRecord> list = new ArrayList<>();
|
||||
IntStream.range(0, numRecords).forEach(i -> {
|
||||
list.add(generateGenericRecord(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID()
|
||||
list.add(generateGenericRecord(UUID.randomUUID().toString(), "0", UUID.randomUUID().toString(), UUID.randomUUID()
|
||||
.toString(), RAND.nextLong()));
|
||||
});
|
||||
return list;
|
||||
|
||||
@@ -40,7 +40,7 @@ public class TestAvroOrcUtils extends HoodieCommonTestHarness {
|
||||
// The following types are tested:
|
||||
// DATE, DECIMAL, LONG, INT, BYTES, ARRAY, RECORD, MAP, STRING, FLOAT, DOUBLE
|
||||
TypeDescription orcSchema = TypeDescription.fromString("struct<"
|
||||
+ "timestamp:bigint,_row_key:string,rider:string,driver:string,begin_lat:double,"
|
||||
+ "timestamp:bigint,_row_key:string,partition_path:string,rider:string,driver:string,begin_lat:double,"
|
||||
+ "begin_lon:double,end_lat:double,end_lon:double,"
|
||||
+ "distance_in_meters:int,seconds_since_epoch:bigint,weight:float,nation:binary,"
|
||||
+ "current_date:date,current_ts:bigint,height:decimal(10,6),"
|
||||
|
||||
@@ -23,11 +23,14 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -50,6 +53,7 @@ import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.METADATA_FIELD_SCHEMA;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@@ -151,6 +155,33 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRecordKeyPartitionPathVirtualKeysFromParquet() throws Exception {
|
||||
List<String> rowKeys = new ArrayList<>();
|
||||
List<HoodieKey> expected = new ArrayList<>();
|
||||
String partitionPath = "path1";
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
String rowKey = UUID.randomUUID().toString();
|
||||
rowKeys.add(rowKey);
|
||||
expected.add(new HoodieKey(rowKey, partitionPath));
|
||||
}
|
||||
|
||||
String filePath = Paths.get(basePath, "test.parquet").toUri().toString();
|
||||
Schema schema = getSchemaWithFields(Arrays.asList(new String[]{"abc", "def"}));
|
||||
writeParquetFile(BloomFilterTypeCode.SIMPLE.name(), filePath, rowKeys, schema, true, partitionPath,
|
||||
false, "abc", "def");
|
||||
|
||||
// Read and verify
|
||||
List<HoodieKey> fetchedRows =
|
||||
parquetUtils.fetchRecordKeyPartitionPath(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath),
|
||||
Option.of(new TestBaseKeyGen("abc","def")));
|
||||
assertEquals(rowKeys.size(), fetchedRows.size(), "Total count does not match");
|
||||
|
||||
for (HoodieKey entry : fetchedRows) {
|
||||
assertTrue(expected.contains(entry), "Record key must be in the given filter");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadCounts() throws Exception {
|
||||
String filePath = Paths.get(basePath, "test.parquet").toUri().toString();
|
||||
@@ -168,22 +199,73 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
private void writeParquetFile(String typeCode, String filePath, List<String> rowKeys, Schema schema, boolean addPartitionPathField, String partitionPath) throws Exception {
|
||||
writeParquetFile(typeCode, filePath, rowKeys, schema, addPartitionPathField, partitionPath,
|
||||
true, null, null);
|
||||
}
|
||||
|
||||
private void writeParquetFile(String typeCode, String filePath, List<String> rowKeys, Schema schema, boolean addPartitionPathField, String partitionPathValue,
|
||||
boolean useMetaFields, String recordFieldName, String partitionFieldName) throws Exception {
|
||||
// Write out a parquet file
|
||||
BloomFilter filter = BloomFilterFactory
|
||||
.createBloomFilter(1000, 0.0001, 10000, typeCode);
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
|
||||
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,
|
||||
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
|
||||
for (String rowKey : rowKeys) {
|
||||
GenericRecord rec = new GenericData.Record(schema);
|
||||
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
|
||||
rec.put(useMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD : recordFieldName, rowKey);
|
||||
if (addPartitionPathField) {
|
||||
rec.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath);
|
||||
rec.put(useMetaFields ? HoodieRecord.PARTITION_PATH_METADATA_FIELD : partitionFieldName, partitionPathValue);
|
||||
}
|
||||
writer.write(rec);
|
||||
writeSupport.add(rowKey);
|
||||
}
|
||||
writer.close();
|
||||
}
|
||||
|
||||
private static Schema getSchemaWithFields(List<String> fields) {
|
||||
List<Schema.Field> toBeAddedFields = new ArrayList<>();
|
||||
Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false);
|
||||
|
||||
for (String field: fields) {
|
||||
Schema.Field schemaField =
|
||||
new Schema.Field(field, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE);
|
||||
toBeAddedFields.add(schemaField);
|
||||
}
|
||||
recordSchema.setFields(toBeAddedFields);
|
||||
return recordSchema;
|
||||
}
|
||||
|
||||
class TestBaseKeyGen extends BaseKeyGenerator {
|
||||
|
||||
private String recordKeyField;
|
||||
private String partitionField;
|
||||
|
||||
public TestBaseKeyGen(String recordKeyField, String partitionField) {
|
||||
super(new TypedProperties());
|
||||
this.recordKeyField = recordKeyField;
|
||||
this.partitionField = partitionField;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return record.get(recordKeyField).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(GenericRecord record) {
|
||||
return record.get(partitionField).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getRecordKeyFields() {
|
||||
return Arrays.asList(new String[]{recordKeyField});
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return Arrays.asList(new String[]{partitionField});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user