HUDI-479: Eliminate or Minimize use of Guava if possible (#1159)
This commit is contained in:
@@ -64,7 +64,6 @@ import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
@@ -77,6 +76,7 @@ import org.apache.spark.storage.StorageLevel;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -746,7 +746,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
String startRollbackInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
// Start the timer
|
||||
final Timer.Context context = startContext();
|
||||
ImmutableMap.Builder<String, List<HoodieRollbackStat>> instantsToStats = ImmutableMap.builder();
|
||||
Map<String, List<HoodieRollbackStat>> instantsToStats = new HashMap<>();
|
||||
table.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, startRollbackInstant));
|
||||
instantsToRollback.forEach(instant -> {
|
||||
@@ -773,7 +773,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
});
|
||||
try {
|
||||
finishRestore(context, instantsToStats.build(),
|
||||
finishRestore(context, Collections.unmodifiableMap(instantsToStats),
|
||||
instantsToRollback.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
|
||||
startRollbackInstant, instantTime);
|
||||
} catch (IOException io) {
|
||||
|
||||
@@ -64,9 +64,8 @@ public class BloomIndexFileInfo implements Serializable {
|
||||
* Does the given key fall within the range (inclusive).
|
||||
*/
|
||||
public boolean isKeyInRange(String recordKey) {
|
||||
assert minRecordKey != null;
|
||||
assert maxRecordKey != null;
|
||||
return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0;
|
||||
return Objects.requireNonNull(minRecordKey).compareTo(recordKey) <= 0
|
||||
&& Objects.requireNonNull(maxRecordKey).compareTo(recordKey) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -54,8 +54,8 @@ import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
@@ -18,19 +18,19 @@
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import javax.management.MBeanServer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import java.lang.management.ManagementFactory;
|
||||
import javax.management.MBeanServer;
|
||||
|
||||
/**
|
||||
* Implementation of Jmx reporter, which used to report jmx metric.
|
||||
*/
|
||||
@@ -92,7 +92,7 @@ public class JmxMetricsReporter extends MetricsReporter {
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
Preconditions.checkNotNull(jmxReporterServer, "jmxReporterServer is not running.");
|
||||
Objects.requireNonNull(jmxReporterServer, "jmxReporterServer is not running.");
|
||||
try {
|
||||
jmxReporterServer.stop();
|
||||
} catch (Exception e) {
|
||||
|
||||
@@ -18,20 +18,23 @@
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.codahale.metrics.jmx.JmxReporter;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.rmi.NoSuchObjectException;
|
||||
import java.rmi.registry.LocateRegistry;
|
||||
import java.rmi.registry.Registry;
|
||||
import java.rmi.server.UnicastRemoteObject;
|
||||
import java.util.Objects;
|
||||
import javax.management.MBeanServer;
|
||||
import javax.management.remote.JMXConnectorServer;
|
||||
import javax.management.remote.JMXConnectorServerFactory;
|
||||
import javax.management.remote.JMXServiceURL;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* A reporter which publishes metric values to a JMX server.
|
||||
@@ -78,10 +81,9 @@ public class JmxReporterServer {
|
||||
}
|
||||
|
||||
public JmxReporterServer build() {
|
||||
Preconditions.checkNotNull(registry, "registry cannot be null!");
|
||||
Preconditions.checkNotNull(mBeanServer, "mBeanServer cannot be null!");
|
||||
Preconditions
|
||||
.checkArgument(!StringUtils.isNullOrEmpty(host), "host cannot be null or empty!");
|
||||
Objects.requireNonNull(registry, "registry cannot be null!");
|
||||
Objects.requireNonNull(mBeanServer, "mBeanServer cannot be null!");
|
||||
ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(host), "host cannot be null or empty!");
|
||||
return new JmxReporterServer(registry, host, port, mBeanServer);
|
||||
}
|
||||
}
|
||||
@@ -110,7 +112,7 @@ public class JmxReporterServer {
|
||||
}
|
||||
|
||||
public void start() {
|
||||
Preconditions.checkArgument(reporter != null && connector != null,
|
||||
ValidationUtils.checkArgument(reporter != null && connector != null,
|
||||
"reporter or connector cannot be null!");
|
||||
try {
|
||||
connector.start();
|
||||
|
||||
@@ -18,12 +18,12 @@
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import com.codahale.metrics.Gauge;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.google.common.io.Closeables;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -53,8 +53,7 @@ public class Metrics {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
|
||||
try {
|
||||
reporter.report();
|
||||
reporter.stop();
|
||||
Closeables.close(reporter.getReporter(), true);
|
||||
FileIOUtils.close(reporter.getReporter(), true);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
@@ -32,6 +32,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
@@ -43,7 +44,6 @@ import org.apache.hudi.table.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -113,7 +113,7 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
String maxInstantTime = metaClient
|
||||
.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
||||
.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
LOG.info("MaxMemoryPerCompaction => " + SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()));
|
||||
|
||||
@@ -214,7 +214,7 @@ public class RollbackHelper implements Serializable {
|
||||
|
||||
private Map<HeaderMetadataType, String> generateHeader(String commit) {
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = new HashMap<>();
|
||||
Map<HeaderMetadataType, String> header = new HashMap<>(3);
|
||||
header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
|
||||
|
||||
@@ -249,17 +249,13 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
|
||||
|
||||
String filename0 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Collections.singletonList(record1),
|
||||
schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Collections.singletonList(record1), schema, null, false);
|
||||
String filename1 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", new ArrayList<>(),
|
||||
schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", new ArrayList<>(), schema, null, false);
|
||||
String filename2 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record2),
|
||||
schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record2), schema, null, false);
|
||||
String filename3 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record4),
|
||||
schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record4), schema, null, false);
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
@@ -45,6 +45,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
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.CollectionUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
@@ -55,9 +56,6 @@ import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -290,7 +288,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
|
||||
for (int i = 0; i < dataFiles.size(); i++) {
|
||||
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
|
||||
Iterables.get(dataFiles, i).getCommitTime(),
|
||||
(dataFiles.get(i)).getCommitTime(),
|
||||
commitedVersions.get(commitedVersions.size() - 1 - i));
|
||||
}
|
||||
}
|
||||
@@ -740,12 +738,14 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
String file1P1C0 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "000");
|
||||
|
||||
HoodieCommitMetadata commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file1P0C0).build())
|
||||
.put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file1P1C0).build())
|
||||
.build());
|
||||
HoodieCommitMetadata commitMetadata = generateCommitMetadata(
|
||||
Collections.unmodifiableMap(new HashMap<String, List<String>>() {
|
||||
{
|
||||
put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, CollectionUtils.createImmutableList(file1P0C0));
|
||||
put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, CollectionUtils.createImmutableList(file1P1C0));
|
||||
}
|
||||
})
|
||||
);
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"),
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
@@ -778,12 +778,12 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0); // update
|
||||
HoodieTestUtils
|
||||
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "001", file1P1C0); // update
|
||||
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file1P0C0).add(file2P0C1).build())
|
||||
.put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file1P1C0).add(file2P1C1).build())
|
||||
.build());
|
||||
commitMetadata = generateCommitMetadata(new HashMap<String, List<String>>() {
|
||||
{
|
||||
put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, CollectionUtils.createImmutableList(file1P0C0, file2P0C1));
|
||||
put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, CollectionUtils.createImmutableList(file1P1C0, file2P1C1));
|
||||
}
|
||||
});
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"),
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
@@ -814,10 +814,9 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
String file3P0C2 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002");
|
||||
|
||||
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file1P0C0).add(file2P0C1).add(file3P0C2).build())
|
||||
.build());
|
||||
commitMetadata = generateCommitMetadata(CollectionUtils
|
||||
.createImmutableMap(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file3P0C2)));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "002"),
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
@@ -840,10 +839,8 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update
|
||||
String file4P0C3 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003");
|
||||
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file1P0C0).add(file2P0C1).add(file4P0C3).build())
|
||||
.build());
|
||||
commitMetadata = generateCommitMetadata(CollectionUtils.createImmutableMap(
|
||||
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file4P0C3)));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "003"),
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
@@ -871,10 +868,8 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// No cleaning on partially written file, with no commit.
|
||||
HoodieTestUtils
|
||||
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "004", file3P0C2); // update
|
||||
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file3P0C2).build())
|
||||
.build());
|
||||
commitMetadata = generateCommitMetadata(CollectionUtils.createImmutableMap(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
CollectionUtils.createImmutableList(file3P0C2)));
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "004"));
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(
|
||||
|
||||
@@ -27,7 +27,6 @@ import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
@@ -122,8 +121,14 @@ public class TestHoodieCompactionStrategy {
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, partitionPaths[2])
|
||||
.put(110 * MB, partitionPaths[2]).put(100 * MB, partitionPaths[1]).put(90 * MB, partitionPaths[0]).build();
|
||||
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
|
||||
{
|
||||
put(120 * MB, partitionPaths[2]);
|
||||
put(110 * MB, partitionPaths[2]);
|
||||
put(100 * MB, partitionPaths[1]);
|
||||
put(90 * MB, partitionPaths[0]);
|
||||
}
|
||||
});
|
||||
|
||||
DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig =
|
||||
@@ -162,9 +167,16 @@ public class TestHoodieCompactionStrategy {
|
||||
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
|
||||
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, currentDay)
|
||||
.put(110 * MB, currentDayMinus1).put(100 * MB, currentDayMinus2).put(80 * MB, currentDayMinus3)
|
||||
.put(90 * MB, currentDayPlus1).put(70 * MB, currentDayPlus5).build();
|
||||
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
|
||||
{
|
||||
put(120 * MB, currentDay);
|
||||
put(110 * MB, currentDayMinus1);
|
||||
put(100 * MB, currentDayMinus2);
|
||||
put(80 * MB, currentDayMinus3);
|
||||
put(90 * MB, currentDayPlus1);
|
||||
put(70 * MB, currentDayPlus5);
|
||||
}
|
||||
});
|
||||
|
||||
BoundedPartitionAwareCompactionStrategy strategy = new BoundedPartitionAwareCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig =
|
||||
@@ -204,9 +216,16 @@ public class TestHoodieCompactionStrategy {
|
||||
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
|
||||
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, currentDay)
|
||||
.put(110 * MB, currentDayMinus1).put(100 * MB, currentDayMinus2).put(80 * MB, currentDayMinus3)
|
||||
.put(90 * MB, currentDayPlus1).put(70 * MB, currentDayPlus5).build();
|
||||
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
|
||||
{
|
||||
put(120 * MB, currentDay);
|
||||
put(110 * MB, currentDayMinus1);
|
||||
put(100 * MB, currentDayMinus2);
|
||||
put(80 * MB, currentDayMinus3);
|
||||
put(90 * MB, currentDayPlus1);
|
||||
put(70 * MB, currentDayPlus5);
|
||||
}
|
||||
});
|
||||
|
||||
UnBoundedPartitionAwareCompactionStrategy strategy = new UnBoundedPartitionAwareCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig =
|
||||
|
||||
Reference in New Issue
Block a user