1
0

[HUDI-912] Refactor and relocate KeyGenerator to support more engines (#2200)

* [HUDI-912] Refactor and relocate KeyGenerator to support more engines

* Rename KeyGenerators
This commit is contained in:
wangxianghu
2020-11-03 05:12:51 +08:00
committed by GitHub
parent 59f995a3f5
commit d160abb437
38 changed files with 1038 additions and 524 deletions

View File

@@ -0,0 +1,120 @@
/*
* 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.ApiMaturityLevel;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Base class for the built-in key generators. Contains methods structured for
* code reuse amongst them.
*/
public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements SparkKeyGeneratorInterface {
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
private static final String NAMESPACE = "hoodieRow";
private transient Function1<Object, Object> converterFn = null;
protected StructType structType;
protected Map<String, List<Integer>> recordKeyPositions = new HashMap<>();
protected Map<String, List<Integer>> partitionPathPositions = new HashMap<>();
protected BuiltinKeyGenerator(TypedProperties config) {
super(config);
}
/**
* 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
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getRecordKey(Row row) {
if (null == converterFn) {
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
return getKey(genericRecord).getRecordKey();
}
/**
* Fetch partition path from {@link Row}.
* @param row instance of {@link Row} from which partition path is requested
* @return the partition path of interest from {@link Row}.
*/
@Override
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getPartitionPath(Row row) {
if (null == converterFn) {
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
return getKey(genericRecord).getPartitionPath();
}
void buildFieldPositionMapIfNeeded(StructType structType) {
if (this.structType == null) {
// parse simple fields
getRecordKeyFields().stream()
.filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
} else {
throw new HoodieKeyException("recordKey value not found for field: \"" + f + "\"");
}
});
// parse nested fields
getRecordKeyFields().stream()
.filter(f -> f.contains("."))
.forEach(f -> recordKeyPositions.put(f, RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, true)));
// parse simple fields
if (getPartitionPathFields() != null) {
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
partitionPathPositions.put(f,
Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
} else {
partitionPathPositions.put(f, Collections.singletonList(-1));
}
});
// parse nested fields
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains("."))
.forEach(f -> partitionPathPositions.put(f,
RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false)));
}
this.structType = structType;
}
}
}

View File

@@ -0,0 +1,67 @@
/*
* 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.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.sql.Row;
import java.util.Arrays;
import java.util.stream.Collectors;
/**
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
*/
public class ComplexKeyGenerator extends BuiltinKeyGenerator {
private final ComplexAvroKeyGenerator complexAvroKeyGenerator;
public ComplexKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props);
}
@Override
public String getRecordKey(GenericRecord record) {
return complexAvroKeyGenerator.getRecordKey(record);
}
@Override
public String getPartitionPath(GenericRecord record) {
return complexAvroKeyGenerator.getPartitionPath(record);
}
@Override
public String getRecordKey(Row row) {
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, partitionPathPositions);
}
}

View File

@@ -0,0 +1,135 @@
/*
* 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.util.Option;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.sql.Row;
import java.io.IOException;
import java.util.Arrays;
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:
*
* properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
*
* The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
*
* Few points to consider: 1. If you want to customize some partition path field on a timestamp basis, you can use field1:timestampBased 2. If you simply want to have the value of your configured
* field in the partition path, use field1:simple 3. If you want your table to be non partitioned, simply leave it as blank.
*
* RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
*/
public class CustomKeyGenerator extends BuiltinKeyGenerator {
private final CustomAvroKeyGenerator customAvroKeyGenerator;
public CustomKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
customAvroKeyGenerator = new CustomAvroKeyGenerator(props);
}
@Override
public String getRecordKey(GenericRecord record) {
return customAvroKeyGenerator.getRecordKey(record);
}
@Override
public String getPartitionPath(GenericRecord record) {
return customAvroKeyGenerator.getPartitionPath(record);
}
@Override
public String getRecordKey(Row row) {
validateRecordKeyFields();
return getRecordKeyFields().size() == 1
? new SimpleKeyGenerator(config).getRecordKey(row)
: new ComplexKeyGenerator(config).getRecordKey(row);
}
@Override
public String getPartitionPath(Row row) {
return getPartitionPath(Option.empty(), Option.of(row));
}
private String getPartitionPath(Option<GenericRecord> record, Option<Row> row) {
if (getPartitionPathFields() == null) {
throw new HoodieKeyException("Unable to find field names for partition path in cfg");
}
String partitionPathField;
StringBuilder partitionPath = new StringBuilder();
//Corresponds to no partition case
if (getPartitionPathFields().size() == 1 && getPartitionPathFields().get(0).isEmpty()) {
return "";
}
for (String field : getPartitionPathFields()) {
String[] fieldWithType = field.split(customAvroKeyGenerator.getSplitRegex());
if (fieldWithType.length != 2) {
throw new HoodieKeyGeneratorException("Unable to find field names for partition path in proper format");
}
partitionPathField = fieldWithType[0];
CustomAvroKeyGenerator.PartitionKeyType keyType = CustomAvroKeyGenerator.PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
switch (keyType) {
case SIMPLE:
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 {
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 HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class");
}
break;
default:
throw new HoodieKeyGeneratorException("Please provide valid PartitionKeyType with fields! You provided: " + keyType);
}
partitionPath.append(customAvroKeyGenerator.getDefaultPartitionPathSeparator());
}
partitionPath.deleteCharAt(partitionPath.length() - 1);
return partitionPath.toString();
}
private void validateRecordKeyFields() {
if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) {
throw new HoodieKeyException("Unable to find field names for record key in cfg");
}
}
}

View File

@@ -0,0 +1,69 @@
/*
* 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.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.sql.Row;
import java.util.ArrayList;
import java.util.Arrays;
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.
*/
public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
private final GlobalAvroDeleteKeyGenerator globalAvroDeleteKeyGenerator;
public GlobalDeleteKeyGenerator(TypedProperties config) {
super(config);
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(","));
globalAvroDeleteKeyGenerator = new GlobalAvroDeleteKeyGenerator(config);
}
@Override
public String getRecordKey(GenericRecord record) {
return globalAvroDeleteKeyGenerator.getRecordKey(record);
}
@Override
public String getPartitionPath(GenericRecord record) {
return globalAvroDeleteKeyGenerator.getPartitionPath(record);
}
@Override
public List<String> getPartitionPathFields() {
return new ArrayList<>();
}
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
}
@Override
public String getPartitionPath(Row row) {
return globalAvroDeleteKeyGenerator.getEmptyPartition();
}
}

View File

@@ -0,0 +1,55 @@
/*
* 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.spark.sql.Row;
import java.util.List;
/**
* Simple Key generator for unpartitioned Hive Tables.
*/
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
private final NonpartitionedAvroKeyGenerator nonpartitionedAvroKeyGenerator;
public NonpartitionedKeyGenerator(TypedProperties config) {
super(config);
nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(config);
}
@Override
public String getPartitionPath(GenericRecord record) {
return nonpartitionedAvroKeyGenerator.getPartitionPath(record);
}
@Override
public List<String> getPartitionPathFields() {
return nonpartitionedAvroKeyGenerator.getPartitionPathFields();
}
@Override
public String getPartitionPath(Row row) {
return nonpartitionedAvroKeyGenerator.getEmptyPartition();
}
}

View File

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

View File

@@ -0,0 +1,75 @@
/*
* 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.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.sql.Row;
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 {
private final SimpleAvroKeyGenerator simpleAvroKeyGenerator;
public SimpleKeyGenerator(TypedProperties props) {
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
}
SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
this(props, null, partitionPathField);
}
SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
super(props);
this.recordKeyFields = recordKeyField == null
? Collections.emptyList()
: Collections.singletonList(recordKeyField);
this.partitionPathFields = Collections.singletonList(partitionPathField);
simpleAvroKeyGenerator = new SimpleAvroKeyGenerator(props, recordKeyField, partitionPathField);
}
@Override
public String getRecordKey(GenericRecord record) {
return simpleAvroKeyGenerator.getRecordKey(record);
}
@Override
public String getPartitionPath(GenericRecord record) {
return simpleAvroKeyGenerator.getPartitionPath(record);
}
@Override
public String getRecordKey(Row row) {
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, partitionPathPositions);
}
}

View File

@@ -0,0 +1,82 @@
/*
* 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.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.sql.Row;
import java.io.IOException;
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.
*/
public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
private final TimestampBasedAvroKeyGenerator timestampBasedAvroKeyGenerator;
public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
}
TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
this(config, null, partitionPathField);
}
TimestampBasedKeyGenerator(TypedProperties config, String recordKeyField, String partitionPathField) throws IOException {
super(config, recordKeyField, partitionPathField);
timestampBasedAvroKeyGenerator = new TimestampBasedAvroKeyGenerator(config, recordKeyField, partitionPathField);
}
@Override
public String getPartitionPath(GenericRecord record) {
return timestampBasedAvroKeyGenerator.getPartitionPath(record);
}
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
}
@Override
public String getPartitionPath(Row row) {
Object fieldVal = null;
buildFieldPositionMapIfNeeded(row.schema());
Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathPositions.get(getPartitionPathFields().get(0)));
try {
if (partitionPathFieldVal == null || partitionPathFieldVal.toString().contains(DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER)
|| partitionPathFieldVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
fieldVal = timestampBasedAvroKeyGenerator.getDefaultPartitionVal();
} else {
fieldVal = partitionPathFieldVal;
}
return timestampBasedAvroKeyGenerator.getPartitionPath(fieldVal);
} catch (Exception e) {
throw new HoodieKeyGeneratorException("Unable to parse input partition field :" + fieldVal, e);
}
}
}