zhangyue19921010 commented on code in PR #13557: URL: https://github.com/apache/hudi/pull/13557#discussion_r2209037005
########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/DefaultInsertPartitioner.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.sink.partitioner; + +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.configuration.FlinkOptions; + +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.configuration.Configuration; + +import java.util.Random; + +/** + * Insert input partitioner. + * + * 1. Split numPartitions into multi bucket based on numPartitions / groupLength + * 2. Route each record into related Bucket based on (partitionPath.hashCode() & Integer.MAX_VALUE) % groupNumber + * 3. Using random to get specific index number. + * + * Need to take care of the last bucket to avoid resource wast. + */ +public class DefaultInsertPartitioner<T extends HoodieKey> implements Partitioner<T> { Review Comment: maybe renamed to GroupedInsertPartitioner ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java: ########## @@ -128,6 +128,20 @@ private FlinkOptions() { .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n" + "This will render any value set for the option in-effective"); + public static final ConfigOption<String> INSERT_PARTITIONER_CLASS_NAME = ConfigOptions + .key("write.insert.partitioner.class.name") + .stringType() + .defaultValue("") + .withDescription("Insert partitioner to use aiming to re-balance records and reducing small file number " + + "in the scenario of multi-level partitioning. For example dt/hour/eventID" + + "Currently support org.apache.hudi.sink.partitioner.DefaultInsertPartitioner"); + + public static final ConfigOption<Integer> DEFAULT_PARALLELISM_PER_PARTITION = ConfigOptions + .key("write.insert.partitioner.default_parallelism_per_partition") + .intType() + .defaultValue(30) Review Comment: use `write.tasks` as default value -- 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]
