1
0

[HUDI-629]: Replace Guava's Hashing with an equivalent in NumericUtils.java (#1350)

* [HUDI-629]: Replace Guava's Hashing with an equivalent in NumericUtils.java
This commit is contained in:
Suneel Marthi
2020-03-13 20:28:05 -04:00
committed by GitHub
parent fb7fba365f
commit 99b7e9eb9e
51 changed files with 308 additions and 228 deletions

View File

@@ -36,13 +36,13 @@ import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.compact.OperationResult;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -138,7 +138,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
// TODO: Add a rollback instant but for compaction
HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant);
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
ValidationUtils.checkArgument(deleted, "Unable to delete compaction instant.");
}
return res;
}
@@ -247,7 +247,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
List<HoodieLogFile> logFilesToBeMoved =
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
return logFilesToBeMoved.stream().map(lf -> {
Preconditions.checkArgument(lf.getLogVersion() - maxVersion > 0, "Expect new log version to be sane");
ValidationUtils.checkArgument(lf.getLogVersion() - maxVersion > 0, "Expect new log version to be sane");
HoodieLogFile newLogFile = new HoodieLogFile(new Path(lf.getPath().getParent(),
FSUtils.makeLogFileName(lf.getFileId(), "." + FSUtils.getFileExtensionFromLog(lf.getPath()),
compactionInstant, lf.getLogVersion() - maxVersion, HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
@@ -266,9 +266,9 @@ public class CompactionAdminClient extends AbstractHoodieClient {
protected static void renameLogFile(HoodieTableMetaClient metaClient, HoodieLogFile oldLogFile,
HoodieLogFile newLogFile) throws IOException {
FileStatus[] statuses = metaClient.getFs().listStatus(oldLogFile.getPath());
Preconditions.checkArgument(statuses.length == 1, "Only one status must be present");
Preconditions.checkArgument(statuses[0].isFile(), "Source File must exist");
Preconditions.checkArgument(oldLogFile.getPath().getParent().equals(newLogFile.getPath().getParent()),
ValidationUtils.checkArgument(statuses.length == 1, "Only one status must be present");
ValidationUtils.checkArgument(statuses[0].isFile(), "Source File must exist");
ValidationUtils.checkArgument(oldLogFile.getPath().getParent().equals(newLogFile.getPath().getParent()),
"Log file must only be moved within the parent directory");
metaClient.getFs().rename(oldLogFile.getPath(), newLogFile.getPath());
}
@@ -300,9 +300,9 @@ public class CompactionAdminClient extends AbstractHoodieClient {
new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
new Path(operation.getDataFileName().get())))
.getPath().toString();
Preconditions.checkArgument(df.isPresent(),
ValidationUtils.checkArgument(df.isPresent(),
"Data File must be present. File Slice was : " + fs + ", operation :" + operation);
Preconditions.checkArgument(df.get().getPath().equals(expPath),
ValidationUtils.checkArgument(df.get().getPath().equals(expPath),
"Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath());
}
Set<HoodieLogFile> logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet());
@@ -310,7 +310,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
try {
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(
FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), new Path(dp)));
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
ValidationUtils.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
return new HoodieLogFile(fileStatuses[0]);
} catch (FileNotFoundException fe) {
throw new CompactionValidationException(fe.getMessage());
@@ -320,12 +320,12 @@ public class CompactionAdminClient extends AbstractHoodieClient {
}).collect(Collectors.toSet());
Set<HoodieLogFile> missing = logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
.collect(Collectors.toSet());
Preconditions.checkArgument(missing.isEmpty(),
ValidationUtils.checkArgument(missing.isEmpty(),
"All log files specified in compaction operation is not present. Missing :" + missing + ", Exp :"
+ logFilesInCompactionOp + ", Got :" + logFilesInFileSlice);
Set<HoodieLogFile> diff = logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
.collect(Collectors.toSet());
Preconditions.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)),
ValidationUtils.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)),
"There are some log-files which are neither specified in compaction plan "
+ "nor present after compaction request instant. Some of these :" + diff);
} else {

View File

@@ -31,13 +31,13 @@ import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.metrics.HoodieMetrics;
import org.apache.hudi.table.HoodieTable;
import com.codahale.metrics.Timer;
import com.google.common.base.Preconditions;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
@@ -147,7 +147,7 @@ public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHo
private HoodieCleanMetadata runClean(HoodieTable<T> table, HoodieInstant cleanInstant,
HoodieCleanerPlan cleanerPlan) {
Preconditions.checkArgument(
ValidationUtils.checkArgument(
cleanInstant.getState().equals(State.REQUESTED) || cleanInstant.getState().equals(State.INFLIGHT));
try {

View File

@@ -43,6 +43,7 @@ import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
@@ -62,7 +63,6 @@ import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import com.codahale.metrics.Timer;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -588,7 +588,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
}
// Cannot allow savepoint time on a commit that could have been cleaned
Preconditions.checkArgument(
ValidationUtils.checkArgument(
HoodieTimeline.compareTimestamps(commitTime, lastCommitRetained, HoodieTimeline.GREATER_OR_EQUAL),
"Could not savepoint commit " + commitTime + " as this is beyond the lookup window " + lastCommitRetained);
@@ -704,8 +704,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
// Make sure the rollback was successful
Option<HoodieInstant> lastInstant =
activeTimeline.reload().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants().lastInstant();
Preconditions.checkArgument(lastInstant.isPresent());
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
ValidationUtils.checkArgument(lastInstant.isPresent());
ValidationUtils.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
savepointTime + "is not the last commit after rolling back " + commitsToRollback + ", last commit was "
+ lastInstant.get().getTimestamp());
return true;
@@ -876,7 +876,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
HoodieTableMetaClient metaClient = createMetaClient(true);
// if there are pending compactions, their instantTime must not be greater than that of this instant time
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending ->
Preconditions.checkArgument(
ValidationUtils.checkArgument(
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), instantTime, HoodieTimeline.LESSER),
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
+ latestPending + ", Ingesting at " + instantTime));
@@ -909,7 +909,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
HoodieTableMetaClient metaClient = createMetaClient(true);
// if there are inflight writes, their instantTime must not be less than that of compaction instant time
metaClient.getCommitsTimeline().filterPendingExcludingCompaction().firstInstant().ifPresent(earliestInflight -> {
Preconditions.checkArgument(
ValidationUtils.checkArgument(
HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), instantTime, HoodieTimeline.GREATER),
"Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
+ ", Compaction scheduled at " + instantTime);
@@ -919,7 +919,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
.getActiveTimeline().getCommitsAndCompactionTimeline().getInstants().filter(instant -> HoodieTimeline
.compareTimestamps(instant.getTimestamp(), instantTime, HoodieTimeline.GREATER_OR_EQUAL))
.collect(Collectors.toList());
Preconditions.checkArgument(conflictingInstants.isEmpty(),
ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
"Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"
+ conflictingInstants);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);

View File

@@ -20,11 +20,10 @@ package org.apache.hudi.config;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.table.compact.strategy.CompactionStrategy;
import org.apache.hudi.table.compact.strategy.LogFileSizeBasedCompactionStrategy;
import com.google.common.base.Preconditions;
import javax.annotation.concurrent.Immutable;
import java.io.File;
@@ -121,12 +120,9 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
try (FileReader reader = new FileReader(propertiesFile)) {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
@@ -292,8 +288,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
int maxInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
int cleanerCommitsRetained =
Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
Preconditions.checkArgument(maxInstantsToKeep > minInstantsToKeep);
Preconditions.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep);
ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
String.format(
"Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
+ "missing data from few instants.",

View File

@@ -96,7 +96,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
private ConsistencyGuardConfig consistencyGuardConfig;
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled

View File

@@ -18,14 +18,13 @@
package org.apache.hudi.index.bloom;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.collection.Pair;
import com.google.common.hash.Hashing;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -144,7 +143,7 @@ public class BucketizedBloomCheckPartitioner extends Partitioner {
@Override
public int getPartition(Object key) {
final Pair<String, String> parts = (Pair<String, String>) key;
final long hashOfKey = Hashing.md5().hashString(parts.getRight(), StandardCharsets.UTF_8).asLong();
final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", parts.getRight());
final List<Integer> candidatePartitions = fileGroupToPartitions.get(parts.getLeft());
final int idx = (int) Math.floorMod(hashOfKey, candidatePartitions.size());
assert idx >= 0;

View File

@@ -37,6 +37,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
@@ -52,7 +53,6 @@ import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
import org.apache.hudi.io.HoodieCreateHandle;
import org.apache.hudi.io.HoodieMergeHandle;
import com.google.common.hash.Hashing;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
@@ -72,7 +72,6 @@ import org.apache.spark.api.java.function.PairFlatMapFunction;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -732,8 +731,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
// pick the target bucket to use based on the weights.
double totalWeight = 0.0;
final long totalInserts = Math.max(1, globalStat.getNumInserts());
final long hashOfKey =
Hashing.md5().hashString(keyLocation._1().getRecordKey(), StandardCharsets.UTF_8).asLong();
final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", keyLocation._1().getRecordKey());
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
for (InsertBucket insertBucket : targetBuckets) {
totalWeight += insertBucket.weight;

View File

@@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCompactionException;
import org.apache.hudi.exception.HoodieIOException;
@@ -42,7 +43,6 @@ import org.apache.hudi.execution.MergeOnReadLazyInsertIterable;
import org.apache.hudi.io.HoodieAppendHandle;
import org.apache.hudi.table.compact.HoodieMergeOnReadTableCompactor;
import com.google.common.base.Preconditions;
import org.apache.hudi.table.rollback.RollbackHelper;
import org.apache.hudi.table.rollback.RollbackRequest;
import org.apache.log4j.LogManager;
@@ -422,7 +422,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
private List<RollbackRequest> generateAppendRollbackBlocksAction(String partitionPath, HoodieInstant rollbackInstant,
HoodieCommitMetadata commitMetadata) {
Preconditions.checkArgument(rollbackInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION));
ValidationUtils.checkArgument(rollbackInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION));
// wStat.getPrevCommit() might not give the right commit time in the following
// scenario : If a compaction was scheduled, the new commitTime associated with the requested compaction will be
@@ -439,9 +439,9 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
if (validForRollback) {
// For sanity, log instant time can never be less than base-commit on which we are rolling back
Preconditions
ValidationUtils
.checkArgument(HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId()),
rollbackInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
rollbackInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
}
return validForRollback && HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(

View File

@@ -35,14 +35,13 @@ import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.compact.strategy.CompactionStrategy;
import org.apache.hudi.table.HoodieCopyOnWriteTable;
import org.apache.hudi.table.HoodieTable;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
@@ -56,6 +55,7 @@ import org.apache.spark.util.AccumulatorV2;
import org.apache.spark.util.LongAccumulator;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
@@ -125,7 +125,7 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
config.getSpillableMapBasePath());
if (!scanner.iterator().hasNext()) {
return Lists.<WriteStatus>newArrayList();
return new ArrayList<>();
}
Option<HoodieBaseFile> oldDataFileOpt =
@@ -169,7 +169,7 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
jsc.sc().register(totalLogFiles);
jsc.sc().register(totalFileSlices);
Preconditions.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
ValidationUtils.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
"Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
+ hoodieTable.getMetaClient().getTableType().name());
@@ -214,7 +214,7 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
// compactions only
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
Preconditions.checkArgument(
ValidationUtils.checkArgument(
compactionPlan.getOperations().stream().noneMatch(
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "

View File

@@ -28,10 +28,10 @@ import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBl
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.PathFilter;
@@ -147,7 +147,7 @@ public class RollbackHelper implements Serializable {
* @return Merged HoodieRollbackStat
*/
private HoodieRollbackStat mergeRollbackStat(HoodieRollbackStat stat1, HoodieRollbackStat stat2) {
Preconditions.checkArgument(stat1.getPartitionPath().equals(stat2.getPartitionPath()));
ValidationUtils.checkArgument(stat1.getPartitionPath().equals(stat2.getPartitionPath()));
final List<String> successDeleteFiles = new ArrayList<>();
final List<String> failedDeleteFiles = new ArrayList<>();
final Map<FileStatus, Long> commandBlocksCount = new HashMap<>();

View File

@@ -37,7 +37,7 @@ public class TestHoodieWriteConfig {
@Test
public void testPropertyLoading() throws IOException {
Builder builder = HoodieWriteConfig.newBuilder().withPath("/tmp");
Map<String, String> params = new HashMap<>();
Map<String, String> params = new HashMap<>(3);
params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1");
params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "5");
params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2");