HUDI-494 fix incorrect record size estimation
This commit is contained in:
committed by
vinoth chandar
parent
9e07cebece
commit
22cd824d99
@@ -54,6 +54,12 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
||||
// By default, treat any file <= 100MB as a small file.
|
||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(104857600);
|
||||
// Hudi will use the previous commit to calculate the estimated record size by totalBytesWritten/totalRecordsWritten.
|
||||
// If the previous commit is too small to make an accurate estimation, Hudi will search commits in the reverse order,
|
||||
// until find a commit has totalBytesWritten larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * RECORD_SIZE_ESTIMATION_THRESHOLD)
|
||||
public static final String RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = "hoodie.record.size.estimation.threshold";
|
||||
public static final String DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD = "1.0";
|
||||
|
||||
/**
|
||||
* Configs related to specific table types.
|
||||
*/
|
||||
@@ -173,6 +179,11 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder compactionRecordSizeEstimateThreshold(double threshold) {
|
||||
props.setProperty(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP, String.valueOf(threshold));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder insertSplitSize(int insertSplitSize) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
||||
return this;
|
||||
@@ -254,6 +265,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES), PARQUET_SMALL_FILE_LIMIT_BYTES,
|
||||
DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP), RECORD_SIZE_ESTIMATION_THRESHOLD_PROP,
|
||||
DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
||||
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
|
||||
|
||||
@@ -272,6 +272,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
|
||||
}
|
||||
|
||||
public double getRecordSizeEstimationThreshold() {
|
||||
return Double.parseDouble(props.getProperty(HoodieCompactionConfig.RECORD_SIZE_ESTIMATION_THRESHOLD_PROP));
|
||||
}
|
||||
|
||||
public int getCopyOnWriteInsertSplitSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
||||
}
|
||||
|
||||
@@ -29,14 +29,12 @@ import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.utils.ParquetReaderIterator;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
@@ -299,33 +297,4 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtains the average record size based on records written during previous commits. Used for estimating how many
|
||||
* records pack into one file.
|
||||
*/
|
||||
protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, int defaultRecordSizeEstimate) {
|
||||
long avgSize = defaultRecordSizeEstimate;
|
||||
try {
|
||||
if (!commitTimeline.empty()) {
|
||||
// Go over the reverse ordered commits to get a more recent estimate of average record size.
|
||||
Iterator<HoodieInstant> instants = commitTimeline.getReverseOrderedInstants().iterator();
|
||||
while (instants.hasNext()) {
|
||||
HoodieInstant instant = instants.next();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
|
||||
long totalBytesWritten = commitMetadata.fetchTotalBytesWritten();
|
||||
long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten();
|
||||
if (totalBytesWritten > 0 && totalRecordsWritten > 0) {
|
||||
avgSize = (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// make this fail safe.
|
||||
LOG.error("Error trying to compute average bytes/record ", t);
|
||||
}
|
||||
return avgSize;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -131,7 +131,7 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
|
||||
Set<String> partitionPaths = profile.getPartitionPaths();
|
||||
long averageRecordSize =
|
||||
averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
|
||||
config.getCopyOnWriteRecordSizeEstimate());
|
||||
config);
|
||||
LOG.info("AvgRecordSize => " + averageRecordSize);
|
||||
|
||||
Map<String, List<SmallFile>> partitionSmallFilesMap =
|
||||
@@ -289,8 +289,9 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
|
||||
* Obtains the average record size based on records written during previous commits. Used for estimating how many
|
||||
* records pack into one file.
|
||||
*/
|
||||
protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, int defaultRecordSizeEstimate) {
|
||||
long avgSize = defaultRecordSizeEstimate;
|
||||
protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConfig hoodieWriteConfig) {
|
||||
long avgSize = hoodieWriteConfig.getCopyOnWriteRecordSizeEstimate();
|
||||
long fileSizeThreshold = (long) (hoodieWriteConfig.getRecordSizeEstimationThreshold() * hoodieWriteConfig.getParquetSmallFileLimit());
|
||||
try {
|
||||
if (!commitTimeline.empty()) {
|
||||
// Go over the reverse ordered commits to get a more recent estimate of average record size.
|
||||
@@ -301,7 +302,7 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
|
||||
.fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
|
||||
long totalBytesWritten = commitMetadata.fetchTotalBytesWritten();
|
||||
long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten();
|
||||
if (totalBytesWritten > 0 && totalRecordsWritten > 0) {
|
||||
if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) {
|
||||
avgSize = (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten);
|
||||
break;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user