[HUDI-595] code cleanup, refactoring code out of PR# 1159 (#1302)
This commit is contained in:
@@ -260,14 +260,13 @@ public class HDFSParquetImporter implements Serializable {
|
||||
public int parallelism = 1;
|
||||
@Parameter(names = {"--schema-file", "-sf"}, description = "path for Avro schema file", required = true)
|
||||
public String schemaFile = null;
|
||||
@Parameter(names = {"--format", "-f"}, description = "Format for the input data.", required = false,
|
||||
validateValueWith = FormatValidator.class)
|
||||
@Parameter(names = {"--format", "-f"}, description = "Format for the input data.", validateValueWith = FormatValidator.class)
|
||||
public String format = null;
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master")
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false)
|
||||
@Parameter(names = {"--retry", "-rt"}, description = "number of retries")
|
||||
public int retry = 0;
|
||||
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
|
||||
+ "hoodie client for importing")
|
||||
|
||||
@@ -114,7 +114,7 @@ public class TimelineServerPerf implements Serializable {
|
||||
d2.close();
|
||||
|
||||
System.out.println("\n\n\nDumping all File Slices");
|
||||
selected.stream().forEach(p -> fsView.getAllFileSlices(p).forEach(s -> System.out.println("\tMyFileSlice=" + s)));
|
||||
selected.forEach(p -> fsView.getAllFileSlices(p).forEach(s -> System.out.println("\tMyFileSlice=" + s)));
|
||||
|
||||
// Waiting for curl queries
|
||||
if (!useExternalTimelineServer && cfg.waitForManualQueries) {
|
||||
@@ -131,17 +131,16 @@ public class TimelineServerPerf implements Serializable {
|
||||
|
||||
public List<PerfStats> runLookups(JavaSparkContext jsc, List<String> partitionPaths, SyncableFileSystemView fsView,
|
||||
int numIterations, int concurrency) {
|
||||
List<PerfStats> perfStats = jsc.parallelize(partitionPaths, cfg.numExecutors).flatMap(p -> {
|
||||
return jsc.parallelize(partitionPaths, cfg.numExecutors).flatMap(p -> {
|
||||
ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(100);
|
||||
final List<PerfStats> result = new ArrayList<>();
|
||||
final List<ScheduledFuture<PerfStats>> futures = new ArrayList<>();
|
||||
List<FileSlice> slices = fsView.getLatestFileSlices(p).collect(Collectors.toList());
|
||||
String fileId = slices.isEmpty() ? "dummyId"
|
||||
: slices.get(new Random(Double.doubleToLongBits(Math.random())).nextInt(slices.size())).getFileId();
|
||||
IntStream.range(0, concurrency).forEach(i -> {
|
||||
futures.add(executor.schedule(() -> runOneRound(fsView, p, fileId, i, numIterations), 0, TimeUnit.NANOSECONDS));
|
||||
});
|
||||
futures.stream().forEach(x -> {
|
||||
IntStream.range(0, concurrency).forEach(i -> futures.add(executor.schedule(() -> runOneRound(fsView, p, fileId,
|
||||
i, numIterations), 0, TimeUnit.NANOSECONDS)));
|
||||
futures.forEach(x -> {
|
||||
try {
|
||||
result.add(x.get());
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
@@ -149,12 +148,9 @@ public class TimelineServerPerf implements Serializable {
|
||||
}
|
||||
});
|
||||
System.out.println("SLICES are=");
|
||||
slices.stream().forEach(s -> {
|
||||
System.out.println("\t\tFileSlice=" + s);
|
||||
});
|
||||
slices.forEach(s -> System.out.println("\t\tFileSlice=" + s));
|
||||
return result.iterator();
|
||||
}).collect();
|
||||
return perfStats;
|
||||
}
|
||||
|
||||
private static PerfStats runOneRound(SyncableFileSystemView fsView, String partition, String fileId, int id,
|
||||
@@ -194,7 +190,7 @@ public class TimelineServerPerf implements Serializable {
|
||||
}
|
||||
|
||||
public void dump(List<PerfStats> stats) {
|
||||
stats.stream().forEach(x -> {
|
||||
stats.forEach(x -> {
|
||||
String row = String.format("%s,%d,%d,%d,%f,%f,%f,%f\n", x.partition, x.id, x.minTime, x.maxTime, x.meanTime,
|
||||
x.medianTime, x.p75, x.p95);
|
||||
System.out.println(row);
|
||||
@@ -260,7 +256,7 @@ public class TimelineServerPerf implements Serializable {
|
||||
@Parameter(names = {"--num-iterations", "-i"}, description = "Number of iterations for each partitions")
|
||||
public Integer numIterations = 10;
|
||||
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master")
|
||||
public String sparkMaster = "local[2]";
|
||||
|
||||
@Parameter(names = {"--server-port", "-p"}, description = " Server Port")
|
||||
|
||||
@@ -38,7 +38,7 @@ public class IncrSourceHelper {
|
||||
private static String getStrictlyLowerTimestamp(String timestamp) {
|
||||
long ts = Long.parseLong(timestamp);
|
||||
Preconditions.checkArgument(ts > 0, "Timestamp must be positive");
|
||||
Long lower = ts - 1;
|
||||
long lower = ts - 1;
|
||||
return "" + lower;
|
||||
}
|
||||
|
||||
@@ -73,7 +73,7 @@ public class IncrSourceHelper {
|
||||
|
||||
Option<HoodieInstant> nthInstant = Option.fromJavaOptional(activeCommitTimeline
|
||||
.findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstants().reduce((x, y) -> y));
|
||||
return Pair.of(beginInstantTime, nthInstant.map(instant -> instant.getTimestamp()).orElse(beginInstantTime));
|
||||
return Pair.of(beginInstantTime, nthInstant.map(HoodieInstant::getTimestamp).orElse(beginInstantTime));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -94,8 +94,7 @@ public class KafkaOffsetGen {
|
||||
|
||||
// Create initial offset ranges for each 'to' partition, with from = to offsets.
|
||||
OffsetRange[] ranges = new OffsetRange[toOffsetMap.size()];
|
||||
toOffsetMap.entrySet().stream().map(e -> {
|
||||
TopicPartition tp = e.getKey();
|
||||
toOffsetMap.keySet().stream().map(tp -> {
|
||||
long fromOffset = fromOffsetMap.getOrDefault(tp, 0L);
|
||||
return OffsetRange.create(tp, fromOffset, fromOffset);
|
||||
}).sorted(byPartition).collect(Collectors.toList()).toArray(ranges);
|
||||
@@ -208,9 +207,7 @@ public class KafkaOffsetGen {
|
||||
maxEventsToReadFromKafka = (maxEventsToReadFromKafka == Long.MAX_VALUE || maxEventsToReadFromKafka == Integer.MAX_VALUE)
|
||||
? Config.maxEventsFromKafkaSource : maxEventsToReadFromKafka;
|
||||
long numEvents = sourceLimit == Long.MAX_VALUE ? maxEventsToReadFromKafka : sourceLimit;
|
||||
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents);
|
||||
|
||||
return offsetRanges;
|
||||
return CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents);
|
||||
}
|
||||
|
||||
// check up checkpoint offsets is valid or not, if true, return checkpoint offsets,
|
||||
|
||||
Reference in New Issue
Block a user