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

2020-05-21 Thread GitBox


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



##
File path: 
hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
##
@@ -0,0 +1,97 @@
+/*
+ * 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.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.exception.HoodieKeyException;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
+
+  private TypedProperties getCommonProps() {
+TypedProperties properties = new TypedProperties();
+properties.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), 
"_row_key");
+properties.put(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(), 
"true");
+return properties;
+  }
+
+  private TypedProperties getPropertiesWithoutPartitionPathProp() {
+return getCommonProps();
+  }
+
+  private TypedProperties getPropertiesWithoutRecordKeyProp() {
+TypedProperties properties = new TypedProperties();
+properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), 
"timestamp");
+return properties;
+  }
+
+  private TypedProperties getWrongRecordKeyFieldProps() {
+TypedProperties properties = new TypedProperties();
+properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), 
"timestamp");
+properties.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), 
"_wrong_key");
+return properties;
+  }
+
+  private TypedProperties getComplexRecordKeyProp() {
+TypedProperties properties = new TypedProperties();
+properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), 
"timestamp");
+properties.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), 
"_row_key,pii_col");
+return properties;
+  }
+
+  private TypedProperties getProps() {
+TypedProperties properties = getCommonProps();
+properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), 
"timestamp");
+return properties;
+  }
+
+  @Test
+  public void testNullPartitionPathFields() {
+Assertions.assertThrows(IllegalArgumentException.class, () -> new 
SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp()));
+  }
+
+  @Test
+  public void testNullRecordKeyFields() {
+Assertions.assertThrows(IllegalArgumentException.class, () -> new 
SimpleKeyGenerator(getPropertiesWithoutRecordKeyProp()));
+  }
+
+  @Test
+  public void testWrongRecordKeyField() {
+SimpleKeyGenerator keyGenerator = new 
SimpleKeyGenerator(getWrongRecordKeyFieldProps());
+Assertions.assertThrows(HoodieKeyException.class, () -> 
keyGenerator.getRecordKey(getRecord()));
+  }
+
+  @Test
+  public void testComplexRecordKeyField() {
+SimpleKeyGenerator keyGenerator = new 
SimpleKeyGenerator(getComplexRecordKeyProp());
+Assertions.assertThrows(HoodieKeyException.class, () -> 
keyGenerator.getRecordKey(getRecord()));
+  }
+
+  @Test
+  public void testHappyFlow() {
+SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getProps());
+HoodieKey key = keyGenerator.getKey(getRecord());
+Assertions.assertEquals(key.getRecordKey(), "key1");
+Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
+  }
+}

Review comment:
   here as well

##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
##
@@ -154,4 +153,4 @@ private long convertLongTimeToMillis(Long partitionVal) {
 
 return MILLISECONDS.convert(partitionVal, timeUnit);
   }
-}
+}

Review comment:
   need a new line

##
File path: 
hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
##
@@ -0,0 +1,169 @@
+/*
+ * 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
+ * 

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

2020-05-21 Thread GitBox


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



##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+/**
+ * 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 / 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, simply leave it as blank.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or 
ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+SIMPLE, TIMESTAMP
+  }
+
+  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);
+return new HoodieKey(recordKey, partitionPath);
+  }
+
+  public String getPartitionPath(GenericRecord record) {
+if (partitionPathFields == 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 (partitionPathFields.size() == 1 && 
partitionPathFields.get(0).isEmpty()) {

Review comment:
   ok, my bad. Just checked the trim docs, it will just trim leading and 
trailing whitespaces if any. 





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:
us...@infra.apache.org




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

2020-05-17 Thread GitBox


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



##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##
@@ -0,0 +1,126 @@
+/*
+ * 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;
+
+/**
+ * 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 / 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 KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+SIMPLE, TIMESTAMP
+  }
+
+  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);
+return new HoodieKey(recordKey, partitionPath);
+  }
+
+  private String getPartitionPath(GenericRecord record) {
+if (partitionPathFields == 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 (partitionPathFields.size() == 1 && 
partitionPathFields.get(0).isEmpty()) {
+  return "";
+}
+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];
+  PartitionKeyType keyType = 
PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
+  switch (keyType) {
+case SIMPLE:
+  partitionPath.append(new 
SimpleKeyGenerator(properties).getPartitionPath(record, partitionPathField));
+  break;
+case TIMESTAMP:
+  partitionPath.append(new 
TimestampBasedKeyGenerator(properties).getPartitionPath(record, 
partitionPathField));
+  break;
+

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

2020-05-17 Thread GitBox


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



##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+/**
+ * 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 / 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, simply leave it as blank.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or 
ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+SIMPLE, TIMESTAMP
+  }
+
+  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);
+return new HoodieKey(recordKey, partitionPath);
+  }
+
+  public String getPartitionPath(GenericRecord record) {
+if (partitionPathFields == 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 (partitionPathFields.size() == 1 && 
partitionPathFields.get(0).isEmpty()) {

Review comment:
   I get it, but have a follow up question and a comment. 
   - I am not sure why would someone set empty string. if users wants to not 
have any partitions, might as well not set it only ? Anyways, guess you 
discussed w/ vinoth on that. Will let you folks decide on whats better. 
   - Assuming, user might set it to empty, in line 69 where we generate  
partitionPathFields, we trim for empty strings isn't? So, the list should be 
empty in that case in my understanding. correct me if I am wrong. 





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, 

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

2020-05-14 Thread GitBox


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



##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
##
@@ -66,6 +68,14 @@ public HoodieKey getKey(GenericRecord record) {
   partitionPath = partitionPathField + "=" + partitionPath;
 }
 
-return new HoodieKey(recordKey, partitionPath);
+return partitionPath;
+  }
+
+  public String getRecordKey(GenericRecord record) {

Review comment:
   did you think if we need to make this an abstract method in 
KeyGenerator? 

##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+/**
+ * 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 / 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:
   minor typo. "customize"

##
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
##
@@ -49,21 +50,44 @@
 
   public ComplexKeyGenerator(TypedProperties props) {
 super(props);
-this.recordKeyFields = 
Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","))
-.stream().map(String::trim).collect(Collectors.toList());
-this.partitionPathFields =
-
Arrays.asList(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","))
-.stream().map(String::trim).collect(Collectors.toList());
+DataSourceUtils.checkRequiredProperties(props, Arrays.asList(
+DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(),
+DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY())
+);
+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());
 this.hiveStylePartitioning = 
props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
 
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
   }
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
-if (recordKeyFields == null || partitionPathFields == null) {
-  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());
+  }
+
+  public String getPartitionPath(GenericRecord record, String 
partitionPathField) {

Review comment:
   does this needs to be public? why not protected or package 

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

2020-03-30 Thread GitBox
nsivabalan commented on a change in pull request #1433: [HUDI-728]: Implement 
custom key generator
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r400521746
 
 

 ##
 File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
 ##
 @@ -0,0 +1,110 @@
+/*
+ * 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.util.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;
+
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List 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) {
+  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 (fieldWithType[1]) {
+case "simple":
+  keyGenerator = new SimpleKeyGenerator(properties);
+  break;
+case "timestampBased":
+  keyGenerator = new TimestampBasedKeyGenerator(properties);
+  break;
+case "noPartition":
+  keyGenerator = new NonpartitionedKeyGenerator(properties);
 
 Review comment:
   Do you think we can add tests for these cases as well? 


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-30 Thread GitBox
nsivabalan commented on a change in pull request #1433: [HUDI-728]: Implement 
custom key generator
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r400521994
 
 

 ##
 File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
 ##
 @@ -0,0 +1,110 @@
+/*
+ * 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.util.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;
+
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List 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) {
+  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");
 
 Review comment:
   Can we have test covering this line?


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-30 Thread GitBox
nsivabalan commented on a change in pull request #1433: [HUDI-728]: Implement 
custom key generator
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r400521890
 
 

 ##
 File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
 ##
 @@ -0,0 +1,110 @@
+/*
+ * 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.util.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;
+
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List 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) {
+  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 (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!");
 
 Review comment:
   same here. In general, lets try to see if we can add tests for all happy and 
exception paths if doable. 


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-30 Thread GitBox
nsivabalan commented on a change in pull request #1433: [HUDI-728]: Implement 
custom key generator
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r400516294
 
 

 ##
 File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
 ##
 @@ -0,0 +1,110 @@
+/*
+ * 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.util.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;
+
+public class CustomKeyGenerator extends KeyGenerator {
 
 Review comment:
   java docs. Also try to see if having an example would help devs. 


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-30 Thread GitBox
nsivabalan commented on a change in pull request #1433: [HUDI-728]: Implement 
custom key generator
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r400516907
 
 

 ##
 File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
 ##
 @@ -0,0 +1,110 @@
+/*
+ * 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.util.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;
+
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List recordKeyFields;
+  protected final List 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) {
 
 Review comment:
   do you think we need to check for empty as well?


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:
us...@infra.apache.org


With regards,
Apache Git Services