diff --git a/core/utils/metadata-utils/src/main/java/datawave/query/util/MetadataHelper.java b/core/utils/metadata-utils/src/main/java/datawave/query/util/MetadataHelper.java index 50d946112c9..2658a51056e 100644 --- a/core/utils/metadata-utils/src/main/java/datawave/query/util/MetadataHelper.java +++ b/core/utils/metadata-utils/src/main/java/datawave/query/util/MetadataHelper.java @@ -5,6 +5,7 @@ import java.io.IOException; import java.nio.charset.CharacterCodingException; import java.time.format.DateTimeParseException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Calendar; import java.util.Collection; @@ -40,6 +41,7 @@ import org.apache.accumulo.core.data.PartialKey; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.FirstEntryInRowIterator; import org.apache.accumulo.core.iterators.ValueFormatException; import org.apache.accumulo.core.iterators.user.RegExFilter; import org.apache.accumulo.core.iterators.user.SummingCombiner; @@ -1692,6 +1694,105 @@ public Map getCountsForFieldsInDateRange(Set fields, Set getMissingFieldsInDateRange(Set fields, Set datatypes, String beginDate, String endDate, Set specialFields) { + SortedSet sortedDatatypes = new TreeSet<>(datatypes); + Set foundFields = new HashSet<>(); + fields = Sets.difference(fields, specialFields); + Set ranges = createExactFieldCountRanges(fields); + StringBuilder dataTypeRegex = new StringBuilder(); + List settings = new ArrayList<>(); + + if (ranges.isEmpty()) { + return Collections.emptySet(); + } + + int index = 0; + for (String dataType : sortedDatatypes) { + if (index < sortedDatatypes.size() - 1) { + dataTypeRegex.append(dataType).append("\u0000.*").append("|"); + index++; + } else { + dataTypeRegex.append(dataType).append("\u0000.*"); + } + } + + AccumuloClient client = accumuloClient; + if (client instanceof WrappedAccumuloClient) { + client = ((WrappedAccumuloClient) client).getReal(); + } + + try (BatchScanner bs = ScannerHelper.createBatchScanner(client, getMetadataTableName(), getAuths(), fields.size())) { + if (!datatypes.isEmpty()) { + IteratorSetting regexFilter = new IteratorSetting(50, "regexFilter", RegExFilter.class); + regexFilter.addOption(RegExFilter.COLQ_REGEX, dataTypeRegex.toString()); + settings.add(regexFilter); + } + + settings.add(new IteratorSetting(51, "firstEntryInRow", FirstEntryInRowIterator.class)); + bs.fetchColumnFamily(ColumnFamilyConstants.COLF_F); + bs.setRanges(ranges); + + for (IteratorSetting setting : settings) { + bs.addScanIterator(setting); + } + + for (Entry entry : bs) { + Text colq = entry.getKey().getColumnQualifier(); + int colqIndex = colq.find(NULL_BYTE); + + String remainder; + try { + remainder = Text.decode(colq.getBytes(), colqIndex + 1, colq.getLength() - (colqIndex + 1)); + } catch (CharacterCodingException e) { + log.warn("Could not deserialize colqual: {} ", entry.getKey()); + continue; + } + if (remainder.equals(FrequencyMetadataAggregator.AGGREGATED)) { + // This is an aggregated entry. + try { + DateFrequencyMap map = new DateFrequencyMap(entry.getValue().get()); + if (!map.subMap(beginDate, endDate).isEmpty()) { + foundFields.add(entry.getKey().getRow().toString()); + } + } catch (IOException e) { + log.error("Failed to convert Value to DateFrequencyMap", e); + } + } else { + // This is an entry with a count for a single date. + try { + Date date = DateHelper.parse(remainder); + // Add the field if we fall within beginDate and endDate, inclusively. + if (date.compareTo(DateHelper.parse(beginDate)) >= 0 && date.compareTo(DateHelper.parse(endDate)) <= 0) { + foundFields.add(entry.getKey().getRow().toString()); + } + } catch (ValueFormatException e) { + log.warn("Could not convert the Value to a long: {}", entry.getValue()); + } catch (DateTimeParseException e) { + log.warn("Could not convert date string: {}", remainder); + } + } + } + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + return Sets.difference(fields, foundFields); + } + /** * Build ranges for the {@link #getCountsForFieldsInDateRange(Set, Set, String, String)} method. *

@@ -1724,6 +1825,21 @@ private Set createFieldCountRanges(Set fields, SortedSet return ranges; } + /** + * Build ranges for the {@link #getMissingFieldsInDateRange(Set, Set, String, String, Set)} method. + * + * @param fields + * the fields + * @return a set of exact ranges for the provided fields. + */ + private Set createExactFieldCountRanges(Set fields) { + Set ranges = new HashSet<>(); + for (String field : fields) { + ranges.add(Range.exact(field, "f")); + } + return ranges; + } + /** * Deserialize a Value that contains a Long * diff --git a/core/utils/metadata-utils/src/test/java/datawave/query/util/MetadataHelperTest.java b/core/utils/metadata-utils/src/test/java/datawave/query/util/MetadataHelperTest.java index 00c16c98af5..94d0148142b 100644 --- a/core/utils/metadata-utils/src/test/java/datawave/query/util/MetadataHelperTest.java +++ b/core/utils/metadata-utils/src/test/java/datawave/query/util/MetadataHelperTest.java @@ -392,4 +392,100 @@ void testMixedEntryFormats() { Assertions.assertEquals(DateHelper.parse("20200103"), helper.getEarliestOccurrenceOfFieldWithType("NAME", "maze", accumuloClient, null)); } } + + /** + * Tests for {@link MetadataHelper#getMissingFieldsInDateRange(Set, Set, String, String, Set)}. + */ + @Nested + public class GetMissingFieldsInDateRangeTest { + /** + * Test against a table that has only non-aggregated entries as matches. + */ + @Test + void testNonAggregatedEntriesOnly() throws TableNotFoundException { + givenNonAggregatedFrequencyRows("NAME", COLF_F, "csv", "20200103", "20200120", 1L); + givenNonAggregatedFrequencyRows("NAME", COLF_F, "wiki", "20200101", "20200120", 2L); + givenNonAggregatedFrequencyRows("NAME", COLF_F, "maze", "20200105", "20200120", 3L); + givenNonAggregatedFrequencyRows("NAME", COLF_F, "data", "20200107", "20200102", 3L); + givenNonAggregatedFrequencyRows("EVENT_DATE", COLF_F, "csv", "20200101", "20200120", 4L); + givenNonAggregatedFrequencyRows("EVENT_DATE", COLF_F, "wiki", "20200101", "20200120", 5L); + givenNonAggregatedFrequencyRows("EVENT_DATE", COLF_F, "maze", "20200101", "20200120", 6L); + writeMutations(); + + // No DataTypes + Assertions.assertEquals(Collections.emptySet(), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Collections.emptySet(), "20200101", + "20200120", Collections.emptySet())); + // Using DataTypes + Assertions.assertEquals(Set.of("EVENT_DATE"), + helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Set.of("data"), "20200101", "20200120", Collections.emptySet())); + // Fictitious field + Assertions.assertEquals(Set.of("FOO"), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE", "FOO"), + Set.of("wiki", "data", "csv", "maze"), "20200101", "20200120", Collections.emptySet())); + // Missing because of date range + Assertions.assertEquals(Set.of("NAME", "EVENT_DATE"), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Set.of("wiki", "data"), + "20190101", "20191231", Collections.emptySet())); + } + + /** + * Test against a table that has only aggregated entries as matches. + */ + @Test + void testAggregatedEntriesOnly() throws TableNotFoundException { + givenAggregatedFrequencyRow("NAME", COLF_F, "csv", createDateFrequencyMap("20200113", 1L, "20200115", 5L, "20200116", 3L)); + givenAggregatedFrequencyRow("NAME", COLF_F, "wiki", createDateFrequencyMap("20200111", 1L, "20200112", 15L, "20200113", 3L)); + givenAggregatedFrequencyRow("NAME", COLF_F, "maze", createDateFrequencyMap("20200102", 1L, "20200104", 55L, "20200105", 3L)); + givenAggregatedFrequencyRow("NAME", COLF_F, "data", createDateFrequencyMap("20200101", 1L, "20200103", 3L)); + givenAggregatedFrequencyRow("EVENT_DATE", COLF_F, "csv", createDateFrequencyMap("20200101", 2L, "20200102", 3L, "20200103", 4L)); + givenAggregatedFrequencyRow("EVENT_DATE", COLF_F, "wiki", createDateFrequencyMap("20200101", 2L, "20200102", 3L, "20200103", 4L)); + givenAggregatedFrequencyRow("EVENT_DATE", COLF_F, "maze", createDateFrequencyMap("20200101", 2L, "20200102", 3L, "20200103", 4L)); + writeMutations(); + + // No DataTypes + Assertions.assertEquals(Collections.emptySet(), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Collections.emptySet(), "20200101", + "20200120", Collections.emptySet())); + // Using DataTypes + Assertions.assertEquals(Set.of("EVENT_DATE"), + helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Set.of("data"), "20200101", "20200120", Collections.emptySet())); + // Fictitious field + Assertions.assertEquals(Set.of("FOO"), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE", "FOO"), + Set.of("wiki", "data", "csv", "maze"), "20200101", "20200120", Collections.emptySet())); + // Missing because of date range + Assertions.assertEquals(Set.of("NAME", "EVENT_DATE"), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Set.of("wiki", "data"), + "20190101", "20191231", Collections.emptySet())); + } + + /** + * Test against a table that has both aggregated and non-aggregated entries as matches. + */ + @Test + void testMixedEntryFormats() throws TableNotFoundException { + givenAggregatedFrequencyRow("NAME", COLF_F, "csv", createDateFrequencyMap("20200111", 1L, "20200112", 5L, "20200113", 3L)); + givenNonAggregatedFrequencyRows("NAME", COLF_F, "csv", "20200111", "20200120", 1L); + givenAggregatedFrequencyRow("NAME", COLF_F, "wiki", createDateFrequencyMap("20200111", 1L, "20200112", 15L, "20200113", 3L)); + givenAggregatedFrequencyRow("NAME", COLF_F, "maze", createDateFrequencyMap("20200111", 1L, "20200112", 55L, "20200113", 3L)); + givenNonAggregatedFrequencyRows("NAME", COLF_F, "maze", "20200103", "20200120", 3L); + givenAggregatedFrequencyRow("NAME", COLF_F, "data", createDateFrequencyMap("20200111", 1L, "20200113", 3L)); + givenNonAggregatedFrequencyRows("NAME", COLF_F, "data", "20200101", "20200115", 3L); + givenAggregatedFrequencyRow("EVENT_DATE", COLF_F, "csv", createDateFrequencyMap("20200101", 2L, "20200102", 3L, "20200103", 4L)); + givenAggregatedFrequencyRow("EVENT_DATE", COLF_F, "wiki", createDateFrequencyMap("20200101", 2L, "20200102", 3L, "20200103", 4L)); + givenAggregatedFrequencyRow("EVENT_DATE", COLF_F, "maze", createDateFrequencyMap("20200101", 2L, "20200102", 3L, "20200103", 4L)); + givenNonAggregatedFrequencyRows("EVENT_DATE", COLF_F, "csv", "20200101", "20200120", 4L); + givenNonAggregatedFrequencyRows("EVENT_DATE", COLF_F, "wiki", "20200101", "20200120", 5L); + givenNonAggregatedFrequencyRows("EVENT_DATE", COLF_F, "maze", "20200101", "20200120", 6L); + writeMutations(); + + // No DataTypes + Assertions.assertEquals(Collections.emptySet(), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Collections.emptySet(), "20200101", + "20200120", Collections.emptySet())); + // Using DataTypes + Assertions.assertEquals(Set.of("EVENT_DATE"), + helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Set.of("data"), "20200101", "20200120", Collections.emptySet())); + // Fictitious field + Assertions.assertEquals(Set.of("FOO"), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE", "FOO"), + Set.of("wiki", "data", "csv", "maze"), "20200101", "20200120", Collections.emptySet())); + // Missing because of date range + Assertions.assertEquals(Set.of("NAME", "EVENT_DATE"), helper.getMissingFieldsInDateRange(Set.of("NAME", "EVENT_DATE"), Set.of("wiki", "data"), + "20190101", "20191231", Collections.emptySet())); + } + } } diff --git a/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/FieldMissingFromDateRangeVisitor.java b/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/FieldMissingFromDateRangeVisitor.java new file mode 100644 index 00000000000..54965125a4e --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/FieldMissingFromDateRangeVisitor.java @@ -0,0 +1,259 @@ +package datawave.query.jexl.visitors; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.commons.jexl3.parser.ASTEQNode; +import org.apache.commons.jexl3.parser.ASTERNode; +import org.apache.commons.jexl3.parser.ASTFunctionNode; +import org.apache.commons.jexl3.parser.ASTGENode; +import org.apache.commons.jexl3.parser.ASTGTNode; +import org.apache.commons.jexl3.parser.ASTIdentifier; +import org.apache.commons.jexl3.parser.ASTJexlScript; +import org.apache.commons.jexl3.parser.ASTLENode; +import org.apache.commons.jexl3.parser.ASTLTNode; +import org.apache.commons.jexl3.parser.ASTNENode; +import org.apache.commons.jexl3.parser.ASTNRNode; +import org.apache.commons.jexl3.parser.ASTOrNode; +import org.apache.commons.jexl3.parser.JexlNode; +import org.apache.hadoop.util.Sets; + +import datawave.query.jexl.JexlASTHelper; +import datawave.query.jexl.functions.JexlFunctionArgumentDescriptorFactory; +import datawave.query.jexl.functions.arguments.JexlArgumentDescriptor; +import datawave.query.util.MetadataHelper; +import datawave.util.time.DateHelper; + +/** + * Class to check that each query node contains a field which exists in the schema for the given date range. + * + *

+ * 1. If a datatype filter was specified, then the existence check is limited to only those datatypes
+ * 2. If a datatype filter is NOT specified (null or empty), this implies ALL datatypes.
+ * 
+ */ +public class FieldMissingFromDateRangeVisitor extends ShortCircuitBaseVisitor { + + private final MetadataHelper helper; + private final Set datatypeFilter; + + public FieldMissingFromDateRangeVisitor(MetadataHelper helper, Set datatypeFilter) { + this.helper = helper; + // if given datatypeFilter is empty or null, assume that means ALL datatypes + if (datatypeFilter == null) { + datatypeFilter = Collections.emptySet(); + } + this.datatypeFilter = datatypeFilter; + } + + @SuppressWarnings("unchecked") + public static Set getNonIngestedFields(MetadataHelper helper, ASTJexlScript script, Set datatypes, Set specialFields, + Date beginDate, Date endDate) { + if (datatypes == null) { + datatypes = Collections.emptySet(); + } + + // Collect the fields + FieldMissingFromDateRangeVisitor visitor = new FieldMissingFromDateRangeVisitor(helper, datatypes); + List> fieldSets = (List>) script.jjtAccept(visitor, new ArrayList<>()); + + if (fieldSets.isEmpty()) { + return Set.of(); + } + + // @formatter:off + Set allFields = fieldSets.stream() + .flatMap(Set::stream) + .collect(Collectors.toSet()); + // @formatter:on + + // If only special field were found, do not return any fields. + if (specialFields.containsAll(allFields)) { + return Set.of(); + } + + // Fetch all fields not found in the target date range. + Set missingFields = visitor.helper.getMissingFieldsInDateRange(allFields, datatypes, DateHelper.format(beginDate), DateHelper.format(endDate), + specialFields); + + // @formatter:off + return fieldSets.stream() + .filter((set) -> !set.isEmpty()) + .filter((set) -> Sets.intersection(specialFields, set).isEmpty()) + .filter(missingFields::containsAll) + .flatMap(Set::stream) + .collect(Collectors.toCollection(LinkedHashSet::new)); + // @formatter:on + } + + // Collect all field names in given node. + private Object collectFields(JexlNode node, Object data) throws TableNotFoundException { + // If data is a set, the node is a descendant of an OR node. + if (data instanceof Set) { + // noinspection unchecked + addFields(node, (Set) data); + } else { + // Otherwise it is a top-level node. Create a new set and collect all fields into it. + @SuppressWarnings("unchecked") + List> fieldSets = (List>) data; + Set fields = new LinkedHashSet<>(); + collectFields(node, fields); + fieldSets.add(fields); + } + return data; + } + + // All fields from the given node to the given candidate node. + private void addFields(JexlNode node, Set data) { + List identifiers; + + // A node could be literal == literal in terms of an identityQuery + try { + identifiers = JexlASTHelper.getIdentifiers(node); + } catch (NoSuchElementException e) { + return; + } + + if (identifiers.isEmpty()) { + // Catch cases where we have two literals + // essentially everything but identifier op literal + return; + } + + for (ASTIdentifier identifier : identifiers) { + String fieldName = JexlASTHelper.deconstructIdentifier(identifier); + data.add(fieldName); + } + } + + @Override + public Object visit(ASTERNode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTNRNode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTEQNode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTNENode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTGENode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTGTNode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTLENode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTLTNode node, Object data) { + try { + return collectFields(node, data); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object visit(ASTFunctionNode node, Object data) { + // If data is a set, this function node is a descendant of an OR node. + if (data instanceof Set) { + // noinspection unchecked + addFields(node, (Set) data); + } else { + // Otherwise this is a top-level node. Create a new set of fields and collect all fields from the function node. + @SuppressWarnings("unchecked") + List> fieldSets = (List>) data; + Set fields = new LinkedHashSet<>(); + addFields(node, fields); + fieldSets.add(fields); + } + return data; + } + + // Add all fields found within the given function node to the given set. + private void addFields(ASTFunctionNode node, Set fields) { + JexlArgumentDescriptor desc = JexlFunctionArgumentDescriptorFactory.F.getArgumentDescriptor(node); + for (String fieldName : desc.fields(this.helper, this.datatypeFilter)) { + fieldName = JexlASTHelper.deconstructIdentifier(fieldName); + fields.add(fieldName); + } + } + + // Descend through these nodes + @Override + public Object visit(ASTJexlScript node, Object data) { + node.childrenAccept(this, data); + return data; + } + + @Override + public Object visit(ASTOrNode node, Object data) { + // If data is a set, this OR node is nested within another OR node. Pass it along to its children. + if (data instanceof Set) { + node.childrenAccept(this, data); + } else { + // Otherwise this is a top-level OR node. Create a new set and collect all fields from the node's children. + @SuppressWarnings("unchecked") + List> fieldSets = (List>) data; + Set fields = new LinkedHashSet<>(); + node.childrenAccept(this, fields); + if (!fields.isEmpty()) { + fieldSets.add(fields); + } + } + return data; + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/rules/FieldExistenceRule.java b/warehouse/query-core/src/main/java/datawave/query/rules/FieldExistenceRule.java index dd21328e50e..addd4fce256 100644 --- a/warehouse/query-core/src/main/java/datawave/query/rules/FieldExistenceRule.java +++ b/warehouse/query-core/src/main/java/datawave/query/rules/FieldExistenceRule.java @@ -9,7 +9,10 @@ import org.apache.commons.jexl3.parser.ASTJexlScript; import org.apache.log4j.Logger; +import datawave.microservice.query.Query; +import datawave.query.jexl.visitors.FieldMissingFromDateRangeVisitor; import datawave.query.jexl.visitors.FieldMissingFromSchemaVisitor; +import datawave.util.time.DateHelper; /** * A {@link QueryRule} implementation that will check a query for any non-existent fields, i.e. not present in the data dictionary. @@ -81,6 +84,15 @@ public QueryRuleResult validate(QueryValidationConfiguration ruleConfiguration) if (!nonExistentFields.isEmpty()) { result.addMessage("Fields not found in data dictionary: " + String.join(", ", nonExistentFields)); } + + // Find all OR branches in the query that consist entirely of fields not present within the query's date range. + Query settings = ruleConfig.getQuerySettings(); + Set missingFields = FieldMissingFromDateRangeVisitor.getNonIngestedFields(ruleConfig.getMetadataHelper(), jexlQuery, Collections.emptySet(), + getSpecialFields(), settings.getBeginDate(), settings.getEndDate()); + if (!missingFields.isEmpty()) { + result.addMessage("There is no data in the following fields " + missingFields + " for the entire date range " + + DateHelper.format(settings.getBeginDate()) + " to " + DateHelper.format(settings.getEndDate()) + "."); + } } catch (Exception e) { // If an exception occurred, log and preserve it in the result. log.error("Error occurred when validating against instance '" + getName() + "' of " + getClass(), e); diff --git a/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/FieldMissingFromDateRangeVisitorTest.java b/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/FieldMissingFromDateRangeVisitorTest.java new file mode 100644 index 00000000000..476e46afb18 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/FieldMissingFromDateRangeVisitorTest.java @@ -0,0 +1,277 @@ +package datawave.query.jexl.visitors; + +import static datawave.query.Constants.ANY_FIELD; +import static datawave.query.Constants.NO_FIELD; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Set; + +import org.apache.commons.jexl3.parser.ASTJexlScript; +import org.apache.commons.jexl3.parser.ParseException; +import org.easymock.EasyMock; +import org.easymock.EasyMockExtension; +import org.easymock.Mock; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import datawave.query.jexl.JexlASTHelper; +import datawave.query.util.MetadataHelper; +import datawave.util.time.DateHelper; + +@ExtendWith(EasyMockExtension.class) +public class FieldMissingFromDateRangeVisitorTest { + + private static final String beginDate = "20201231"; + private static final String endDate = "20251231"; + private static final Set specialFields = Set.of(ANY_FIELD, NO_FIELD); + + @Mock + private MetadataHelper helper; + + private String query; + + private final Set dataTypes = new HashSet<>(); + private final Set expectedImmaterialFields = new LinkedHashSet<>(); + + private boolean helperMocked = false; + + @AfterEach + void tearDown() { + this.query = null; + this.dataTypes.clear(); + this.expectedImmaterialFields.clear(); + } + + /** + * Test query with ANDed fields where all exist during the date range. + */ + @Test + public void testWithAndFieldsThatAllExist() throws ParseException { + givenQuery("AGE == 'foo' && GENDER == 'bar'"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER"), Set.of()); + + assertResult(); + } + + /** + * Test query with multiple ORed fields that all exist during the date range. + */ + @Test + public void testWithORFieldsThatAllExist() throws ParseException { + givenQuery("AGE == 'foo' || GENDER == 'bar' || JOB == 'foo'"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of()); + + assertResult(); + } + + /** + * Test query with ANDed fields but only some exist during the date range. + */ + @Test + public void testWithAndFieldsThatSomeExist() throws ParseException { + givenQuery("AGE == 'foo' && GENDER == 'bar' && JOB == 'foo'"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("JOB")); + + expectImmaterialFields("JOB"); + + assertResult(); + } + + /** + * Test query with ANDed fields but only some exist during the date range with a datatype filter. + */ + @Test + public void testWithAndFieldsThatSomeExistWithNonEmptyDataTypes() throws ParseException { + givenQuery("AGE == 'foo' && GENDER == 'bar' && JOB == 'foo'"); + givenDataTypes("attr"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("JOB")); + + expectImmaterialFields("JOB"); + + assertResult(); + } + + /** + * Test query with only special fields ANDed. + */ + @Test + public void testAndWithOnlySpecialFields() throws ParseException { + givenQuery("_ANYFIELD_ == 'foo' && _NOFIELD_ == 'bar'"); + + assertResult(); + } + + /** + * Test query with only special fields ORd. + */ + @Test + public void testOrWithOnlySpecialFields() throws ParseException { + givenQuery("_ANYFIELD_ == 'foo' || _NOFIELD_ == 'bar'"); + + assertResult(); + } + + /** + * Test query with ORed fields but only some exist during the date range. Fields will only be returned as NonIngested if ALL ORed fields are not found. + */ + @Test + public void testWithOrFieldsThatSomeExist() throws ParseException { + givenQuery("AGE == 'foo' || GENDER == 'bar' || JOB == 'foo'"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("AGE", "GENDER")); + + assertResult(); + } + + /** + * Test query with ORed fields but ALL do not exist during the date range. Fields will only be returned as NonIngested if ALL ORed fields are not found. + */ + @Test + public void testWithOrFieldsThatAllDoNotExist() throws ParseException { + givenQuery("AGE == 'foo' || GENDER == 'bar' || JOB == 'foo'"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("AGE", "GENDER", "JOB")); + + expectImmaterialFields("AGE", "GENDER", "JOB"); + + assertResult(); + } + + /** + * Test query with function for a field that exists in the date range. + */ + @Test + public void testRegexFunctionWithFieldThatExists() throws ParseException { + givenQuery("filter:includeRegex(GENDER, 'bar.*')"); + + expectHelperToBeCalled(Set.of("GENDER"), Set.of()); + + assertResult(); + } + + /** + * Test query with function for a special field. + */ + @Test + public void testRegexFunctionWithSpecialField() throws ParseException { + givenQuery("filter:includeRegex(_ANYFIELD_, 'bar.*')"); + + assertResult(); + } + + /** + * Test query with function for a field that does not exist in the date range. + */ + @Test + public void testRegexFunctionWithFieldThatDoesNotExist() throws ParseException { + givenQuery("filter:includeRegex(JOB, 'bar.*')"); + + expectHelperToBeCalled(Set.of("JOB"), Set.of("JOB")); + + expectImmaterialFields("JOB"); + + assertResult(); + } + + @Test + void testNestedFunction() throws ParseException { + givenQuery("AGE == 'foo' && (NAME == 'bar' || filter:includeRegex(JOB, 'bar.*'))"); + + expectHelperToBeCalled(Set.of("AGE", "NAME", "JOB"), Set.of("NAME", "JOB")); + + expectImmaterialFields("JOB", "NAME"); + + assertResult(); + } + + @Test + public void testNestedIntersectionWithSomeMissingFields() throws ParseException { + givenQuery("AGE == 'foo' || (GENDER == 'bar' && JOB == 'foo')"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("GENDER", "JOB")); + + assertResult(); + } + + @Test + public void testNestedUnion() throws ParseException { + givenQuery("AGE == 'foo' && (GENDER == 'bar' || JOB == 'foo')"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("GENDER", "JOB")); + + expectImmaterialFields("GENDER", "JOB"); + + assertResult(); + } + + @Test + public void testDoubleNestedIntersectionWhereSomeMissing() throws ParseException { + givenQuery("(AGE == 'foo' && GENDER == 'foo') || (GENDER == 'bar' && JOB == 'foo')"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("AGE", "GENDER")); + + assertResult(); + } + + @Test + public void testDoubleNestedIntersectionWhereAllMissing() throws ParseException { + givenQuery("(AGE == 'foo' && GENDER == 'foo') || (GENDER == 'bar' && JOB == 'foo')"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "JOB"), Set.of("AGE", "GENDER", "JOB")); + + expectImmaterialFields("AGE", "GENDER", "JOB"); + + assertResult(); + } + + @Test + public void testDoubleNestedUnionWithSomeORsMissingAllFields() throws ParseException { + givenQuery("(AGE == 'foo' || GENDER == 'foo') && (NAME == 'bar' || JOB == 'foo') && (ORG == 'hr' || NAME == 'hat')"); + + expectHelperToBeCalled(Set.of("AGE", "GENDER", "NAME", "JOB", "ORG"), Set.of("AGE", "GENDER", "NAME", "JOB")); + + expectImmaterialFields("AGE", "GENDER", "NAME", "JOB"); + + assertResult(); + } + + private void assertResult() throws ParseException { + // If we expect any fields to be found, mock up a call to the helper function. + ASTJexlScript script = JexlASTHelper.parseJexlQuery(query); + Set actualImmaterialFields = FieldMissingFromDateRangeVisitor.getNonIngestedFields(this.helper, script, dataTypes, specialFields, + DateHelper.parse(beginDate), DateHelper.parse(endDate)); + + if (helperMocked) { + // If the helper function was mocked, verify that the arguments to the function matched what we expected. + EasyMock.verify(this.helper); + } + + assertThat(actualImmaterialFields).isEqualTo(expectedImmaterialFields); + } + + private void givenQuery(String query) { + this.query = query; + } + + private void expectHelperToBeCalled(Set foundFields, Set missingFields) { + EasyMock.expect(this.helper.getMissingFieldsInDateRange(foundFields, dataTypes, beginDate, endDate, specialFields)).andReturn(missingFields); + EasyMock.replay(this.helper); + helperMocked = true; + } + + private void givenDataTypes(String... dataTypes) { + this.dataTypes.addAll(Arrays.asList(dataTypes)); + } + + private void expectImmaterialFields(String... fields) { + this.expectedImmaterialFields.addAll(Arrays.asList(fields)); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/rules/FieldExistenceRuleTest.java b/warehouse/query-core/src/test/java/datawave/query/rules/FieldExistenceRuleTest.java index cd14cfe6106..5b968215046 100644 --- a/warehouse/query-core/src/test/java/datawave/query/rules/FieldExistenceRuleTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/rules/FieldExistenceRuleTest.java @@ -1,87 +1,131 @@ package datawave.query.rules; +import static datawave.query.Constants.ANY_FIELD; +import static datawave.query.Constants.NO_FIELD; + +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Set; +import org.apache.accumulo.core.client.TableNotFoundException; import org.easymock.EasyMock; +import org.easymock.EasyMockExtension; +import org.easymock.Mock; +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.extension.ExtendWith; +import datawave.microservice.query.Query; import datawave.query.util.MetadataHelper; -import datawave.query.util.MockMetadataHelper; +import datawave.util.time.DateHelper; +@ExtendWith(EasyMockExtension.class) public class FieldExistenceRuleTest extends ShardQueryRuleTest { - private static final Set ALL_FIELDS = Set.of("FOO", "BAR", "BAT"); - private static final String ANYFIELD = "_ANYFIELD_"; - private static final MockMetadataHelper defaultMetadataHelper = new MockMetadataHelper(); + private static final String beginDate = "20201231"; + private static final String endDate = "20251231"; + private static final Set specialFields = Set.of(ANY_FIELD, NO_FIELD); + + @Mock + private MetadataHelper helper; - private final Set fieldExceptions = new HashSet<>(); + @Mock + private Query settings; + + private final Set existingFields = new HashSet<>(); + private final Set foundFields = new HashSet<>(); + private final Set missingFields = new HashSet<>(); @BeforeAll - public static void beforeClass() throws Exception { - defaultMetadataHelper.addFields(ALL_FIELDS); - } + public static void beforeClass() throws Exception {} @BeforeEach public void setUp() throws Exception { givenRuleName(RULE_NAME); - givenMetadataHelper(defaultMetadataHelper); + givenMetadataHelper(helper); expectRuleName(RULE_NAME); } + @AfterEach + @Override + void tearDown() { + query = null; + existingFields.clear(); + foundFields.clear(); + missingFields.clear(); + } + /** - * Test a query where all fields exist. + * Test no non-existent fields or immaterial query subsets. */ @Test - public void testAllFieldsExist() throws Exception { - givenQuery("FOO == 'abc' || BAR =~ 'abc' || filter:includeRegex(BAT, '45*')"); + void testNoIssuesFound() throws Exception { + givenQuery("AGE == 'foo' || GENDER == 'bar'"); + + givenExistingFields("AGE", "GENDER"); + expectFoundFields("AGE", "GENDER"); assertResult(); + + EasyMock.verify(settings, helper); } /** - * Test a query where some fields do not exist. + * Test the presence of a non-existent field. */ @Test - public void testNonExistentFields() throws Exception { - givenQuery("TOMFOOLERY == 'abc' || CHAOS =~ 'abc' || filter:includeRegex(SHENANIGANS, '45.8') || FOO == 'aa'"); - expectMessage("Fields not found in data dictionary: TOMFOOLERY, CHAOS, SHENANIGANS"); + void testNonExistentFieldFound() throws Exception { + givenQuery("AGE == 'foo' || GENDER == 'bar'"); + + givenExistingFields("AGE"); + expectFoundFields("AGE", "GENDER"); + givenMissingFields("GENDER"); + + expectMessage("Fields not found in data dictionary: GENDER"); + assertResult(); + + EasyMock.verify(settings, helper); } /** - * Test a query that has a non-existent field that is a special field. + * Test the presence of an immaterial query subset. */ @Test - public void testSpecialField() throws Exception { - givenQuery("FOO == 'abc' || TOMFOOLERY == 'abc' || _ANYFIELD_ = 'abc'"); - givenFieldException(ANYFIELD); - expectMessage("Fields not found in data dictionary: TOMFOOLERY"); + void testImmaterialFieldsFound() throws Exception { + givenQuery("AGE == 'foo' || GENDER == 'bar'"); + + givenExistingFields("AGE", "GENDER"); + expectFoundFields("AGE", "GENDER"); + givenMissingFields("AGE", "GENDER"); + + expectMessage("There is no data in the following fields [AGE, GENDER] for the entire date range 20201231 to 20251231."); + assertResult(); + + EasyMock.verify(settings, helper); } /** - * Test a scenario where an exception gets thrown by the metadata helper. + * Test the presence of a non-existent field and an immaterial query subset. */ @Test - public void testExceptionThrown() throws Exception { - MetadataHelper mockHelper = EasyMock.createMock(MetadataHelper.class); - Exception exception = new IllegalArgumentException("Failed to fetch all fields"); - EasyMock.expect(mockHelper.getAllFields(Collections.emptySet())).andThrow(exception); - EasyMock.replay(mockHelper); + void testNonExistentFieldsAndImmaterialFieldsFound() throws Exception { + givenQuery("NAME == 'hat' && (AGE == 'foo' || GENDER == 'bar')"); - givenQuery("FOO == 'abc'"); - givenMetadataHelper(mockHelper); + givenExistingFields("AGE", "GENDER"); + expectFoundFields("AGE", "GENDER", "NAME"); + givenMissingFields("AGE", "GENDER"); + + expectMessage("Fields not found in data dictionary: NAME"); + expectMessage("There is no data in the following fields [AGE, GENDER] for the entire date range 20201231 to 20251231."); - expectException(exception); assertResult(); - } - private void givenFieldException(String exception) { - this.fieldExceptions.add(exception); + EasyMock.verify(settings, helper); } @Override @@ -91,8 +135,37 @@ protected Object parseQuery() throws Exception { @Override protected ShardQueryRule getNewRule() { + EasyMock.expect(settings.getBeginDate()).andReturn(DateHelper.parse(beginDate)).anyTimes(); + EasyMock.expect(settings.getEndDate()).andReturn(DateHelper.parse(endDate)).anyTimes(); + givenQuerySettings(settings); + + try { + EasyMock.expect(helper.getAllFields(Collections.emptySet())).andReturn(existingFields); + } catch (TableNotFoundException e) { + throw new RuntimeException(e); + } + + if (!foundFields.isEmpty()) { + EasyMock.expect(helper.getMissingFieldsInDateRange(foundFields, Collections.emptySet(), beginDate, endDate, specialFields)) + .andReturn(missingFields); + } + + EasyMock.replay(settings, helper); + FieldExistenceRule rule = new FieldExistenceRule(ruleName); - rule.setSpecialFields(fieldExceptions); + rule.setSpecialFields(specialFields); return rule; } + + private void givenExistingFields(String... fields) { + this.existingFields.addAll(Arrays.asList(fields)); + } + + private void expectFoundFields(String... fields) { + this.foundFields.addAll(Arrays.asList(fields)); + } + + private void givenMissingFields(String... fields) { + this.missingFields.addAll(Arrays.asList(fields)); + } }