From dd063f100fd24cabad1557bf2258005f3a4f345c Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Mon, 16 Jun 2025 22:56:06 +0000 Subject: [PATCH 01/26] Initial approach --- .../ingest/mapreduce/job/IngestJob.java | 16 +- .../job/MultiRFileOutputFormatter.java | 43 +-- .../ingest/mapreduce/job/SplitsCache.java | 26 ++ .../mapreduce/job/SplitsCacheFactory.java | 30 ++ .../ingest/mapreduce/job/SplitsConstants.java | 10 + .../ingest/mapreduce/job/SplitsFile.java | 258 ++++++++++++++++-- .../mapreduce/job/TableSplitsCache.java | 19 +- .../job/reindex/RepartitionerJob.java | 4 +- .../job/reindex/ShardReindexJob.java | 4 +- .../partition/BalancedShardPartitioner.java | 159 +---------- .../partition/MultiTableRangePartitioner.java | 24 +- .../TabletLocationHashPartitioner.java | 55 +--- .../TabletLocationNamePartitioner.java | 139 ---------- .../ingest/util/GenerateSplitsFile.java | 5 +- .../job/MultiRFileOutputFormatterTest.java | 8 - .../ingest/mapreduce/job/SplitsFileTest.java | 97 +++---- .../mapreduce/job/TableSplitsCacheTest.java | 4 +- .../MultiTableRRRangePartitionerTest.java | 21 +- .../MultiTableRangePartitionerTest.java | 9 +- .../SplitBasedHashPartitionerTest.java | 5 +- .../TabletLocationNamePartitionerTest.java | 82 ------ .../partition/TestShardGenerator.java | 7 +- 22 files changed, 434 insertions(+), 591 deletions(-) create mode 100644 warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java create mode 100644 warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java create mode 100644 warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java delete mode 100644 warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitioner.java delete mode 100644 warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitionerTest.java diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java index 66331cab147..e259aa449aa 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java @@ -354,13 +354,11 @@ public int run(String[] args) throws Exception { configureJob(job, conf, workDirPath, outputFs); // Log configuration - log.info("Types: {}", TypeRegistry.getTypeNames()); - log.info("Tables: {}", tableNames); - log.info("InputFormat: {}", job.getInputFormatClass().getName()); - log.info("Mapper: {}", job.getMapperClass().getName()); - log.info("Reduce tasks: {}", (useMapOnly ? 0 : reduceTasks)); - log.info("Split File: {} / {}", conf.get(TableSplitsCache.SPLITS_CACHE_DIR), - conf.get(TableSplitsCache.SPLITS_CACHE_FILE, TableSplitsCache.DEFAULT_SPLITS_CACHE_FILE)); + log.info("Types: " + TypeRegistry.getTypeNames()); + log.info("Tables: " + tableNames); + log.info("InputFormat: " + job.getInputFormatClass().getName()); + log.info("Mapper: " + job.getMapperClass().getName()); + log.info("Reduce tasks: " + (useMapOnly ? 0 : reduceTasks)); // Note that if we run any other jobs in the same vm (such as a sampler), then we may // need to catch and throw away an exception here @@ -661,7 +659,7 @@ protected Configuration parseArguments(String[] args, Configuration conf) throws } else if (args[i].equals("-disableRefreshSplits")) { conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); } else if (args[i].equals("-splitsCacheDir")) { - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, args[++i]); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, args[++i]); } else if (args[i].equals("-multipleNumShardsCacheDir")) { conf.set(NumShards.MULTIPLE_NUMSHARDS_CACHE_PATH, args[++i]); } else if (args[i].equals("-enableAccumuloConfigCache")) { @@ -845,7 +843,7 @@ protected void configureBulkPartitionerAndOutputFormatter(Job job, AccumuloHelpe conf.setInt("splits.num.reduce", this.reduceTasks); // used by the output formatter and the sharded partitioner long before = System.currentTimeMillis(); - SplitsFile.setupFile(job, conf); + SplitsCache.getInstance(conf).setupJob(job); long after = System.currentTimeMillis(); log.info("Sharded splits files setup time: {}ms", (after - before)); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index a146fbcb737..dc5a7e5cf5b 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -96,6 +96,7 @@ public class MultiRFileOutputFormatter extends FileOutputFormat tableConfigs; protected Set tableIds = null; + protected SplitsCache splitsCache; protected long maxRFileSize = 0; protected int maxRFileEntries = 0; protected boolean generateMapFileRowKeys = false; @@ -435,10 +436,12 @@ protected void setTableIdsAndConfigs() throws IOException { private SafeFileOutputCommitter _committer = null; @Override - public synchronized OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException { - if (_committer == null) { - Path output = getOutputPath(context); - _committer = new SafeFileOutputCommitter(output, context); + public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException { + synchronized (this) { + if (_committer == null) { + Path output = getOutputPath(context); + _committer = new SafeFileOutputCommitter(output, context); + } } return _committer; } @@ -451,6 +454,7 @@ public RecordWriter getRecordWriter(final TaskAttemptContex FileOutputCommitter committer = (FileOutputCommitter) getOutputCommitter(context); workDir = committer.getWorkPath(); conf = context.getConfiguration(); + splitsCache = SplitsCache.getInstance(conf); setTableIdsAndConfigs(); @@ -621,12 +625,8 @@ private SizeTrackingWriter getOrCreateWriter(TaskAttemptContext context, String if (generateMapFilePerShardLocation) { // Look up the shard location (tablet server serving shard ID rowKey) // If we don't have a location, then just use the rowKey itself. - Map shardLocs = getShardLocations(tableName); - shardLocation = shardLocs.containsKey(rowKey) ? shardLocs.get(rowKey) : null; - if (shardLocation == null) { - // in this case we have a shard id that has no split. Lets put this in one "extra" file - shardLocation = "extra"; - } + // in the case we have a shard id that has no split. Lets put this in one "extra" file + shardLocation = splitsCache.getExactLocation(tableName, rowKey, () -> "extra"); } // Combine table name with shard location so that we end up // with all of the shard map files under directories that can be @@ -652,27 +652,4 @@ private SizeTrackingWriter getOrCreateWriter(TaskAttemptContext context, String } }; } - - /** - * Read in the sequence file (that was created at job startup) for the given table that contains a list of shard IDs and the corresponding tablet server to - * which that shard is assigned. - * - * @param tableName - * the table name - * @return a mapping of the shard ids and tablet server - * @throws IOException - * if there is an issue with read or write - */ - protected Map getShardLocations(String tableName) throws IOException { - // Create the Map of sharded table name to [shardId -> server] - if (this.tableShardLocations == null) { - this.tableShardLocations = new HashMap<>(); - } - - if (null == this.tableShardLocations.get(tableName)) { - this.tableShardLocations.put(tableName, SplitsFile.getSplitsAndLocations(conf, tableName)); - } - - return tableShardLocations.get(tableName); - } } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java new file mode 100644 index 00000000000..8c6235c1269 --- /dev/null +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java @@ -0,0 +1,26 @@ +package datawave.ingest.mapreduce.job; + +import java.io.IOException; +import java.util.function.Supplier; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; + +public interface SplitsCache extends AutoCloseable { + static SplitsCache getInstance(final Configuration conf) { + return SplitsCacheFactory.getSplitsCache(conf); + } + + void setupJob(final Job job) throws IOException; + + boolean hasSplits(); + + int getExactIndex(String table, Text key); + + int getExactPartition(String table, Text key); + + int getNearestPartition(String table, Text key); + + String getExactLocation(String table, Text key, Supplier defaultFn); +} diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java new file mode 100644 index 00000000000..854855ef844 --- /dev/null +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java @@ -0,0 +1,30 @@ +package datawave.ingest.mapreduce.job; + +import static datawave.ingest.mapreduce.job.SplitsConstants.DEFAULT_SPLITS_CACHE_IMPL; +import static datawave.ingest.mapreduce.job.SplitsConstants.SPLITS_CACHE_IMPL; + +import java.lang.reflect.InvocationTargetException; + +import org.apache.hadoop.conf.Configuration; + +public class SplitsCacheFactory { + static volatile SplitsCache INSTANCE; + + public static SplitsCache getSplitsCache(final Configuration conf) { + if (INSTANCE == null) { + synchronized (SplitsCacheFactory.class) { + if (INSTANCE == null) { + try { + final String splitsCacheImpl = conf.get(SPLITS_CACHE_IMPL, DEFAULT_SPLITS_CACHE_IMPL); + // noinspection unchecked + final Class clazz = (Class) Class.forName(splitsCacheImpl); + INSTANCE = clazz.getDeclaredConstructor().newInstance(); + } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException ex) { + throw new RuntimeException(ex); + } + } + } + } + return INSTANCE; + } +} diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java new file mode 100644 index 00000000000..1e6bc9e5575 --- /dev/null +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java @@ -0,0 +1,10 @@ +package datawave.ingest.mapreduce.job; + +public class SplitsConstants { + public static final String SPLITS_CACHE_DIR = "datawave.ingest.splits.cache.dir"; + public static final String SPLITS_CACHE_FILE = "datawave.ingest.splits.cache.file"; + public static final String SPLITS_CACHE_IMPL = "datawave.ingest.splits.cache.impl"; + + public static final String DEFAULT_SPLITS_CACHE_IMPL = ""; + public static final String DEFAULT_SPLITS_CACHE_DIR = "/data/splitsCache"; +} diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java index 2bda9ba6bc4..71a9ba4608f 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java @@ -1,14 +1,20 @@ package datawave.ingest.mapreduce.job; +import static datawave.ingest.mapreduce.job.SplitsConstants.SPLITS_CACHE_DIR; +import static datawave.ingest.mapreduce.job.SplitsConstants.SPLITS_CACHE_FILE; + import java.io.IOException; +import java.io.UncheckedIOException; import java.net.URI; -import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; -import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.commons.lang.time.DateUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.conf.Configuration; @@ -17,14 +23,15 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; -import org.apache.log4j.Logger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import datawave.ingest.mapreduce.handler.shard.ShardIdFactory; import datawave.ingest.mapreduce.handler.shard.ShardedDataTypeHandler; import datawave.util.time.DateHelper; -public class SplitsFile { - private static final Logger log = Logger.getLogger(SplitsFile.class); +public class SplitsFile implements SplitsCache { + private static final Logger log = LoggerFactory.getLogger(SplitsFile.class); public static final String SPLIT_WORK_DIR = "split.work.dir"; public static final String MAX_SHARDS_PER_TSERVER = "shardedMap.max.shards.per.tserver"; @@ -33,7 +40,22 @@ public class SplitsFile { public static final String CONFIGURED_SHARDED_TABLE_NAMES = ShardedDataTypeHandler.SHARDED_TNAMES + ".configured"; public static final String DIST_CACHE_LABEL = "splitsFile"; - public static void setupFile(Job job, Configuration conf) throws IOException, URISyntaxException, AccumuloSecurityException, AccumuloException { + private static final int NUMBER_MILLIS_BACK = 0; + + private final TableSplitsCache instance; + private final Configuration conf; + private final ConcurrentHashMap> shardPartitionsByTable; + private final String today; + + public SplitsFile(Configuration conf) throws IOException { + this.conf = conf; + this.shardPartitionsByTable = new ConcurrentHashMap<>(); + this.today = DateHelper.format(NUMBER_MILLIS_BACK); + instance = TableSplitsCache.getCurrentCache(conf); + } + + @Override + public void setupJob(Job job) throws IOException { Path baseSplitsPath = TableSplitsCache.getSplitsPath(conf); FileSystem sourceFs = baseSplitsPath.getFileSystem(conf); @@ -45,12 +67,11 @@ public static void setupFile(Job job, Configuration conf) throws IOException, UR try { log.info("Base splits: " + baseSplitsPath); - Path destSplits = new Path( - conf.get(SPLIT_WORK_DIR) + "/" + conf.get(TableSplitsCache.SPLITS_CACHE_FILE, TableSplitsCache.DEFAULT_SPLITS_CACHE_FILE)); + Path destSplits = new Path(conf.get(SPLIT_WORK_DIR) + "/" + conf.get(SPLITS_CACHE_FILE, TableSplitsCache.DEFAULT_SPLITS_CACHE_FILE)); log.info("Dest splits: " + destSplits); FileUtil.copy(sourceFs, baseSplitsPath, destFs, destSplits, false, conf); - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, conf.get(SPLIT_WORK_DIR)); + conf.set(SPLITS_CACHE_DIR, conf.get(SPLIT_WORK_DIR)); // // if we want the freshest splits, go ahead and update them in the work dir // if (TableSplitsCache.shouldRefreshSplits(conf)) { @@ -65,11 +86,89 @@ public static void setupFile(Job job, Configuration conf) throws IOException, UR } catch (Exception e) { log.error("Unable to use splits file because " + e.getMessage()); - throw e; + throw new IOException(e); + } + } + + @Override + public void close() { + // no code + } + + @Override + public boolean hasSplits() { + try { + return !instance.getSplits().isEmpty(); + } catch (IOException e) { + throw new UncheckedIOException(e); } } - private static void validate(Configuration conf) throws IOException { + @Override + public String getExactLocation(String table, Text key, Supplier defaultFn) { + String location; + try { + Map locationByTable = instance.getSplitsAndLocationByTable(table); + String keyValue = locationByTable.get(key); + location = keyValue == null ? defaultFn.get() : keyValue; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return location; + } + + @Override + public int getExactIndex(String table, Text key) { + List splits; + try { + splits = instance.getSplits(table); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return splits != null ? Collections.binarySearch(splits, key) : -1; + } + + @Override + public int getExactPartition(String tableName, Text shardId) { + Map assignments; + try { + assignments = lazilyCreateAssignments(tableName); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + Integer partitionId = assignments.get(shardId); + if (partitionId != null) { + return partitionId; + } + return -1; + } + + @Override + public int getNearestPartition(String tableName, Text shardId) { + Map assignments = null; + try { + assignments = lazilyCreateAssignments(tableName); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + Integer partitionId = assignments.get(shardId); + if (partitionId != null) { + return partitionId; + } + + List keys = new ArrayList<>(assignments.keySet()); + Collections.sort(keys); + int closestAssignment = Collections.binarySearch(keys, shardId); + if (closestAssignment >= 0) { + log.warn("Something is screwy, found {} on the second try", shardId); + return assignments.get(shardId); + } + // insertion point in the index of the key greater + Text shardString = keys.get(Math.abs(closestAssignment + 1)); + return assignments.get(shardString); + } + + public void validate(Configuration conf) throws IOException { TableSplitsCache cache = TableSplitsCache.getCurrentCache(conf); int daysToVerify = conf.getInt(SHARDS_BALANCED_DAYS_TO_VERIFY, 2) - 1; @@ -79,7 +178,7 @@ private static void validate(Configuration conf) throws IOException { } - public static void validateShardIdLocations(Configuration conf, String tableName, int daysToVerify, Map shardIdToLocation) { + public void validateShardIdLocations(Configuration conf, String tableName, int daysToVerify, Map shardIdToLocation) { ShardIdFactory shardIdFactory = new ShardIdFactory(conf); // assume true unless proven otherwise boolean isValid = true; @@ -107,7 +206,121 @@ public static void validateShardIdLocations(Configuration conf, String tableName } } - private static boolean prefixMatches(byte[] prefixBytes, byte[] keyBytes, int keyLen) { + /** + * For a given tablename, provides the mapping from {@code shard id -> partition} + * + * @param tableName + * the name of the table + * @return a list of the shard mappings + * @throws IOException + * if there is an issue with read or write + */ + private Map lazilyCreateAssignments(String tableName) throws IOException { + Map assignments = this.shardPartitionsByTable.get(tableName); + if (null == assignments) { + assignments = getPartitionsByShardId(tableName); + this.shardPartitionsByTable.put(tableName, assignments); + } + return assignments; + } + + /** + * Loads the splits file for the table name and uses it to assign partitions. + * + * @param tableName + * name of the table + * @return a map of the partitions + * @throws IOException + * if there is an issue with read or write + */ + private HashMap getPartitionsByShardId(String tableName) throws IOException { + if (log.isDebugEnabled()) + log.debug("Loading splits data for " + tableName); + + List sortedSplits = instance.getSplits(tableName); + Map shardIdToLocation = instance.getSplitsAndLocationByTable(tableName); + + if (log.isDebugEnabled()) + log.debug("Assigning partitioners for each shard in " + tableName); + return assignPartitionsForEachShard(sortedSplits, shardIdToLocation); + } + + /** + * 1. sorts the the tablet assignments by shard id, starting with the most recent going backwards
+ * 2. assigns partitions to each tservers, starting from the beginning, but skipping future dates
+ * 3. assigns partitions to each shardid by looking up its tserver's assignments
+ * 4. returns the {@code shard id -> partition} map + *

+ * e.g.,
+ * 1. sorted assignments
+ * 2. tserver map
+ * 3. shard map: {@code future->tserver7 *no change* future->2 shard4->tserver2 tserver2->0 shard4->0 + * shard3->tserver3 tserver3->1 shard3->1 shard2->tserver2 *no change* shard2->0 shard1->tserver7 tserver7->2 shard1->2} + * + * @param shardIdToLocations + * the map of shard ids and their location + * @return shardId to + */ + private HashMap assignPartitionsForEachShard(List sortedShardIds, Map shardIdToLocations) { + int totalNumUniqueTServers = calculateNumberOfUniqueTservers(shardIdToLocations); + + HashMap partitionsByTServer = getTServerAssignments(totalNumUniqueTServers, sortedShardIds, shardIdToLocations); + HashMap partitionsByShardId = getShardIdAssignments(shardIdToLocations, partitionsByTServer); + + if (log.isDebugEnabled()) + log.debug("Number of shardIds assigned: " + partitionsByShardId.size()); + + return partitionsByShardId; + } + + private int calculateNumberOfUniqueTservers(Map shardIdToLocations) { + + int totalNumUniqueTServers = new HashSet(shardIdToLocations.values()).size(); + if (log.isDebugEnabled()) + log.debug("Total TServers involved: " + totalNumUniqueTServers); + return totalNumUniqueTServers; + } + + private HashMap getTServerAssignments(int totalNumTServers, List sortedShardIds, Map shardIdsToTservers) { + HashMap partitionsByTServer = new HashMap<>(totalNumTServers); + int nextAvailableSlot = 0; + boolean alreadySkippedFutureShards = false; + for (Text shard : sortedShardIds) { + if (alreadySkippedFutureShards || !isFutureShard(shard)) { // short circuiting for performance + alreadySkippedFutureShards = true; + String location = shardIdsToTservers.get(shard); + Integer assignedPartition = partitionsByTServer.get(location); + if (null == assignedPartition) { + assignedPartition = nextAvailableSlot; + partitionsByTServer.put(location, assignedPartition); + nextAvailableSlot++; + } + if (partitionsByTServer.size() == totalNumTServers) { + // all the tservers have been assigned partitions, so we can stop + return partitionsByTServer; + } + } + } + return partitionsByTServer; + } + + private boolean isFutureShard(Text shardId) { + String shardIdStr = shardId.toString().intern(); + if (shardIdStr.length() < 8) { + return true; + } + return shardIdStr.substring(0, 8).compareTo(today) > 0; + } + + private HashMap getShardIdAssignments(Map shardIdsToTservers, HashMap partitionsByTServer) { + HashMap partitionsByShardId = new HashMap<>(); + for (Map.Entry entry : shardIdsToTservers.entrySet()) { + partitionsByShardId.put(entry.getKey(), partitionsByTServer.get(entry.getValue())); + } + return partitionsByShardId; + } + + private boolean prefixMatches(byte[] prefixBytes, byte[] keyBytes, int keyLen) { // if key length is less than prefix size, no use comparing if (prefixBytes.length > keyLen) { return false; @@ -132,7 +345,7 @@ private static boolean prefixMatches(byte[] prefixBytes, byte[] keyBytes, int ke * that should exist * @return if the number of shards for the given date are as expected */ - private static boolean shardsExistForDate(Map locations, String datePrefix, int expectedNumberOfShards) { + private boolean shardsExistForDate(Map locations, String datePrefix, int expectedNumberOfShards) { int count = 0; byte[] prefixBytes = datePrefix.getBytes(); for (Text key : locations.keySet()) { @@ -152,7 +365,7 @@ private static boolean shardsExistForDate(Map locations, String dat * to check * @return if the shards are distributed in a balanced fashion */ - private static boolean shardsAreBalanced(Map locations, String datePrefix, int maxShardsPerTserver) { + private boolean shardsAreBalanced(Map locations, String datePrefix, int maxShardsPerTserver) { // assume true unless proven wrong boolean dateIsBalanced = true; @@ -184,16 +397,11 @@ private static boolean shardsAreBalanced(Map locations, String date return dateIsBalanced; } - public static Map> getSplits(Configuration conf) throws IOException { - return TableSplitsCache.getCurrentCache(conf).getSplits(); - - } - - public static Map getSplitsAndLocations(Configuration conf, String tableName) throws IOException { - return TableSplitsCache.getCurrentCache(conf).getSplitsAndLocationByTable(tableName); + public Map getSplitsAndLocations(Configuration conf, String tableName) throws IOException { + return instance.getSplitsAndLocationByTable(tableName); } - public static List getSplits(Configuration conf, String tableName) throws IOException { - return TableSplitsCache.getCurrentCache(conf).getSplits(tableName); + public List getSplits(Configuration conf, String tableName) throws IOException { + return instance.getSplits(tableName); } } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/TableSplitsCache.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/TableSplitsCache.java index ab901845675..1cf8e6725d6 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/TableSplitsCache.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/TableSplitsCache.java @@ -1,5 +1,8 @@ package datawave.ingest.mapreduce.job; +import static datawave.ingest.mapreduce.job.SplitsConstants.SPLITS_CACHE_DIR; +import static datawave.ingest.mapreduce.job.SplitsConstants.SPLITS_CACHE_FILE; + import java.io.BufferedOutputStream; import java.io.BufferedReader; import java.io.File; @@ -51,14 +54,12 @@ public class TableSplitsCache extends BaseHdfsFileCacheUtil { public static final String REFRESH_SPLITS = "datawave.ingest.refresh.splits"; - public static final String SPLITS_CACHE_DIR = "datawave.ingest.splits.cache.dir"; - public static final String SPLITS_CACHE_FILE = "datawave.ingest.splits.cache.fileName"; + public static final String DEFAULT_SPLITS_CACHE_FILE = "all-splits.txt"; - public static final String MAX_SPLIT_DECREASE = "datawave.ingest.splits.max.decrease.number"; - public static final String MAX_SPLIT_PERCENTAGE_DECREASE = "datawave.ingest.splits.max.decrease.percentage"; + private static final String MAX_SPLIT_DECREASE = "datawave.ingest.splits.max.decrease.number"; + private static final String MAX_SPLIT_PERCENTAGE_DECREASE = "datawave.ingest.splits.max.decrease.percentage"; private static final Logger log = Logger.getLogger(TableSplitsCache.class); private static final String DEFAULT_SPLITS_CACHE_DIR = "/data/splitsCache"; - public static final String DEFAULT_SPLITS_CACHE_FILE = "all-splits.txt"; private static final short DEFAULT_MAX_SPLIT_DECREASE = 42; private static final double DEFAULT_MAX_SPLIT_PERCENTAGE_DECREASE = .5; private static final boolean DEFAULT_REFRESH_SPLITS = true; @@ -407,7 +408,7 @@ private String dedup(Map dedupMap, String value) { * @throws IOException * for issues with read or write */ - public List getSplits(String table) throws IOException { + List getSplits(String table) throws IOException { if (this.splits.isEmpty()) { read(); } @@ -425,7 +426,7 @@ public List getSplits(String table) throws IOException { * @throws IOException * for issues with read or write */ - public List getSplits(String table, int maxSplits) throws IOException { + List getSplits(String table, int maxSplits) throws IOException { return trimSplits(getSplits(table), maxSplits); } @@ -434,7 +435,7 @@ public List getSplits(String table, int maxSplits) throws IOException { * @throws IOException * for issues with read or write */ - public Map> getSplits() throws IOException { + Map> getSplits() throws IOException { if (this.splits.isEmpty()) read(); return Collections.unmodifiableMap(splits); @@ -445,7 +446,7 @@ public Map> getSplits() throws IOException { * @return map of splits to tablet locations for the table * @throws IOException */ - public Map getSplitsAndLocationByTable(String table) throws IOException { + Map getSplitsAndLocationByTable(String table) throws IOException { if (this.splitLocations.isEmpty()) read(); if (this.splitLocations.containsKey(table)) { diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/RepartitionerJob.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/RepartitionerJob.java index 2b13c25d7be..ba8e694766f 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/RepartitionerJob.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/RepartitionerJob.java @@ -41,7 +41,7 @@ import datawave.ingest.mapreduce.job.DelegatingPartitioner; import datawave.ingest.mapreduce.job.IngestJob; import datawave.ingest.mapreduce.job.RFileInputFormat; -import datawave.ingest.mapreduce.job.SplitsFile; +import datawave.ingest.mapreduce.job.SplitsCache; import datawave.ingest.mapreduce.job.reduce.BulkIngestKeyDedupeCombiner; import datawave.ingest.mapreduce.job.writer.AbstractContextWriter; import datawave.ingest.mapreduce.job.writer.ContextWriter; @@ -103,7 +103,7 @@ private Job setupJob() throws URISyntaxException, IOException, AccumuloException Configuration config = j.getConfiguration(); // setup and cache table from config - SplitsFile.setupFile(j, config); + SplitsCache.getInstance(config).setupJob(j); Set tableNames = IngestJob.setupAndCacheTables(config, false); config.setInt("splits.num.reduce", jobConfig.reducers); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/ShardReindexJob.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/ShardReindexJob.java index 55f202d41e7..05ebb9f8413 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/ShardReindexJob.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/reindex/ShardReindexJob.java @@ -61,7 +61,7 @@ import datawave.ingest.mapreduce.job.IngestJob; import datawave.ingest.mapreduce.job.MultiRFileOutputFormatter; import datawave.ingest.mapreduce.job.RFileInputFormat; -import datawave.ingest.mapreduce.job.SplitsFile; +import datawave.ingest.mapreduce.job.SplitsCache; import datawave.ingest.mapreduce.job.reduce.BulkIngestKeyAggregatingReducer; import datawave.ingest.mapreduce.job.reduce.BulkIngestKeyDedupeCombiner; import datawave.ingest.mapreduce.job.util.AccumuloUtil; @@ -230,7 +230,7 @@ private Job setupJob() throws IOException, ParseException, AccumuloException, Ta // all changes to configuration must be before this line Job j = Job.getInstance(getConf()); Configuration config = j.getConfiguration(); - SplitsFile.setupFile(j, config); + SplitsCache.getInstance(config).setupJob(j); // check if using some form of accumulo in input if (jobConfig.inputFiles == null) { diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java index 8c48e537167..0ef1970d4e3 100755 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java @@ -1,15 +1,10 @@ package datawave.ingest.mapreduce.partition; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.List; import java.util.Map; import org.apache.accumulo.core.data.Value; -import org.apache.commons.lang.time.DateUtils; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -20,8 +15,7 @@ import datawave.ingest.mapreduce.handler.shard.ShardIdFactory; import datawave.ingest.mapreduce.handler.shard.ShardedDataTypeHandler; import datawave.ingest.mapreduce.job.BulkIngestKey; -import datawave.ingest.mapreduce.job.SplitsFile; -import datawave.util.time.DateHelper; +import datawave.ingest.mapreduce.job.SplitsCache; /** * The BalancedShardPartitioner takes advantage of the way that shards are balanced. See ShardedTableTabletBalancer. * The partitioner is designed to have no @@ -34,11 +28,11 @@ */ public class BalancedShardPartitioner extends Partitioner implements Configurable, DelegatePartitioner { private static final Logger log = Logger.getLogger(BalancedShardPartitioner.class); - private static final long now = System.currentTimeMillis(); - private static final String today = formatDay(0); private Configuration conf; private Map> shardPartitionsByTable; private Map offsetsFactorByTable; + private SplitsCache splitsCache; + private String missingShardStrategy; int missingShardIdCount = 0; public static final String MISSING_SHARD_STRATEGY_PROP = "datawave.ingest.mapreduce.partition.BalancedShardPartitioner.missing.shard.strategy"; @@ -46,7 +40,7 @@ public class BalancedShardPartitioner extends Partitioner i private ShardIdFactory shardIdFactory = null; @Override - public synchronized int getPartition(BulkIngestKey key, Value value, int numReduceTasks) { + public int getPartition(BulkIngestKey key, Value value, int numReduceTasks) { try { // partition will be balanced for a given day, more so for recent days int partition = getAssignedPartition(key.getTableName().toString(), key.getKey().getRow()); @@ -64,18 +58,15 @@ public synchronized int getPartition(BulkIngestKey key, Value value, int numRedu } private int getAssignedPartition(String tableName, Text shardId) throws IOException { - Map assignments = lazilyCreateAssignments(tableName); - - Integer partitionId = assignments.get(shardId); - if (partitionId != null) { - return partitionId; - } // if the partitionId is not there, either shards were not created for the day // or not all shards were created for the day - String missingShardStrategy = conf.get(MISSING_SHARD_STRATEGY_PROP, "hash"); switch (missingShardStrategy) { case "hash": + int partition = splitsCache.getExactPartition(tableName, shardId); + if (partition >= 0) { + return partition; + } // only warn a few times per partitioner to avoid flooding the logs if (missingShardIdCount < 10) { log.warn("shardId didn't have a partition assigned to it: " + shardId); @@ -83,141 +74,12 @@ private int getAssignedPartition(String tableName, Text shardId) throws IOExcept } return (shardId.hashCode() & Integer.MAX_VALUE); case "collapse": - ArrayList keys = new ArrayList<>(assignments.keySet()); - Collections.sort(keys); - int closestAssignment = Collections.binarySearch(keys, shardId); - if (closestAssignment >= 0) { - // Should have found it earlier, but just in case go ahead and return it - log.warn("Something is screwy, found " + shardId + " on the second try"); - return assignments.get(shardId); - } - // (-(insertion point) - 1) // insertion point in the index of the key greater - Text shardString = keys.get(Math.abs(closestAssignment + 1)); - return assignments.get(shardString); + return splitsCache.getNearestPartition(tableName, shardId); default: throw new RuntimeException("Unsupported missing shard strategy " + MISSING_SHARD_STRATEGY_PROP + "=" + missingShardStrategy); } } - /** - * For a given tablename, provides the mapping from {@code shard id -> partition} - * - * @param tableName - * the name of the table - * @return a list of the shard mappings - * @throws IOException - * if there is an issue with read or write - */ - private Map lazilyCreateAssignments(String tableName) throws IOException { - if (this.shardPartitionsByTable == null) { - this.shardPartitionsByTable = new HashMap<>(); - } - if (null == this.shardPartitionsByTable.get(tableName)) { - this.shardPartitionsByTable.put(tableName, getPartitionsByShardId(tableName)); - } - return this.shardPartitionsByTable.get(tableName); - } - - /** - * Loads the splits file for the table name and uses it to assign partitions. - * - * @param tableName - * name of the table - * @return a map of the partitions - * @throws IOException - * if there is an issue with read or write - */ - private HashMap getPartitionsByShardId(String tableName) throws IOException { - if (log.isDebugEnabled()) - log.debug("Loading splits data for " + tableName); - - List sortedSplits = SplitsFile.getSplits(conf, tableName); - Map shardIdToLocation = SplitsFile.getSplitsAndLocations(conf, tableName); - - if (log.isDebugEnabled()) - log.debug("Assigning partitioners for each shard in " + tableName); - return assignPartitionsForEachShard(sortedSplits, shardIdToLocation); - } - - /** - * 1. sorts the the tablet assignments by shard id, starting with the most recent going backwards
- * 2. assigns partitions to each tservers, starting from the beginning, but skipping future dates
- * 3. assigns partitions to each shardid by looking up its tserver's assignments
- * 4. returns the {@code shard id -> partition} map - *

- * e.g.,
- * 1. sorted assignments
- * 2. tserver map
- * 3. shard map: {@code future->tserver7 *no change* future->2 shard4->tserver2 tserver2->0 shard4->0 - * shard3->tserver3 tserver3->1 shard3->1 shard2->tserver2 *no change* shard2->0 shard1->tserver7 tserver7->2 shard1->2} - * - * @param shardIdToLocations - * the map of shard ids and their location - * @return shardId to - */ - private HashMap assignPartitionsForEachShard(List sortedShardIds, Map shardIdToLocations) { - int totalNumUniqueTServers = calculateNumberOfUniqueTservers(shardIdToLocations); - - HashMap partitionsByTServer = getTServerAssignments(totalNumUniqueTServers, sortedShardIds, shardIdToLocations); - HashMap partitionsByShardId = getShardIdAssignments(shardIdToLocations, partitionsByTServer); - - if (log.isDebugEnabled()) - log.debug("Number of shardIds assigned: " + partitionsByShardId.size()); - - return partitionsByShardId; - } - - private int calculateNumberOfUniqueTservers(Map shardIdToLocations) { - - int totalNumUniqueTServers = new HashSet(shardIdToLocations.values()).size(); - if (log.isDebugEnabled()) - log.debug("Total TServers involved: " + totalNumUniqueTServers); - return totalNumUniqueTServers; - } - - private HashMap getTServerAssignments(int totalNumTServers, List sortedShardIds, Map shardIdsToTservers) { - HashMap partitionsByTServer = new HashMap<>(totalNumTServers); - int nextAvailableSlot = 0; - boolean alreadySkippedFutureShards = false; - for (Text shard : sortedShardIds) { - if (alreadySkippedFutureShards || !isFutureShard(shard)) { // short circuiting for performance - alreadySkippedFutureShards = true; - String location = shardIdsToTservers.get(shard); - Integer assignedPartition = partitionsByTServer.get(location); - if (null == assignedPartition) { - assignedPartition = nextAvailableSlot; - partitionsByTServer.put(location, assignedPartition); - nextAvailableSlot++; - } - if (partitionsByTServer.size() == totalNumTServers) { - // all the tservers have been assigned partitions, so we can stop - return partitionsByTServer; - } - } - } - return partitionsByTServer; - } - - private static boolean isFutureShard(Text shardId) { - String shardIdStr = shardId.toString().intern(); - if (shardIdStr.length() < 8) { - return true; - } - return shardIdStr.substring(0, 8).compareTo(today) > 0; - } - - private static String formatDay(int numDaysBack) { - return DateHelper.format(now - (DateUtils.MILLIS_PER_DAY * numDaysBack)); - } - - private HashMap getShardIdAssignments(Map shardIdsToTservers, HashMap partitionsByTServer) { - HashMap partitionsByShardId = new HashMap<>(); - for (Map.Entry entry : shardIdsToTservers.entrySet()) { - partitionsByShardId.put(entry.getKey(), partitionsByTServer.get(entry.getValue())); - } - return partitionsByShardId; - } - @Override public void configureWithPrefix(String prefix) {/* no op */} @@ -248,6 +110,9 @@ public Configuration getConf() { public void setConf(Configuration conf) { this.conf = conf; shardIdFactory = new ShardIdFactory(conf); + missingShardStrategy = conf.get(MISSING_SHARD_STRATEGY_PROP, "hash"); + splitsCache = SplitsCache.getInstance(conf); + defineOffsetsForTables(conf); } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java index 19d44c9e1f7..9de6591deb5 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java @@ -3,8 +3,6 @@ import java.io.IOException; import java.text.DecimalFormat; import java.util.Arrays; -import java.util.Collections; -import java.util.List; import org.apache.accumulo.core.data.Value; import org.apache.hadoop.conf.Configuration; @@ -16,7 +14,7 @@ import org.apache.log4j.Logger; import datawave.ingest.mapreduce.job.BulkIngestKey; -import datawave.ingest.mapreduce.job.SplitsFile; +import datawave.ingest.mapreduce.job.SplitsCache; /** * Range partitioner that uses a split file with the format: {@code tableNamesplitPointtabletLocation} @@ -36,14 +34,14 @@ public class MultiTableRangePartitioner extends Partitioner private DecimalFormat formatter = new DecimalFormat("000"); private Configuration conf; private PartitionLimiter partitionLimiter; - protected Object semaphore = new Object(); + private SplitsCache splitsCache; private void readCacheFilesIfNecessary() { if (cacheFilesRead) { return; } - synchronized (semaphore) { + synchronized (this) { if (cacheFilesRead) { return; } @@ -61,7 +59,7 @@ private void readCacheFilesIfNecessary() { } try { - if (SplitsFile.getSplits(conf).isEmpty()) { + if (!splitsCache.hasSplits()) { log.error("Non-sharded splits by table cannot be empty. If this is a development system, please create at least one split in one of the non-sharded tables (see bin/ingest/seed_index_splits.sh)."); throw new IOException("splits by table cannot be empty"); } @@ -80,19 +78,12 @@ public int getPartition(BulkIngestKey key, Value value, int numPartitions) { String tableName = key.getTableName().toString(); - List cutPointArray = null; - try { - cutPointArray = SplitsFile.getSplits(conf, tableName); - } catch (IOException e) { - log.error("Failed to read splits in MultiTableRangePartitioner for " + tableName); - } - if (null == cutPointArray) { + int index = splitsCache.getExactIndex(tableName, holder); + if (index < 0) { return (tableName.hashCode() & Integer.MAX_VALUE) % numPartitions; } - key.getKey().getRow(holder); - int index = Collections.binarySearch(cutPointArray, holder); - index = calculateIndex(index, numPartitions, tableName, cutPointArray.size()); + index = calculateIndex(index, numPartitions, tableName, 0); index = partitionLimiter.limit(numPartitions, index); TaskInputOutputContext c = context; @@ -118,6 +109,7 @@ public static void setContext(TaskInputOutputContext context) { public void setConf(Configuration conf) { this.conf = conf; partitionLimiter = new PartitionLimiter(conf); + splitsCache = SplitsCache.getInstance(conf); if (partitionLimiter.getNumPartitions() == 0) { partitionLimiter.setMaxPartitions(Integer.MAX_VALUE); } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitioner.java index 14560705b45..7d3cabda861 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitioner.java @@ -1,7 +1,5 @@ package datawave.ingest.mapreduce.partition; -import java.io.IOException; -import java.util.HashMap; import java.util.Map; import org.apache.accumulo.core.data.Value; @@ -13,7 +11,7 @@ import org.apache.log4j.Logger; import datawave.ingest.mapreduce.job.BulkIngestKey; -import datawave.ingest.mapreduce.job.SplitsFile; +import datawave.ingest.mapreduce.job.SplitsCache; /** * The TabletLocationHashPartitioner will generate partitions for the shard table using the hashCode method on the tserver location string @@ -23,6 +21,7 @@ public class TabletLocationHashPartitioner extends Partitioner> shardHashes; + private SplitsCache splitsCache; /** * Given a map of shard IDs to tablet server locations, this method determines a partition for a given key's shard ID. The goal is that we want to ensure @@ -35,18 +34,14 @@ public class TabletLocationHashPartitioner extends Partitioner shardHash = getShardHashes(key.getTableName().toString()); - Integer hash = shardHash.get(shardId); - if (hash != null) { - return (hash & Integer.MAX_VALUE) % numReduceTasks; - } else { - return (shardId.hashCode() & Integer.MAX_VALUE) % numReduceTasks; - } - } catch (IOException e) { - throw new RuntimeException(e); + public int getPartition(BulkIngestKey key, Value value, int numReduceTasks) { + Text shardId = key.getKey().getRow(); + String location = splitsCache.getExactLocation(key.getTableName().toString(), shardId, () -> null); + if (location != null) { + int hash = location.hashCode(); + return (hash & Integer.MAX_VALUE) % numReduceTasks; + } else { + return (shardId.hashCode() & Integer.MAX_VALUE) % numReduceTasks; } } @@ -58,35 +53,7 @@ public Configuration getConf() { @Override public void setConf(Configuration conf) { this.conf = conf; - } - - /** - * hashCode of the tserver name Read in the sequence file (that was created at job startup) that contains a list of shard IDs and the corresponding tablet - * server to which that shard is assigned. The hash is a simple hashCode of the location string. - * - * @param tableName - * the table name - * @throws IOException - * for issues with read or write - * @return a mapping of the shard hashes - */ - private Map getShardHashes(String tableName) throws IOException { - if (this.shardHashes == null) { - this.shardHashes = new HashMap<>(); - } - - if (null == this.shardHashes.get(tableName)) { - Map hashedForTable = new HashMap<>(); - - for (Map.Entry entry : SplitsFile.getSplitsAndLocations(conf, tableName).entrySet()) { - - hashedForTable.put(entry.getKey(), entry.getValue().toString().hashCode()); - } - - this.shardHashes.put(tableName, hashedForTable); - } - - return this.shardHashes.get(tableName); + this.splitsCache = SplitsCache.getInstance(conf); } @Override diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitioner.java deleted file mode 100644 index c2a327bde61..00000000000 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitioner.java +++ /dev/null @@ -1,139 +0,0 @@ -package datawave.ingest.mapreduce.partition; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Map; -import java.util.SortedSet; -import java.util.TreeSet; - -import org.apache.accumulo.core.data.Value; -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.Partitioner; -import org.apache.log4j.Logger; - -import datawave.ingest.mapreduce.job.BulkIngestKey; -import datawave.ingest.mapreduce.job.SplitsFile; - -/** - * The TabletLocationHashPartitioner will look up the shard tablet servers, sort those tablet servers and then partition based on the index into that sorted - * list. - */ -public class TabletLocationNamePartitioner extends Partitioner implements Configurable, DelegatePartitioner { - private static final Logger log = Logger.getLogger(TabletLocationNamePartitioner.class); - private Configuration conf; - private Map> shardLocations; - - @Override - public synchronized int getPartition(BulkIngestKey key, Value value, int numReduceTasks) { - - try { - return getLocationPartition(key.getKey().getRow(), getShardLocations(key.getTableName().toString()), numReduceTasks); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public Configuration getConf() { - return conf; - } - - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } - - // this is used by two schemes - /** - * Given a map of shard IDs to tablet server locations, this method determines a partition for a given key's shard ID. The goal is that we want to ensure - * that all shard IDs served by a given tablet server get sent to the same reducer. To do this, we look up where the shard ID is supposed to be stored and - * use a hash of that (modded by the number of reduces) to come up with the final allocation. This mapping needs to be computed at job startup and, so long - * as no migration goes on during a job, will produce a single map file per tablet server. Note that it is also possible that we receive data for a day that - * hasn't been loaded yet. In that case, we'll just hash the shard ID and send data to that reducer. This will spread out the data for a given day, but the - * map files produced for it will not belong to any given tablet server. So, in the worst case, we have other older data when is already assigned to a - * tablet server and new data which is not. In this case, we'd end up sending two map files to each tablet server. Of course, if tablets get moved around - * between when the job starts and the map files are loaded, then we may end up sending multiple map files to each tablet server. - * - * @param shardId - * the shard id - * @param shardHash - * the shard hash map - * @param numReduceTasks - * the number of reducer tasks - * @return the partition - */ - private int getLocationPartition(Text shardId, Map shardHash, int numReduceTasks) { - int partition = 0; - Integer hash = shardHash.get(shardId); - if (hash != null) { - partition = (hash & Integer.MAX_VALUE) % numReduceTasks; - } else { - partition = (shardId.hashCode() & Integer.MAX_VALUE) % numReduceTasks; - } - return partition; - } - - /** - * Sorts the tserver locations for all the shard ids. Each shardId is assigned a number which corresponds to where its tserver is found in that sorted list. - * sort by most recent shard id, tserver - * - * suggested change: sort by most recent day's locations first, not by location id - * - * Read in the sequence file (that was created at job startup) that contains a list of shard IDs and the corresponding tablet server to which that shard is - * assigned. The location is a number generated by adding the tuples of the location ip address and port. - * - * @param tableName - * the table name - * @return a map of the shard locations - * @throws IOException - * for read write issues - */ - private Map getShardLocations(String tableName) throws IOException { - if (this.shardLocations == null) { - this.shardLocations = new HashMap<>(); - } - - if (null == this.shardLocations.get(tableName)) { - Map shards = SplitsFile.getSplitsAndLocations(conf, tableName); - - // now sort the locations - SortedSet locations = new TreeSet<>(); - locations.addAll(shards.values()); - - ArrayList locList = new ArrayList<>(locations); - HashMap localShardLocations = new HashMap<>(); - for (Map.Entry entry : shards.entrySet()) { - localShardLocations.put(entry.getKey(), locList.indexOf(entry.getValue())); - } - - this.shardLocations.put(tableName, localShardLocations); - } - - return this.shardLocations.get(tableName); - } - - @Override - public void configureWithPrefix(String prefix) {/* no op */} - - @Override - public int getNumPartitions() { - return Integer.MAX_VALUE; - } - - @Override - public void initializeJob(Job job) {} - - @Override - public boolean needSplits() { - return true; - } - - @Override - public boolean needSplitLocations() { - return true; - } -} diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/util/GenerateSplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/util/GenerateSplitsFile.java index 9f84c591e55..df4b0d95a36 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/util/GenerateSplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/util/GenerateSplitsFile.java @@ -8,6 +8,7 @@ import org.apache.commons.cli.ParseException; import org.apache.hadoop.conf.Configuration; +import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; /** @@ -40,8 +41,8 @@ public static void main(String[] args) { System.exit(1); } if (cl.hasOption("sp")) { - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, cl.getOptionValue("sp")); - log.info("Set " + TableSplitsCache.SPLITS_CACHE_DIR + " to " + cl.getOptionValue("sp")); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, cl.getOptionValue("sp")); + log.info("Set " + SplitsConstants.SPLITS_CACHE_DIR + " to " + cl.getOptionValue("sp")); } if (cl.hasOption("cs")) { configSuffix = cl.getOptionValue("cs"); diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java index 64db2a6483d..bc569f6fc3b 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java @@ -373,14 +373,6 @@ protected int getSeqFileBlockSize() { return 1; } - @Override - protected Map getShardLocations(String tableName) throws IOException { - Map locations = new HashMap<>(); - locations.put(new Text("20100101_1"), "server1"); - locations.put(new Text("20100101_2"), "server2"); - return locations; - } - @Override protected SizeTrackingWriter openWriter(String filename, AccumuloConfiguration tableConf) { filenames.add(filename); diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java index bd9898ca208..369561ca215 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java @@ -1,10 +1,5 @@ package datawave.ingest.mapreduce.job; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; - import java.io.BufferedOutputStream; import java.io.File; import java.io.IOException; @@ -22,12 +17,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import datawave.ingest.mapreduce.handler.shard.ShardIdFactory; import datawave.ingest.mapreduce.handler.shard.ShardedDataTypeHandler; @@ -39,20 +33,21 @@ public class SplitsFileTest { private static final String TABLE_NAME = "unitTestTable"; private static final int SHARDS_PER_DAY = 10; private static Configuration conf; + private SplitsFile uut; - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir + private java.nio.file.Path tempDir; - @BeforeClass + @BeforeAll public static void defineShardLocationsFile() throws IOException { conf = new Configuration(); conf.setInt(ShardIdFactory.NUM_SHARDS, SHARDS_PER_DAY); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, TableName.SHARD); } - @Before - public void clearCache() { - TableSplitsCache.getCurrentCache(conf).clear(); + @BeforeEach + public void setUp() throws IOException { + uut = new SplitsFile(conf); } private void createSplitsFile(Map splits, Configuration conf, int expectedNumRows, String tableName) throws IOException { @@ -64,13 +59,13 @@ private void createSplitsFile(Map splits, Configuration conf, int e long actualCount = splits.size(); Map SplitsFiles = new HashMap<>(); // SplitsFile.addToConf(conf, SplitsFiles); - Assert.assertEquals("IngestJob#writeSplitsFile failed to create the expected number of rows", expectedNumRows, actualCount); + Assertions.assertEquals(expectedNumRows, actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); // return file; } private void writeBaseSplitsFile(Map locations, Configuration conf, String tableName) throws IOException { - File tmpBaseSplitDir = temporaryFolder.newFolder(); + File tmpBaseSplitDir = tempDir.toFile(); String splitsFile = TableSplitsCache.DEFAULT_SPLITS_CACHE_FILE; Path splitsPath = new Path(tmpBaseSplitDir.getAbsolutePath() + "/" + splitsFile); @@ -82,24 +77,24 @@ private void writeBaseSplitsFile(Map locations, Configuration conf, out.println(tableName + '\t' + new String(Base64.encodeBase64(e.getKey().toString().getBytes())) + '\t' + e.getValue()); } } - Assert.assertTrue(fs.exists(splitsPath)); + Assertions.assertTrue(fs.exists(splitsPath)); - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, tmpBaseSplitDir.getAbsolutePath()); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, tmpBaseSplitDir.getAbsolutePath()); } private FileSystem setWorkingDirectory(Configuration conf) throws IOException { FileSystem fs = FileSystem.getLocal(conf); - File tempWorkDir = temporaryFolder.newFolder(); + File tempWorkDir = tempDir.toFile(); fs.setWorkingDirectory(new Path(tempWorkDir.toString())); conf.set(SplitsFile.SPLIT_WORK_DIR, tempWorkDir.toString()); return fs; } - @Test(expected = IOException.class) + @Test public void testGetAllSplitsFilesWithoutPath() throws Exception { Configuration conf = new Configuration(); - File tempWorkDir = temporaryFolder.newFolder(); + File tempWorkDir = tempDir.toFile(); conf.set(FileSystem.FS_DEFAULT_NAME_KEY, tempWorkDir.toURI().toString()); FileSystem fs = FileSystem.get(tempWorkDir.toURI(), conf); fs.setWorkingDirectory(new Path(tempWorkDir.toString())); @@ -109,8 +104,8 @@ public void testGetAllSplitsFilesWithoutPath() throws Exception { conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "shard_ingest_unit_test_table_1,shard_ingest_unit_test_table_2,shard_ingest_unit_test_table_3"); String[] tableNames = new String[] {TABLE_NAME}; - SplitsFile.setupFile(Job.getInstance(conf), conf); - SplitsFile.getSplitsAndLocations(conf, TABLE_NAME); + uut.setupJob(Job.getInstance(conf)); + uut.getSplitsAndLocations(conf, TABLE_NAME); } @Test @@ -118,72 +113,70 @@ public void testSingleDaySplitsCreated_AndValid() throws Exception { String tableName = "validSplits"; SortedMap splits = createDistributedLocations(tableName); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, all should be good, none of these should error - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); - SplitsFile.validateShardIdLocations(conf, tableName, 1, locations); - SplitsFile.validateShardIdLocations(conf, tableName, 2, locations); + uut.validateShardIdLocations(conf, tableName, 0, locations); + uut.validateShardIdLocations(conf, tableName, 1, locations); + uut.validateShardIdLocations(conf, tableName, 2, locations); } - @Test(expected = IllegalStateException.class) + @Test public void testMissingAllOfTodaysSplits() throws Exception { String tableName = "missingTodaysSplits"; SortedMap splits = simulateMissingSplitsForDay(0, tableName); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be invalid, which makes the rest bad too - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); - // shouldn't make it here - fail(); + Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); } - @Test(expected = IllegalStateException.class) + @Test public void testUnbalancedTodaysSplits() throws Exception { String tableName = "unbalancedTodaysSplits"; SortedMap splits = simulateUnbalancedSplitsForDay(0, tableName); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be invalid, which makes the rest bad too - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); + Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); } - @Test(expected = IllegalStateException.class) + @Test public void testMissingAllOfYesterdaysSplits() throws Exception { String tableName = "missingYesterdaysSplits"; SortedMap splits = simulateMissingSplitsForDay(1, tableName); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); - assertThat(splits.size(), is(equalTo(locations.size()))); + Map locations = uut.getSplitsAndLocations(conf, tableName); + Assertions.assertEquals(splits.size(), locations.size()); // three days of splits, today should be valid // yesterday and all other days invalid - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); + uut.validateShardIdLocations(conf, tableName, 0, locations); // this should cause the exception - SplitsFile.validateShardIdLocations(conf, tableName, 1, locations); + Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 1, locations)); } - @Test(expected = IllegalStateException.class) + @Test public void testUnbalancedYesterdaysSplits() throws Exception { String tableName = "unbalancedYesterdaysSplits"; SortedMap splits = simulateUnbalancedSplitsForDay(1, tableName); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be valid // yesterday and all other days invalid - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); + uut.validateShardIdLocations(conf, tableName, 0, locations); // this should cause the exception - SplitsFile.validateShardIdLocations(conf, tableName, 1, locations); + Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 1, locations)); } - @Test(expected = IllegalStateException.class) + @Test public void testUnbalancedMaxMoreThanConfigured() throws Exception { String tableName = "unbalancedMoreSplitsThenMaxPer"; SortedMap splits = simulateMultipleShardsPerTServer(tableName, 3); conf.setInt(SplitsFile.MAX_SHARDS_PER_TSERVER, 2); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); + Map locations = uut.getSplitsAndLocations(conf, tableName); // this should cause the exception - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); + Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); } @Test @@ -193,9 +186,9 @@ public void testUnbalancedButNotMoreThanConfigured() throws Exception { conf.setInt(SplitsFile.MAX_SHARDS_PER_TSERVER, 3); createSplitsFile(splits, conf, splits.size(), tableName); - Map locations = SplitsFile.getSplitsAndLocations(conf, tableName); + Map locations = uut.getSplitsAndLocations(conf, tableName); // this should NOT cause an exception - SplitsFile.validateShardIdLocations(conf, tableName, 0, locations); + uut.validateShardIdLocations(conf, tableName, 0, locations); } private SortedMap simulateUnbalancedSplitsForDay(int daysAgo, String tableName) throws IOException { diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/TableSplitsCacheTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/TableSplitsCacheTest.java index d9672d6b28c..c65f4fad858 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/TableSplitsCacheTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/TableSplitsCacheTest.java @@ -168,11 +168,11 @@ public void setup() throws Exception { public void setSplitsCacheDir() { URL url = TableSplitsCacheTest.class.getResource("/datawave/ingest/mapreduce/job/all-splits.txt"); Assert.assertNotNull("TableSplitsCacheTest#setup failed to load test cache directory.", url); - mockConfiguration.put(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf(Path.SEPARATOR))); + mockConfiguration.put(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf(Path.SEPARATOR))); } public void setSplitsCacheDir(String splitsCacheDir) { - mockConfiguration.put(TableSplitsCache.SPLITS_CACHE_DIR, splitsCacheDir); + mockConfiguration.put(SplitsConstants.SPLITS_CACHE_DIR, splitsCacheDir); } @After diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java index d0bf03cdb5b..c27ec0776f1 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java @@ -27,6 +27,7 @@ import org.junit.Test; import datawave.ingest.mapreduce.job.BulkIngestKey; +import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; @@ -81,8 +82,8 @@ public void testEmptySplitsThrowsException() throws IOException, URISyntaxExcept String filename = "full_empty_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); getPartition("23432"); } @@ -91,8 +92,8 @@ public void testProblemGettingLocalCacheFiles() throws IOException, URISyntaxExc String filename = "full_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); MultiTableRangePartitioner.setContext(new MapContextImpl(configuration, new TaskAttemptID(), null, null, null, null, null) { @Override @@ -128,8 +129,8 @@ public void testAllDataForOneSplitGoesToOnePartitioner() { String filename = "full_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); int numPartitions = 581; MultiTableRRRangePartitioner partitioner = new MultiTableRRRangePartitioner(); @@ -166,8 +167,8 @@ public void testPartitionerSpaceIsValid() { String filename = "full_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); int numPartitions = 581; MultiTableRRRangePartitioner partitioner = new MultiTableRRRangePartitioner(); @@ -272,8 +273,8 @@ private MultiTableRRRangePartitioner createPartitionerFromSplits() { String filename = "full_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); MultiTableRRRangePartitioner partitioner = new MultiTableRRRangePartitioner(); partitioner.setConf(configuration); return partitioner; diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java index d1187b1d65f..47f5196b17f 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java @@ -18,6 +18,7 @@ import org.junit.Test; import datawave.ingest.mapreduce.job.BulkIngestKey; +import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableConfigurationUtil; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; @@ -41,8 +42,8 @@ public void testGoodSplitsFile() throws IOException, URISyntaxException { String filename = "trimmed_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); Assert.assertEquals(5, getPartition()); } @@ -51,8 +52,8 @@ public void testEmptySplitsThrowsException() throws IOException, URISyntaxExcept String filename = "trimmed_empty_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); - configuration.set(TableSplitsCache.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); - configuration.set(TableSplitsCache.SPLITS_CACHE_FILE, filename); + configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); + configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); getPartition(); } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/SplitBasedHashPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/SplitBasedHashPartitionerTest.java index aa5226a3e43..f8814adbe2c 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/SplitBasedHashPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/SplitBasedHashPartitionerTest.java @@ -15,6 +15,7 @@ import org.junit.Test; import datawave.ingest.mapreduce.job.BulkIngestKey; +import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; public class SplitBasedHashPartitionerTest { @@ -27,8 +28,8 @@ public void before() { final String testFilePath = SplitBasedHashPartitionerTest.class.getClassLoader().getResource(TEST_FILE_LOCATION).getPath(); conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, testFilePath.substring(0, testFilePath.lastIndexOf('/'))); - conf.set(TableSplitsCache.SPLITS_CACHE_FILE, "full_splits.txt"); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, testFilePath.substring(0, testFilePath.lastIndexOf('/'))); + conf.set(SplitsConstants.SPLITS_CACHE_FILE, "full_splits.txt"); TableSplitsCache.getCurrentCache(conf).clear(); diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitionerTest.java deleted file mode 100644 index db4190bfb3f..00000000000 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationNamePartitionerTest.java +++ /dev/null @@ -1,82 +0,0 @@ -package datawave.ingest.mapreduce.partition; - -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.net.URL; -import java.util.HashMap; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import datawave.ingest.mapreduce.job.BulkIngestKey; -import datawave.ingest.mapreduce.job.TableSplitsCache; - -public class TabletLocationNamePartitionerTest { - Configuration conf = new Configuration(); - TabletLocationNamePartitioner partitioner = null; - - @Before - public void setUp() { - conf = new Configuration(); - conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); - partitioner = new TabletLocationNamePartitioner(); - partitioner.setConf(conf); - } - - @After - public void tearDown() { - conf.clear(); - conf = null; - partitioner = null; - } - - @Test - public void testSequentialLocationScheme() throws Exception { - Map shardedTableMapFiles = new HashMap<>(); - // setup the location partition sheme - URL file = getClass().getResource("/datawave/ingest/mapreduce/partition/_shards.lst"); - shardedTableMapFiles.put("shard", new Path(file.toURI().toString())); - - // now read in a list of shards and display the distribution - file = getClass().getResource("/datawave/ingest/mapreduce/partition/shards.list"); - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, file.getPath().substring(0, file.getPath().lastIndexOf('/'))); - conf.set(TableSplitsCache.SPLITS_CACHE_FILE, "shards_n_locs.list"); - BufferedReader reader = new BufferedReader(new InputStreamReader(file.openStream())); - String line = reader.readLine(); - int[] partitions = new int[912]; - while (line != null) { - String shardId = line.trim(); - Key key = new Key(shardId); - Value value = new Value(); - int partition = partitioner.getPartition(new BulkIngestKey(new Text("shard"), key), value, 912); - partitions[partition]++; - line = reader.readLine(); - } - boolean errored = false; - int[] distribution = new int[8]; - int[] expected = new int[] {77, 207, 266, 196, 112, 42, 9, 3}; - for (int i = 0; i < 912; i++) { - distribution[partitions[i]]++; - } - for (int i = 0; i < 8; i++) { - try { - Assert.assertEquals("Unexpected distribution: ", expected[i], distribution[i]); - } catch (Throwable e) { - System.err.println(e.getMessage()); - errored = true; - } - } - if (errored) { - Assert.fail("Failed to get expected distribution. See console for unexpected entries"); - } - - } -} diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TestShardGenerator.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TestShardGenerator.java index cfcccb78ee8..51af9cb8c38 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TestShardGenerator.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TestShardGenerator.java @@ -17,6 +17,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.time.DateHelper; @@ -43,7 +44,7 @@ public TestShardGenerator(Configuration conf, File tmpDir, int numDays, int shar registerSomeTServers(); Map locations = simulateTabletAssignments(tableNames); // adding sorting here since it now happens when we generate the splits file - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, tmpDir.getAbsolutePath()); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, tmpDir.getAbsolutePath()); TableSplitsCache.getCurrentCache(conf).clear(); Map sortedLocations = TableSplitsCache.getCurrentCache(conf).reverseSortByShardIds(locations); String tmpDirectory = tmpDir + "/"; @@ -69,7 +70,7 @@ public TestShardGenerator(Configuration conf, File tmpDir, Map loca // constructor that takes a created list of locations String tmpDirectory = tmpDir + "/"; Path splitsPath = new Path(tmpDir.getAbsolutePath() + "/all-splits.txt"); - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, tmpDir.getAbsolutePath()); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, tmpDir.getAbsolutePath()); Map sortedLocations = TableSplitsCache.getCurrentCache(conf).reverseSortByShardIds(locations); try (PrintStream out = new PrintStream(new BufferedOutputStream(fs.create(splitsPath)))) { @@ -80,7 +81,7 @@ public TestShardGenerator(Configuration conf, File tmpDir, Map loca } } } - conf.set(TableSplitsCache.SPLITS_CACHE_DIR, tmpDir.getAbsolutePath()); + conf.set(SplitsConstants.SPLITS_CACHE_DIR, tmpDir.getAbsolutePath()); } // create splits for all the shards from today back NUM_DAYS, From 55adf22054bdda9685b9033c6a92e7390447f1d8 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Wed, 18 Jun 2025 21:44:41 +0000 Subject: [PATCH 02/26] Check for committer initialized before singleton initialize --- .../mapreduce/job/MultiRFileOutputFormatter.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index dc5a7e5cf5b..b3cd478276a 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -433,14 +433,16 @@ protected void setTableIdsAndConfigs() throws IOException { } // Creating because super class does not allow overridding - private SafeFileOutputCommitter _committer = null; + private volatile SafeFileOutputCommitter _committer = null; @Override public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException { - synchronized (this) { - if (_committer == null) { - Path output = getOutputPath(context); - _committer = new SafeFileOutputCommitter(output, context); + if (_committer == null) { + synchronized (this) { + if (_committer == null) { + Path output = getOutputPath(context); + _committer = new SafeFileOutputCommitter(output, context); + } } } return _committer; From 0f07837b19deaefcd3b4e440442c46fdf7ee0b35 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Wed, 18 Jun 2025 21:45:45 +0000 Subject: [PATCH 03/26] Refactor SplitsCache implementers to require default constructor and use init method --- .../ingest/mapreduce/job/SplitsCache.java | 2 ++ .../mapreduce/job/SplitsCacheFactory.java | 1 + .../ingest/mapreduce/job/SplitsFile.java | 19 +++++++++++-------- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java index 8c6235c1269..03eb80b6587 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java @@ -12,6 +12,8 @@ static SplitsCache getInstance(final Configuration conf) { return SplitsCacheFactory.getSplitsCache(conf); } + void init(Configuration conf); + void setupJob(final Job job) throws IOException; boolean hasSplits(); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java index 854855ef844..7ccd97c5b70 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java @@ -19,6 +19,7 @@ public static SplitsCache getSplitsCache(final Configuration conf) { // noinspection unchecked final Class clazz = (Class) Class.forName(splitsCacheImpl); INSTANCE = clazz.getDeclaredConstructor().newInstance(); + INSTANCE.init(conf); } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException ex) { throw new RuntimeException(ex); } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java index 71a9ba4608f..27bbb59bc65 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java @@ -41,17 +41,20 @@ public class SplitsFile implements SplitsCache { public static final String DIST_CACHE_LABEL = "splitsFile"; private static final int NUMBER_MILLIS_BACK = 0; + private static final String TODAY = DateHelper.format(NUMBER_MILLIS_BACK); - private final TableSplitsCache instance; - private final Configuration conf; private final ConcurrentHashMap> shardPartitionsByTable; - private final String today; + private TableSplitsCache instance; + private Configuration conf; - public SplitsFile(Configuration conf) throws IOException { - this.conf = conf; + public SplitsFile() { this.shardPartitionsByTable = new ConcurrentHashMap<>(); - this.today = DateHelper.format(NUMBER_MILLIS_BACK); - instance = TableSplitsCache.getCurrentCache(conf); + } + + @Override + public void init(Configuration conf) { + this.conf = conf; + this.instance = TableSplitsCache.getCurrentCache(conf); } @Override @@ -309,7 +312,7 @@ private boolean isFutureShard(Text shardId) { if (shardIdStr.length() < 8) { return true; } - return shardIdStr.substring(0, 8).compareTo(today) > 0; + return shardIdStr.substring(0, 8).compareTo(TODAY) > 0; } private HashMap getShardIdAssignments(Map shardIdsToTservers, HashMap partitionsByTServer) { From cc28943255c71b93e4cb121e6aff3f1dea5a3d2c Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 15:04:33 +0000 Subject: [PATCH 04/26] SplitsFileTest fixes for test failures --- .../ingest/mapreduce/job/SplitsFile.java | 6 +-- .../ingest/mapreduce/job/SplitsFileTest.java | 39 +++++++++++++------ 2 files changed, 30 insertions(+), 15 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java index 27bbb59bc65..43911952119 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java @@ -88,7 +88,7 @@ public void setupJob(Job job) throws IOException { } } catch (Exception e) { - log.error("Unable to use splits file because " + e.getMessage()); + log.error("Unable to use splits file because {}", e.getMessage(), e); throw new IOException(e); } } @@ -174,11 +174,9 @@ public int getNearestPartition(String tableName, Text shardId) { public void validate(Configuration conf) throws IOException { TableSplitsCache cache = TableSplitsCache.getCurrentCache(conf); int daysToVerify = conf.getInt(SHARDS_BALANCED_DAYS_TO_VERIFY, 2) - 1; - for (String table : conf.getStrings(ShardedDataTypeHandler.SHARDED_TNAMES)) { validateShardIdLocations(conf, table, daysToVerify, cache.getSplitsAndLocationByTable(table)); } - } public void validateShardIdLocations(Configuration conf, String tableName, int daysToVerify, Map shardIdToLocation) { @@ -389,7 +387,7 @@ private boolean shardsAreBalanced(Map locations, String datePrefix, // if shard is assigned to more tservers than allowed, then the shards are not balanced if (cnt.intValue() > maxShardsPerTserver) { - log.warn(cnt.toInteger() + " Shards for " + datePrefix + " assigned to tablet " + value); + log.warn("{} Shards for {} assigned to tablet {}", cnt.toInteger(), datePrefix, value); dateIsBalanced = false; } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java index 369561ca215..15feb259473 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java @@ -47,7 +47,10 @@ public static void defineShardLocationsFile() throws IOException { @BeforeEach public void setUp() throws IOException { - uut = new SplitsFile(conf); + // Wipe the underlying cache that the SplitsFile uses + // state is saved that will impact multiple test iterations + TableSplitsCache.clear(); + uut = new SplitsFile(); } private void createSplitsFile(Map splits, Configuration conf, int expectedNumRows, String tableName) throws IOException { @@ -57,11 +60,7 @@ private void createSplitsFile(Map splits, Configuration conf, int e writeBaseSplitsFile(splits, conf, tableName); long actualCount = splits.size(); - Map SplitsFiles = new HashMap<>(); - // SplitsFile.addToConf(conf, SplitsFiles); Assertions.assertEquals(expectedNumRows, actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); - - // return file; } private void writeBaseSplitsFile(Map locations, Configuration conf, String tableName) throws IOException { @@ -100,12 +99,11 @@ public void testGetAllSplitsFilesWithoutPath() throws Exception { fs.setWorkingDirectory(new Path(tempWorkDir.toString())); Path workDir = fs.makeQualified(new Path("work")); conf.set(SplitsFile.SPLIT_WORK_DIR, workDir.toString()); - conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "shard_ingest_unit_test_table_1,shard_ingest_unit_test_table_2,shard_ingest_unit_test_table_3"); - String[] tableNames = new String[] {TABLE_NAME}; - uut.setupJob(Job.getInstance(conf)); - uut.getSplitsAndLocations(conf, TABLE_NAME); + uut.init(conf); + + Assertions.assertThrows(IOException.class, () -> uut.setupJob(Job.getInstance(conf))); } @Test @@ -113,6 +111,9 @@ public void testSingleDaySplitsCreated_AndValid() throws Exception { String tableName = "validSplits"; SortedMap splits = createDistributedLocations(tableName); createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, all should be good, none of these should error uut.validateShardIdLocations(conf, tableName, 0, locations); @@ -125,6 +126,9 @@ public void testMissingAllOfTodaysSplits() throws Exception { String tableName = "missingTodaysSplits"; SortedMap splits = simulateMissingSplitsForDay(0, tableName); createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be invalid, which makes the rest bad too Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); @@ -135,6 +139,9 @@ public void testUnbalancedTodaysSplits() throws Exception { String tableName = "unbalancedTodaysSplits"; SortedMap splits = simulateUnbalancedSplitsForDay(0, tableName); createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be invalid, which makes the rest bad too Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); @@ -145,6 +152,9 @@ public void testMissingAllOfYesterdaysSplits() throws Exception { String tableName = "missingYesterdaysSplits"; SortedMap splits = simulateMissingSplitsForDay(1, tableName); createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); Assertions.assertEquals(splits.size(), locations.size()); // three days of splits, today should be valid @@ -159,6 +169,9 @@ public void testUnbalancedYesterdaysSplits() throws Exception { String tableName = "unbalancedYesterdaysSplits"; SortedMap splits = simulateUnbalancedSplitsForDay(1, tableName); createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be valid // yesterday and all other days invalid @@ -172,8 +185,10 @@ public void testUnbalancedMaxMoreThanConfigured() throws Exception { String tableName = "unbalancedMoreSplitsThenMaxPer"; SortedMap splits = simulateMultipleShardsPerTServer(tableName, 3); conf.setInt(SplitsFile.MAX_SHARDS_PER_TSERVER, 2); - createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); // this should cause the exception Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); @@ -184,8 +199,10 @@ public void testUnbalancedButNotMoreThanConfigured() throws Exception { String tableName = "unbalancedNotMoreSplitsThenMaxPer"; SortedMap splits = simulateMultipleShardsPerTServer(tableName, 3); conf.setInt(SplitsFile.MAX_SHARDS_PER_TSERVER, 3); - createSplitsFile(splits, conf, splits.size(), tableName); + + uut.init(conf); + Map locations = uut.getSplitsAndLocations(conf, tableName); // this should NOT cause an exception uut.validateShardIdLocations(conf, tableName, 0, locations); From f0a5da75b2b9ca9db081a7147468280f29f5b633 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 15:09:57 +0000 Subject: [PATCH 05/26] SplitsFile as default implementation setup --- .../ingest/mapreduce/job/SplitsCacheFactory.java | 9 ++++----- .../datawave/ingest/mapreduce/job/SplitsConstants.java | 3 --- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java index 7ccd97c5b70..9916a0eb3b6 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java @@ -1,13 +1,12 @@ package datawave.ingest.mapreduce.job; -import static datawave.ingest.mapreduce.job.SplitsConstants.DEFAULT_SPLITS_CACHE_IMPL; -import static datawave.ingest.mapreduce.job.SplitsConstants.SPLITS_CACHE_IMPL; - import java.lang.reflect.InvocationTargetException; import org.apache.hadoop.conf.Configuration; public class SplitsCacheFactory { + public static final String SPLITS_CACHE_IMPL = "datawave.ingest.splits.cache.impl"; + static volatile SplitsCache INSTANCE; public static SplitsCache getSplitsCache(final Configuration conf) { @@ -15,9 +14,9 @@ public static SplitsCache getSplitsCache(final Configuration conf) { synchronized (SplitsCacheFactory.class) { if (INSTANCE == null) { try { - final String splitsCacheImpl = conf.get(SPLITS_CACHE_IMPL, DEFAULT_SPLITS_CACHE_IMPL); + String splitsCacheImpl = conf.get(SPLITS_CACHE_IMPL); // noinspection unchecked - final Class clazz = (Class) Class.forName(splitsCacheImpl); + Class clazz = splitsCacheImpl != null ? (Class) Class.forName(splitsCacheImpl) : SplitsFile.class; INSTANCE = clazz.getDeclaredConstructor().newInstance(); INSTANCE.init(conf); } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException ex) { diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java index 1e6bc9e5575..fe89dcff99e 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsConstants.java @@ -3,8 +3,5 @@ public class SplitsConstants { public static final String SPLITS_CACHE_DIR = "datawave.ingest.splits.cache.dir"; public static final String SPLITS_CACHE_FILE = "datawave.ingest.splits.cache.file"; - public static final String SPLITS_CACHE_IMPL = "datawave.ingest.splits.cache.impl"; - - public static final String DEFAULT_SPLITS_CACHE_IMPL = ""; public static final String DEFAULT_SPLITS_CACHE_DIR = "/data/splitsCache"; } From 6397aedf0e259eb801528b08d9b73f3d8e81156c Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 17:30:07 +0000 Subject: [PATCH 06/26] MultiTableRangePartitioner fix for holder variable --- .../ingest/mapreduce/partition/MultiTableRangePartitioner.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java index 9de6591deb5..b808e6b16f1 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java @@ -75,8 +75,10 @@ private void readCacheFilesIfNecessary() { @Override public int getPartition(BulkIngestKey key, Value value, int numPartitions) { readCacheFilesIfNecessary(); + holder.clear(); String tableName = key.getTableName().toString(); + key.getKey().getRow(holder); int index = splitsCache.getExactIndex(tableName, holder); if (index < 0) { From 0e9e944e854fe7216ea42ca60972416bf88c2b54 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 18:51:00 +0000 Subject: [PATCH 07/26] Splits size method --- .../datawave/ingest/mapreduce/job/SplitsCache.java | 2 ++ .../datawave/ingest/mapreduce/job/SplitsFile.java | 12 ++++++++++++ .../partition/MultiTableRangePartitioner.java | 7 ++++++- 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java index 03eb80b6587..ab1ff01e178 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java @@ -18,6 +18,8 @@ static SplitsCache getInstance(final Configuration conf) { boolean hasSplits(); + int getSplitsCount(String table); + int getExactIndex(String table, Text key); int getExactPartition(String table, Text key); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java index 43911952119..0e7e7b43e55 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java @@ -107,6 +107,18 @@ public boolean hasSplits() { } } + @Override + public int getSplitsCount(String table) { + int count = 0; + try { + List splits = instance.getSplits(table); + count = splits == null ? -1 : splits.size(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return count; + } + @Override public String getExactLocation(String table, Text key, Supplier defaultFn) { String location; diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java index b808e6b16f1..2227347db90 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java @@ -85,7 +85,12 @@ public int getPartition(BulkIngestKey key, Value value, int numPartitions) { return (tableName.hashCode() & Integer.MAX_VALUE) % numPartitions; } - index = calculateIndex(index, numPartitions, tableName, 0); + int splitSize = splitsCache.getSplitsCount(tableName); + + // Note that cut-point length may be used by derived classes + // even though its not used below + index = calculateIndex(index, numPartitions, tableName, splitSize); + index = partitionLimiter.limit(numPartitions, index); TaskInputOutputContext c = context; From 76e49edbf0befef5d90d596fd3882cd9d03ed04b Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 19:05:10 +0000 Subject: [PATCH 08/26] Specifically look for missing splits and splits that are before split-space --- .../mapreduce/partition/MultiTableRangePartitioner.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java index 2227347db90..bf6eea59f7e 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java @@ -80,12 +80,12 @@ public int getPartition(BulkIngestKey key, Value value, int numPartitions) { String tableName = key.getTableName().toString(); key.getKey().getRow(holder); - int index = splitsCache.getExactIndex(tableName, holder); - if (index < 0) { + int splitSize= splitsCache.getSplitsCount(tableName); + if (splitSize < 0) { return (tableName.hashCode() & Integer.MAX_VALUE) % numPartitions; } - int splitSize = splitsCache.getSplitsCount(tableName); + int index = splitsCache.getExactIndex(tableName, holder); // Note that cut-point length may be used by derived classes // even though its not used below From 36a0afdddec0013f8ae8e4ed18882cf8626fa4f6 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 19:05:22 +0000 Subject: [PATCH 09/26] JUnit5 test update --- .../MultiTableRRRangePartitionerTest.java | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java index c27ec0776f1..51cf07f61ae 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java @@ -22,14 +22,15 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; import datawave.ingest.mapreduce.job.BulkIngestKey; import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class MultiTableRRRangePartitionerTest { @@ -45,7 +46,7 @@ public class MultiTableRRRangePartitionerTest { Configuration configuration; Job mockJob; - @Before + @BeforeEach public void before() throws IOException { mockJob = new Job(); configuration = mockJob.getConfiguration(); @@ -73,21 +74,21 @@ public void testCalculateIndex() { int resultFour = instance.calculateIndex(indexFour, numPartitions, tableName, cutPointArrayLength); assertEquals(result, resultTwo); assertEquals(result, expectedResult); - Assert.assertEquals(4, resultThree); - Assert.assertEquals(0, resultFour); + Assertions.assertEquals(4, resultThree); + Assertions.assertEquals(0, resultFour); } - @Test(expected = RuntimeException.class) + @Test public void testEmptySplitsThrowsException() throws IOException, URISyntaxException { String filename = "full_empty_splits.txt"; URL url = createUrl(filename); mockContextForLocalCacheFile(url); configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); - getPartition("23432"); + Assertions.assertThrows(RuntimeException.class, () -> getPartition("23432")); } - @Test(expected = RuntimeException.class) + @Test public void testProblemGettingLocalCacheFiles() throws IOException, URISyntaxException { String filename = "full_splits.txt"; URL url = createUrl(filename); @@ -102,7 +103,7 @@ public org.apache.hadoop.fs.Path[] getLocalCacheFiles() throws IOException { } }); - getPartition("23432"); + Assertions.assertThrows(RuntimeException.class, () -> getPartition("23432")); } private URL createUrl(String fileName) { @@ -157,8 +158,8 @@ public void testCalculateIndexRangeIsValid() { MultiTableRRRangePartitioner partitioner = new MultiTableRRRangePartitioner(); for (int i = -1 * cutPointArrayLength - 1; i < cutPointArrayLength; i++) { int result = partitioner.calculateIndex(i, numPartitions, "someTableName", cutPointArrayLength); - Assert.assertTrue("i: " + i + " result: " + result, 0 <= result); - Assert.assertTrue("i: " + i + " result: " + result, result < numPartitions); + Assertions.assertTrue(0 <= result, "i: " + i + " result: " + result); + Assertions.assertTrue(result < numPartitions, "i: " + i + " result: " + result); } } @@ -179,15 +180,15 @@ public void testPartitionerSpaceIsValid() { for (int i = 0; i < numSplits; i++) { String rowStr = Character.toString((char) ("a".codePointAt(0) + i)); int result = partitioner.getPartition(getBulkIngestKey(rowStr), new Value(), numPartitions); - Assert.assertTrue("rowStr: " + rowStr + " partition: " + result, numPartitions - numSplits - 1 <= result); - Assert.assertTrue("rowStr: " + rowStr + " partition: " + result, result < numPartitions); + Assertions.assertTrue(numPartitions - numSplits - 1 <= result, "rowStr: " + rowStr + " partition: " + result); + Assertions.assertTrue(result < numPartitions, "rowStr: " + rowStr + " partition: " + result); } // test rows before and after each split for (int i = -1; i < numSplits + 1; i++) { int result = partitioner.getPartition(getBulkIngestKey(Character.toString((char) ("a".codePointAt(0) + i)) + "_"), new Value(), numPartitions); - Assert.assertTrue("i: " + i + " partition: " + result, numPartitions - numSplits - 1 <= result); - Assert.assertTrue("i: " + i + " partition: " + result, result < numPartitions); + Assertions.assertTrue(numPartitions - numSplits - 1 <= result, "i: " + i + " partition: " + result); + Assertions.assertTrue(result < numPartitions, "i: " + i + " partition: " + result); } } @@ -200,12 +201,11 @@ public void testEvenDistributionWithExtraReducers() { // first split is a, last is z countPartitions(numberTimesPartitionSeen, numPartitions, partitioner); - Assert.assertEquals( - "Should have seen a total of 27 different partitions. There is a split for each letter of the alphabet and the null split which is not in the file", - 27, numberTimesPartitionSeen.size()); + Assertions.assertEquals(27, numberTimesPartitionSeen.size(), + "Should have seen a total of 27 different partitions. There is a split for each letter of the alphabet and the null split which is not in the file"); for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { - Assert.assertEquals("We haven't used the partition space so they should all be even, but partition " + partitionAndNumSeen.getKey().intValue() - + " did not see 2.", 2, partitionAndNumSeen.getValue().intValue()); + Assertions.assertEquals(2, partitionAndNumSeen.getValue().intValue(), "We haven't used the partition space so they should all be even, but partition " + partitionAndNumSeen.getKey().intValue() + + " did not see 2."); } } @@ -218,15 +218,15 @@ public void testEvenDistributionWithFewerReducers() { // first split is a, last is z countPartitions(numberTimesPartitionSeen, numPartitions, partitioner); - Assert.assertEquals("Should have seen a total of 10 different partitions given the small reducer space", 10, numberTimesPartitionSeen.size()); + Assertions.assertEquals(10, numberTimesPartitionSeen.size(), "Should have seen a total of 10 different partitions given the small reducer space"); System.out.println(numberTimesPartitionSeen); // we partitioned 27 splits // over a space of 10 partitioners // so each partitioners should have 2 splits or 3 splits assigned to it // we partitioned two rows per split, so each partition should have seen 4 or 6 rows for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { - Assert.assertTrue(partitionAndNumSeen.toString(), 4 <= partitionAndNumSeen.getValue().intValue()); - Assert.assertTrue(partitionAndNumSeen.toString(), partitionAndNumSeen.getValue().intValue() <= 6); + Assertions.assertTrue(4 <= partitionAndNumSeen.getValue().intValue(), partitionAndNumSeen.toString()); + Assertions.assertTrue(partitionAndNumSeen.getValue().intValue() <= 6, partitionAndNumSeen.toString()); } } @@ -241,7 +241,7 @@ public void testPartitionsInNonContiguousWay() { String row = Character.toString((char) ("a".codePointAt(0) + i)); partitionsFound.add(partitioner.getPartition(getBulkIngestKey(row), new Value(), numPartitions)); } - Assert.assertEquals(10, partitionsFound.size()); + Assertions.assertEquals(10, partitionsFound.size()); // k - t go to different partitions partitionsFound.clear(); @@ -249,7 +249,7 @@ public void testPartitionsInNonContiguousWay() { String row = Character.toString((char) ("a".codePointAt(0) + i)); partitionsFound.add(partitioner.getPartition(getBulkIngestKey(row), new Value(), numPartitions)); } - Assert.assertEquals(10, partitionsFound.size()); + Assertions.assertEquals(10, partitionsFound.size()); } @Test From c51e2fd710c5f7ba37d4d61eb509ebaa73253ee5 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 19:15:58 +0000 Subject: [PATCH 10/26] More JUnit5 updates --- .../partition/MultiTableRRRangePartitionerTest.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java index 51cf07f61ae..bb5b81b94c8 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java @@ -27,7 +27,6 @@ import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; -import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -145,8 +144,7 @@ public void testAllDataForOneSplitGoesToOnePartitioner() { String rowStr = Character.toString((char) ("a".codePointAt(0) + i)); int resultRow = partitioner.getPartition(getBulkIngestKey(rowStr), new Value(), numPartitions); - Assert.assertEquals("These should have matched: resultRow: " + resultRow + " , resultForPrecedingRow: " + resultForPrecedingRow, resultRow, - resultForPrecedingRow); + Assertions.assertEquals(resultRow, resultForPrecedingRow, "These should have matched: resultRow: " + resultRow + " , resultForPrecedingRow: " + resultForPrecedingRow); } } @@ -264,7 +262,7 @@ public void testOverlapsAtLastPartitions() { int previousCount = 0; for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { int currentCount = partitionAndNumSeen.getValue().intValue(); - Assert.assertTrue(partitionAndNumSeen.toString(), previousCount <= currentCount); + Assertions.assertTrue(previousCount <= currentCount, partitionAndNumSeen.toString()); previousCount = currentCount; } } From 6c36e10725672bd20b4ce607f7e7db332c5fc798 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 19:25:58 +0000 Subject: [PATCH 11/26] Splits cache factory reset for tests --- .../datawave/ingest/mapreduce/job/SplitsCacheFactory.java | 4 ++++ .../mapreduce/partition/BalancedShardPartitionerTest.java | 4 +++- .../partition/MultiTableRRRangePartitionerTest.java | 5 +++-- .../mapreduce/partition/MultiTableRangePartitionerTest.java | 4 +++- .../partition/TabletLocationHashPartitionerTest.java | 4 +++- 5 files changed, 16 insertions(+), 5 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java index 9916a0eb3b6..8a4c4e25f45 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java @@ -27,4 +27,8 @@ public static SplitsCache getSplitsCache(final Configuration conf) { } return INSTANCE; } + + public static void clearInstance() { + INSTANCE = null; + } } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index 312c20923b7..7887ef25496 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -12,6 +12,7 @@ import java.util.TreeMap; import java.util.TreeSet; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.commons.lang.time.DateUtils; @@ -56,7 +57,8 @@ public static void defineShardLocationsFile() throws IOException { @Before public void setUp() throws IOException { conf = new Configuration(); - TableSplitsCache.getCurrentCache(conf).clear(); + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); conf.setInt(ShardIdFactory.NUM_SHARDS, SHARDS_PER_DAY); partitioner = new BalancedShardPartitioner(); diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java index bb5b81b94c8..aeb86990678 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java @@ -13,6 +13,7 @@ import java.util.Map; import java.util.TreeMap; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; @@ -52,8 +53,8 @@ public void before() throws IOException { configuration.set("job.output.table.names", TableName.SHARD); configuration.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); - TableSplitsCache.getCurrentCache(configuration).clear(); - + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); } @Test diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java index 47f5196b17f..f6dfb6e6e2c 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java @@ -4,6 +4,7 @@ import java.net.URISyntaxException; import java.net.URL; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; @@ -33,8 +34,9 @@ public void before() throws IOException { mockJob = Job.getInstance(); configuration = mockJob.getConfiguration(); configuration.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); - TableSplitsCache.getCurrentCache(configuration).clear(); configuration.set(TableConfigurationUtil.JOB_OUTPUT_TABLE_NAMES, TableName.SHARD); + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); } @Test diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java index 2271e04a998..4d7a543ada1 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java @@ -4,6 +4,7 @@ import java.util.HashSet; import java.util.Set; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.hadoop.conf.Configuration; import org.junit.After; import org.junit.Assert; @@ -33,7 +34,8 @@ public void setUp() { conf = new Configuration(); conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); - TableSplitsCache.getCurrentCache(conf).clear(); + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); partitioner = new TabletLocationHashPartitioner(); partitioner.setConf(conf); From 7c910ee2a0bd0fe1bf3c8426c6f4d8306ad83383 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Tue, 24 Jun 2025 19:36:47 +0000 Subject: [PATCH 12/26] Update to use JUnit 5 semantics --- .../BalancedShardPartitionerTest.java | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index 7887ef25496..0de97e36b7e 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -1,9 +1,8 @@ package datawave.ingest.mapreduce.partition; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -19,13 +18,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Partitioner; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; import datawave.ingest.mapreduce.handler.shard.ShardIdFactory; import datawave.ingest.mapreduce.handler.shard.ShardedDataTypeHandler; @@ -33,6 +25,14 @@ import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; import datawave.util.time.DateHelper; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; public class BalancedShardPartitionerTest { private static final int TOTAL_TSERVERS = 600; @@ -41,20 +41,20 @@ public class BalancedShardPartitionerTest { private static final int NUM_REDUCE_TASKS = 270; private static Configuration conf; - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir + public Path temporaryFolder; private BalancedShardPartitioner partitioner = null; private ShardIdFactory shardIdFactory = new ShardIdFactory(conf); - @BeforeClass + @BeforeAll public static void defineShardLocationsFile() throws IOException { conf = new Configuration(); conf.setInt(ShardIdFactory.NUM_SHARDS, SHARDS_PER_DAY); } - @Before + @BeforeEach public void setUp() throws IOException { conf = new Configuration(); TableSplitsCache.clear(); @@ -63,7 +63,7 @@ public void setUp() throws IOException { conf.setInt(ShardIdFactory.NUM_SHARDS, SHARDS_PER_DAY); partitioner = new BalancedShardPartitioner(); // gotta load this every test, or using different values bleeds into other tests - new TestShardGenerator(conf, temporaryFolder.newFolder(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD); + new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("root")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD); conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "shard"); shardIdFactory = new ShardIdFactory(conf); @@ -71,7 +71,7 @@ public void setUp() throws IOException { } - @After + @AfterEach public void tearDown() { partitioner = null; conf.unset(BalancedShardPartitioner.MISSING_SHARD_STRATEGY_PROP); @@ -87,7 +87,7 @@ public void testNoCollisionsTodayAndBack2Days() throws Exception { @Test public void testTwoTablesAreOffsetted() throws Exception { // create another split files for this test that contains two tables. register the tables names for both shard and error shard - new TestShardGenerator(conf, temporaryFolder.newFolder(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD, TableName.ERROR_SHARD); + new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("test-two-tables")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD, TableName.ERROR_SHARD); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "errorShard,shard"); partitioner.setConf(conf); @@ -105,8 +105,8 @@ public void testTwoTablesAreOffsetted() throws Exception { private void verifyOffsetGroup(int group, int partitionId, String date) { int numShards = shardIdFactory.getNumShards(date); - Assert.assertTrue("partitionId " + partitionId + " is not >= " + (numShards * group), partitionId >= numShards * group); - Assert.assertTrue("partitionId " + partitionId + " is not < " + (numShards * (group + 1)), partitionId < numShards * (group + 1)); + assertTrue(partitionId >= numShards * group, "partitionId " + partitionId + " is not >= " + (numShards * group)); + assertTrue(partitionId < numShards * (group + 1), "partitionId " + partitionId + " is not < " + (numShards * (group + 1))); } @Test @@ -231,7 +231,7 @@ private void simulateDifferentNumberShardsPerDay(String missingShardStrategy, St locations.put(new Text(day + "_" + currShard), Integer.toString(tserverId++)); } } - new TestShardGenerator(conf, temporaryFolder.newFolder(), locations, tableName); + new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("simulated-diff")).toFile(), locations, tableName); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, tableName); partitioner.setConf(conf); if (missingShardStrategy != null) { @@ -273,7 +273,7 @@ public static void assertExpectedCollisions(Partitioner partitionerIn, int daysB partitionsUsed.add(partition); } // 9 is what we get by hashing the shardId - Assert.assertTrue("For " + daysBack + " days ago, we had a different number of collisions: " + collisions, expectedCollisions >= collisions); + assertTrue(expectedCollisions >= collisions, "For " + daysBack + " days ago, we had a different number of collisions: " + collisions); // this // has // more to From 4695c263dca4abae910589aebc11e19a37d2f661 Mon Sep 17 00:00:00 2001 From: dtspence <33552925+dtspence@users.noreply.github.com> Date: Mon, 4 Aug 2025 02:26:42 +0000 Subject: [PATCH 13/26] Formatting changes --- .../mapreduce/job/SplitsCacheFactory.java | 3 ++- .../partition/MultiTableRangePartitioner.java | 2 +- .../ingest/mapreduce/job/SplitsFileTest.java | 1 - .../BalancedShardPartitionerTest.java | 24 ++++++++++--------- .../MultiTableRRRangePartitionerTest.java | 16 +++++++------ .../MultiTableRangePartitionerTest.java | 2 +- .../TabletLocationHashPartitionerTest.java | 2 +- 7 files changed, 27 insertions(+), 23 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java index 8a4c4e25f45..480724d76be 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCacheFactory.java @@ -16,7 +16,8 @@ public static SplitsCache getSplitsCache(final Configuration conf) { try { String splitsCacheImpl = conf.get(SPLITS_CACHE_IMPL); // noinspection unchecked - Class clazz = splitsCacheImpl != null ? (Class) Class.forName(splitsCacheImpl) : SplitsFile.class; + Class clazz = splitsCacheImpl != null ? (Class) Class.forName(splitsCacheImpl) + : SplitsFile.class; INSTANCE = clazz.getDeclaredConstructor().newInstance(); INSTANCE.init(conf); } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException ex) { diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java index bf6eea59f7e..7d105ed8fcc 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitioner.java @@ -80,7 +80,7 @@ public int getPartition(BulkIngestKey key, Value value, int numPartitions) { String tableName = key.getTableName().toString(); key.getKey().getRow(holder); - int splitSize= splitsCache.getSplitsCount(tableName); + int splitSize = splitsCache.getSplitsCount(tableName); if (splitSize < 0) { return (tableName.hashCode() & Integer.MAX_VALUE) % numPartitions; } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java index 15feb259473..c234c6ceaa8 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java @@ -5,7 +5,6 @@ import java.io.IOException; import java.io.PrintStream; import java.net.URI; -import java.util.HashMap; import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index 0de97e36b7e..04e32bb9f5e 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -1,5 +1,8 @@ package datawave.ingest.mapreduce.partition; +import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; @@ -11,28 +14,25 @@ import java.util.TreeMap; import java.util.TreeSet; -import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.commons.lang.time.DateUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Partitioner; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import datawave.ingest.mapreduce.handler.shard.ShardIdFactory; import datawave.ingest.mapreduce.handler.shard.ShardedDataTypeHandler; import datawave.ingest.mapreduce.job.BulkIngestKey; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; import datawave.util.time.DateHelper; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import static org.junit.Assert.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; public class BalancedShardPartitionerTest { private static final int TOTAL_TSERVERS = 600; @@ -63,7 +63,8 @@ public void setUp() throws IOException { conf.setInt(ShardIdFactory.NUM_SHARDS, SHARDS_PER_DAY); partitioner = new BalancedShardPartitioner(); // gotta load this every test, or using different values bleeds into other tests - new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("root")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD); + new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("root")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, + TableName.SHARD); conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "shard"); shardIdFactory = new ShardIdFactory(conf); @@ -87,7 +88,8 @@ public void testNoCollisionsTodayAndBack2Days() throws Exception { @Test public void testTwoTablesAreOffsetted() throws Exception { // create another split files for this test that contains two tables. register the tables names for both shard and error shard - new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("test-two-tables")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD, TableName.ERROR_SHARD); + new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("test-two-tables")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, + TableName.SHARD, TableName.ERROR_SHARD); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "errorShard,shard"); partitioner.setConf(conf); diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java index aeb86990678..4ed7114dbb9 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java @@ -13,7 +13,6 @@ import java.util.Map; import java.util.TreeMap; -import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; @@ -23,14 +22,15 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.task.MapContextImpl; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import datawave.ingest.mapreduce.job.BulkIngestKey; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableSplitsCache; import datawave.util.TableName; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; public class MultiTableRRRangePartitionerTest { @@ -145,7 +145,8 @@ public void testAllDataForOneSplitGoesToOnePartitioner() { String rowStr = Character.toString((char) ("a".codePointAt(0) + i)); int resultRow = partitioner.getPartition(getBulkIngestKey(rowStr), new Value(), numPartitions); - Assertions.assertEquals(resultRow, resultForPrecedingRow, "These should have matched: resultRow: " + resultRow + " , resultForPrecedingRow: " + resultForPrecedingRow); + Assertions.assertEquals(resultRow, resultForPrecedingRow, + "These should have matched: resultRow: " + resultRow + " , resultForPrecedingRow: " + resultForPrecedingRow); } } @@ -203,8 +204,9 @@ public void testEvenDistributionWithExtraReducers() { Assertions.assertEquals(27, numberTimesPartitionSeen.size(), "Should have seen a total of 27 different partitions. There is a split for each letter of the alphabet and the null split which is not in the file"); for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { - Assertions.assertEquals(2, partitionAndNumSeen.getValue().intValue(), "We haven't used the partition space so they should all be even, but partition " + partitionAndNumSeen.getKey().intValue() - + " did not see 2."); + Assertions.assertEquals(2, partitionAndNumSeen.getValue().intValue(), + "We haven't used the partition space so they should all be even, but partition " + partitionAndNumSeen.getKey().intValue() + + " did not see 2."); } } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java index f6dfb6e6e2c..49bd8a87cc4 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRangePartitionerTest.java @@ -4,7 +4,6 @@ import java.net.URISyntaxException; import java.net.URL; -import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; @@ -19,6 +18,7 @@ import org.junit.Test; import datawave.ingest.mapreduce.job.BulkIngestKey; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import datawave.ingest.mapreduce.job.SplitsConstants; import datawave.ingest.mapreduce.job.TableConfigurationUtil; import datawave.ingest.mapreduce.job.TableSplitsCache; diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java index 4d7a543ada1..7eb6c84dc8e 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java @@ -4,7 +4,6 @@ import java.util.HashSet; import java.util.Set; -import datawave.ingest.mapreduce.job.SplitsCacheFactory; import org.apache.hadoop.conf.Configuration; import org.junit.After; import org.junit.Assert; @@ -14,6 +13,7 @@ import org.junit.rules.TemporaryFolder; import datawave.ingest.mapreduce.handler.shard.ShardIdFactory; +import datawave.ingest.mapreduce.job.SplitsCacheFactory; import datawave.ingest.mapreduce.job.TableSplitsCache; public class TabletLocationHashPartitionerTest { From 47225b44ed8edee0d2ed7853f5771b8cd8753fb8 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Mon, 30 Mar 2026 14:17:15 +0000 Subject: [PATCH 14/26] Adding in unit test qol updates from TX Dylan --- .../ingest/mapreduce/job/IngestJob.java | 6 ++-- .../job/MultiRFileOutputFormatter.java | 3 ++ .../mapreduce/job/SplitsJobContext.java | 10 ++++++ .../job/MultiRFileOutputFormatterTest.java | 22 ++++++++++-- .../ingest/mapreduce/job/SplitsFileTest.java | 4 ++- .../BalancedShardPartitionerTest.java | 34 +++++++++++++------ .../TabletLocationHashPartitionerTest.java | 3 ++ 7 files changed, 64 insertions(+), 18 deletions(-) create mode 100644 warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java index e259aa449aa..ba4f600ffc9 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/IngestJob.java @@ -722,7 +722,7 @@ protected Configuration parseArguments(String[] args, Configuration conf) throws } else if (args[i].equals("-compressionType")) { compressionType = args[++i]; } else if (args[i].equals("-compressionTableDisallowList")) { - String[] tables = StringUtils.split(args[++i], ','); + String[] tables = args[++i].split(","); compressionTableDisallowList.addAll(Arrays.asList(tables)); } else if (args[i].equals("-maxRFileUndeduppedEntries")) { maxRFileEntries = Integer.parseInt(args[++i]); @@ -1152,7 +1152,7 @@ protected boolean markJobComplete(FileSystem fs, Path workDir) throws IOExceptio * if there is an issue with read or write */ protected Path[] getFilesToProcess(FileSystem fs, boolean inputFileLists, String inputFileListMarker, String inputPaths) throws IOException { - String[] paths = StringUtils.trimAndRemoveEmptyStrings(StringUtils.split(inputPaths, ',')); + String[] paths = StringUtils.trimAndRemoveEmptyStrings(inputPaths.split(",")); List inputPathList = new ArrayList<>(inputFileLists ? paths.length * 100 : paths.length); for (String inputPath : paths) { // if we are to treat the input paths as file lists, then expand here @@ -1385,7 +1385,7 @@ protected void startDaemonProcesses(Configuration configuration) { if (daemonClassNames == null) { return; } - for (String className : StringUtils.split(daemonClassNames, ',')) { + for (String className : daemonClassNames.split(",")) { try { @SuppressWarnings("unchecked") Class daemonClass = (Class) Class.forName(className.trim()); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index b3cd478276a..dc580edfea5 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -627,6 +627,9 @@ private SizeTrackingWriter getOrCreateWriter(TaskAttemptContext context, String if (generateMapFilePerShardLocation) { // Look up the shard location (tablet server serving shard ID rowKey) // If we don't have a location, then just use the rowKey itself. + // in the case we have a shard id that has no split. Let's put this in one "extra" file, if not defined by row key (unit testing). + String fallback = conf.get("shard.fallback.name." + rowKey.toString(), "extra"); + shardLocation = splitsCache.getExactLocation(tableName, rowKey, () -> fallback); // in the case we have a shard id that has no split. Lets put this in one "extra" file shardLocation = splitsCache.getExactLocation(tableName, rowKey, () -> "extra"); } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java new file mode 100644 index 00000000000..3a3e638ca00 --- /dev/null +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java @@ -0,0 +1,10 @@ + +package datawave.ingest.mapreduce.job; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; + +public interface SplitsJobContext { + void prepareContext(Configuration conf) throws IOException; +} diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java index bc569f6fc3b..08f667df8b3 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java @@ -415,9 +415,7 @@ public void append(Key key, Value value) throws IOException {} public void before() { formatter = createFormatter(); conf = new Configuration(); - conf.set("mapred.output.dir", "/tmp"); - conf.set(SplitsFile.CONFIGURED_SHARDED_TABLE_NAMES, TableName.SHARD); - + setConfiguration(conf, 2); } @Test @@ -431,6 +429,9 @@ public void testTableSeparation() throws IOException, InterruptedException { @Test public void testTableSeparationWithFilePerShardLoc() throws IOException, InterruptedException { + // conf.set("shard.fallback.name.20100101_1", "server1"); + // conf.set("shard.fallback.name.20100101_2", "server2"); + MultiRFileOutputFormatter.setGenerateMapFilePerShardLocation(conf, true); RecordWriter writer = createWriter(formatter, conf); writeShardPairs(writer, 2); @@ -458,6 +459,9 @@ private void expectShardFiles(int num) { @Test public void testRFileFileSizeLimitWithFilePerShardLoc() throws IOException, InterruptedException { + // conf.set("shard.fallback.name.20100101_1", "server1"); + // conf.set("shard.fallback.name.20100101_2", "server2"); + MultiRFileOutputFormatter.setGenerateMapFilePerShardLocation(conf, true); // each key we write is 16 characters total, so a limit of 32 should allow two keys per file MultiRFileOutputFormatter.setRFileLimits(conf, 0, 32); @@ -482,6 +486,9 @@ public void testRFileFileSizeLimit() throws IOException, InterruptedException { @Test public void testRFileEntrySizeLimitWithFilePerShardLoc() throws IOException, InterruptedException { + // conf.set("shard.fallback.name.20100101_1", "server1"); + // conf.set("shard.fallback.name.20100101_2", "server2"); + MultiRFileOutputFormatter.setRFileLimits(conf, 1, 0); MultiRFileOutputFormatter.setGenerateMapFilePerShardLocation(conf, true); RecordWriter writer = createWriter(formatter, conf); @@ -522,4 +529,13 @@ private void writeShardEntry(RecordWriter writer, int shard private void assertNumFileNames(int expectedNumFiles) { Assert.assertEquals(filenames.toString(), expectedNumFiles, filenames.size()); } + + private void setConfiguration(Configuration conf, int count) { + conf.set("mapred.output.dir", "/tmp"); + conf.set(SplitsFile.CONFIGURED_SHARDED_TABLE_NAMES, TableName.SHARD); + + for (int i = 1; i <= count; i++) { + conf.set(String.format("shard.fallback.name.20100101_%d", i), "server" + i); + } + } } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java index c234c6ceaa8..0a710fdccb9 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java @@ -1,5 +1,7 @@ package datawave.ingest.mapreduce.job; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.BufferedOutputStream; import java.io.File; import java.io.IOException; @@ -59,7 +61,7 @@ private void createSplitsFile(Map splits, Configuration conf, int e writeBaseSplitsFile(splits, conf, tableName); long actualCount = splits.size(); - Assertions.assertEquals(expectedNumRows, actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); + assertEquals(expectedNumRows, actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); } private void writeBaseSplitsFile(Map locations, Configuration conf, String tableName) throws IOException { diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index 04e32bb9f5e..07a5f9c0784 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -88,12 +88,17 @@ public void testNoCollisionsTodayAndBack2Days() throws Exception { @Test public void testTwoTablesAreOffsetted() throws Exception { // create another split files for this test that contains two tables. register the tables names for both shard and error shard - new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("test-two-tables")).toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, - TableName.SHARD, TableName.ERROR_SHARD); + Path twoTablesDir = Files.createDirectory(temporaryFolder.resolve("test-two-tables")); + new TestShardGenerator(conf, twoTablesDir.toFile(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, TableName.SHARD, TableName.ERROR_SHARD); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "errorShard,shard"); + conf.set("datawave.ingest.splits.cache.dir", twoTablesDir.toString()); + conf.set("datawave.ingest.splits.cache.file", "all-splits.txt"); - partitioner.setConf(conf); + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); + conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, true); + partitioner.setConf(conf); // For a shard from today, we can assume that they're well balanced. // If offsetting is working, they will not go to the same partitions String today = formatDay(0); @@ -188,7 +193,7 @@ public void testDifferentNumberShardsPerDayHash() throws IOException { // hashing is the default implementation, so null is passed in String tableName = "shard3"; - simulateDifferentNumberShardsPerDay(null, tableName); + simulateDifferentNumberShardsPerDay("hash", tableName); // 1 day ago should get SHARDS_PER_DAY partitions assertPartitionsForDay(partitioner, tableName, 1, SHARDS_PER_DAY); @@ -214,13 +219,9 @@ public void testDifferentNumberShardsPerDayCollapseButOutsideRange() throws IOEx } private void simulateDifferentNumberShardsPerDay(String missingShardStrategy, String tableName) throws IOException { - // This emulates today, yesterday and the day before have SHARDS_PER_DAY splits and - // 3 days ago and 4 days ago only have 2 splits, _0 and _1. - SortedMap locations = new TreeMap<>(); long now = System.currentTimeMillis(); int tserverId = 1; - Text prevEndRow = new Text(); for (int daysAgo = 0; daysAgo <= 2; daysAgo++) { String day = DateHelper.format(now - (daysAgo * DateUtils.MILLIS_PER_DAY)); for (int currShard = 0; currShard < SHARDS_PER_DAY; currShard++) { @@ -233,14 +234,25 @@ private void simulateDifferentNumberShardsPerDay(String missingShardStrategy, St locations.put(new Text(day + "_" + currShard), Integer.toString(tserverId++)); } } - new TestShardGenerator(conf, Files.createDirectory(temporaryFolder.resolve("simulated-diff")).toFile(), locations, tableName); + + Path simulatedDiffDir = Files.createDirectory(temporaryFolder.resolve("simulated-diff")); + new TestShardGenerator(conf, simulatedDiffDir.toFile(), locations, tableName); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, tableName); + conf.set("datawave.ingest.splits.cache.dir", simulatedDiffDir.toString()); + conf.set("datawave.ingest.splits.cache.file", "all-splits.txt"); + conf.set(BalancedShardPartitioner.MISSING_SHARD_STRATEGY_PROP, missingShardStrategy); + + // Clear stale cache and force a reload + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); + conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, true); + partitioner.setConf(conf); if (missingShardStrategy != null) { conf.set(BalancedShardPartitioner.MISSING_SHARD_STRATEGY_PROP, missingShardStrategy); } - // check we made enough tservers - assertEquals(SHARDS_PER_DAY * 3 + 2 + 2, tserverId - 1); // since it already ++'d for next one + + assertEquals(SHARDS_PER_DAY * 3 + 2 + 2, tserverId - 1); } private void assertPartitionsForDay(BalancedShardPartitioner partioner, String tableName, int daysAgo, int expectedNumOfPartitions) { diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java index 7eb6c84dc8e..c3b5a4a79b4 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/TabletLocationHashPartitionerTest.java @@ -52,6 +52,9 @@ public void tearDown() { public void testLocationHashPartitioner() throws Exception { conf.setInt(ShardIdFactory.NUM_SHARDS, SHARDS_PER_DAY); new TestShardGenerator(conf, temporaryFolder.newFolder(), NUM_DAYS, SHARDS_PER_DAY, TOTAL_TSERVERS, "shard"); + TableSplitsCache.clear(); + SplitsCacheFactory.clearInstance(); + TabletLocationHashPartitioner partitionerTwo = new TabletLocationHashPartitioner(); partitionerTwo.setConf(conf); From ca0e08d4ce1590a67ffd77e794261d81b3cd8807 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Thu, 2 Apr 2026 13:52:22 +0000 Subject: [PATCH 15/26] Fixing import warnings --- .../job/MultiRFileOutputFormatter.java | 12 ----- .../ingest/mapreduce/job/SplitsFileTest.java | 21 ++++----- .../MultiTableRRRangePartitionerTest.java | 44 +++++++++---------- 3 files changed, 33 insertions(+), 44 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index 70abb13ead3..36614d7ce3e 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -276,7 +276,6 @@ protected SizeTrackingWriter openWriter(String filename, AccumuloConfiguration t var builder = org.apache.accumulo.core.client.rfile.RFile.newWriter().to(filename).withFileSystem(fs).withTableProperties(tableConf); if(this.loadPlanningEnabled) { SplitsCache splits = SplitsCache.getInstance(conf); - //SplitsFile file = (SplitsFile) SplitsCacheFactory.getSplitsCache(conf); LoadPlan.SplitResolver splitResolver = row->{ try { return findContainingSplits(row,((SplitsFile) splits).getSplits(conf, table)); @@ -390,17 +389,6 @@ static LoadPlan.TableSplits findContainingSplits(Text lookupRow, List tabl return new LoadPlan.TableSplits(prevRow, endRow); } - /* - * public static class CustomSplitsResolver implements LoadPlan.SplitResolver { private SplitsCache cache; - * - * public CustomSplitsResolver(SplitsCache cache) { this.cache = cache; } - * - * public LoadPlan.TableSplits apply(Text row) { SplitsFile file = SplitsCacheFactory.getSplitsCache(); SplitsFile f = new SplitsFile(); f.init(conf); - * cache.getSplits(); //LoadPlan.TableSplits splits = new LoadPlan.TableSplits(st.begin, toString().endsWith()); } - * - * } - */ - public static class SizeTrackingWriter { private RFileWriter delegate; long size = 0; diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java index db7f7b787a0..1c357b20087 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java @@ -1,6 +1,8 @@ package datawave.ingest.mapreduce.job; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.BufferedOutputStream; import java.io.File; @@ -18,7 +20,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -63,7 +64,7 @@ private void createSplitsFile(Map splits, Configuration conf, int e writeBaseSplitsFile(splits, conf, tableName); long actualCount = splits.size(); - assertEquals(expectedNumRows, actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); + assertEquals(expectedNumRows == actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); } private void writeBaseSplitsFile(Map locations, Configuration conf, String tableName) throws IOException { @@ -79,7 +80,7 @@ private void writeBaseSplitsFile(Map locations, Configuration conf, TableSplitsCache.getCurrentCache(conf).writeHeaderLine(out, Collections.singleton(tableName)); TableSplitsCache.getCurrentCache(conf).writeLocations(out, tableName, Lists.newArrayList(locations.keySet()), locations); } - Assertions.assertTrue(fs.exists(splitsPath)); + assertTrue(fs.exists(splitsPath)); conf.set(SplitsConstants.SPLITS_CACHE_DIR, tmpBaseSplitDir.getAbsolutePath()); @@ -106,7 +107,7 @@ public void testGetAllSplitsFilesWithoutPath() throws Exception { uut.init(conf); - Assertions.assertThrows(IOException.class, () -> uut.setupJob(Job.getInstance(conf))); + assertThrows(IOException.class, () -> uut.setupJob(Job.getInstance(conf))); } @Test @@ -134,7 +135,7 @@ public void testMissingAllOfTodaysSplits() throws Exception { Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be invalid, which makes the rest bad too - Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); + assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); } @Test @@ -147,7 +148,7 @@ public void testUnbalancedTodaysSplits() throws Exception { Map locations = uut.getSplitsAndLocations(conf, tableName); // three days of splits, today should be invalid, which makes the rest bad too - Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); + assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); } @Test @@ -159,12 +160,12 @@ public void testMissingAllOfYesterdaysSplits() throws Exception { uut.init(conf); Map locations = uut.getSplitsAndLocations(conf, tableName); - Assertions.assertEquals(splits.size(), locations.size()); + assertEquals(splits.size(), locations.size()); // three days of splits, today should be valid // yesterday and all other days invalid uut.validateShardIdLocations(conf, tableName, 0, locations); // this should cause the exception - Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 1, locations)); + assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 1, locations)); } @Test @@ -180,7 +181,7 @@ public void testUnbalancedYesterdaysSplits() throws Exception { // yesterday and all other days invalid uut.validateShardIdLocations(conf, tableName, 0, locations); // this should cause the exception - Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 1, locations)); + assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 1, locations)); } @Test @@ -194,7 +195,7 @@ public void testUnbalancedMaxMoreThanConfigured() throws Exception { Map locations = uut.getSplitsAndLocations(conf, tableName); // this should cause the exception - Assertions.assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); + assertThrows(IllegalStateException.class, () -> uut.validateShardIdLocations(conf, tableName, 0, locations)); } @Test diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java index f4575b3890c..b09f6d29b26 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/MultiTableRRRangePartitionerTest.java @@ -5,6 +5,8 @@ package datawave.ingest.mapreduce.partition; import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.net.URISyntaxException; @@ -22,7 +24,6 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -74,8 +75,8 @@ public void testCalculateIndex() { int resultFour = instance.calculateIndex(indexFour, numPartitions, tableName, cutPointArrayLength); assertEquals(result, resultTwo); assertEquals(result, expectedResult); - Assertions.assertEquals(4, resultThree); - Assertions.assertEquals(0, resultFour); + assertEquals(4, resultThree); + assertEquals(0, resultFour); } @Test @@ -85,7 +86,7 @@ public void testEmptySplitsThrowsException() throws IOException, URISyntaxExcept mockContextForLocalCacheFile(url); configuration.set(SplitsConstants.SPLITS_CACHE_DIR, url.getPath().substring(0, url.getPath().lastIndexOf('/'))); configuration.set(SplitsConstants.SPLITS_CACHE_FILE, filename); - Assertions.assertThrows(RuntimeException.class, () -> getPartition("23432")); + assertThrows(RuntimeException.class, () -> getPartition("23432")); } @Test @@ -103,7 +104,7 @@ public org.apache.hadoop.fs.Path[] getLocalCacheFiles() throws IOException { } }); - Assertions.assertThrows(RuntimeException.class, () -> getPartition("23432")); + assertThrows(RuntimeException.class, () -> getPartition("23432")); } private URL createUrl(String fileName) { @@ -145,7 +146,7 @@ public void testAllDataForOneSplitGoesToOnePartitioner() { String rowStr = Character.toString((char) ("a".codePointAt(0) + i)); int resultRow = partitioner.getPartition(getBulkIngestKey(rowStr), new Value(), numPartitions); - Assertions.assertEquals(resultRow, resultForPrecedingRow, + assertTrue(resultRow == resultForPrecedingRow, "These should have matched: resultRow: " + resultRow + " , resultForPrecedingRow: " + resultForPrecedingRow); } } @@ -158,8 +159,8 @@ public void testCalculateIndexRangeIsValid() { MultiTableRRRangePartitioner partitioner = new MultiTableRRRangePartitioner(); for (int i = -1 * cutPointArrayLength - 1; i < cutPointArrayLength; i++) { int result = partitioner.calculateIndex(i, numPartitions, "someTableName", cutPointArrayLength); - Assertions.assertTrue(0 <= result, "i: " + i + " result: " + result); - Assertions.assertTrue(result < numPartitions, "i: " + i + " result: " + result); + assertTrue(0 <= result, "i: " + i + " result: " + result); + assertTrue(result < numPartitions, "i: " + i + " result: " + result); } } @@ -180,15 +181,15 @@ public void testPartitionerSpaceIsValid() { for (int i = 0; i < numSplits; i++) { String rowStr = Character.toString((char) ("a".codePointAt(0) + i)); int result = partitioner.getPartition(getBulkIngestKey(rowStr), new Value(), numPartitions); - Assertions.assertTrue(numPartitions - numSplits - 1 <= result, "rowStr: " + rowStr + " partition: " + result); - Assertions.assertTrue(result < numPartitions, "rowStr: " + rowStr + " partition: " + result); + assertTrue(numPartitions - numSplits - 1 <= result, "rowStr: " + rowStr + " partition: " + result); + assertTrue(result < numPartitions, "rowStr: " + rowStr + " partition: " + result); } // test rows before and after each split for (int i = -1; i < numSplits + 1; i++) { int result = partitioner.getPartition(getBulkIngestKey(Character.toString((char) ("a".codePointAt(0) + i)) + "_"), new Value(), numPartitions); - Assertions.assertTrue(numPartitions - numSplits - 1 <= result, "i: " + i + " partition: " + result); - Assertions.assertTrue(result < numPartitions, "i: " + i + " partition: " + result); + assertTrue(numPartitions - numSplits - 1 <= result, "i: " + i + " partition: " + result); + assertTrue(result < numPartitions, "i: " + i + " partition: " + result); } } @@ -201,12 +202,11 @@ public void testEvenDistributionWithExtraReducers() { // first split is a, last is z countPartitions(numberTimesPartitionSeen, numPartitions, partitioner); - Assertions.assertEquals(27, numberTimesPartitionSeen.size(), + assertTrue(27 == numberTimesPartitionSeen.size(), "Should have seen a total of 27 different partitions. There is a split for each letter of the alphabet and the null split which is not in the file"); for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { - Assertions.assertEquals(2, partitionAndNumSeen.getValue().intValue(), - "We haven't used the partition space so they should all be even, but partition " + partitionAndNumSeen.getKey().intValue() - + " did not see 2."); + assertTrue(2 == partitionAndNumSeen.getValue().intValue(), "We haven't used the partition space so they should all be even, but partition " + + partitionAndNumSeen.getKey().intValue() + " did not see 2."); } } @@ -219,15 +219,15 @@ public void testEvenDistributionWithFewerReducers() { // first split is a, last is z countPartitions(numberTimesPartitionSeen, numPartitions, partitioner); - Assertions.assertEquals(10, numberTimesPartitionSeen.size(), "Should have seen a total of 10 different partitions given the small reducer space"); + assertTrue(numberTimesPartitionSeen.size() == 10, "Should have seen a total of 10 different partitions given the small reducer space"); System.out.println(numberTimesPartitionSeen); // we partitioned 27 splits // over a space of 10 partitioners // so each partitioners should have 2 splits or 3 splits assigned to it // we partitioned two rows per split, so each partition should have seen 4 or 6 rows for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { - Assertions.assertTrue(4 <= partitionAndNumSeen.getValue().intValue(), partitionAndNumSeen.toString()); - Assertions.assertTrue(partitionAndNumSeen.getValue().intValue() <= 6, partitionAndNumSeen.toString()); + assertTrue(4 <= partitionAndNumSeen.getValue().intValue(), partitionAndNumSeen.toString()); + assertTrue(partitionAndNumSeen.getValue().intValue() <= 6, partitionAndNumSeen.toString()); } } @@ -242,7 +242,7 @@ public void testPartitionsInNonContiguousWay() { String row = Character.toString((char) ("a".codePointAt(0) + i)); partitionsFound.add(partitioner.getPartition(getBulkIngestKey(row), new Value(), numPartitions)); } - Assertions.assertEquals(10, partitionsFound.size()); + assertEquals(10, partitionsFound.size()); // k - t go to different partitions partitionsFound.clear(); @@ -250,7 +250,7 @@ public void testPartitionsInNonContiguousWay() { String row = Character.toString((char) ("a".codePointAt(0) + i)); partitionsFound.add(partitioner.getPartition(getBulkIngestKey(row), new Value(), numPartitions)); } - Assertions.assertEquals(10, partitionsFound.size()); + assertEquals(10, partitionsFound.size()); } @Test @@ -265,7 +265,7 @@ public void testOverlapsAtLastPartitions() { int previousCount = 0; for (Map.Entry partitionAndNumSeen : numberTimesPartitionSeen.entrySet()) { int currentCount = partitionAndNumSeen.getValue().intValue(); - Assertions.assertTrue(previousCount <= currentCount, partitionAndNumSeen.toString()); + assertTrue(previousCount <= currentCount, partitionAndNumSeen.toString()); previousCount = currentCount; } } From 278cdb9c12f750e6ea5de3a8da38aac32430b0d9 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Thu, 2 Apr 2026 17:51:56 +0000 Subject: [PATCH 16/26] Working through some unit testing --- .../test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java | 2 +- .../mapreduce/partition/BalancedShardPartitionerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java index 1c357b20087..47276018ef6 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/SplitsFileTest.java @@ -64,7 +64,7 @@ private void createSplitsFile(Map splits, Configuration conf, int e writeBaseSplitsFile(splits, conf, tableName); long actualCount = splits.size(); - assertEquals(expectedNumRows == actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); + assertEquals(expectedNumRows, actualCount, "IngestJob#writeSplitsFile failed to create the expected number of rows"); } private void writeBaseSplitsFile(Map locations, Configuration conf, String tableName) throws IOException { diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index 3cd17e4a6ae..afbc7f2be3d 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -1,6 +1,6 @@ package datawave.ingest.mapreduce.partition; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; From 1c9e55a687e59846a0c3e916f8033c8ce04e8294 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Thu, 2 Apr 2026 19:16:03 +0000 Subject: [PATCH 17/26] Adding in getInstance --- .../datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index 36614d7ce3e..0dec7eced17 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -555,6 +555,7 @@ public RecordWriter getRecordWriter(final TaskAttemptContex FileOutputCommitter committer = (FileOutputCommitter) getOutputCommitter(context); workDir = committer.getWorkPath(); conf = context.getConfiguration(); + splitsCache = SplitsCache.getInstance(conf); setTableIdsAndConfigs(); From c3c7842a237e21a3d4c2d393d0567429fd5a576e Mon Sep 17 00:00:00 2001 From: avgAGB Date: Fri, 3 Apr 2026 13:49:47 +0000 Subject: [PATCH 18/26] Working on MROF unit tests --- .../mapreduce/job/MultiRFileOutputFormatterTest.java | 9 +++++++++ .../partition/BalancedShardPartitionerTest.java | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java index 931b0a57ae0..bb9f903af80 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java @@ -505,6 +505,9 @@ public void testTableSeparation() throws IOException, InterruptedException { @Test public void testTableSeparationWithFilePerShardLoc() throws IOException, InterruptedException { + conf.set("shard.fallback.name.20100101_1", "server1"); + conf.set("shard.fallback.name.20100101_2", "server2"); + MultiRFileOutputFormatter.setGenerateMapFilePerShardLocation(conf, true); RecordWriter writer = createWriter(formatter, conf); writeShardPairs(writer, 2); @@ -532,6 +535,9 @@ private void expectShardFiles(int num) { @Test public void testRFileFileSizeLimitWithFilePerShardLoc() throws IOException, InterruptedException { + conf.set("shard.fallback.name.20100101_1", "server1"); + conf.set("shard.fallback.name.20100101_2", "server2"); + MultiRFileOutputFormatter.setGenerateMapFilePerShardLocation(conf, true); // each key we write is 16 characters total, so a limit of 32 should allow two keys per file MultiRFileOutputFormatter.setRFileLimits(conf, 0, 32); @@ -558,6 +564,9 @@ public void testRFileFileSizeLimit() throws IOException, InterruptedException { @Test public void testRFileEntrySizeLimitWithFilePerShardLoc() throws IOException, InterruptedException { + conf.set("shard.fallback.name.20100101_1", "server1"); + conf.set("shard.fallback.name.20100101_2", "server2"); + MultiRFileOutputFormatter.setRFileLimits(conf, 1, 0); MultiRFileOutputFormatter.setGenerateMapFilePerShardLocation(conf, true); RecordWriter writer = createWriter(formatter, conf); diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index afbc7f2be3d..994147312b1 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -67,6 +67,7 @@ public void setUp() throws IOException { TableName.SHARD); conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "shard"); + shardIdFactory = new ShardIdFactory(conf); partitioner.setConf(conf); @@ -99,7 +100,6 @@ public void testTwoTablesAreOffsetted() throws Exception { conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, true); partitioner.setConf(conf); - // For a shard from today, we can assume that they're well balanced. // If offsetting is working, they will not go to the same partitions String today = formatDay(0); From ea87a827bbf2f9a38005cab02960cc715c296757 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Fri, 3 Apr 2026 15:45:53 +0000 Subject: [PATCH 19/26] More ut tweaks --- .../java/datawave/ingest/mapreduce/job/SplitsFile.java | 10 +++++++--- .../partition/BalancedShardPartitionerTest.java | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java index 0e7e7b43e55..4ab56b6ac1d 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java @@ -40,10 +40,10 @@ public class SplitsFile implements SplitsCache { public static final String CONFIGURED_SHARDED_TABLE_NAMES = ShardedDataTypeHandler.SHARDED_TNAMES + ".configured"; public static final String DIST_CACHE_LABEL = "splitsFile"; - private static final int NUMBER_MILLIS_BACK = 0; - private static final String TODAY = DateHelper.format(NUMBER_MILLIS_BACK); + private static final long NOW = System.currentTimeMillis(); + private static final String TODAY = formatDay(0); - private final ConcurrentHashMap> shardPartitionsByTable; + private ConcurrentHashMap> shardPartitionsByTable; private TableSplitsCache instance; private Configuration conf; @@ -325,6 +325,10 @@ private boolean isFutureShard(Text shardId) { return shardIdStr.substring(0, 8).compareTo(TODAY) > 0; } + private static String formatDay(int numDaysBack) { + return DateHelper.format(NOW - (DateUtils.MILLIS_PER_DAY * numDaysBack)); + } + private HashMap getShardIdAssignments(Map shardIdsToTservers, HashMap partitionsByTServer) { HashMap partitionsByShardId = new HashMap<>(); for (Map.Entry entry : shardIdsToTservers.entrySet()) { diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java index 994147312b1..021e8fcf2e0 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/partition/BalancedShardPartitionerTest.java @@ -67,7 +67,7 @@ public void setUp() throws IOException { TableName.SHARD); conf.setBoolean(TableSplitsCache.REFRESH_SPLITS, false); conf.set(ShardedDataTypeHandler.SHARDED_TNAMES, "shard"); - + shardIdFactory = new ShardIdFactory(conf); partitioner.setConf(conf); From 7ae18ba50dd1cb82a8147f05e0584e8778874a43 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Mon, 20 Apr 2026 16:57:06 +0000 Subject: [PATCH 20/26] Making tweaks to move the getsplits deeper in the stack --- .../job/MultiRFileOutputFormatter.java | 13 +++++------ .../ingest/mapreduce/job/SplitsCache.java | 3 +++ .../ingest/mapreduce/job/SplitsFile.java | 22 +++++++++++++++++++ 3 files changed, 30 insertions(+), 8 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index 0dec7eced17..18fbacd10bd 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -276,14 +276,11 @@ protected SizeTrackingWriter openWriter(String filename, AccumuloConfiguration t var builder = org.apache.accumulo.core.client.rfile.RFile.newWriter().to(filename).withFileSystem(fs).withTableProperties(tableConf); if(this.loadPlanningEnabled) { SplitsCache splits = SplitsCache.getInstance(conf); - LoadPlan.SplitResolver splitResolver = row->{ - try { - return findContainingSplits(row,((SplitsFile) splits).getSplits(conf, table)); - } catch (IOException e) { - throw new RuntimeException(e); - } - }; - builder = builder.withSplitResolver(splitResolver); + List splitsFile = splits.getSplits(conf, table); + if(splitsFile != null && !splitsFile.isEmpty()) { + LoadPlan.SplitResolver splitResolver = row->SplitsFile.findContainingSplits(row, splitsFile); + builder = builder.withSplitResolver(splitResolver); + } } return new SizeTrackingWriter(builder.build()); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java index ab1ff01e178..42cb83ab0e6 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsCache.java @@ -1,6 +1,7 @@ package datawave.ingest.mapreduce.job; import java.io.IOException; +import java.util.List; import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; @@ -27,4 +28,6 @@ static SplitsCache getInstance(final Configuration conf) { int getNearestPartition(String table, Text key); String getExactLocation(String table, Text key, Supplier defaultFn); + + List getSplits(Configuration conf, String tableName) throws IOException; } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java index 4ab56b6ac1d..518c7752350 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsFile.java @@ -15,6 +15,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; +import org.apache.accumulo.core.data.LoadPlan; import org.apache.commons.lang.time.DateUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.conf.Configuration; @@ -421,4 +422,25 @@ public Map getSplitsAndLocations(Configuration conf, String tableNa public List getSplits(Configuration conf, String tableName) throws IOException { return instance.getSplits(tableName); } + + /** + * Finds two contiguous table splits that contain the specified row should reside + * + * @param lookupRow + * Row value to be mapped + * @param tableSplits + * Splits for the table in question + * @return KeyExtent mapping for the given row + */ + static LoadPlan.TableSplits findContainingSplits(Text lookupRow, List tableSplits) { + int position = Collections.binarySearch(tableSplits, lookupRow); + if (position < 0) { + position = -1 * (position + 1); + } + + Text prevRow = position == 0 ? null : tableSplits.get(position - 1); + Text endRow = position == tableSplits.size() ? null : tableSplits.get(position); + + return new LoadPlan.TableSplits(prevRow, endRow); + } } From d8c255dd905e1eeec26f9974b24e3f1a94ac9fff Mon Sep 17 00:00:00 2001 From: avgAGB Date: Mon, 20 Apr 2026 18:37:36 +0000 Subject: [PATCH 21/26] Formatting tweak --- .../ingest/mapreduce/job/MultiRFileOutputFormatterTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java index bb9f903af80..f160953e434 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java @@ -1,7 +1,6 @@ package datawave.ingest.mapreduce.job; import static datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.BULK_IMPORT_MODE_CONFIG; -import static datawave.ingest.mapreduce.job.MultiRFileOutputFormatter.findContainingSplits; import static org.junit.Assert.assertEquals; import java.io.IOException; @@ -233,7 +232,7 @@ public void testPlanning() { expectedExtents.add(new LoadPlan.TableSplits(new Text("20170603_0b"), new Text("20170603_0c"))); List tableSplits = getSplits(); - Set extents = rfileRows.stream().map(row -> findContainingSplits(row, tableSplits)) + Set extents = rfileRows.stream().map(row -> SplitsFile.findContainingSplits(row, tableSplits)) .collect(Collectors.toCollection(HashSet::new)); assertEquals(expectedExtents, extents); From c53b46a9c69a8e7f02ce84cc98cd32e4b355f428 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Wed, 22 Apr 2026 10:45:48 +0000 Subject: [PATCH 22/26] Pruning out unused method --- .../job/MultiRFileOutputFormatter.java | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index 18fbacd10bd..8c2c54068f6 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -365,27 +365,6 @@ private void writeLoadPlans(TaskAttemptContext context) { log.debug("Finished writing bulk load plans to disk"); } - /** - * Finds two contiguous table splits that contain the specified row should reside - * - * @param lookupRow - * Row value to be mapped - * @param tableSplits - * Splits for the table in question - * @return KeyExtent mapping for the given row - */ - static LoadPlan.TableSplits findContainingSplits(Text lookupRow, List tableSplits) { - int position = Collections.binarySearch(tableSplits, lookupRow); - if (position < 0) { - position = -1 * (position + 1); - } - - Text prevRow = position == 0 ? null : tableSplits.get(position - 1); - Text endRow = position == tableSplits.size() ? null : tableSplits.get(position); - - return new LoadPlan.TableSplits(prevRow, endRow); - } - public static class SizeTrackingWriter { private RFileWriter delegate; long size = 0; From e8d368aceb3669384ee85ddb92ec688d0a2658d6 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Tue, 2 Jun 2026 01:05:56 +0000 Subject: [PATCH 23/26] Adding in enum --- .../partition/BalancedShardPartitioner.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java index 0ef1970d4e3..f723bc284c7 100755 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java @@ -35,6 +35,22 @@ public class BalancedShardPartitioner extends Partitioner i private String missingShardStrategy; int missingShardIdCount = 0; + public enum MissingShardStrategy { + HASH, COLLAPSE; + + public static MissingShardStrategy getStrategy(String stratString) { + if (HASH.name().equals(stratString)) { + return HASH; + } else if (COLLAPSE.name().equals(stratString)) { + return COLLAPSE; + } else { + throw new EnumConstantNotPresentException(BalancedShardPartitioner.MissingShardStrategy.class, stratString); + } + } + } + + private MissingShardStrategy strategy; + public static final String MISSING_SHARD_STRATEGY_PROP = "datawave.ingest.mapreduce.partition.BalancedShardPartitioner.missing.shard.strategy"; private ShardIdFactory shardIdFactory = null; @@ -111,6 +127,7 @@ public void setConf(Configuration conf) { this.conf = conf; shardIdFactory = new ShardIdFactory(conf); missingShardStrategy = conf.get(MISSING_SHARD_STRATEGY_PROP, "hash"); + MissingShardStrategy shardStrat = MissingShardStrategy.getStrategy(conf.get(MISSING_SHARD_STRATEGY_PROP)); splitsCache = SplitsCache.getInstance(conf); defineOffsetsForTables(conf); From 7a93dfe3552571aef0fbb8c468583e11e9217729 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Tue, 2 Jun 2026 01:18:24 +0000 Subject: [PATCH 24/26] Removing SplitsJobContext --- .../ingest/mapreduce/job/SplitsJobContext.java | 10 ---------- 1 file changed, 10 deletions(-) delete mode 100644 warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java deleted file mode 100644 index 3a3e638ca00..00000000000 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/SplitsJobContext.java +++ /dev/null @@ -1,10 +0,0 @@ - -package datawave.ingest.mapreduce.job; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; - -public interface SplitsJobContext { - void prepareContext(Configuration conf) throws IOException; -} From e5b1dd5c74a98ecf2fd3bdc258953f06adcf7284 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Tue, 2 Jun 2026 01:53:24 +0000 Subject: [PATCH 25/26] Fixing unit tests --- .../mapreduce/partition/BalancedShardPartitioner.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java index f723bc284c7..e5dd8615f74 100755 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java @@ -36,13 +36,13 @@ public class BalancedShardPartitioner extends Partitioner i int missingShardIdCount = 0; public enum MissingShardStrategy { - HASH, COLLAPSE; + hash, collapse; public static MissingShardStrategy getStrategy(String stratString) { - if (HASH.name().equals(stratString)) { - return HASH; - } else if (COLLAPSE.name().equals(stratString)) { - return COLLAPSE; + if (hash.name().equals(stratString)) { + return hash; + } else if (collapse.name().equals(stratString)) { + return collapse; } else { throw new EnumConstantNotPresentException(BalancedShardPartitioner.MissingShardStrategy.class, stratString); } From df763336c6c95e1052d111b98f85178d840ee268 Mon Sep 17 00:00:00 2001 From: avgAGB Date: Tue, 2 Jun 2026 02:10:04 +0000 Subject: [PATCH 26/26] Tweaking enum to allow a default --- .../mapreduce/partition/BalancedShardPartitioner.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java index e5dd8615f74..d0e75f4b068 100755 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/partition/BalancedShardPartitioner.java @@ -39,12 +39,10 @@ public enum MissingShardStrategy { hash, collapse; public static MissingShardStrategy getStrategy(String stratString) { - if (hash.name().equals(stratString)) { - return hash; - } else if (collapse.name().equals(stratString)) { + if (collapse.name().equals(stratString)) { return collapse; } else { - throw new EnumConstantNotPresentException(BalancedShardPartitioner.MissingShardStrategy.class, stratString); + return hash; } } }