adding support for complex keys (#728)
- Resolving the issue related to ambiguity in recordKey by creating and parsing json object as string. - added unit test for ComplexKeyGenerator - minor changes
This commit is contained in:
committed by
vinoth chandar
parent
1b61eb45e0
commit
17e878f721
@@ -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 com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.util.TypedProperties;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
/**
|
||||
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as
|
||||
* configs.
|
||||
*/
|
||||
public class ComplexKeyGenerator extends KeyGenerator {
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH = "default";
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
|
||||
protected final List<String> partitionPathFields;
|
||||
|
||||
public ComplexKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","));
|
||||
this.partitionPathFields = Arrays.asList(props
|
||||
.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyFields == null || partitionPathFields == null) {
|
||||
throw new HoodieException(
|
||||
"Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (String recordKeyField : recordKeyFields) {
|
||||
recordKey.append(recordKeyField + ":" + DataSourceUtils.getNestedFieldValAsString(record, recordKeyField) + ",");
|
||||
}
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
try {
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
partitionPath.append(DataSourceUtils.getNestedFieldValAsString(record, partitionPathField));
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
} catch (HoodieException e) {
|
||||
partitionPath = partitionPath.append(DEFAULT_PARTITION_PATH);
|
||||
}
|
||||
|
||||
return new HoodieKey(recordKey.toString(), partitionPath.toString());
|
||||
}
|
||||
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user