diff --git a/src/main/java/datawave/iterators/MetadataFColumnSeekingFilter.java b/src/main/java/datawave/iterators/MetadataFColumnSeekingFilter.java new file mode 100644 index 0000000..a55e9c1 --- /dev/null +++ b/src/main/java/datawave/iterators/MetadataFColumnSeekingFilter.java @@ -0,0 +1,173 @@ +package datawave.iterators; + +import java.io.IOException; +import java.util.Map; +import java.util.TreeSet; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.PartialKey; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.IteratorEnvironment; +import org.apache.accumulo.core.iterators.OptionDescriber; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.user.SeekingFilter; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Splitter; + +import datawave.data.ColumnFamilyConstants; + +/** + * A {@link SeekingFilter} that operates on the metadata table's {@link ColumnFamilyConstants#COLF_F} column. + *

+ * This filter solves the problem of calculating field cardinality for a small date range on a system that contains many days worth of data, i.e., it is not + * practical to simply filter based on date and/or datatype. + *

+ * Given that the F column is simply the datatype and date concatenated with a null byte, it is easy to calculate a seek range that limits the time spent + * iterating across useless keys. + */ +public class MetadataFColumnSeekingFilter extends SeekingFilter implements OptionDescriber { + + private static final Logger log = LoggerFactory.getLogger(MetadataFColumnSeekingFilter.class); + + public static final String DATATYPES_OPT = "datatypes"; + public static final String START_DATE = "start.date"; + public static final String END_DATE = "end.date"; + + private TreeSet datatypes; + private String startDate; + private String endDate; + + @Override + public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { + if (!validateOptions(options)) { + throw new IllegalArgumentException("Iterator not configured with correct options"); + } + + String opt = options.get(DATATYPES_OPT); + if (StringUtils.isBlank(opt)) { + datatypes = new TreeSet<>(); + } else { + datatypes = new TreeSet<>(Splitter.on(',').splitToList(opt)); + } + + startDate = options.get(START_DATE); + endDate = options.get(END_DATE); + + super.init(source, options, env); + } + + @Override + public IteratorOptions describeOptions() { + IteratorOptions opts = new IteratorOptions(getClass().getName(), "Filter keys by datatype and date range", null, null); + opts.addNamedOption(DATATYPES_OPT, "The set of datatypes used as a filter"); + opts.addNamedOption(START_DATE, "The start date, used for seeking"); + opts.addNamedOption(END_DATE, "The end date, used for seeking"); + return null; + } + + @Override + public boolean validateOptions(Map options) { + return options.containsKey(DATATYPES_OPT) && options.containsKey(START_DATE) && options.containsKey(END_DATE); + } + + /** + * A key is filtered if one of the following three conditions is met. Otherwise, the source will call next. + *

    + *
  1. datatype miss
  2. + *
  3. key date is before the start date
  4. + *
  5. key date is after the end date
  6. + *
+ * + * @param k + * a key + * @param v + * a value + * @return a {@link FilterResult} + */ + @Override + public FilterResult filter(Key k, Value v) { + if (log.isTraceEnabled()) { + log.trace("filter key: {}", k.toStringNoTime()); + } + String cq = k.getColumnQualifier().toString(); + int index = cq.indexOf('\u0000'); + String datatype = cq.substring(0, index); + if (!datatypes.isEmpty() && !datatypes.contains(datatype)) { + return new FilterResult(false, AdvanceResult.USE_HINT); + } + + String date = cq.substring(index + 1); + if (date.compareTo(startDate) < 0) { + return new FilterResult(false, AdvanceResult.USE_HINT); + } + + if (date.compareTo(endDate) > 0) { + return new FilterResult(false, AdvanceResult.USE_HINT); + } + + return new FilterResult(true, AdvanceResult.NEXT); + } + + @Override + public Key getNextKeyHint(Key k, Value v) { + if (log.isTraceEnabled()) { + log.trace("get next hint for key: {}", k.toStringNoTime()); + } + + Key hint; + String cq = k.getColumnQualifier().toString(); + int index = cq.indexOf('\u0000'); + String datatype = cq.substring(0, index); + + if (!datatypes.isEmpty() && !datatypes.contains(datatype)) { + hint = getSeekToNextDatatypeKey(k, datatype); + } else { + String date = cq.substring(index + 1); + if (date.compareTo(startDate) < 0) { + hint = getSeekToStartDateKey(k, datatype); + } else if (date.compareTo(endDate) > 0) { + hint = getDatatypeRolloverKey(k, datatype); + } else { + hint = k.followingKey(PartialKey.ROW_COLFAM_COLQUAL); + } + } + + log.trace("hint: {}", hint); + return hint; + } + + private Key getSeekToNextDatatypeKey(Key key, String datatype) { + if (datatypes.isEmpty()) { + // no datatypes provided, so we must instead produce a 'rollover' start key + return getDatatypeRolloverKey(key, datatype); + } + + // otherwise datatypes were provided + String nextDatatype = datatypes.higher(datatype); + if (nextDatatype != null) { + log.trace("seek to next datatype"); + Text nextColumnQualifier = new Text(nextDatatype + '\u0000' + startDate); + return new Key(key.getRow(), key.getColumnFamily(), nextColumnQualifier); + } else { + log.trace("seek to next ROW_COLFAM"); + // out of datatypes, we're done. This partial range will trigger a "beyond source" condition + return key.followingKey(PartialKey.ROW_COLFAM); + } + } + + private Key getDatatypeRolloverKey(Key key, String datatype) { + log.trace("seek to rollover datatype"); + Text cq = new Text(datatype + '\u0000' + '\uffff'); + return new Key(key.getRow(), key.getColumnFamily(), cq); + } + + private Key getSeekToStartDateKey(Key k, String datatype) { + log.trace("seek to start date"); + Text cq = new Text(datatype + '\u0000' + startDate); + return new Key(k.getRow(), k.getColumnFamily(), cq); + } +} diff --git a/src/main/java/datawave/query/util/MetadataHelper.java b/src/main/java/datawave/query/util/MetadataHelper.java index d1a391c..c8189c6 100644 --- a/src/main/java/datawave/query/util/MetadataHelper.java +++ b/src/main/java/datawave/query/util/MetadataHelper.java @@ -12,12 +12,13 @@ import java.util.Date; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.SortedSet; import java.util.TimeZone; +import java.util.TreeSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -26,6 +27,7 @@ import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.BatchWriterConfig; import org.apache.accumulo.core.client.IteratorSetting; @@ -53,6 +55,7 @@ import org.springframework.stereotype.Component; import com.google.common.base.Function; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; @@ -67,6 +70,7 @@ import datawave.data.MetadataCardinalityCounts; import datawave.data.type.Type; import datawave.iterators.EdgeMetadataCombiner; +import datawave.iterators.MetadataFColumnSeekingFilter; import datawave.iterators.filter.EdgeMetadataCQStrippingIterator; import datawave.marking.MarkingFunctions; import datawave.query.composite.CompositeMetadata; @@ -1533,6 +1537,147 @@ protected HashMap getCountsByFieldInDayWithTypes(String fieldName, return datatypeToCounts; } + /** + * Get counts for each field across the date range. + *

+ * Note: it is highly recommended to use this method instead {@link #getCountsForFieldsInDateRange(Set, Set, Date, Date)}. + * + * @param fields + * the fields + * @param begin + * the start date + * @param end + * the end date + * @return a map of field counts + */ + public Map getCountsForFieldsInDateRange(Set fields, Date begin, Date end) { + return getCountsForFieldsInDateRange(fields, Collections.emptySet(), begin, end); + } + + /** + * Get counts for each field across the date range. Optionally filter by datatypes if provided. + * + * @param fields + * the fields + * @param datatypes + * the datatypes + * @param begin + * the start date + * @param end + * the end date + * @return a map of field counts + */ + public Map getCountsForFieldsInDateRange(Set fields, Set datatypes, Date begin, Date end) { + Date truncatedBegin = DateUtils.truncate(begin, Calendar.DATE); + Date truncatedEnd = DateUtils.truncate(end, Calendar.DATE); + String startDate = DateHelper.format(truncatedBegin); + String endDate = DateHelper.format(truncatedEnd); + return getCountsForFieldsInDateRange(fields, datatypes, startDate, endDate); + } + + /** + * Get counts for each field across the date range. Optionally filter by datatypes if provided. + * + * @param fields + * the fields + * @param datatypes + * the datatypes + * @param beginDate + * the start date + * @param endDate + * the end date + * @return a map of field counts + */ + public Map getCountsForFieldsInDateRange(Set fields, Set datatypes, String beginDate, String endDate) { + + SortedSet sortedDatatypes = new TreeSet<>(datatypes); + Map fieldCounts = new HashMap<>(); + Set ranges = createFieldCountRanges(fields, sortedDatatypes, beginDate, endDate); + + if (ranges.isEmpty()) { + return fieldCounts; + } + + try (BatchScanner bs = ScannerHelper.createBatchScanner(accumuloClient, getMetadataTableName(), getAuths(), fields.size())) { + + bs.setRanges(ranges); + bs.fetchColumnFamily(ColumnFamilyConstants.COLF_F); + + IteratorSetting setting = new IteratorSetting(50, "MetadataFrequencySeekingIterator", MetadataFColumnSeekingFilter.class); + setting.addOption(MetadataFColumnSeekingFilter.DATATYPES_OPT, Joiner.on(',').join(sortedDatatypes)); + setting.addOption(MetadataFColumnSeekingFilter.START_DATE, beginDate); + setting.addOption(MetadataFColumnSeekingFilter.END_DATE, endDate); + bs.addScanIterator(setting); + + for (Entry entry : bs) { + + String field = entry.getKey().getRow().toString(); + Long count = readLongFromValue(entry.getValue()); + + if (fieldCounts.containsKey(field)) { + Long existingCount = fieldCounts.get(field); + existingCount += count; + fieldCounts.put(field, existingCount); + } else { + fieldCounts.put(field, count); + } + } + + } catch (TableNotFoundException | IOException e) { + throw new RuntimeException(e); + } + return fieldCounts; + } + + /** + * Build ranges for the {@link #getCountsForFieldsInDateRange(Set, Set, String, String)} method. + *

+ * The {@link MetadataFColumnSeekingFilter} can handle a field range, but providing datatypes enables more precise ranges. + * + * @param fields + * the fields + * @param datatypes + * the datatypes + * @param beginDate + * the start date + * @param endDate + * the end date + * @return a set of ranges for the provided fields, bounded by date and optionally datatypes + */ + private Set createFieldCountRanges(Set fields, SortedSet datatypes, String beginDate, String endDate) { + Set ranges = new HashSet<>(); + for (String field : fields) { + if (datatypes.isEmpty()) { + // punt the hard work to the MetadataFColumnSeekingFilter + ranges.add(Range.exact(field, "f")); + } else { + // more precise range, the MetadataFColumnSeekingFilter will handle seeing between the first and + // last datatypes as necessary + Key start = new Key(field, "f", datatypes.first() + '\u0000' + beginDate); + Key end = new Key(field, "f", datatypes.last() + '\u0000' + endDate + '\u0000'); + ranges.add(new Range(start, true, end, false)); + } + } + return ranges; + } + + /** + * Deserialize a Value that contains a Long + * + * @param value + * an accumulo Value + * @return a long + * @throws IOException + * if there is a deserialization problem + */ + private Long readLongFromValue(Value value) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(value.get())) { + try (DataInputStream inputStream = new DataInputStream(bais)) { + return WritableUtils.readVLong(inputStream); + } + } + } + /** * Get the earliest occurrence of a field across all datatypes * diff --git a/src/test/java/datawave/iterators/MetadataFColumnSeekingFilterTest.java b/src/test/java/datawave/iterators/MetadataFColumnSeekingFilterTest.java new file mode 100644 index 0000000..e733bb5 --- /dev/null +++ b/src/test/java/datawave/iterators/MetadataFColumnSeekingFilterTest.java @@ -0,0 +1,358 @@ +package datawave.iterators; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +import java.util.Calendar; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.TimeZone; +import java.util.TreeSet; + +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.BatchScanner; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.BatchWriterConfig; +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.LongCombiner; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Joiner; + +import datawave.accumulo.inmemory.InMemoryAccumuloClient; +import datawave.accumulo.inmemory.InMemoryInstance; +import datawave.data.ColumnFamilyConstants; +import datawave.util.time.DateHelper; + +/** + * Run the seeking filter through a series of tests for the following combinations + *

+ */ +class MetadataFColumnSeekingFilterTest { + + private static final Logger log = LoggerFactory.getLogger(MetadataFColumnSeekingFilterTest.class); + + private static final String METADATA_TABLE_NAME = "DatawaveMetadata"; + private static final LongCombiner.VarLenEncoder encoder = new LongCombiner.VarLenEncoder(); + private static AccumuloClient client; + + private int expectedKeys = 0; + private long expectedCount = 0; + + private Set fields; + private Set datatypes; + private String startDate; + private String endDate; + + @BeforeAll + public static void setup() throws Exception { + InMemoryInstance instance = new InMemoryInstance(MetadataFColumnSeekingFilterTest.class.getName()); + client = new InMemoryAccumuloClient("", instance); + client.tableOperations().create(METADATA_TABLE_NAME); + writeData(); + } + + private static void writeData() throws Exception { + + BatchWriterConfig config = new BatchWriterConfig(); + try (BatchWriter bw = client.createBatchWriter(METADATA_TABLE_NAME, config)) { + + Set fields = Set.of("FIELD_A", "FIELD_B", "FIELD_C", "FIELD_D"); + Set datatypes = Set.of("datatype-a", "datatype-b"); + + Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + cal.setTime(DateHelper.parse("20240501")); + for (int i = 0; i < 15; i++) { + for (String field : fields) { + for (String datatype : datatypes) { + String date = DateHelper.format(cal.getTime()); + String cq = datatype + '\u0000' + date; + write(bw, field, "f", cq, createValue(i + 1)); + } + } + cal.add(Calendar.DAY_OF_MONTH, 1); + } + } + } + + private static void write(BatchWriter bw, String row, String cf, String cq, Value value) throws Exception { + Mutation m = new Mutation(row); + m.put(cf, cq, value); + bw.addMutation(m); + } + + private static Value createValue(long count) { + return new Value(encoder.encode(count)); + } + + @BeforeEach + public void beforeEach() { + expectedCount = -1L; + expectedKeys = -1; + fields = null; + datatypes = null; + startDate = null; + endDate = null; + } + + @Test + public void testSingleField_SingleDay_ZeroDatatype() throws Exception { + withExpectedKeysAndCount(2, 10L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of()); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testSingleField_SingleDay_SingleDatatype() throws Exception { + withExpectedKeysAndCount(1, 5L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of("datatype-a")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testSingleField_SingleDay_MultiDatatype() throws Exception { + withExpectedKeysAndCount(2, 10L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of("datatype-a", "datatype-b")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testSingleField_SingleDay_ExtraDatatype() throws Exception { + withExpectedKeysAndCount(2, 10L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of("datatype-a", "datatype-b", "datatype-c")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testSingleField_MultiDay_ZeroDatatype() throws Exception { + withExpectedKeysAndCount(12, 90L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of()); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testSingleField_MultiDay_SingleDatatype() throws Exception { + withExpectedKeysAndCount(6, 45L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of("datatype-a")); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testSingleField_MultiDay_MultiDatatype() throws Exception { + withExpectedKeysAndCount(12, 90L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of("datatype-a", "datatype-b")); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testSingleField_MultiDay_ExtraDatatype() throws Exception { + withExpectedKeysAndCount(12, 90L); + withFieldsAndDatatypes(Set.of("FIELD_A"), Set.of("datatype-a", "datatype-b", "datatype-c")); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testMultiField_SingleDay_ZeroDatatype() throws Exception { + withExpectedKeysAndCount(4, 20L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_B"), Set.of()); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testMultiField_SingleDay_SingleDatatype() throws Exception { + withExpectedKeysAndCount(2, 10L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_B"), Set.of("datatype-a")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testMultiField_SingleDay_MultiDatatype() throws Exception { + withExpectedKeysAndCount(4, 20L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_B"), Set.of("datatype-a", "datatype-b")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testMultiField_SingleDay_ExtraDatatype() throws Exception { + withExpectedKeysAndCount(4, 20L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_B"), Set.of("datatype-a", "datatype-b", "datatype-c")); + withDates("20240505", "20240505"); + test(); + } + + // multi field with a gap + + @Test + public void testMultiFieldWithGap_SingleDay_ZeroDatatype() throws Exception { + withExpectedKeysAndCount(4, 20L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_C"), Set.of()); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testMultiFieldWithGap_SingleDay_SingleDatatype() throws Exception { + withExpectedKeysAndCount(2, 10L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_C"), Set.of("datatype-a")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testMultiFieldWithGap_SingleDay_MultiDatatype() throws Exception { + withExpectedKeysAndCount(4, 20L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_C"), Set.of("datatype-a", "datatype-b")); + withDates("20240505", "20240505"); + test(); + } + + @Test + public void testMultiFieldGap_SingleDay_ExtraDatatype() throws Exception { + withExpectedKeysAndCount(4, 20L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_C"), Set.of("datatype-a", "datatype-b", "datatype-c")); + withDates("20240505", "20240505"); + test(); + } + + // tests for field that doesn't exist + + @Test + public void testSingleFieldNoField_MultiDay_ZeroDatatype() throws Exception { + withExpectedKeysAndCount(12, 90L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_Z"), Set.of()); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testSingleFieldNoField_MultiDay_SingleDatatype() throws Exception { + withExpectedKeysAndCount(6, 45L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_Z"), Set.of("datatype-a")); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testSingleFieldNoField_MultiDay_MultiDatatype() throws Exception { + withExpectedKeysAndCount(12, 90L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_Z"), Set.of("datatype-a", "datatype-b")); + withDates("20240505", "20240510"); + test(); + } + + @Test + public void testSingleFieldNoField_MultiDay_ExtraDatatype() throws Exception { + withExpectedKeysAndCount(12, 90L); + withFieldsAndDatatypes(Set.of("FIELD_A", "FIELD_Z"), Set.of("datatype-a", "datatype-b", "datatype-c")); + withDates("20240505", "20240510"); + test(); + } + + private void test() throws Exception { + assertNotEquals(-1, expectedCount, "expected count must be non-negative"); + assertNotEquals(-1, expectedKeys, "expected keys must be non-negative"); + assertNotNull(fields, "fields must not be null"); + assertNotNull(datatypes, "datatypes must not be null"); + assertNotNull(startDate, "start date must be non-null"); + assertNotNull(endDate, "end date must be non-null"); + + Set ranges = createExactFieldRanges(); + scanRanges(ranges); + + if (!datatypes.isEmpty()) { + ranges = createDateBoundedRanges(); + scanRanges(ranges); + } + } + + private void scanRanges(Collection ranges) throws Exception { + try (BatchScanner scanner = client.createBatchScanner(METADATA_TABLE_NAME)) { + scanner.setRanges(ranges); + scanner.fetchColumnFamily(ColumnFamilyConstants.COLF_F); + + IteratorSetting setting = new IteratorSetting(50, "MetadataFColumnSeekingFilter", MetadataFColumnSeekingFilter.class); + setting.addOption(MetadataFColumnSeekingFilter.DATATYPES_OPT, Joiner.on(',').join(datatypes)); + setting.addOption(MetadataFColumnSeekingFilter.START_DATE, startDate); + setting.addOption(MetadataFColumnSeekingFilter.END_DATE, endDate); + scanner.addScanIterator(setting); + + int keys = 0; + int count = 0; + for (Map.Entry entry : scanner) { + log.debug("tk: {}", entry.getKey()); + + keys++; + count += encoder.decode(entry.getValue().get()); + } + + assertEquals(expectedKeys, keys); + assertEquals(expectedCount, count); + } + } + + private Set createExactFieldRanges() { + Set ranges = new HashSet<>(); + for (String field : fields) { + ranges.add(Range.exact(field)); + } + return ranges; + } + + private Set createDateBoundedRanges() { + assertNotNull(datatypes, "cannot build date bounded ranges with null datatypes"); + assertFalse(datatypes.isEmpty(), "cannot build date bounded ranges with empty datatypes"); + + Set ranges = new HashSet<>(); + TreeSet sortedTypes = new TreeSet<>(datatypes); + for (String field : fields) { + // build a range bounded by both datatype and date + // the MetadataFColumnSeekingFilter will handle any other datatypes that fall inside the bounds + Key start = new Key(field, "f", sortedTypes.first() + '\u0000' + startDate); + Key end = new Key(field, "f", sortedTypes.last() + '\u0000' + endDate + '\u0000'); + ranges.add(new Range(start, true, end, false)); + } + return ranges; + } + + private void withFieldsAndDatatypes(Set fields, Set datatypes) { + this.fields = fields; + this.datatypes = datatypes; + } + + private void withDates(String startDate, String endDate) { + this.startDate = startDate; + this.endDate = endDate; + } + + private void withExpectedKeysAndCount(int expectedKeys, long expectedCount) { + this.expectedKeys = expectedKeys; + this.expectedCount = expectedCount; + } +} diff --git a/src/test/java/datawave/query/util/MetadataHelperTableTest.java b/src/test/java/datawave/query/util/MetadataHelperTableTest.java index 006224a..efc6679 100644 --- a/src/test/java/datawave/query/util/MetadataHelperTableTest.java +++ b/src/test/java/datawave/query/util/MetadataHelperTableTest.java @@ -812,6 +812,40 @@ public void testGetMetadataTableName() { assertEquals(METADATA_TABLE_NAME, helper.getMetadataTableName()); } + @Test + public void testGetCountsForFieldsInDateRange_SingleFieldSingleDatatypeSingleDayRange() { + // presuming a query like "SHAPE == 'foo' && COLOR == 'bar'" + Set fields = Set.of("SHAPE"); + Set datatypes = Set.of("datatype-a"); + Map counts = helper.getCountsForFieldsInDateRange(fields, datatypes, "20240302", "20240302"); + + assertTrue(counts.containsKey("SHAPE")); + assertEquals(14L, counts.get("SHAPE")); + } + + @Test + public void testGetCountsForFieldsInDateRange_MultiFieldMultiDatatypeSingleDayRange() { + // presuming a query like "SHAPE == 'foo' && COLOR == 'bar'" + Set fields = Set.of("SHAPE", "DEFINITION"); + Set datatypes = Set.of("datatype-a", "datatype-b"); + Map counts = helper.getCountsForFieldsInDateRange(fields, datatypes, "20240302", "20240302"); + + assertTrue(counts.containsKey("SHAPE")); + assertTrue(counts.containsKey("DEFINITION")); + assertEquals(14L, counts.get("SHAPE")); + assertEquals(14L, counts.get("DEFINITION")); + } + + @Test + public void testGetCountsForFieldsInDateRange_SingleFieldNoDatatypesMultiDayRange() { + // presuming a query like "SHAPE == 'foo' && COLOR == 'bar'" + Set fields = Set.of("SHAPE"); + Map counts = helper.getCountsForFieldsInDateRange(fields, Set.of(), "20240302", "20240304"); + + assertTrue(counts.containsKey("SHAPE")); + assertEquals(536L, counts.get("SHAPE")); + } + /** * Assert that an iterable matches expectations *