[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:
@@ -20,12 +20,12 @@ package org.apache.hudi.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.javalin.Javalin;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.client.methods.HttpGet;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.impl.client.HttpClientBuilder;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
@@ -87,7 +87,7 @@ public class HoodieWithTimelineServer implements Serializable {
|
||||
IntStream.range(0, cfg.numPartitions).forEach(i -> messages.add("Hello World"));
|
||||
List<String> gotMessages = jsc.parallelize(messages).map(msg -> sendRequest(driverHost, cfg.serverPort)).collect();
|
||||
System.out.println("Got Messages :" + gotMessages);
|
||||
Preconditions.checkArgument(gotMessages.equals(messages), "Got expected reply from Server");
|
||||
ValidationUtils.checkArgument(gotMessages.equals(messages), "Got expected reply from Server");
|
||||
}
|
||||
|
||||
public String sendRequest(String driverHost, int port) {
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.util.DFSPropertiesConfiguration;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -36,7 +37,6 @@ import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.Source;
|
||||
import org.apache.hudi.utilities.transform.Transformer;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -137,7 +137,7 @@ public class UtilHelpers {
|
||||
TypedProperties properties = new TypedProperties();
|
||||
props.forEach(x -> {
|
||||
String[] kv = x.split("=");
|
||||
Preconditions.checkArgument(kv.length == 2);
|
||||
ValidationUtils.checkArgument(kv.length == 2);
|
||||
properties.setProperty(kv[0], kv[1]);
|
||||
});
|
||||
return properties;
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.utilities.deltastreamer;
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
@@ -49,7 +50,6 @@ import org.apache.hudi.utilities.sources.InputBatch;
|
||||
import org.apache.hudi.utilities.transform.Transformer;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -502,10 +502,10 @@ public class DeltaSync implements Serializable {
|
||||
HoodieWriteConfig config = builder.build();
|
||||
|
||||
// Validate what deltastreamer assumes of write-config to be really safe
|
||||
Preconditions.checkArgument(config.isInlineCompaction() == cfg.isInlineCompactionEnabled());
|
||||
Preconditions.checkArgument(!config.shouldAutoCommit());
|
||||
Preconditions.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes);
|
||||
Preconditions.checkArgument(config.shouldCombineBeforeUpsert());
|
||||
ValidationUtils.checkArgument(config.isInlineCompaction() == cfg.isInlineCompactionEnabled());
|
||||
ValidationUtils.checkArgument(!config.shouldAutoCommit());
|
||||
ValidationUtils.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes);
|
||||
ValidationUtils.checkArgument(config.shouldCombineBeforeUpsert());
|
||||
|
||||
return config;
|
||||
}
|
||||
|
||||
@@ -29,6 +29,7 @@ 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.TypedProperties;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -41,7 +42,6 @@ import com.beust.jcommander.IStringConverter;
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.beust.jcommander.ParameterException;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -353,7 +353,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath, false);
|
||||
tableType = meta.getTableType();
|
||||
// This will guarantee there is no surprise with table type
|
||||
Preconditions.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.tableType)),
|
||||
ValidationUtils.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.tableType)),
|
||||
"Hoodie table is of type " + tableType + " but passed in CLI argument is " + cfg.tableType);
|
||||
} else {
|
||||
tableType = HoodieTableType.valueOf(cfg.tableType);
|
||||
|
||||
@@ -22,12 +22,14 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class IncrSourceHelper {
|
||||
|
||||
/**
|
||||
@@ -37,7 +39,7 @@ public class IncrSourceHelper {
|
||||
*/
|
||||
private static String getStrictlyLowerTimestamp(String timestamp) {
|
||||
long ts = Long.parseLong(timestamp);
|
||||
Preconditions.checkArgument(ts > 0, "Timestamp must be positive");
|
||||
ValidationUtils.checkArgument(ts > 0, "Timestamp must be positive");
|
||||
long lower = ts - 1;
|
||||
return "" + lower;
|
||||
}
|
||||
@@ -54,7 +56,7 @@ public class IncrSourceHelper {
|
||||
*/
|
||||
public static Pair<String, String> calculateBeginAndEndInstants(JavaSparkContext jssc, String srcBasePath,
|
||||
int numInstantsPerFetch, Option<String> beginInstant, boolean readLatestOnMissingBeginInstant) {
|
||||
Preconditions.checkArgument(numInstantsPerFetch > 0,
|
||||
ValidationUtils.checkArgument(numInstantsPerFetch > 0,
|
||||
"Make sure the config hoodie.deltastreamer.source.hoodieincr.num_instants is set to a positive value");
|
||||
HoodieTableMetaClient srcMetaClient = new HoodieTableMetaClient(jssc.hadoopConfiguration(), srcBasePath, true);
|
||||
|
||||
@@ -85,11 +87,11 @@ public class IncrSourceHelper {
|
||||
* @param endInstant end instant of the batch
|
||||
*/
|
||||
public static void validateInstantTime(Row row, String instantTime, String sinceInstant, String endInstant) {
|
||||
Preconditions.checkNotNull(instantTime);
|
||||
Preconditions.checkArgument(HoodieTimeline.compareTimestamps(instantTime, sinceInstant, HoodieTimeline.GREATER),
|
||||
Objects.requireNonNull(instantTime);
|
||||
ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, sinceInstant, HoodieTimeline.GREATER),
|
||||
"Instant time(_hoodie_commit_time) in row (" + row + ") was : " + instantTime + "but expected to be between "
|
||||
+ sinceInstant + "(excl) - " + endInstant + "(incl)");
|
||||
Preconditions.checkArgument(
|
||||
ValidationUtils.checkArgument(
|
||||
HoodieTimeline.compareTimestamps(instantTime, endInstant, HoodieTimeline.LESSER_OR_EQUAL),
|
||||
"Instant time(_hoodie_commit_time) in row (" + row + ") was : " + instantTime + "but expected to be between "
|
||||
+ sinceInstant + "(excl) - " + endInstant + "(incl)");
|
||||
|
||||
Reference in New Issue
Block a user