Fixing UpsertPartitioner to ensure that input records are deterministically assigned to output partitions
This commit is contained in:
committed by
vinoth chandar
parent
e2d13c6305
commit
ec40d04d51
@@ -16,37 +16,26 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.func.LazyInsertIterable;
|
||||
import com.uber.hoodie.io.HoodieCleanHelper;
|
||||
import com.uber.hoodie.io.HoodieMergeHandle;
|
||||
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.Partitioner;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
@@ -56,13 +45,11 @@ import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -73,8 +60,8 @@ import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import scala.Tuple2;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where
|
||||
@@ -90,12 +77,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
|
||||
|
||||
// seed for random number generator. No particular significance, just makes testing deterministic
|
||||
private static final long RANDOM_NUMBER_SEED = 356374L;
|
||||
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
|
||||
|
||||
enum BucketType {
|
||||
@@ -169,6 +150,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
*/
|
||||
private int totalBuckets = 0;
|
||||
|
||||
/**
|
||||
* Stat for the current workload. Helps in determining total inserts, upserts etc.
|
||||
*/
|
||||
private WorkloadStat globalStat;
|
||||
|
||||
/**
|
||||
* Helps decide which bucket an incoming update should go to.
|
||||
*/
|
||||
@@ -187,17 +173,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
*/
|
||||
private HashMap<Integer, BucketInfo> bucketInfoMap;
|
||||
|
||||
|
||||
/**
|
||||
* Random number generator to use for splitting inserts into buckets by weight
|
||||
*/
|
||||
private Random rand = new Random(RANDOM_NUMBER_SEED);
|
||||
|
||||
|
||||
UpsertPartitioner(WorkloadProfile profile) {
|
||||
updateLocationToBucket = new HashMap<>();
|
||||
partitionPathToInsertBuckets = new HashMap<>();
|
||||
bucketInfoMap = new HashMap<>();
|
||||
globalStat = profile.getGlobalStat();
|
||||
|
||||
assignUpdates(profile);
|
||||
assignInserts(profile);
|
||||
@@ -379,7 +359,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
List<InsertBucket> targetBuckets = partitionPathToInsertBuckets.get(keyLocation._1().getPartitionPath());
|
||||
// pick the target bucket to use based on the weights.
|
||||
double totalWeight = 0.0;
|
||||
double r = rand.nextDouble();
|
||||
final long totalInserts = Math.max(1, globalStat.getNumInserts());
|
||||
final double r = 1.0 * Math.floorMod(Hashing.md5().hashString(keyLocation._1().getRecordKey()).asLong(),
|
||||
totalInserts) / totalInserts;
|
||||
for (InsertBucket insertBucket: targetBuckets) {
|
||||
totalWeight += insertBucket.weight;
|
||||
if (r <= totalWeight) {
|
||||
|
||||
Reference in New Issue
Block a user