You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/04/24 05:40:50 UTC

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1433: [HUDI-728]: Implement custom key generator

vinothchandar commented on a change in pull request #1433:
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r411494931



##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
##########
@@ -40,4 +40,22 @@ protected KeyGenerator(TypedProperties config) {
    * Generate a Hoodie Key out of provided generic record.
    */
   public abstract HoodieKey getKey(GenericRecord record);
+
+  public abstract String getPartitionPath(GenericRecord record, String partitionPathField);
+
+  public abstract String getRecordKey(GenericRecord record);
+
+  public enum PartitionKeyType {
+    simple("simple"), complex("complex"), timestampBased("timestampBased"), noPartition("noPartition");

Review comment:
       please name enum elements like constants.. all caps

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
##########
@@ -40,4 +40,22 @@ protected KeyGenerator(TypedProperties config) {
    * Generate a Hoodie Key out of provided generic record.
    */
   public abstract HoodieKey getKey(GenericRecord record);
+
+  public abstract String getPartitionPath(GenericRecord record, String partitionPathField);

Review comment:
       javadocs for these abstract methods? 
   
   Also I am not sure if it makes sense to add these component methods here.. We already have a top level one `getKey()`

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.noPartition;
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.simple;
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.timestampBased;
+
+/**
+ * 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 customise some partition path field on a timestamp basis, you can use field1:timestampBased

Review comment:
       all this needs to be documented for the user? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
##########
@@ -64,6 +62,32 @@ public HoodieKey getKey(GenericRecord record) {
       throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
     }
 
+    String recordKey = getRecordKey(record);
+    StringBuilder partitionPath = new StringBuilder();
+    for (String partitionPathField : partitionPathFields) {
+      partitionPath.append(getPartitionPath(record, partitionPathField));
+      partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
+    }
+    partitionPath.deleteCharAt(partitionPath.length() - 1);
+
+    return new HoodieKey(recordKey, partitionPath.toString());
+  }
+
+  @Override
+  public String getPartitionPath(GenericRecord record, String partitionPathField) {

Review comment:
       I assume this whol;e change is just restructuring code 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
##########
@@ -40,4 +40,22 @@ protected KeyGenerator(TypedProperties config) {
    * Generate a Hoodie Key out of provided generic record.
    */
   public abstract HoodieKey getKey(GenericRecord record);
+
+  public abstract String getPartitionPath(GenericRecord record, String partitionPathField);
+
+  public abstract String getRecordKey(GenericRecord record);
+
+  public enum PartitionKeyType {
+    simple("simple"), complex("complex"), timestampBased("timestampBased"), noPartition("noPartition");
+
+    private String type;

Review comment:
       this is not needed right? we can just use ordinal/name from the name itself?

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
##########
@@ -40,4 +40,22 @@ protected KeyGenerator(TypedProperties config) {
    * Generate a Hoodie Key out of provided generic record.
    */
   public abstract HoodieKey getKey(GenericRecord record);
+
+  public abstract String getPartitionPath(GenericRecord record, String partitionPathField);

Review comment:
       I understand you want to reuse the other key generators.. I suggest exposing the methods in those concrete classes without leaking it to the abstract class here.. This will otherwise break all custom key generator implementations out there.. 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.noPartition;
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.simple;
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.timestampBased;
+
+/**
+ * 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 customise 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, use field1:noPartition.

Review comment:
       or can they leave the partitionPathFields as empty?  that seems more intuitive? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.noPartition;
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.simple;
+import static org.apache.hudi.keygen.KeyGenerator.PartitionKeyType.timestampBased;
+
+/**
+ * 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 customise 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, use field1:noPartition.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List<String> recordKeyFields;
+  protected final List<String> partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  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());
+  }
+
+  @Override
+  public HoodieKey getKey(GenericRecord record) {
+    //call function to get the record key
+    String recordKey = getRecordKey(record);
+    //call function to get the partition key based on the type for that partition path field
+    String partitionPath = getPartitionPath(record, null);
+    return new HoodieKey(recordKey, partitionPath);
+  }
+
+  @Override
+  public String getPartitionPath(GenericRecord record, String pathField) {
+    if (partitionPathFields == null || partitionPathFields.isEmpty()) {
+      throw new HoodieKeyException("Unable to find field names for partition path in cfg");
+    }
+
+    String partitionPathField;
+    KeyGenerator keyGenerator;
+    StringBuilder partitionPath = new StringBuilder();
+    boolean nonPartitionedTable = false;
+    for (String field : partitionPathFields) {
+      String[] fieldWithType = field.split(SPLIT_REGEX);
+      if (fieldWithType.length != 2) {
+        throw new HoodieKeyException("Unable to find field names for partition path in proper format");
+      }
+
+      partitionPathField = fieldWithType[0];
+      switch (getPartitionKeyType(fieldWithType[1])) {
+        case simple:
+          keyGenerator = new SimpleKeyGenerator(properties);
+          break;
+        case timestampBased:
+          keyGenerator = new TimestampBasedKeyGenerator(properties);
+          break;
+        case noPartition:
+          keyGenerator = new NonpartitionedKeyGenerator(properties);
+          nonPartitionedTable = true;
+          break;
+        default:
+          throw new HoodieDeltaStreamerException("Please provide valid PartitionKeyType with fields!");
+      }
+
+      if (nonPartitionedTable) {
+        return "";
+      }
+      partitionPath.append(keyGenerator.getPartitionPath(record, partitionPathField));
+      partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
+    }
+    partitionPath.deleteCharAt(partitionPath.length() - 1);
+
+    return partitionPath.toString();
+  }
+
+  private PartitionKeyType getPartitionKeyType(String type) {
+    switch (type) {
+      case "simple":

Review comment:
       let's turn this into an enum?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org