yihua commented on code in PR #11758:
URL: https://github.com/apache/hudi/pull/11758#discussion_r1747950600
##########
hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java:
##########
@@ -60,7 +60,7 @@ public static Map<String, Long>
generatePartitionNumRecords(List<HoodieRecord> r
}
@ParameterizedTest
- @ValueSource(strings = {"rider", "rider,driver"})
+ @ValueSource(strings = {"rider", "rider, driver"})
Review Comment:
Revert the change
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java:
##########
@@ -80,31 +71,4 @@ default String getFileIdPfx(int partitionId) {
default Option<WriteHandleFactory> getWriteHandleFactory(int partitionId) {
return Option.empty();
}
-
- /*
- * If possible, we want to sort the data by partition path. Doing so will
reduce the number of files written.
- * This will not change the desired sort order, it is just a performance
improvement.
- **/
- static String[] tryPrependPartitionPathColumns(String[] columnNames,
HoodieWriteConfig config) {
- String partitionPath;
- if (config.populateMetaFields()) {
- partitionPath =
HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.getFieldName();
- } else {
- partitionPath =
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key());
- }
- if (isNullOrEmpty(partitionPath)) {
- return columnNames;
- }
- Set<String> sortCols = new
LinkedHashSet<>(StringUtils.split(partitionPath, ","));
- sortCols.addAll(Arrays.asList(columnNames));
- return sortCols.toArray(new String[0]);
- }
-
- static Object[] prependPartitionPath(String partitionPath, Object[]
columnValues) {
- Object[] prependColumnValues = new Object[columnValues.length + 1];
- System.arraycopy(columnValues, 0, prependColumnValues, 1,
columnValues.length);
- prependColumnValues[0] = partitionPath;
- return prependColumnValues;
- }
-
-}
+}
Review Comment:
Keep new line
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -293,6 +293,16 @@ public class HoodieWriteConfig extends HoodieConfig {
.withDocumentation("Columns to sort the data by when use
org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner as user
defined partitioner during bulk_insert. "
+ "For example 'column1,column2'");
+ public static final ConfigProperty<Boolean>
BULKINSERT_SUFFIX_RECORD_KEY_SORT_COLUMNS = ConfigProperty
+ .key("hoodie.bulkinsert.sort.suffix.record_key")
+ .defaultValue(false)
+ .markAdvanced()
+ .withDocumentation(
Review Comment:
add `.sinceVersion("1.0.0")`
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java:
##########
@@ -59,8 +65,39 @@ public boolean arePartitionRecordsSorted() {
return true;
}
- private String[] getSortColumnName(HoodieWriteConfig config) {
+ /*
+ * If possible, we want to sort the data by partition path. Doing so will
reduce the number of files written.
+ * Suffix the recordKeys as well if
BULKINSERT_SUFFIX_RECORD_KEY_FOR_USER_DEFINED_SORT_COLUMNS is enabled, this
reduces skew.
+ * This will not change the desired sort order, it is just a performance
improvement.
+ * NOTE: This function is used by BULK_INSERT in row writer mode where
Dataset<Row> can be sorted by providing the sort columns.
+ **/
+ static String[] tryPrependPartitionPathAndSuffixRecordKeyColumns(String[]
columnNames, HoodieWriteConfig config) {
+ String partitionPath;
+ String recordKeyFieldName;
+ if (config.populateMetaFields()) {
+ partitionPath =
HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.getFieldName();
+ recordKeyFieldName =
HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName();
+ } else {
+ // TODO: Remove the else block as sorting in row-writer mode is always
used with populateMetaFields turned on as it allows easy access to hoodie meta
fields
+ // without the need for generating them again. Disabling
populateMetaFields defeats the purpose of sorting using row-writer and is
generally never used.
+ // https://issues.apache.org/jira/browse/HUDI-8101
+ partitionPath =
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key());
+ recordKeyFieldName =
config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key());
+ }
+ Set<String> sortCols = new LinkedHashSet<>();
+ if (!isNullOrEmpty(partitionPath)) {
+ sortCols = new LinkedHashSet<>(StringUtils.split(partitionPath, ","));
+ }
+ sortCols.addAll(Arrays.asList(columnNames));
+ boolean suffixRecordKey =
config.getBoolean(BULKINSERT_SUFFIX_RECORD_KEY_SORT_COLUMNS);
+ if (suffixRecordKey) {
+ sortCols.add(recordKeyFieldName);
+ }
+ return sortCols.toArray(new String[0]);
+ }
+
+ private static String[] getSortColumnName(HoodieWriteConfig config) {
return
Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(","))
.map(String::trim).toArray(String[]::new);
}
-}
+}
Review Comment:
Keep new line for all classes.
##########
hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java:
##########
@@ -42,29 +43,29 @@ public class JavaCustomColumnsSortPartitioner<T>
private final String[] sortColumnNames;
private final Schema schema;
private final boolean consistentLogicalTimestampEnabled;
+ private final boolean suffixRecordKey;
public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema,
HoodieWriteConfig config) {
this.sortColumnNames = columnNames;
this.schema = schema;
this.consistentLogicalTimestampEnabled =
config.isConsistentLogicalTimestampEnabled();
+ this.suffixRecordKey =
config.getBoolean(BULKINSERT_SUFFIX_RECORD_KEY_SORT_COLUMNS);
}
@Override
public List<HoodieRecord<T>> repartitionRecords(
List<HoodieRecord<T>> records, int outputPartitions) {
- return records.stream().sorted((o1, o2) -> {
- FlatLists.ComparableList<Comparable> values1 =
FlatLists.ofComparableArray(
- BulkInsertPartitioner.prependPartitionPath(o1.getPartitionPath(),
HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord) o1, sortColumnNames,
schema, consistentLogicalTimestampEnabled))
- );
- FlatLists.ComparableList<Comparable> values2 =
FlatLists.ofComparableArray(
- BulkInsertPartitioner.prependPartitionPath(o2.getPartitionPath(),
HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord) o2, sortColumnNames,
schema, consistentLogicalTimestampEnabled))
- );
- return values1.compareTo(values2);
- }).collect(Collectors.toList());
+ return records.stream()
+ .sorted((o1, o2) ->
getComparableSortColumns(o1).compareTo(getComparableSortColumns(o2)))
+ .collect(Collectors.toList());
}
@Override
public boolean arePartitionRecordsSorted() {
return true;
}
-}
+
+ FlatLists.ComparableList<Comparable<HoodieRecord>>
getComparableSortColumns(HoodieRecord record) {
+ return SortUtils.getComparableSortColumns(record, sortColumnNames, schema,
suffixRecordKey, consistentLogicalTimestampEnabled);
+ }
+}
Review Comment:
Keep new line
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java:
##########
@@ -77,4 +76,4 @@ private String[] getSortColumnName(HoodieWriteConfig config) {
return
Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(","))
.map(String::trim).toArray(String[]::new);
}
-}
+}
Review Comment:
Keep the new line
##########
hudi-common/src/main/java/org/apache/hudi/common/util/SortUtils.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.common.util;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.collection.FlatLists;
+
+import org.apache.avro.Schema;
+
+/**
+ * Utility functions used by BULK_INSERT practitioners while sorting records.
+ */
+public class SortUtils {
+ static Object[] prependPartitionPath(String partitionPath, Object[]
columnValues) {
+ Object[] prependColumnValues = new Object[columnValues.length + 1];
+ System.arraycopy(columnValues, 0, prependColumnValues, 1,
columnValues.length);
+ prependColumnValues[0] = partitionPath;
+ return prependColumnValues;
+ }
+
+ static Object[] prependPartitionPathAndSuffixRecordKey(String partitionPath,
String recordKey, Object[] columnValues) {
+ Object[] newColumnValues = new Object[columnValues.length + 2];
+ System.arraycopy(columnValues, 0, newColumnValues, 1, columnValues.length);
+ newColumnValues[0] = partitionPath;
+ newColumnValues[newColumnValues.length - 1] = recordKey;
+ return newColumnValues;
+ }
+
+ /**
+ * Given a hoodie record, returns a comparable list of sorted columns.
+ *
+ * @param record HoodieRecord (Spark or Avro)
+ * @param sortColumnNames user provided sort columns
+ * @param schema schema for table
+ * @param suffixRecordKey
HoodieWriteConfig.BULKINSERT_SUFFIX_RECORD_KEY_SORT_COLUMNS
+ * @param consistentLogicalTimestampEnabled
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED
+ */
+ public static FlatLists.ComparableList<Comparable<HoodieRecord>>
getComparableSortColumns(
+ HoodieRecord record,
+ String[] sortColumnNames,
+ Schema schema,
+ boolean suffixRecordKey,
+ boolean consistentLogicalTimestampEnabled
+ ) {
+ if (record.getRecordType() == HoodieRecord.HoodieRecordType.SPARK) {
+ Object[] columnValues = record.getColumnValues(schema, sortColumnNames,
consistentLogicalTimestampEnabled);
+ if (suffixRecordKey) {
+ return FlatLists.ofComparableArray(
+ prependPartitionPathAndSuffixRecordKey(record.getPartitionPath(),
record.getRecordKey(), columnValues));
+ }
+ return
FlatLists.ofComparableArray(prependPartitionPath(record.getPartitionPath(),
columnValues));
+ } else if (record.getRecordType() == HoodieRecord.HoodieRecordType.AVRO) {
+ return FlatLists.ofComparableArray(
+ HoodieAvroUtils.getSortColumnValuesWithPartitionPathAndRecordKey(
+ record, sortColumnNames, schema, suffixRecordKey,
consistentLogicalTimestampEnabled
+ ));
+ }
+ throw new IllegalArgumentException("Invalid recordType" +
record.getRecordType());
+ }
+}
Review Comment:
New line
##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestRowCustomColumnsSortPartitioner.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.execution.bulkinsert;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.config.HoodieWriteConfig;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static
org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
+
+public class TestRowCustomColumnsSortPartitioner {
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ void tryPrependPartitionPathAndSuffixRecordKeyColumns(boolean
suffixRecordKey) {
+ TypedProperties properties = new TypedProperties();
+
properties.setProperty(HoodieWriteConfig.BULKINSERT_SUFFIX_RECORD_KEY_SORT_COLUMNS.key(),
String.valueOf(suffixRecordKey));
+ HoodieWriteConfig writeConfig = HoodieWriteConfig
+ .newBuilder()
+ .withPath("/")
+ .withSchema(TRIP_EXAMPLE_SCHEMA)
+ .withProperties(properties)
+ .build();
+
+ String[] columnValues = new String[] {"col1", "col2", "col3"};
+ String[] sortColumns =
RowCustomColumnsSortPartitioner.tryPrependPartitionPathAndSuffixRecordKeyColumns(columnValues,
writeConfig);
+ if (suffixRecordKey) {
+ Assertions.assertArrayEquals(new Object[] {"_hoodie_partition_path",
"col1", "col2", "col3", "_hoodie_record_key"}, sortColumns);
+ } else {
+ Assertions.assertArrayEquals(new Object[] {"_hoodie_partition_path",
"col1", "col2", "col3"}, sortColumns);
+ }
+ }
+}
Review Comment:
Same here on new line
##########
hudi-common/src/test/java/org/apache/hudi/common/util/TestSortUtils.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.common.util;
+
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.RewriteAvroPayload;
+import org.apache.hudi.common.util.collection.FlatLists;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestSortUtils {
+
+ private static final String SCHEMA = "{\"type\": \"record\",\"name\":
\"testrec\",\"fields\": [ "
+ + "{\"name\": \"timestamp\",\"type\": \"double\"},{\"name\":
\"_row_key\", \"type\": \"string\"},"
+ + "{\"name\": \"non_pii_col\", \"type\": \"string\"},"
+ + "{\"name\": \"pii_col\", \"type\": \"string\", \"column_category\":
\"user_profile\"}]}";
+
+ @Test
+ void testPrependPartitionPath() {
+ Object[] columnValues = new Object[] {"col1", "col2", "col3"};
+ Object[] sortColumns = SortUtils.prependPartitionPath("partition_path",
columnValues);
+ Assertions.assertArrayEquals(new Object[] {"partition_path", "col1",
"col2", "col3"}, sortColumns);
+ }
+
+ @Test
+ void testPrependPartitionPathAndSuffixRecordKey() {
+ Object[] columnValues = new Object[] {"col1", "col2", "col3"};
+ Object[] sortColumns =
SortUtils.prependPartitionPathAndSuffixRecordKey("partition_path",
"record_key", columnValues);
+ Assertions.assertArrayEquals(new Object[] {"partition_path", "col1",
"col2", "col3", "record_key"}, sortColumns);
+ }
+
+ @ParameterizedTest
+ @MethodSource("getArguments")
+ void testGetComparableSortColumnsAvroRecord(HoodieRecordType recordType,
boolean suffixRecordKey) {
+ Schema schema = new Schema.Parser().parse(SCHEMA);
+ GenericRecord genericRecord = new GenericData.Record(schema);
+ genericRecord.put("non_pii_col", "val1");
+ genericRecord.put("pii_col", "val2");
+ genericRecord.put("timestamp", 3.5);
+ HoodieRecordPayload payload = new RewriteAvroPayload(genericRecord);
+
+ HoodieRecord record;
+ if (recordType == HoodieRecordType.AVRO) {
+ record = new HoodieAvroRecord(new HoodieKey("record1", "partition1"),
payload);
+ } else {
+ record = new TestSparkRecord(new HoodieKey("record1", "partition1"),
payload);
+ }
+ String[] userSortColumns = new String[] {"non_pii_col", "timestamp"};
+ FlatLists.ComparableList<Comparable<HoodieRecord>> comparableList =
SortUtils.getComparableSortColumns(record, userSortColumns,
Schema.parse(SCHEMA), suffixRecordKey, true);
+ Object[] expectedSortColumnValues;
+ if (suffixRecordKey) {
+ expectedSortColumnValues = new Object[] {"partition1", "val1", 3.5,
"record1"};
+ } else {
+ expectedSortColumnValues = new Object[] {"partition1", "val1", 3.5};
+ }
+ assertEquals(FlatLists.ofComparableArray(expectedSortColumnValues),
comparableList);
+ }
+
+ private static Stream<Arguments> getArguments() {
+ return Stream.of(
+ Arguments.of(HoodieRecordType.SPARK, true),
+ Arguments.of(HoodieRecordType.SPARK, false),
+ Arguments.of(HoodieRecordType.AVRO, true),
+ Arguments.of(HoodieRecordType.AVRO, false)
+ );
+ }
+
+ public static class TestSparkRecord<T extends HoodieRecordPayload> extends
HoodieAvroRecord {
+
+ public TestSparkRecord(HoodieKey key, T data) {
+ super(key, data);
+ }
+
+ @Override
+ public HoodieRecordType getRecordType() {
+ return HoodieRecordType.SPARK;
+ }
+ }
+}
Review Comment:
New 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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]