From 7f309fdb85c16f3877484b7e4ff33f8e16a1ac54 Mon Sep 17 00:00:00 2001 From: Ivan Bella <347158+ivakegg@users.noreply.github.com> Date: Wed, 15 Jan 2025 13:22:04 -0500 Subject: [PATCH] Feature/most recent unique (#2570) * Refactored the mostRecentUnique functionality to avoid the ivarator paths * Implementation a file backed sorted map * Created additional document map for the return cache --- .../map/BulkResultsFileOutputMapper.java | 1 + core/utils/metadata-utils | 2 +- .../input/microservice/QueryLogicFactory.xml | 16 + .../input/webservice/QueryLogicFactory.xml | 12 +- .../microservice/QueryLogicFactory.xml | 16 + pom.xml | 2 +- ...DatawaveFieldIndexCachingIteratorJexl.java | 7 +- .../iterators/filesystem/FileSystemCache.java | 6 +- .../java/datawave/query/QueryParameters.java | 1 + .../datawave/query/attributes/Attribute.java | 15 + .../query/attributes/UniqueFields.java | 139 ++- .../query/common/grouping/GroupFields.java | 2 +- .../query/config/ShardQueryConfiguration.java | 22 +- .../query/iterator/QueryIterator.java | 20 +- .../datawave/query/iterator/QueryOptions.java | 21 +- .../FinalDocumentTrackingIterator.java | 4 +- .../query/jexl/functions/QueryFunctions.java | 2 + .../functions/QueryFunctionsDescriptor.java | 9 + .../QueryOptionsFromQueryVisitor.java | 30 +- .../functions/jexl/MostRecentUnique.java | 72 ++ .../functions/jexl/MostRecentUniqueByDay.java | 22 + .../jexl/MostRecentUniqueByHour.java | 23 + .../jexl/MostRecentUniqueByMinute.java | 23 + .../jexl/MostRecentUniqueByMonth.java | 23 + .../jexl/MostRecentUniqueBySecond.java | 23 + .../jexl/MostRecentUniqueByTenthOfHour.java | 23 + .../jexl/MostRecentUniqueByYear.java | 23 + .../language/functions/jexl/UniqueByDay.java | 4 - .../language/functions/jexl/UniqueByHour.java | 6 +- .../functions/jexl/UniqueByMinute.java | 4 - .../query/planner/DefaultQueryPlanner.java | 5 + .../query/planner/QueryOptionsSwitch.java | 6 + .../query/tables/ShardQueryLogic.java | 56 +- .../tables/async/event/VisitorFunction.java | 2 +- .../query/transformer/DocumentTransform.java | 1 + .../query/transformer/GroupingTransform.java | 5 +- .../query/transformer/UniqueTransform.java | 448 +++++-- .../BufferedFileBackedSortedMap.java | 545 +++++++++ .../sortedmap/FileByteDocumentSortedMap.java | 207 ++++ .../sortedmap/FileKeyDocumentSortedMap.java | 206 ++++ .../util/sortedmap/FileKeyValueSortedMap.java | 206 ++++ .../sortedmap/FileSerializableSortedMap.java | 289 +++++ .../query/util/sortedmap/FileSortedMap.java | 1073 +++++++++++++++++ .../util/sortedmap/HdfsBackedSortedMap.java | 301 +++++ .../sortedmap/MultiMapBackedSortedMap.java | 407 +++++++ .../util/sortedmap/RewritableSortedMap.java | 19 + .../sortedmap/SortedMapTempFileHandler.java | 63 + .../rfile/KeyValueByteDocumentTransforms.java | 83 ++ .../rfile/RFileByteDocumentInputStream.java | 30 + .../rfile/RFileByteDocumentOutputStream.java | 26 + .../rfile/RFileKeyDocumentInputStream.java | 28 + .../rfile/RFileKeyDocumentOutputStream.java | 22 + .../rfile/RFileKeyValueInputStream.java | 26 + .../rfile/RFileKeyValueInputStreamBase.java | 79 ++ .../rfile/RFileKeyValueOutputStream.java | 20 + .../rfile/RFileKeyValueOutputStreamBase.java | 54 + .../test/java/datawave/query/UniqueTest.java | 96 +- .../query/attributes/UniqueFieldsTest.java | 8 +- .../config/ShardQueryConfigurationTest.java | 2 + .../QueryOptionsFromQueryVisitorTest.java | 103 +- .../UniqueTransformMostRecentTest.java | 86 ++ .../transformer/UniqueTransformTest.java | 363 +++--- .../datawave/query/util/WiseGuysIngest.java | 321 ++--- ...edFileBackedByteDocumentSortedMapTest.java | 71 ++ ...redFileBackedKeyDocumentSortedMapTest.java | 75 ++ ...fferedFileBackedKeyValueSortedMapTest.java | 58 + ...eredFileBackedRewritableSortedMapTest.java | 98 ++ .../BufferedFileBackedSortedMapTest.java | 395 ++++++ .../util/sortedmap/FileSortedMapTest.java | 407 +++++++ .../sortedmap/HdfsBackedSortedMapTest.java | 254 ++++ .../util/sortedmap/MergeSortIteratorTest.java | 241 ++++ .../MultiMapBackedKeyValueSortedMapTest.java | 53 + ...MultiMapBackedRewritableSortedMapTest.java | 89 ++ .../MultiMapBackedSortedMapTest.java | 327 +++++ .../KeyValueByteDocumenTransformsTest.java | 107 ++ .../datawave/query/QueryLogicFactory.xml | 14 +- .../results/cached/CachedRunningQuery.java | 4 +- .../datawave/query/QueryLogicFactory.xml | 12 +- 78 files changed, 7436 insertions(+), 528 deletions(-) create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUnique.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByDay.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByHour.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMinute.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMonth.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueBySecond.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByTenthOfHour.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByYear.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileByteDocumentSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyDocumentSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyValueSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSerializableSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/HdfsBackedSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiMapBackedSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/RewritableSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedMapTempFileHandler.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumentTransforms.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentInputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentOutputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentInputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentOutputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStreamBase.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStreamBase.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyDocumentSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MergeSortIteratorTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedKeyValueSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedRewritableSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedSortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumenTransformsTest.java diff --git a/core/map-reduce/src/main/java/datawave/core/mapreduce/bulkresults/map/BulkResultsFileOutputMapper.java b/core/map-reduce/src/main/java/datawave/core/mapreduce/bulkresults/map/BulkResultsFileOutputMapper.java index 1ce75bf2cd1..62b7f2d403e 100644 --- a/core/map-reduce/src/main/java/datawave/core/mapreduce/bulkresults/map/BulkResultsFileOutputMapper.java +++ b/core/map-reduce/src/main/java/datawave/core/mapreduce/bulkresults/map/BulkResultsFileOutputMapper.java @@ -82,6 +82,7 @@ protected void setup(org.apache.hadoop.mapreduce.Mapper.Con QueryLogic logic = (QueryLogic) super.applicationContext.getBean(logicName); t = logic.getEnrichedTransformer(query); + Assert.notNull(logic.getMarkingFunctions()); Assert.notNull(logic.getResponseObjectFactory()); this.format = SerializationFormat.valueOf(context.getConfiguration().get(RESULT_SERIALIZATION_FORMAT)); diff --git a/core/utils/metadata-utils b/core/utils/metadata-utils index 9b84bb8b5ac..7f05830ac67 160000 --- a/core/utils/metadata-utils +++ b/core/utils/metadata-utils @@ -1 +1 @@ -Subproject commit 9b84bb8b5ac291b2293619f7e1bd294579990a5d +Subproject commit 7f05830ac6788d492a8e38ef99f173d20bbaef65 diff --git a/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml b/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml index c11b1c258c3..e6233a4fac1 100644 --- a/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml +++ b/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml @@ -233,7 +233,23 @@ + + + + + + + + + + + + + + + + diff --git a/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml b/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml index 306770277de..8c40461dcdf 100644 --- a/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml +++ b/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml @@ -38,15 +38,23 @@ - + - + + + + + + + + + diff --git a/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml b/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml index a28dad51d0f..bb07d23456d 100644 --- a/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml +++ b/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml @@ -233,7 +233,23 @@ + + + + + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml index b96ba02f806..1a198e3ee98 100644 --- a/pom.xml +++ b/pom.xml @@ -71,7 +71,7 @@ 3.0.0 1.0.0 4.0.7 - 3.0.3 + 3.1.2 1.9.0 5.2.0 2.15.0 diff --git a/warehouse/query-core/src/main/java/datawave/core/iterators/DatawaveFieldIndexCachingIteratorJexl.java b/warehouse/query-core/src/main/java/datawave/core/iterators/DatawaveFieldIndexCachingIteratorJexl.java index f9a7ce19cb8..85980d15a78 100644 --- a/warehouse/query-core/src/main/java/datawave/core/iterators/DatawaveFieldIndexCachingIteratorJexl.java +++ b/warehouse/query-core/src/main/java/datawave/core/iterators/DatawaveFieldIndexCachingIteratorJexl.java @@ -1303,8 +1303,11 @@ protected void setupRowBasedHdfsBackedSet(String row) throws IOException { this.createdRowDir = false; } - this.set = new HdfsBackedSortedSet<>(null, hdfsBackedSetBufferSize, ivaratorCacheDirs, row, maxOpenFiles, numRetries, persistOptions, - new FileKeySortedSet.Factory()); + // noinspection unchecked + this.set = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder().withBufferPersistThreshold(hdfsBackedSetBufferSize) + .withIvaratorCacheDirs(ivaratorCacheDirs).withUniqueSubPath(row).withMaxOpenFiles(maxOpenFiles).withNumRetries(numRetries) + .withPersistOptions(persistOptions).withSetFactory(new FileKeySortedSet.Factory()).build(); + this.threadSafeSet = Collections.synchronizedSortedSet(this.set); this.currentRow = row; this.setControl.takeOwnership(row, this); diff --git a/warehouse/query-core/src/main/java/datawave/core/iterators/filesystem/FileSystemCache.java b/warehouse/query-core/src/main/java/datawave/core/iterators/filesystem/FileSystemCache.java index e3fcd9890b5..1796dd4aca8 100644 --- a/warehouse/query-core/src/main/java/datawave/core/iterators/filesystem/FileSystemCache.java +++ b/warehouse/query-core/src/main/java/datawave/core/iterators/filesystem/FileSystemCache.java @@ -25,8 +25,10 @@ public class FileSystemCache { public FileSystemCache(String hdfsSiteConfigs) throws MalformedURLException { conf = new Configuration(); - for (String url : org.apache.commons.lang.StringUtils.split(hdfsSiteConfigs, ',')) { - conf.addResource(new URL(url)); + if (hdfsSiteConfigs != null) { + for (String url : org.apache.commons.lang.StringUtils.split(hdfsSiteConfigs, ',')) { + conf.addResource(new URL(url)); + } } } diff --git a/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java b/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java index 48979701c3d..7faba47fc2a 100644 --- a/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java +++ b/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java @@ -184,6 +184,7 @@ public class QueryParameters { public static final String GROUP_FIELDS_BATCH_SIZE = "group.fields.batch.size"; public static final String UNIQUE_FIELDS = "unique.fields"; + public static final String MOST_RECENT_UNIQUE = "most.recent.unique"; /** * Used to specify fields which are excluded from QueryModel expansion diff --git a/warehouse/query-core/src/main/java/datawave/query/attributes/Attribute.java b/warehouse/query-core/src/main/java/datawave/query/attributes/Attribute.java index b425e6451f0..c27ff50c289 100644 --- a/warehouse/query-core/src/main/java/datawave/query/attributes/Attribute.java +++ b/warehouse/query-core/src/main/java/datawave/query/attributes/Attribute.java @@ -9,6 +9,7 @@ import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.security.ColumnVisibility; +import org.apache.commons.lang.builder.EqualsBuilder; import org.apache.commons.lang.builder.HashCodeBuilder; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; @@ -228,6 +229,20 @@ public boolean isFromIndex() { return fromIndex; } + @Override + public boolean equals(Object o) { + if (!(o instanceof Attribute)) { + return false; + } + Attribute other = (Attribute) o; + EqualsBuilder equals = new EqualsBuilder().append(this.isMetadataSet(), other.isMetadataSet()); + if (this.isMetadataSet()) { + equals.append(this.getMetadata(), other.getMetadata()); + } + return equals.isEquals(); + } + + @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(145, 11); hcb.append(this.isMetadataSet()); diff --git a/warehouse/query-core/src/main/java/datawave/query/attributes/UniqueFields.java b/warehouse/query-core/src/main/java/datawave/query/attributes/UniqueFields.java index 0fab5c560ce..0c861f54e6b 100644 --- a/warehouse/query-core/src/main/java/datawave/query/attributes/UniqueFields.java +++ b/warehouse/query-core/src/main/java/datawave/query/attributes/UniqueFields.java @@ -4,14 +4,17 @@ import java.util.Collection; import java.util.HashSet; import java.util.Iterator; +import java.util.NavigableSet; import java.util.Objects; import java.util.Set; +import java.util.SortedSet; import org.apache.commons.lang.StringUtils; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.collect.Multimap; +import com.google.common.collect.Multimaps; import com.google.common.collect.Sets; import com.google.common.collect.SortedSetMultimap; import com.google.common.collect.TreeMultimap; @@ -25,9 +28,11 @@ * captured as a parameter string using {@link UniqueFields#toString()}, and transformed back into a {@link UniqueFields} instance via * {@link UniqueFields#from(String)}. */ -public class UniqueFields implements Serializable { +public class UniqueFields implements Serializable, Cloneable { - private Multimap fieldMap; + private final TreeMultimap fieldMap = TreeMultimap.create(); + private boolean mostRecent = false; + private static String MOST_RECENT_UNIQUE = "_MOST_RECENT_"; /** * Returns a new {@link UniqueFields} parsed from this string. The provided string is expected to have the format returned by @@ -72,8 +77,12 @@ public static UniqueFields from(String string) { if (nextComma == -1 && nextStartBracket == -1) { String field = string.substring(currentIndex); if (!field.isEmpty()) { - // Add the field only if its not blank. Ignore cases with consecutive trailing commas like field1[ALL],, - uniqueFields.put(field, UniqueGranularity.ALL); + if (field.equals(MOST_RECENT_UNIQUE)) { + uniqueFields.setMostRecent(true); + } else { + // Add the field only if its not blank. Ignore cases with consecutive trailing commas like field1[ALL],, + uniqueFields.put(field, UniqueGranularity.ALL); + } } break; // There are no more fields to be parsed. } else if (nextComma != -1 && (nextStartBracket == -1 || nextComma < nextStartBracket)) { @@ -87,8 +96,12 @@ public static UniqueFields from(String string) { // Add the field with the ALL granularity. String field = string.substring(currentIndex, nextComma); if (!field.isEmpty()) { - // Add the field only if its not blank. Ignore cases with consecutive commas like field1,,field2[DAY] - uniqueFields.put(field, UniqueGranularity.ALL); + if (field.equals(MOST_RECENT_UNIQUE)) { + uniqueFields.setMostRecent(true); + } else { + // Add the field only if its not blank. Ignore cases with consecutive commas like field1,,field2[DAY] + uniqueFields.put(field, UniqueGranularity.ALL); + } } currentIndex = nextComma + 1; // Advance to the start of the next field. } else { @@ -100,14 +113,18 @@ public static UniqueFields from(String string) { String field = string.substring(currentIndex, nextStartBracket); int nextEndBracket = string.indexOf(Constants.BRACKET_END, currentIndex); if (!field.isEmpty()) { - String granularityList = string.substring((nextStartBracket + 1), nextEndBracket); - // An empty granularity list, e.g. field[] is equivalent to field[ALL]. - if (granularityList.isEmpty()) { - uniqueFields.put(field, UniqueGranularity.ALL); + if (field.equals(MOST_RECENT_UNIQUE)) { + uniqueFields.setMostRecent(true); } else { - String[] granularities = StringUtils.split(granularityList, Constants.COMMA); - for (String granularity : granularities) { - uniqueFields.put(field, parseGranularity(granularity)); + String granularityList = string.substring((nextStartBracket + 1), nextEndBracket); + // An empty granularity list, e.g. field[] is equivalent to field[ALL]. + if (granularityList.isEmpty()) { + uniqueFields.put(field, UniqueGranularity.ALL); + } else { + String[] granularities = StringUtils.split(granularityList, Constants.COMMA); + for (String granularity : granularities) { + uniqueFields.put(field, parseGranularity(granularity)); + } } } } @@ -128,24 +145,19 @@ private static UniqueGranularity parseGranularity(String granularity) { } /** - * Return a copy of the given {@link UniqueFields}. + * Return a clone of this class * - * @param other - * the other instance to copy * @return the copy */ - public static UniqueFields copyOf(UniqueFields other) { - if (other == null) { - return null; - } - UniqueFields uniqueFields = new UniqueFields(); - uniqueFields.fieldMap = TreeMultimap.create(other.fieldMap); - return uniqueFields; + @Override + public UniqueFields clone() { + UniqueFields newFields = new UniqueFields(); + newFields.fieldMap.putAll(this.fieldMap); + newFields.mostRecent = this.mostRecent; + return newFields; } - public UniqueFields() { - fieldMap = TreeMultimap.create(); - } + public UniqueFields() {} /** * Create a new {@link UniqueFields} with the provided map as the underlying field map. @@ -154,7 +166,24 @@ public UniqueFields() { * the field map to use */ public UniqueFields(SortedSetMultimap fieldMap) { - this.fieldMap = fieldMap; + putAll(fieldMap); + } + + /** + * Clear out the field map + */ + public UniqueFields clear() { + this.fieldMap.clear(); + return this; + } + + /** + * Set the field map + * + * @param fields + */ + public UniqueFields set(Multimap fields) { + return clear().putAll(fields); } /** @@ -165,8 +194,9 @@ public UniqueFields(SortedSetMultimap fieldMap) { * @param uniqueGranularity * the granularity */ - public void put(String field, UniqueGranularity uniqueGranularity) { - fieldMap.put(field, uniqueGranularity); + public UniqueFields put(String field, UniqueGranularity uniqueGranularity) { + fieldMap.put(JexlASTHelper.deconstructIdentifier(field).toUpperCase(), uniqueGranularity); + return this; } /** @@ -175,10 +205,13 @@ public void put(String field, UniqueGranularity uniqueGranularity) { * @param fieldMap * the field map to add entries from */ - public void putAll(Multimap fieldMap) { + public UniqueFields putAll(Multimap fieldMap) { if (fieldMap != null) { - this.fieldMap.putAll(fieldMap); + for (String field : fieldMap.keySet()) { + this.fieldMap.putAll(JexlASTHelper.deconstructIdentifier(field).toUpperCase(), fieldMap.get(field)); + } } + return this; } /** @@ -195,12 +228,12 @@ public void replace(String field, String replacement) { } /** - * Return a copy of the fields within this {@link UniqueFields}. Modifications to this set will not modify the fields in this {@link UniqueFields}. + * Return the fields within this {@link UniqueFields}. Modifications to this set will modify the fields in this {@link UniqueFields}. * * @return a copy of the fields */ - public Set getFields() { - return Sets.newHashSet(fieldMap.keySet()); + public NavigableSet getFields() { + return fieldMap.keySet(); } /** @@ -208,26 +241,10 @@ public Set getFields() { * * @return the field map */ - public Multimap getFieldMap() { + public TreeMultimap getFieldMap() { return fieldMap; } - /** - * Replace any identifier fields with their deconstructed version. - */ - public void deconstructIdentifierFields() { - Multimap newFieldMap = TreeMultimap.create(); - for (String field : fieldMap.keySet()) { - String newField = JexlASTHelper.deconstructIdentifier(field); - if (newField.equals(field)) { - newFieldMap.putAll(field, fieldMap.get(field)); - } else { - newFieldMap.putAll(newField, fieldMap.get(field)); - } - } - this.fieldMap = newFieldMap; - } - /** * Remap all fields to include any matches from the provided model. The original field entries will be retained. * @@ -238,12 +255,11 @@ public void remapFields(Multimap model) { Multimap newFieldMap = TreeMultimap.create(fieldMap); for (String field : fieldMap.keySet()) { Collection granularities = fieldMap.get(field); - field = field.toUpperCase(); if (model.containsKey(field)) { model.get(field).forEach((newField) -> newFieldMap.putAll(newField, granularities)); } } - this.fieldMap = newFieldMap; + set(newFieldMap); } /** @@ -305,6 +321,10 @@ public String transformValue(String field, String value) { @Override public String toString() { StringBuilder sb = new StringBuilder(); + if (mostRecent) { + sb.append(MOST_RECENT_UNIQUE); + sb.append(Constants.COMMA); + } Iterator fieldIterator = fieldMap.keySet().iterator(); while (fieldIterator.hasNext()) { // Write the field. @@ -326,6 +346,15 @@ public String toString() { return sb.toString(); } + public boolean isMostRecent() { + return mostRecent; + } + + public UniqueFields setMostRecent(boolean mostRecent) { + this.mostRecent = mostRecent; + return this; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -335,12 +364,12 @@ public boolean equals(Object o) { return false; } UniqueFields that = (UniqueFields) o; - return Objects.equals(fieldMap, that.fieldMap); + return Objects.equals(fieldMap, that.fieldMap) && mostRecent == that.mostRecent; } @Override public int hashCode() { - return Objects.hash(fieldMap); + return Objects.hash(fieldMap, mostRecent); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/common/grouping/GroupFields.java b/warehouse/query-core/src/main/java/datawave/query/common/grouping/GroupFields.java index d51f64cd65c..5f7cd133e2b 100644 --- a/warehouse/query-core/src/main/java/datawave/query/common/grouping/GroupFields.java +++ b/warehouse/query-core/src/main/java/datawave/query/common/grouping/GroupFields.java @@ -311,7 +311,7 @@ public void deconstructIdentifiers() { // Return a copy of the given set with all identifiers deconstructed. private Set deconstructIdentifiers(Set set) { - return set.stream().map(JexlASTHelper::deconstructIdentifier).collect(Collectors.toSet()); + return set.stream().map(JexlASTHelper::deconstructIdentifier).map(String::toUpperCase).collect(Collectors.toSet()); } /** diff --git a/warehouse/query-core/src/main/java/datawave/query/config/ShardQueryConfiguration.java b/warehouse/query-core/src/main/java/datawave/query/config/ShardQueryConfiguration.java index a6dd448a4ec..70b210460b7 100644 --- a/warehouse/query-core/src/main/java/datawave/query/config/ShardQueryConfiguration.java +++ b/warehouse/query-core/src/main/java/datawave/query/config/ShardQueryConfiguration.java @@ -357,6 +357,8 @@ public class ShardQueryConfiguration extends GenericQueryConfiguration implement private List ivaratorCacheDirConfigs = Collections.emptyList(); private String ivaratorFstHdfsBaseURIs = null; private int ivaratorCacheBufferSize = 10000; + + private int uniqueCacheBufferSize = 100; private long ivaratorCacheScanPersistThreshold = 100000L; private long ivaratorCacheScanTimeout = 1000L * 60 * 60; private int maxFieldIndexRangeSplit = 11; @@ -731,7 +733,9 @@ public void copyFrom(ShardQueryConfiguration other) { this.setCompositeFilterFunctionsEnabled(other.isCompositeFilterFunctionsEnabled()); this.setGroupFieldsBatchSize(other.getGroupFieldsBatchSize()); this.setAccrueStats(other.getAccrueStats()); - this.setUniqueFields(UniqueFields.copyOf(other.getUniqueFields())); + this.setUniqueFields(other.getUniqueFields()); + log.info("Checkpointing with " + getUniqueFields()); + this.setUniqueCacheBufferSize(other.getUniqueCacheBufferSize()); this.setCacheModel(other.getCacheModel()); this.setTrackSizes(other.isTrackSizes()); this.setContentFieldNames(null == other.getContentFieldNames() ? null : Lists.newArrayList(other.getContentFieldNames())); @@ -1535,6 +1539,14 @@ public void setIvaratorFstHdfsBaseURIs(String ivaratorFstHdfsBaseURIs) { this.ivaratorFstHdfsBaseURIs = ivaratorFstHdfsBaseURIs; } + public int getUniqueCacheBufferSize() { + return uniqueCacheBufferSize; + } + + public void setUniqueCacheBufferSize(int uniqueCacheBufferSize) { + this.uniqueCacheBufferSize = uniqueCacheBufferSize; + } + public int getIvaratorCacheBufferSize() { return ivaratorCacheBufferSize; } @@ -1890,11 +1902,7 @@ public UniqueFields getUniqueFields() { } public void setUniqueFields(UniqueFields uniqueFields) { - this.uniqueFields = uniqueFields; - // If unique fields are present, make sure they are deconstructed by this point. - if (uniqueFields != null) { - uniqueFields.deconstructIdentifierFields(); - } + this.uniqueFields = uniqueFields.clone(); } public boolean isHitList() { @@ -2938,6 +2946,7 @@ public boolean equals(Object o) { getGroupFieldsBatchSize() == that.getGroupFieldsBatchSize() && getAccrueStats() == that.getAccrueStats() && Objects.equals(getUniqueFields(), that.getUniqueFields()) && + getUniqueCacheBufferSize() == that.getUniqueCacheBufferSize() && getCacheModel() == that.getCacheModel() && isTrackSizes() == that.isTrackSizes() && getEnforceUniqueConjunctionsWithinExpression() == that.getEnforceUniqueConjunctionsWithinExpression() && @@ -3208,6 +3217,7 @@ public int hashCode() { getAccrueStats(), getGroupFields(), getUniqueFields(), + getUniqueCacheBufferSize(), getCacheModel(), isTrackSizes(), getContentFieldNames(), diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/QueryIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/QueryIterator.java index c99cd2d5fcb..c51168de6e1 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/QueryIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/QueryIterator.java @@ -121,6 +121,7 @@ import datawave.query.util.Tuple3; import datawave.query.util.TupleToEntry; import datawave.query.util.TypeMetadata; +import datawave.query.util.sortedset.FileSortedSet; import datawave.util.StringUtils; /** @@ -452,8 +453,7 @@ else if (documentRange != null && (!this.isContainsIndexOnlyTerms() && this.getT // now apply the unique iterator if requested UniqueTransform uniquify = getUniqueTransform(); if (uniquify != null) { - // pipelineDocuments = uniquify; - pipelineDocuments = Iterators.filter(pipelineDocuments, uniquify.getUniquePredicate()); + pipelineDocuments = uniquify.getIterator(pipelineDocuments); } // apply the grouping iterator if requested and if the batch size is greater than zero @@ -1553,11 +1553,23 @@ public Comparator getValueComparator(Tuple3 hitsOnlySet = new HashSet<>(); @@ -518,6 +522,7 @@ public void deepCopy(QueryOptions other) { this.ivaratorCacheDirConfigs = (other.ivaratorCacheDirConfigs == null) ? null : new ArrayList<>(other.ivaratorCacheDirConfigs); this.hdfsSiteConfigURLs = other.hdfsSiteConfigURLs; this.ivaratorCacheBufferSize = other.ivaratorCacheBufferSize; + this.uniqueCacheBufferSize = other.uniqueCacheBufferSize; this.ivaratorCacheScanPersistThreshold = other.ivaratorCacheScanPersistThreshold; this.ivaratorCacheScanTimeout = other.ivaratorCacheScanTimeout; this.hdfsFileCompressionCodec = other.hdfsFileCompressionCodec; @@ -1057,6 +1062,14 @@ public void setIvaratorCacheBufferSize(int ivaratorCacheBufferSize) { this.ivaratorCacheBufferSize = ivaratorCacheBufferSize; } + public int getUniqueCacheBufferSize() { + return uniqueCacheBufferSize; + } + + public void setUniqueCacheBufferSize(int uniqueCacheBufferSize) { + this.uniqueCacheBufferSize = uniqueCacheBufferSize; + } + public long getIvaratorCacheScanPersistThreshold() { return ivaratorCacheScanPersistThreshold; } @@ -1202,7 +1215,7 @@ public UniqueFields getUniqueFields() { } public void setUniqueFields(UniqueFields uniqueFields) { - this.uniqueFields = uniqueFields; + this.uniqueFields = uniqueFields.clone(); } public Set getHitsOnlySet() { @@ -1698,6 +1711,12 @@ public boolean validateOptions(Map options) { if (options.containsKey(UNIQUE_FIELDS)) { this.setUniqueFields(UniqueFields.from(options.get(UNIQUE_FIELDS))); + if (options.containsKey(MOST_RECENT_UNIQUE)) { + this.getUniqueFields().setMostRecent(Boolean.valueOf(options.get(MOST_RECENT_UNIQUE))); + if (options.containsKey(UNIQUE_CACHE_BUFFER_SIZE)) { + this.setUniqueCacheBufferSize(Integer.parseInt(options.get(UNIQUE_CACHE_BUFFER_SIZE))); + } + } } if (options.containsKey(HIT_LIST)) { diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/profile/FinalDocumentTrackingIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/profile/FinalDocumentTrackingIterator.java index e8af4ae0931..c00f60784f0 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/profile/FinalDocumentTrackingIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/profile/FinalDocumentTrackingIterator.java @@ -24,8 +24,8 @@ public class FinalDocumentTrackingIterator implements Iterator *
  • {@code f:unique_by_second()}: Expects a comma-delimited list of fields to be unique with a granularity level of by SECOND, e.g. * {@code unique_by_second('field1','field2')}
  • + *
  • {@code f:most_recent_unique...} Adding most_recent_ before any unique function will set the most.recent.unique flag to true, e.g. + * {@code most_recent_unique_by_day('field1','field2')}
  • *
  • {@code f:rename}: Expects a comma-delimited list field/field mappings e.g. {@code f:rename('field1=field2','field3=field4')}
  • * */ @@ -59,9 +61,18 @@ public class QueryOptionsFromQueryVisitor extends RebuildingVisitor { QueryFunctions.UNIQUE_FUNCTION, UniqueFunction.UNIQUE_BY_DAY_FUNCTION, UniqueFunction.UNIQUE_BY_HOUR_FUNCTION, UniqueFunction.UNIQUE_BY_MINUTE_FUNCTION, UniqueFunction.UNIQUE_BY_TENTH_OF_HOUR_FUNCTION, UniqueFunction.UNIQUE_BY_MONTH_FUNCTION, UniqueFunction.UNIQUE_BY_SECOND_FUNCTION, UniqueFunction.UNIQUE_BY_MILLISECOND_FUNCTION, UniqueFunction.UNIQUE_BY_YEAR_FUNCTION, - QueryFunctions.GROUPBY_FUNCTION, QueryFunctions.EXCERPT_FIELDS_FUNCTION, QueryFunctions.NO_EXPANSION, - QueryFunctions.LENIENT_FIELDS_FUNCTION, QueryFunctions.STRICT_FIELDS_FUNCTION, QueryFunctions.SUM, QueryFunctions.MIN, QueryFunctions.MAX, - QueryFunctions.AVERAGE, QueryFunctions.COUNT, QueryFunctions.RENAME_FUNCTION); + QueryFunctions.MOST_RECENT_PREFIX + QueryFunctions.UNIQUE_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_DAY_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_HOUR_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_MINUTE_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_TENTH_OF_HOUR_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_MONTH_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_SECOND_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_MILLISECOND_FUNCTION, + QueryFunctions.MOST_RECENT_PREFIX + UniqueFunction.UNIQUE_BY_YEAR_FUNCTION, QueryFunctions.GROUPBY_FUNCTION, + QueryFunctions.EXCERPT_FIELDS_FUNCTION, QueryFunctions.NO_EXPANSION, QueryFunctions.LENIENT_FIELDS_FUNCTION, + QueryFunctions.STRICT_FIELDS_FUNCTION, QueryFunctions.SUM, QueryFunctions.MIN, QueryFunctions.MAX, QueryFunctions.AVERAGE, + QueryFunctions.COUNT, QueryFunctions.RENAME_FUNCTION); @SuppressWarnings("unchecked") public static T collect(T node, Object data) { @@ -188,7 +199,16 @@ private Object visit(ASTFunctionNode node, Map optionsMap) { ASTNamespaceIdentifier nsIdentifier = (ASTNamespaceIdentifier) node.jjtGetChild(0); // if this is the f:options function, create a List for the userData to be passed to the child nodes if (nsIdentifier.getNamespace().equals(QueryFunctions.QUERY_FUNCTION_NAMESPACE)) { - switch (String.valueOf(nsIdentifier.getName())) { + String function = String.valueOf(nsIdentifier.getName()); + + // check for the most recent flag for the unique functions only + boolean mostRecent = function.startsWith(QueryFunctions.MOST_RECENT_PREFIX + QueryFunctions.UNIQUE_FUNCTION); + if (mostRecent) { + function = function.substring(QueryFunctions.MOST_RECENT_PREFIX.length()); + optionsMap.put(QueryParameters.MOST_RECENT_UNIQUE, "true"); + } + + switch (function) { case QueryFunctions.OPTIONS_FUNCTION: { List optionsList = new ArrayList<>(); this.visit(node, optionsList); @@ -234,7 +254,7 @@ private Object visit(ASTFunctionNode node, Map optionsMap) { case UniqueFunction.UNIQUE_BY_SECOND_FUNCTION: case UniqueFunction.UNIQUE_BY_TENTH_OF_HOUR_FUNCTION: { UniqueFields uniqueFields = new UniqueFields(); - updateUniqueFields(node, uniqueFields, optionsMap, UniqueFunction.findByName(nsIdentifier.getName())); + updateUniqueFields(node, uniqueFields, optionsMap, UniqueFunction.findByName(function)); return null; } case QueryFunctions.GROUPBY_FUNCTION: { diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUnique.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUnique.java new file mode 100644 index 00000000000..ed94abec1ae --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUnique.java @@ -0,0 +1,72 @@ +package datawave.query.language.functions.jexl; + +import java.text.MessageFormat; +import java.util.ArrayList; +import java.util.stream.Collectors; + +import datawave.query.attributes.UniqueFields; +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.language.functions.QueryFunction; +import datawave.webservice.query.exception.BadRequestQueryException; +import datawave.webservice.query.exception.DatawaveErrorCode; + +/** + * Function to determine most recent uniqueness among documents given a set of fields and the levels of granularity that should be used for each fields. This + * function accepts a list of fields with specified granularity levels in the format {@code field[ALL],dateField[DAY,HOUR,MINUTE]}. See {@link UniqueFields} for + * additional documentation on supported formatting. + */ +public class MostRecentUnique extends JexlQueryFunction { + + public MostRecentUnique() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryFunctions.UNIQUE_FUNCTION, new ArrayList<>()); + } + + /** + * query options contain a list of fields. Cannot be the empty list. + * + * @throws IllegalArgumentException + * for illegal arguments + */ + @Override + public void validate() throws IllegalArgumentException { + if (this.parameterList.isEmpty()) { + BadRequestQueryException qe = new BadRequestQueryException(DatawaveErrorCode.INVALID_FUNCTION_ARGUMENTS, + MessageFormat.format("{0} requires at least one argument", this.name)); + throw new IllegalArgumentException(qe); + } else { + String parameters = String.join(",", parameterList); + try { + UniqueFields.from(parameters); + } catch (Exception e) { + BadRequestQueryException qe = new BadRequestQueryException(DatawaveErrorCode.INVALID_FUNCTION_ARGUMENTS, + MessageFormat.format("Unable to parse unique fields from arguments for function {0}", this.name)); + throw new IllegalArgumentException(qe); + } + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + + sb.append(QueryFunctions.QUERY_FUNCTION_NAMESPACE).append(':').append(QueryFunctions.MOST_RECENT_PREFIX).append(QueryFunctions.UNIQUE_FUNCTION); + if (parameterList.isEmpty()) { + sb.append("()"); + } else { + char separator = '('; + for (String parm : parameterList) { + sb.append(separator).append(escapeString(parm)); + separator = ','; + } + sb.append(')'); + } + + return sb.toString(); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUnique(); + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByDay.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByDay.java new file mode 100644 index 00000000000..845bd863ae3 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByDay.java @@ -0,0 +1,22 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a unique result for every day for a given list of fields. This function is equivalent to {@code #MOST_RECENT_UNIQUE(field[DAY])}. + */ +public class MostRecentUniqueByDay extends UniqueByFunction { + + public MostRecentUniqueByDay() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_DAY_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueByDay(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByHour.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByHour.java new file mode 100644 index 00000000000..c831dac8aec --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByHour.java @@ -0,0 +1,23 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a unique result for every hour of the day for a given list of fields. This function is equivalent to + * {@code #MOST_RECENT_UNIQUE(field[HOUR])}. + */ +public class MostRecentUniqueByHour extends UniqueByFunction { + + public MostRecentUniqueByHour() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_HOUR_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueByHour(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMinute.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMinute.java new file mode 100644 index 00000000000..f8b04bc4050 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMinute.java @@ -0,0 +1,23 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a most recent unique result for every minute of the hour for a given list of fields. This function is equivalent to + * {@code #MOST_RECENT_UNIQUE(field[MINUTE])}. + */ +public class MostRecentUniqueByMinute extends UniqueByFunction { + + public MostRecentUniqueByMinute() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_MINUTE_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueByMinute(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMonth.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMonth.java new file mode 100644 index 00000000000..3c611479dd5 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMonth.java @@ -0,0 +1,23 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a most recent unique result for every month of the year for a given list of fields. This function is equivalent to + * {@code #MOST_RECENT_UNIQUE(field[MONTH])}. + */ +public class MostRecentUniqueByMonth extends UniqueByFunction { + + public MostRecentUniqueByMonth() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_MONTH_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueByMonth(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueBySecond.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueBySecond.java new file mode 100644 index 00000000000..8ff9eedbb45 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueBySecond.java @@ -0,0 +1,23 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a most recent unique result for every second for a given list of fields. This function is equivalent to + * {@code #MOST_RECENT_UNIQUE(field[SECOND])}. + */ +public class MostRecentUniqueBySecond extends UniqueByFunction { + + public MostRecentUniqueBySecond() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_SECOND_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueBySecond(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByTenthOfHour.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByTenthOfHour.java new file mode 100644 index 00000000000..81948a62cb3 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByTenthOfHour.java @@ -0,0 +1,23 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a most recent_unique result for every tenth of an hour for a given list of fields. This function is equivalent to + * {@code #MOST_RECENT_UNIQUE(field[TENTH_OF_HOUR])}. + */ +public class MostRecentUniqueByTenthOfHour extends UniqueByFunction { + + public MostRecentUniqueByTenthOfHour() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_TENTH_OF_HOUR_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueByTenthOfHour(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByYear.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByYear.java new file mode 100644 index 00000000000..24d8c8c4471 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByYear.java @@ -0,0 +1,23 @@ +package datawave.query.language.functions.jexl; + +import java.util.ArrayList; + +import datawave.query.jexl.functions.QueryFunctions; +import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; +import datawave.query.language.functions.QueryFunction; + +/** + * Function to return a most recent unique result for the year for a given list of fields. This function is equivalent to + * {@code #MOST_RECENT_UNIQUE(field[YEAR])}. + */ +public class MostRecentUniqueByYear extends UniqueByFunction { + + public MostRecentUniqueByYear() { + super(QueryFunctions.MOST_RECENT_PREFIX + QueryOptionsFromQueryVisitor.UniqueFunction.UNIQUE_BY_YEAR_FUNCTION, new ArrayList<>()); + } + + @Override + public QueryFunction duplicate() { + return new MostRecentUniqueByYear(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByDay.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByDay.java index a354b79c5a0..c6fd2c0f040 100644 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByDay.java +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByDay.java @@ -1,13 +1,9 @@ package datawave.query.language.functions.jexl; -import java.text.MessageFormat; import java.util.ArrayList; -import datawave.query.jexl.functions.QueryFunctions; import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; import datawave.query.language.functions.QueryFunction; -import datawave.webservice.query.exception.BadRequestQueryException; -import datawave.webservice.query.exception.DatawaveErrorCode; /** * Function to return a unique result for every day for a given list of fields. This function is equivalent to {@code #unique(field[DAY])}. diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByHour.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByHour.java index 78afb7e50b1..c6f298f0c0b 100644 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByHour.java +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByHour.java @@ -1,16 +1,12 @@ package datawave.query.language.functions.jexl; -import java.text.MessageFormat; import java.util.ArrayList; -import datawave.query.jexl.functions.QueryFunctions; import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; import datawave.query.language.functions.QueryFunction; -import datawave.webservice.query.exception.BadRequestQueryException; -import datawave.webservice.query.exception.DatawaveErrorCode; /** - * Function to return a unique result for every hour of the day for a given list of fields. This function is equivalent to {@code #unique(field[DAY])}. + * Function to return a unique result for every hour of the day for a given list of fields. This function is equivalent to {@code #unique(field[HOUR])}. */ public class UniqueByHour extends UniqueByFunction { diff --git a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByMinute.java b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByMinute.java index 90bc82720fc..b05d880f4fd 100644 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByMinute.java +++ b/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/UniqueByMinute.java @@ -1,13 +1,9 @@ package datawave.query.language.functions.jexl; -import java.text.MessageFormat; import java.util.ArrayList; -import datawave.query.jexl.functions.QueryFunctions; import datawave.query.jexl.visitors.QueryOptionsFromQueryVisitor; import datawave.query.language.functions.QueryFunction; -import datawave.webservice.query.exception.BadRequestQueryException; -import datawave.webservice.query.exception.DatawaveErrorCode; /** * Function to return a unique result for every minute of the hour for a given list of fields. This function is equivalent to {@code #unique(field[MINUTE])}. diff --git a/warehouse/query-core/src/main/java/datawave/query/planner/DefaultQueryPlanner.java b/warehouse/query-core/src/main/java/datawave/query/planner/DefaultQueryPlanner.java index 4bc1f729049..0324c7a69fd 100644 --- a/warehouse/query-core/src/main/java/datawave/query/planner/DefaultQueryPlanner.java +++ b/warehouse/query-core/src/main/java/datawave/query/planner/DefaultQueryPlanner.java @@ -627,6 +627,11 @@ private void configureIterator(ShardQueryConfiguration config, IteratorSetting c addOption(cfg, QueryOptions.GROUP_FIELDS, config.getGroupFields().toString(), true); addOption(cfg, QueryOptions.GROUP_FIELDS_BATCH_SIZE, config.getGroupFieldsBatchSizeAsString(), true); addOption(cfg, QueryOptions.UNIQUE_FIELDS, config.getUniqueFields().toString(), true); + if (config.getUniqueFields().isMostRecent()) { + // this may be redundant with the uniqueFields.toString(), but other code relies on this explicitly being set + addOption(cfg, QueryOptions.MOST_RECENT_UNIQUE, Boolean.toString(true), false); + addOption(cfg, QueryOptions.UNIQUE_CACHE_BUFFER_SIZE, Integer.toString(config.getUniqueCacheBufferSize()), false); + } addOption(cfg, QueryOptions.HIT_LIST, Boolean.toString(config.isHitList()), false); addOption(cfg, QueryOptions.TERM_FREQUENCY_FIELDS, Joiner.on(',').join(config.getQueryTermFrequencyFields()), false); addOption(cfg, QueryOptions.TERM_FREQUENCIES_REQUIRED, Boolean.toString(config.isTermFrequenciesRequired()), false); diff --git a/warehouse/query-core/src/main/java/datawave/query/planner/QueryOptionsSwitch.java b/warehouse/query-core/src/main/java/datawave/query/planner/QueryOptionsSwitch.java index 249b33d2b26..9c6eaedf486 100644 --- a/warehouse/query-core/src/main/java/datawave/query/planner/QueryOptionsSwitch.java +++ b/warehouse/query-core/src/main/java/datawave/query/planner/QueryOptionsSwitch.java @@ -62,8 +62,14 @@ public static void apply(Map optionsMap, ShardQueryConfiguration break; case QueryParameters.UNIQUE_FIELDS: UniqueFields uniqueFields = UniqueFields.from(value); + // preserve the most recent flag + uniqueFields.setMostRecent(config.getUniqueFields().isMostRecent()); config.setUniqueFields(uniqueFields); break; + case QueryParameters.MOST_RECENT_UNIQUE: + log.info("Setting unique fields to be most recent"); + config.getUniqueFields().setMostRecent(Boolean.parseBoolean(value)); + break; case QueryParameters.EXCERPT_FIELDS: ExcerptFields excerptFields = ExcerptFields.from(value); config.setExcerptFields(excerptFields); diff --git a/warehouse/query-core/src/main/java/datawave/query/tables/ShardQueryLogic.java b/warehouse/query-core/src/main/java/datawave/query/tables/ShardQueryLogic.java index bacab5def54..1e764715d8c 100644 --- a/warehouse/query-core/src/main/java/datawave/query/tables/ShardQueryLogic.java +++ b/warehouse/query-core/src/main/java/datawave/query/tables/ShardQueryLogic.java @@ -101,6 +101,7 @@ import datawave.query.util.MetadataHelper; import datawave.query.util.MetadataHelperFactory; import datawave.query.util.QueryStopwatch; +import datawave.query.util.sortedset.FileSortedSet; import datawave.util.time.TraceStopwatch; import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.result.event.ResponseObjectFactory; @@ -277,7 +278,9 @@ public static BatchScanner createBatchScanner(ShardQueryConfiguration config, Sc @Override public GenericQueryConfiguration initialize(AccumuloClient client, Query settings, Set auths) throws Exception { + // whenever we reinitialize, ensure we have a fresh transformer this.transformerInstance = null; + this.config = ShardQueryConfiguration.create(this, settings); if (log.isTraceEnabled()) log.trace("Initializing ShardQueryLogic: " + System.identityHashCode(this) + '(' @@ -650,7 +653,11 @@ protected String getStopwatchHeader(ShardQueryConfiguration config) { @Override public QueryLogicTransformer getTransformer(Query settings) { if (this.transformerInstance != null) { - addConfigBasedTransformers(); + try { + addConfigBasedTransformers(); + } catch (QueryException e) { + throw new DatawaveFatalQueryException("Unable to configure transformers", e); + } return this.transformerInstance; } @@ -673,7 +680,11 @@ public QueryLogicTransformer getTransformer(Query settings) { transformer.setPrimaryToSecondaryFieldMap(primaryToSecondaryFieldMap); transformer.setQm(queryModel); this.transformerInstance = transformer; - addConfigBasedTransformers(); + try { + addConfigBasedTransformers(); + } catch (QueryException e) { + throw new DatawaveFatalQueryException("Unable to configure transformers", e); + } return this.transformerInstance; } @@ -690,7 +701,7 @@ public boolean isLongRunningQuery() { /** * If the configuration didn't exist, OR IT CHANGED, we need to create or update the transformers that have been added. */ - private void addConfigBasedTransformers() { + private void addConfigBasedTransformers() throws QueryException { if (getConfig() != null) { ((DocumentTransformer) this.transformerInstance).setProjectFields(getConfig().getProjectFields()); ((DocumentTransformer) this.transformerInstance).setDisallowlistedFields(getConfig().getDisallowlistedFields()); @@ -698,10 +709,26 @@ private void addConfigBasedTransformers() { if (getConfig().getUniqueFields() != null && !getConfig().getUniqueFields().isEmpty()) { DocumentTransform alreadyExists = ((DocumentTransformer) this.transformerInstance).containsTransform(UniqueTransform.class); if (alreadyExists != null) { - ((UniqueTransform) alreadyExists).updateConfig(getConfig().getUniqueFields(), getQueryModel()); + ((UniqueTransform) alreadyExists).updateConfig(getConfig().getUniqueFields()); } else { - ((DocumentTransformer) this.transformerInstance) - .addTransform(new UniqueTransform(this, getConfig().getUniqueFields(), this.getQueryExecutionForPageTimeout())); + try { + // @formatter:off + ((DocumentTransformer) this.transformerInstance).addTransform(new UniqueTransform.Builder() + .withUniqueFields(getConfig().getUniqueFields()) + .withQueryExecutionForPageTimeout(this.getQueryExecutionForPageTimeout()) + .withModel(getQueryModel()) + .withBufferPersistThreshold(getUniqueCacheBufferSize()) + .withIvaratorCacheDirConfigs(getIvaratorCacheDirConfigs()) + .withHdfsSiteConfigURLs(getHdfsSiteConfigURLs()) + .withSubDirectory(getConfig().getQuery().getId().toString()) + .withMaxOpenFiles(getIvaratorMaxOpenFiles()) + .withNumRetries(getIvaratorNumRetries()) + .withPersistOptions(new FileSortedSet.PersistOptions(true, false, 0)) + .build()); + // @formatter:on + } catch (IOException ioe) { + throw new QueryException("Unable to create a unique transform", ioe); + } } } @@ -971,11 +998,18 @@ protected void loadQueryParameters(ShardQueryConfiguration config, Query setting UniqueFields uniqueFields = UniqueFields.from(uniqueFieldsParam); // Only set the unique fields if we were actually given some if (!uniqueFields.isEmpty()) { - this.setUniqueFields(uniqueFields); + // preserve the most recent flag + uniqueFields.setMostRecent(config.getUniqueFields().isMostRecent()); config.setUniqueFields(uniqueFields); } } + // Get the most recent flag + String mostRecentUnique = settings.findParameter(QueryParameters.MOST_RECENT_UNIQUE).getParameterValue().trim(); + if (StringUtils.isNotBlank(mostRecentUnique)) { + config.getUniqueFields().setMostRecent(Boolean.valueOf(mostRecentUnique)); + } + // Get the EXCERPT_FIELDS parameter if given String excerptFieldsParam = settings.findParameter(QueryParameters.EXCERPT_FIELDS).getParameterValue().trim(); if (StringUtils.isNotBlank(excerptFieldsParam)) { @@ -1977,6 +2011,14 @@ public void setIvaratorFstHdfsBaseURIs(String ivaratorFstHdfsBaseURIs) { getConfig().setIvaratorFstHdfsBaseURIs(ivaratorFstHdfsBaseURIs); } + public int getUniqueCacheBufferSize() { + return getConfig().getUniqueCacheBufferSize(); + } + + public void setUniqueCacheBufferSize(int uniqueCacheBufferSize) { + getConfig().setUniqueCacheBufferSize(uniqueCacheBufferSize); + } + public int getIvaratorCacheBufferSize() { return getConfig().getIvaratorCacheBufferSize(); } diff --git a/warehouse/query-core/src/main/java/datawave/query/tables/async/event/VisitorFunction.java b/warehouse/query-core/src/main/java/datawave/query/tables/async/event/VisitorFunction.java index 081959f8777..5fda16f5d12 100644 --- a/warehouse/query-core/src/main/java/datawave/query/tables/async/event/VisitorFunction.java +++ b/warehouse/query-core/src/main/java/datawave/query/tables/async/event/VisitorFunction.java @@ -414,7 +414,7 @@ protected void pruneEmptyOptions(IteratorSetting settings) { * an {@link IteratorSetting} */ protected void pruneIvaratorConfigs(ASTJexlScript script, IteratorSetting settings) { - if (script != null && !IvaratorRequiredVisitor.isIvaratorRequired(script)) { + if (script != null && !settings.getOptions().containsKey(QueryOptions.MOST_RECENT_UNIQUE) && !IvaratorRequiredVisitor.isIvaratorRequired(script)) { settings.removeOption(QueryOptions.IVARATOR_CACHE_BUFFER_SIZE); settings.removeOption(QueryOptions.IVARATOR_CACHE_DIR_CONFIG); settings.removeOption(QueryOptions.IVARATOR_NUM_RETRIES); diff --git a/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransform.java b/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransform.java index f78177ce039..ea9d6fca930 100644 --- a/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransform.java +++ b/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransform.java @@ -37,6 +37,7 @@ class DefaultDocumentTransform implements DocumentTransform { public void initialize(Query settings, MarkingFunctions markingFunctions) { this.settings = settings; this.markingFunctions = markingFunctions; + this.queryExecutionForPageStartTime = System.currentTimeMillis(); } @Override diff --git a/warehouse/query-core/src/main/java/datawave/query/transformer/GroupingTransform.java b/warehouse/query-core/src/main/java/datawave/query/transformer/GroupingTransform.java index 8327188210f..c9cfca151db 100644 --- a/warehouse/query-core/src/main/java/datawave/query/transformer/GroupingTransform.java +++ b/warehouse/query-core/src/main/java/datawave/query/transformer/GroupingTransform.java @@ -23,7 +23,6 @@ import datawave.query.common.grouping.GroupingUtils; import datawave.query.common.grouping.Groups; import datawave.query.iterator.profile.FinalDocumentTrackingIterator; -import datawave.query.model.QueryModel; /** * GroupingTransform mimics GROUP BY with a COUNT in SQL. For the given fields, this transform will group into unique combinations of values and assign a count @@ -91,6 +90,10 @@ public Entry apply(@Nullable Entry keyDocumentEntry) return keyDocumentEntry; } + if (keyDocumentEntry.getValue().isIntermediateResult()) { + return keyDocumentEntry; + } + keys.add(keyDocumentEntry.getKey()); log.trace("{} get list key counts for: {}", "web-server", keyDocumentEntry); DocumentGrouper.group(keyDocumentEntry, groupFields, groups); diff --git a/warehouse/query-core/src/main/java/datawave/query/transformer/UniqueTransform.java b/warehouse/query-core/src/main/java/datawave/query/transformer/UniqueTransform.java index 07fb702b7de..f405f224837 100644 --- a/warehouse/query-core/src/main/java/datawave/query/transformer/UniqueTransform.java +++ b/warehouse/query-core/src/main/java/datawave/query/transformer/UniqueTransform.java @@ -4,21 +4,24 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.Serializable; +import java.net.URI; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.TreeSet; -import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.Logger; -import com.google.common.base.Predicate; import com.google.common.collect.HashMultimap; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -26,26 +29,38 @@ import com.google.common.hash.Funnel; import com.google.common.hash.PrimitiveSink; -import datawave.core.query.logic.BaseQueryLogic; +import datawave.core.iterators.filesystem.FileSystemCache; import datawave.query.attributes.Attribute; import datawave.query.attributes.Attributes; import datawave.query.attributes.Document; +import datawave.query.attributes.DocumentKey; import datawave.query.attributes.UniqueFields; +import datawave.query.exceptions.DatawaveFatalQueryException; +import datawave.query.iterator.ivarator.IvaratorCacheDir; +import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; import datawave.query.iterator.profile.FinalDocumentTrackingIterator; import datawave.query.model.QueryModel; -import datawave.query.tables.ShardQueryLogic; +import datawave.query.util.sortedmap.FileByteDocumentSortedMap; +import datawave.query.util.sortedmap.FileKeyDocumentSortedMap; +import datawave.query.util.sortedmap.FileSortedMap; +import datawave.query.util.sortedmap.HdfsBackedSortedMap; +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.FileSortedSet; /** * This iterator will filter documents based on uniqueness across a set of configured fields. Only the first instance of an event with a unique set of those - * fields will be returned. This transform is thread safe. + * fields will be returned unless mostRecentUnique is specified in which case the most recent instance of an event will be returned. This transform is thread + * safe. */ public class UniqueTransform extends DocumentTransform.DefaultDocumentTransform { private static final Logger log = Logger.getLogger(UniqueTransform.class); private BloomFilter bloom; - private UniqueFields uniqueFields; - private Multimap modelMapping; + private UniqueFields uniqueFields = new UniqueFields(); + private HdfsBackedSortedMap map; + private HdfsBackedSortedMap returnSet; + private Iterator> setIterator; /** * Length of time in milliseconds that a client will wait while results are collected. If a full page is not collected before the timeout, a blank page will @@ -66,66 +81,43 @@ public class UniqueTransform extends DocumentTransform.DefaultDocumentTransform public UniqueTransform(UniqueFields uniqueFields, long queryExecutionForPageTimeout) { this.queryExecutionForPageTimeout = queryExecutionForPageTimeout; this.uniqueFields = uniqueFields; - this.uniqueFields.deconstructIdentifierFields(); - this.bloom = BloomFilter.create(new ByteFunnel(), 500000, 1e-15); if (log.isTraceEnabled()) { log.trace("unique fields: " + this.uniqueFields.getFields()); } } /** - * Create a new {@link UniqueTransform} that will use a bloom filter to return on those results that are unique per the uniqueFields. Special uniqueness can - * be requested for date/time fields (@see UniqueFields). The logic will be used to get a query model to include the reverse mappings in the unique field - * set + * Update the configuration of this transform. If the configuration is actually changing, then the bloom filter will be reset as well. * - * @param logic - * The query logic from whih to pull the query model * @param uniqueFields - * The unique fields - * @param queryExecutionForPageTimeout - * If this timeout is passed before since the last result was returned, then an "intermediate" result is returned denoting we are still looking - * for the next unique result. + * The new set of unique fields. */ - public UniqueTransform(BaseQueryLogic> logic, UniqueFields uniqueFields, long queryExecutionForPageTimeout) { - this(uniqueFields, queryExecutionForPageTimeout); - QueryModel model = ((ShardQueryLogic) logic).getQueryModel(); - if (model != null) { - modelMapping = HashMultimap.create(); - // reverse the reverse query mapping which will give us a mapping from the final field name to the original field name(s) - for (Map.Entry entry : model.getReverseQueryMapping().entrySet()) { - modelMapping.put(entry.getValue(), entry.getKey()); - } - } - } - - public void updateConfig(UniqueFields uniqueFields, QueryModel model) { - if (this.uniqueFields != uniqueFields) { - uniqueFields.deconstructIdentifierFields(); - if (!this.uniqueFields.equals(uniqueFields)) { - this.uniqueFields = uniqueFields; - log.info("Resetting unique fields on the unique transform"); - this.bloom = BloomFilter.create(new ByteFunnel(), 500000, 1e-15); - if (log.isTraceEnabled()) { - log.trace("unique fields: " + this.uniqueFields.getFields()); - } + public void updateConfig(UniqueFields uniqueFields) { + // only reset the bloom filter if changing the field set + if (!this.uniqueFields.equals(uniqueFields)) { + this.uniqueFields = uniqueFields.clone(); + log.info("Resetting unique fields on the unique transform"); + if (map != null) { + map.clear(); + returnSet.clear(); + } else { + bloom = BloomFilter.create(new ByteFunnel(), 500000, 1e-15); } - } - if (model != null) { - modelMapping = HashMultimap.create(); - // reverse the reverse query mapping which will give us a mapping from the final field name to the original field name(s) - for (Map.Entry entry : model.getReverseQueryMapping().entrySet()) { - modelMapping.put(entry.getValue(), entry.getKey()); + if (log.isTraceEnabled()) { + log.trace("unique fields: " + this.uniqueFields.getFields()); } } } /** - * Get a predicate that will apply this transform. + * Add phrase excerpts to the documents from the given iterator. * - * @return A unique transform predicate + * @param in + * the iterator source + * @return an iterator that will supply the enriched documents */ - public Predicate> getUniquePredicate() { - return input -> UniqueTransform.this.apply(input) != null; + public Iterator> getIterator(final Iterator> in) { + return new UniqueTransformIterator(in); } /** @@ -143,27 +135,77 @@ public Entry apply(@Nullable Entry keyDocumentEntry) return keyDocumentEntry; } + if (keyDocumentEntry.getValue().isIntermediateResult()) { + return keyDocumentEntry; + } + try { - if (isDuplicate(keyDocumentEntry.getValue())) { - keyDocumentEntry = null; - } else { + if (map != null) { + byte[] signature = getBytes(keyDocumentEntry.getValue()); + synchronized (map) { + this.map.put(signature, keyDocumentEntry.getValue()); + } + return null; + } else if (!isDuplicate(keyDocumentEntry.getValue())) { return keyDocumentEntry; } } catch (IOException ioe) { log.error("Failed to convert document to bytes. Returning document as unique.", ioe); } - } - long elapsedExecutionTimeForCurrentPage = System.currentTimeMillis() - this.queryExecutionForPageStartTime; - if (elapsedExecutionTimeForCurrentPage > this.queryExecutionForPageTimeout) { - Document intermediateResult = new Document(); - intermediateResult.setIntermediateResult(true); - return Maps.immutableEntry(new Key(), intermediateResult); + long elapsedExecutionTimeForCurrentPage = System.currentTimeMillis() - this.queryExecutionForPageStartTime; + if (elapsedExecutionTimeForCurrentPage > this.queryExecutionForPageTimeout) { + Document intermediateResult = new Document(); + intermediateResult.setIntermediateResult(true); + return Maps.immutableEntry(keyDocumentEntry.getKey(), intermediateResult); + } } return null; } + /** + * This will start pulling data from the hdfs backed set if one exists (only if mostRecent is true). + * + * @return The next unique document from the set. + */ + @Override + public Map.Entry flush() { + if (map != null) { + synchronized (map) { + // persist the map so that we do not loose these results and we compact the files for the final iteration. + try { + map.persist(); + } catch (IOException ioe) { + throw new DatawaveFatalQueryException("Unable to persist the most recent unique maps", ioe); + } + if (setIterator == null) { + setupIterator(); + } + if (setIterator.hasNext()) { + return setIterator.next(); + } + } + } + return null; + } + + /** + * This will run through the set and create a new set ordered by Key, Document + */ + private void setupIterator() { + for (Map.Entry entry : map.entrySet()) { + returnSet.put(getDocKey(entry.getValue()), entry.getValue()); + } + // now persist the return set so that we don't lose the results and compact the sets + try { + returnSet.persist(); + } catch (IOException ioe) { + throw new DatawaveFatalQueryException("Could not persist unique document return set", ioe); + } + setIterator = returnSet.entrySet().iterator(); + } + /** * Determine if a document is unique per the fields specified. If we have seen this set of fields and values before, then it is not unique. * @@ -212,71 +254,90 @@ byte[] getBytes(Document document) throws IOException { * if we failed to generate the byte array */ private void outputSortedFieldValues(Document document, DataOutputStream output) throws IOException { - int count = 0; - String lastField = ""; - List values = new ArrayList<>(); + Multimap values = HashMultimap.create(); for (String documentField : new TreeSet<>(document.getDictionary().keySet())) { String field = getUniqueField(documentField); if (field != null) { - if (!field.equals(lastField)) { - count = dumpValues(count, lastField, values, output); - lastField = field; - } addValues(field, document.get(documentField), values); } } - dumpValues(count, lastField, values, output); + // Always dump the fields in the same order (uniqueFields.getFields is a sorted collection) + for (String field : uniqueFields.getFields()) { + dumpValues(field, values.get(field), output); + } output.flush(); } /** * Dump a list of values, sorted, to the data output stream * - * @param count - * value count * @param field * a field * @param values * the list of values * @param output * the output stream - * @return The next field count * @throws IOException * for issues with read/write */ - private int dumpValues(int count, String field, List values, DataOutputStream output) throws IOException { + private void dumpValues(String field, Collection values, DataOutputStream output) throws IOException { + String separator = "f-" + field + ":"; if (!values.isEmpty()) { - Collections.sort(values); - String separator = "f-" + field + '/' + (count++) + ":"; - for (String value : values) { + List valueList = new ArrayList<>(values); + // always output values in sorted order. + Collections.sort(valueList); + for (String value : valueList) { output.writeUTF(separator); output.writeUTF(value); separator = ","; } - values.clear(); + } else { + // dump at least a header for empty value sets to ensure we have some bytes to check against + // in the bloom filter. + output.writeUTF(separator); } - return count; } - // Return the set of values for the provided attribute. - private void addValues(final String field, Attribute attribute, List values) { + /** + * Add the attribute values to the list of values. + * + * @param field + * The attribute field + * @param attribute + * The attribute + * @param values + * The map of values to be updated + */ + private void addValues(final String field, Attribute attribute, Multimap values) { if (attribute instanceof Attributes) { // @formatter:off ((Attributes) attribute).getAttributes().stream() .forEach(a -> addValues(field, a, values)); // @formatter:on } else { - values.add(uniqueFields.transformValue(field, String.valueOf(attribute.getData()))); + values.put(field, uniqueFields.transformValue(field, String.valueOf(attribute.getData()))); } } - // Return the query-specified field that the provided document matches, if one exists, or otherwise return null. + /** + * Return the query-specified field that the provided document matches, if one exists, or otherwise return null. + * + * @param documentField + * The document field + * @return The query specified field + */ private String getUniqueField(String documentField) { String baseDocumentField = getFieldWithoutGrouping(documentField); return uniqueFields.getFields().stream().filter((field) -> isMatchingField(baseDocumentField, field)).findFirst().orElse(null); } - // Return the provided field with any grouping context removed. + /** + * Return the provided field with any grouping context removed. + * + * @param field + * The field + * @return The field with grouping stripped + */ private String getFieldWithoutGrouping(String field) { int index = field.indexOf('.'); if (index < 0) { @@ -286,14 +347,22 @@ private String getFieldWithoutGrouping(String field) { } } - // Return whether or not the provided document field is considered a case-insensitive match for the provided field, applying reverse model mappings if - // configured. + /** + * Return whether or not the provided document field is considered a case-insensitive match for the provided field + * + * @param baseField + * The base field + * @param field + * The field to match with + * @return true if matching + */ private boolean isMatchingField(String baseField, String field) { - baseField = baseField.toUpperCase(); - field = field.toUpperCase(); - return field.equals(baseField) || (modelMapping != null && modelMapping.get(field).contains(baseField)); + return baseField.equalsIgnoreCase(field); } + /** + * A funnel to use for the bloom filter + */ public static class ByteFunnel implements Funnel, Serializable { private static final long serialVersionUID = -2126172579955897986L; @@ -303,4 +372,209 @@ public void funnel(byte[] from, PrimitiveSink into) { into.putBytes(from); } } + + /** + * An iterator of documents for this unique transform given an underlying iterator of documents. + */ + public class UniqueTransformIterator implements Iterator> { + private final Iterator> iterator; + private Map.Entry next = null; + + public UniqueTransformIterator(Iterator> iterator) { + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + if (next == null) { + next = getNext(); + } + return (next != null); + } + + @Override + public Map.Entry next() { + Map.Entry o = null; + if (next == null) { + o = getNext(); + } else { + o = next; + next = null; + } + return o; + } + + private Map.Entry getNext() { + Map.Entry o = null; + while (o == null && iterator.hasNext()) { + o = apply(iterator.next()); + } + // see if there are any results cached by the transform + if (o == null) { + o = flush(); + } + return o; + } + + } + + /** + * A builder of unique transforms + */ + public static class Builder { + private UniqueFields uniqueFields; + private Comparator keyComparator; + private FileSortedMap.RewriteStrategy keyValueComparator; + private QueryModel model; + private int bufferPersistThreshold; + private List ivaratorCacheDirConfigs; + private String hdfsSiteConfigURLs; + private String subDirectory; + private int maxOpenFiles; + private int numRetries; + private long queryExecutionForPageTimeout; + private FileSortedSet.PersistOptions persistOptions; + + public Builder() { + keyComparator = new ByteArrayComparator(); + + keyValueComparator = (key, original, update) -> { + long ts1 = getTimestamp(original); + long ts2 = getTimestamp(update); + return (ts2 > ts1); + }; + } + + /** + * Build a list of potential hdfs directories based on each ivarator cache dir configs. + * + * @param ivaratorCacheDirConfigs + * @param hdfsSiteConfigURLs + * @param subdirectory + * @return A path + * @throws IOException + * for issues with read/write + */ + private static List getIvaratorCacheDirs(List ivaratorCacheDirConfigs, String hdfsSiteConfigURLs, + String subdirectory) throws IOException { + // build a list of ivarator cache dirs from the configs + List pathAndFs = new ArrayList<>(); + if (ivaratorCacheDirConfigs != null && !ivaratorCacheDirConfigs.isEmpty()) { + for (IvaratorCacheDirConfig config : ivaratorCacheDirConfigs) { + + // first, make sure the cache configuration is valid + if (config.isValid()) { + Path path = new Path(config.getBasePathURI(), subdirectory); + URI uri = path.toUri(); + FileSystem fs = new FileSystemCache(hdfsSiteConfigURLs).getFileSystem(uri); + pathAndFs.add(new IvaratorCacheDir(config, fs, uri.toString())); + } + } + } + + if (pathAndFs.isEmpty()) + throw new IOException("Unable to find a usable hdfs cache dir out of " + ivaratorCacheDirConfigs); + + return pathAndFs; + } + + public Builder withUniqueFields(UniqueFields fields) { + this.uniqueFields = fields; + return this; + } + + public Builder withModel(QueryModel model) { + this.model = model; + return this; + } + + public Builder withBufferPersistThreshold(int bufferPersistThreshold) { + this.bufferPersistThreshold = bufferPersistThreshold; + return this; + } + + public Builder withIvaratorCacheDirConfigs(List ivaratorCacheDirConfigs) { + this.ivaratorCacheDirConfigs = ivaratorCacheDirConfigs; + return this; + } + + public Builder withHdfsSiteConfigURLs(String hdfsSiteConfigURLs) { + this.hdfsSiteConfigURLs = hdfsSiteConfigURLs; + return this; + } + + public Builder withSubDirectory(String subDirectory) { + this.subDirectory = subDirectory; + return this; + } + + public Builder withMaxOpenFiles(int maxOpenFiles) { + this.maxOpenFiles = maxOpenFiles; + return this; + } + + public Builder withNumRetries(int numRetries) { + this.numRetries = numRetries; + return this; + } + + public Builder withPersistOptions(FileSortedSet.PersistOptions persistOptions) { + this.persistOptions = persistOptions; + return this; + } + + public Builder withQueryExecutionForPageTimeout(long timeout) { + this.queryExecutionForPageTimeout = timeout; + return this; + } + + public UniqueTransform build() throws IOException { + UniqueTransform transform = new UniqueTransform(uniqueFields, queryExecutionForPageTimeout); + + if (transform.uniqueFields.isMostRecent()) { + // @formatter:off + // noinspection unchecked + transform.map = (HdfsBackedSortedMap) HdfsBackedSortedMap.builder() + .withComparator(keyComparator) + .withRewriteStrategy(keyValueComparator) + .withBufferPersistThreshold(bufferPersistThreshold) + .withIvaratorCacheDirs(getIvaratorCacheDirs(ivaratorCacheDirConfigs, hdfsSiteConfigURLs, subDirectory)) + .withUniqueSubPath("byUniqueKey") + .withMaxOpenFiles(maxOpenFiles) + .withNumRetries(numRetries) + .withPersistOptions(persistOptions) + .withMapFactory(new FileByteDocumentSortedMap.Factory()) + .build(); + + // noinspection unchecked + transform.returnSet = (HdfsBackedSortedMap) HdfsBackedSortedMap.builder() + .withBufferPersistThreshold(bufferPersistThreshold) + .withIvaratorCacheDirs(getIvaratorCacheDirs(ivaratorCacheDirConfigs, hdfsSiteConfigURLs, subDirectory)) + .withUniqueSubPath("byDocKey") + .withMaxOpenFiles(maxOpenFiles) + .withNumRetries(numRetries) + .withPersistOptions(persistOptions) + .withMapFactory(new FileKeyDocumentSortedMap.Factory()) + .build(); + // @formatter:on + } else { + transform.bloom = BloomFilter.create(new ByteFunnel(), 500000, 1e-15); + } + + return transform; + } + } + + private static long getTimestamp(Document doc) { + return getDocKeyAttr(doc).getTimestamp(); + } + + private static DocumentKey getDocKeyAttr(Document doc) { + return (DocumentKey) (doc.get(Document.DOCKEY_FIELD_NAME)); + } + + private static Key getDocKey(Document doc) { + return getDocKeyAttr(doc).getDocKey(); + } + } diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMap.java new file mode 100644 index 00000000000..b1eb4bed782 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMap.java @@ -0,0 +1,545 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.stream.Collectors; + +import org.apache.log4j.Logger; + +import datawave.query.util.sortedmap.FileSortedMap.SortedMapFileHandler; + +/** + * This is a sorted map that will hold up to a specified number of entries before flushing the data to disk. Files will be created as needed. An additional + * "persist" call is supplied to force flushing to disk. The iterator.remove and the submap operations will work up until any buffer has been flushed to disk. + * After that, those operations will not work as specified by the underlying FileSortedMap. + * + * @param + * key of the map + * @param + * value of the map + **/ +public class BufferedFileBackedSortedMap implements SortedMap, RewritableSortedMap { + private static final Logger log = Logger.getLogger(BufferedFileBackedSortedMap.class); + protected static final int DEFAULT_BUFFER_PERSIST_THRESHOLD = 1000; + protected static final int DEFAULT_MAX_OPEN_FILES = 100; + protected static final int DEFAULT_NUM_RETRIES = 2; + + protected MultiMapBackedSortedMap map = new MultiMapBackedSortedMap<>(); + protected int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; + protected FileSortedMap buffer = null; + protected FileSortedMap.FileSortedMapFactory mapFactory = null; + protected final Comparator comparator; + protected FileSortedMap.RewriteStrategy rewriteStrategy; + protected boolean sizeModified = false; + protected int size = 0; + protected int numRetries = DEFAULT_NUM_RETRIES; + + protected List handlerFactories; + protected int bufferPersistThreshold = DEFAULT_BUFFER_PERSIST_THRESHOLD; + + /** + * A factory for SortedMapFileHandlers + * + * + * + */ + public interface SortedMapFileHandlerFactory { + SortedMapFileHandler createHandler() throws IOException; + + boolean isValid(); + } + + public static class Builder,K,V> { + private int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; + private FileSortedMap.FileSortedMapFactory mapFactory = new FileSerializableSortedMap.Factory(); + private Comparator comparator; + private FileSortedMap.RewriteStrategy rewriteStrategy; + private int numRetries = DEFAULT_NUM_RETRIES; + private List handlerFactories = new ArrayList<>(); + private int bufferPersistThreshold = DEFAULT_BUFFER_PERSIST_THRESHOLD; + + public Builder() {} + + @SuppressWarnings("unchecked") + protected B self() { + return (B) this; + } + + public B withMaxOpenFiles(int maxOpenFiles) { + this.maxOpenFiles = maxOpenFiles; + return self(); + } + + @SuppressWarnings("unchecked") + public B withMapFactory(FileSortedMap.FileSortedMapFactory mapFactory) { + this.mapFactory = (FileSortedMap.FileSortedMapFactory) mapFactory; + return self(); + } + + @SuppressWarnings("unchecked") + public B withComparator(Comparator comparator) { + this.comparator = (Comparator) comparator; + return self(); + } + + @SuppressWarnings("unchecked") + public B withRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = (FileSortedMap.RewriteStrategy) rewriteStrategy; + return self(); + } + + public B withNumRetries(int numRetries) { + this.numRetries = numRetries; + return self(); + } + + public B withHandlerFactories(List handlerFactories) { + this.handlerFactories = handlerFactories; + return self(); + } + + public B withBufferPersistThreshold(int bufferPersistThreshold) { + this.bufferPersistThreshold = bufferPersistThreshold; + return self(); + } + + public BufferedFileBackedSortedMap build() throws Exception { + return new BufferedFileBackedSortedMap<>(this); + } + } + + public static Builder builder() { + return new Builder<>(); + } + + protected BufferedFileBackedSortedMap(BufferedFileBackedSortedMap other) { + this.comparator = other.comparator; + this.rewriteStrategy = other.rewriteStrategy; + this.handlerFactories = new ArrayList<>(other.handlerFactories); + this.mapFactory = other.mapFactory; + this.bufferPersistThreshold = other.bufferPersistThreshold; + this.numRetries = other.numRetries; + this.maxOpenFiles = other.maxOpenFiles; + for (SortedMap submap : other.map.getMaps()) { + FileSortedMap clone = ((FileSortedMap) submap).clone(); + this.map.addMap(clone); + if (!clone.isPersisted()) { + this.buffer = clone; + } + } + this.sizeModified = other.sizeModified; + this.size = other.size; + } + + protected BufferedFileBackedSortedMap(Builder builder) { + this.comparator = builder.comparator; + this.rewriteStrategy = builder.rewriteStrategy; + this.handlerFactories = new ArrayList<>(builder.handlerFactories); + this.mapFactory = builder.mapFactory; + this.bufferPersistThreshold = builder.bufferPersistThreshold; + this.numRetries = builder.numRetries; + this.maxOpenFiles = builder.maxOpenFiles; + } + + private SortedMapFileHandler createFileHandler(SortedMapFileHandlerFactory handlerFactory) throws IOException { + if (handlerFactory.isValid()) { + try { + return handlerFactory.createHandler(); + } catch (IOException e) { + log.warn("Unable to create file handler using handler factory: " + handlerFactory, e); + } + } + + return null; + } + + public void persist() throws IOException { + if (buffer != null) { + // go through the handler factories and try to persist the sorted map + for (int i = 0; i < handlerFactories.size() && !buffer.isPersisted(); i++) { + SortedMapFileHandlerFactory handlerFactory = handlerFactories.get(i); + SortedMapFileHandler handler = createFileHandler(handlerFactory); + + // if we have a valid handler, try to persist + if (handler != null) { + Exception cause = null; + for (int attempts = 0; attempts <= numRetries && !buffer.isPersisted(); attempts++) { + try { + buffer.persist(handler); + } catch (IOException e) { + if (attempts == numRetries) + cause = e; + } + } + + if (!buffer.isPersisted()) { + log.warn("Unable to persist the sorted map using the file handler: " + handler, cause); + + // if this was an hdfs file handler, decrement the count + if (handlerFactory instanceof HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) { + HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) handlerFactory); + hdfsHandlerFactory.mapFileCount(hdfsHandlerFactory.getFileCount() - 1); + } + } + } else { + log.warn("Unable to create a file handler using the handler factory: " + handlerFactory); + } + } + + // if the buffer was not persisted, throw an exception + if (!buffer.isPersisted()) + throw new IOException("Unable to persist the sorted map using the configured handler factories."); + + buffer = null; + compact(maxOpenFiles); + } + } + + protected List> getMaps() { + List> maps = new ArrayList<>(); + for (SortedMap submap : map.getMaps()) { + maps.add((FileSortedMap) submap); + } + return maps; + } + + protected void addMap(FileSortedMap submap) { + map.addMap(submap); + size += submap.size(); + } + + public boolean hasPersistedData() { + for (SortedMap submap : map.getMaps()) { + if (((FileSortedMap) submap).isPersisted()) { + return true; + } + } + return false; + } + + public boolean isPersisted() { + // we are (completely) persisted iff the buffer is persisted + return (buffer == null || buffer.isPersisted()); + } + + @Override + public int size() { + if (sizeModified) { + this.size = map.size(); + sizeModified = false; + } + return this.size; + } + + public int getBufferPersistThreshold() { + return this.bufferPersistThreshold; + } + + public int getBufferSize() { + return (this.buffer == null ? 0 : this.buffer.size()); + } + + @Override + public boolean isEmpty() { + return size() == 0; + } + + @Override + public boolean containsKey(Object o) { + // try the cheap operation first + if (buffer != null && buffer.containsKey(o)) { + return true; + } else { + return map.containsKey(o); + } + } + + @Override + public boolean containsValue(Object value) { + return false; + } + + private String printHandlerFactories() { + return String.join(", ", handlerFactories.stream().map(SortedMapFileHandlerFactory::toString).collect(Collectors.toList())); + } + + /** + * If the number of maps is over maxFiles, then start compacting those files down. The goal is to get the number of files down around 50% of maxFiles. + * + * @param maxFiles + * the max number of files + * @throws IOException + * for IO Exceptions + */ + public void compact(int maxFiles) throws IOException { + // if we have more maps than we are allowed, then we need to compact this down + if (maxFiles > 0 && map.getMaps().size() > maxFiles) { + if (log.isDebugEnabled()) { + log.debug("Compacting [" + printHandlerFactories() + "]"); + } + // create a copy of the map list (sorting below) + List> maps = new ArrayList<>(map.getMaps()); + + // calculate the number of maps to compact + int nummaps = maps.size(); + int excessmaps = nummaps - (maxFiles / 2); // those over 50% of maxFiles + int mapsPerCompaction = Math.min(excessmaps + 1, nummaps); // Add in 1 to account for the compacted map being added back in + + // sort the maps by size (compact up smaller maps first) + maps.sort(Comparator.comparing(SortedMap::size).reversed()); + + // newmap will be the final multimap + MultiMapBackedSortedMap newmap = new MultiMapBackedSortedMap<>(); + + // create a map for those maps to be compacted into one file + MultiMapBackedSortedMap mapToCompact = new MultiMapBackedSortedMap<>(); + for (int i = 0; i < mapsPerCompaction; i++) { + mapToCompact.addMap(maps.remove(maps.size() - 1)); + } + + // compact it + if (log.isDebugEnabled()) { + log.debug("Starting compaction for " + mapToCompact); + } + long start = System.currentTimeMillis(); + FileSortedMap compaction = compact(mapToCompact); + if (log.isDebugEnabled()) { + long delta = System.currentTimeMillis() - start; + log.debug("Compacted " + mapToCompact + " -> " + compaction + " in " + delta + "ms"); + } + + // add the compacted map to our final multimap + newmap.addMap(compaction); + + // clear the compactions map to remove the files that were compacted + mapToCompact.clear(); + + // now add in the maps we did not compact + for (int i = 0; i < maps.size(); i++) { + newmap.addMap(maps.get(i)); + } + + // and replace our map + this.map = newmap; + } + } + + private FileSortedMap compact(MultiMapBackedSortedMap mapToCompact) throws IOException { + FileSortedMap compactedmap = null; + + // go through the handler factories and try to persist the sorted map + for (int i = 0; i < handlerFactories.size() && compactedmap == null; i++) { + SortedMapFileHandlerFactory handlerFactory = handlerFactories.get(i); + SortedMapFileHandler handler = createFileHandler(handlerFactory); + + // if we have a valid handler, try to persist + if (handler != null) { + Exception cause = null; + for (int attempts = 0; attempts <= numRetries && compactedmap == null; attempts++) { + try { + compactedmap = mapFactory.newInstance(mapToCompact, handlerFactory.createHandler(), true); + } catch (IOException e) { + if (attempts == numRetries) + cause = e; + } + } + + if (compactedmap == null) { + log.warn("Unable to compact the sorted map using the file handler: " + handler, cause); + + // if this was an hdfs file handler, decrement the count + if (handlerFactory instanceof HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) { + HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) handlerFactory); + hdfsHandlerFactory.mapFileCount(hdfsHandlerFactory.getFileCount() - 1); + } + } + } else { + log.warn("Unable to create a file handler using the handler factory: " + handlerFactory); + } + } + + // if the sorted maps were not compacted, throw an exception + if (compactedmap == null) + throw new IOException("Unable to persist the sorted map using the configured handler factories."); + + return compactedmap; + } + + @Override + public V put(K key, V value) { + if (buffer == null) { + try { + buffer = mapFactory.newInstance(comparator, rewriteStrategy, null, false); + } catch (Exception ex) { + throw new IllegalStateException("Unable to create an underlying FileSortedMap", ex); + } + + map.addMap(buffer); + } + V previous = buffer.put(key, value); + sizeModified = true; + if (previous != null) { + if (buffer.size() >= bufferPersistThreshold) { + try { + persist(); + } catch (Exception ex) { + throw new IllegalStateException("Unable to persist or compact FileSortedMap", ex); + } + } + return previous; + } + return null; + } + + @Override + public void putAll(Map c) { + if (buffer == null) { + try { + buffer = mapFactory.newInstance(comparator, rewriteStrategy, null, false); + } catch (Exception ex) { + throw new IllegalStateException("Unable to create an underlying FileSortedMap", ex); + } + map.addMap(buffer); + } + buffer.putAll(c); + sizeModified = true; + if (buffer.size() >= bufferPersistThreshold) { + try { + persist(); + } catch (Exception ex) { + throw new IllegalStateException("Unable to persist or compact FileSortedMap", ex); + } + } + } + + @Override + public V remove(Object o) { + V value = null; + for (SortedMap map : map.getMaps()) { + FileSortedMap filemap = (FileSortedMap) map; + boolean persist = false; + if (filemap.isPersisted()) { + try { + filemap.load(); + persist = true; + } catch (Exception e) { + throw new IllegalStateException("Unable to remove item from underlying files", e); + } + } + + V testValue = map.remove(o); + if (testValue != null) { + if (value != null) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { + value = testValue; + } + } else { + value = testValue; + } + } + + if (persist) { + try { + filemap.persist(); + } catch (Exception e) { + throw new IllegalStateException("Unable to remove item from underlying files", e); + } + } + } + if (value != null) { + this.sizeModified = true; + } + return value; + } + + @Override + public void clear() { + // This will cause the MultimapBackedSortedMap to call clear on each map in its map of maps, including the buffer + // It will also call clear on its map of maps, emptying the contents + map.clear(); + // Null the buffer so that it will start new on the next add + buffer = null; + this.size = 0; + this.sizeModified = false; + } + + @Override + public Comparator comparator() { + return comparator; + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return map.subMap(fromKey, toKey); + } + + @Override + public SortedMap headMap(K toKey) { + return map.headMap(toKey); + } + + @Override + public SortedMap tailMap(K fromKey) { + return map.tailMap(fromKey); + } + + @Override + public K firstKey() { + return map.firstKey(); + } + + @Override + public K lastKey() { + return map.lastKey(); + } + + @Override + public Set keySet() { + return map.keySet(); + } + + @Override + public Collection values() { + return map.values(); + } + + @Override + public Set> entrySet() { + return map.entrySet(); + } + + @Override + public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = rewriteStrategy; + } + + @Override + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; + } + + @Override + public V get(Object o) { + V value = null; + for (SortedMap map : map.getMaps()) { + V testValue = map.get(o); + if (testValue != null) { + if (value != null) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { + value = testValue; + } + } else { + value = testValue; + } + } + } + return value; + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileByteDocumentSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileByteDocumentSortedMap.java new file mode 100644 index 00000000000..76c34e0d63b --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileByteDocumentSortedMap.java @@ -0,0 +1,207 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.util.Comparator; +import java.util.SortedMap; + +import org.apache.log4j.Logger; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.rfile.RFileByteDocumentInputStream; +import datawave.query.util.sortedmap.rfile.RFileByteDocumentOutputStream; +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.FileSortedSet; + +/** + * A sorted map that can be persisted into a file and still be read in its persisted state. The map can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying maps iff a comparator is supplied that can handle null values. + * + * The persisted file will contain the serialized entries, followed by the actual size. + * + */ +public class FileByteDocumentSortedMap extends FileSortedMap { + private static Logger log = Logger.getLogger(FileByteDocumentSortedMap.class); + + public final static class DefaultByteComparator implements Comparator { + + @Override + public int compare(byte[] o1, byte[] o2) { + return new ByteArrayComparator().compare(o1, o2); + } + } + + /** + * Create a file sorted map from another one + * + * @param other + * the other sorted map + */ + public FileByteDocumentSortedMap(FileByteDocumentSortedMap other) { + super(other); + } + + /** + * Create a file sorted submap from another one + * + * @param other + * the other sorted map + * @param from + * the from key + * @param to + * the to key + */ + public FileByteDocumentSortedMap(FileByteDocumentSortedMap other, byte[] from, byte[] to) { + super(other, from, to); + } + + /** + * Create a persisted sorted map + * + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + */ + public FileByteDocumentSortedMap(SortedMapFileHandler handler, boolean persisted) { + this(new DefaultByteComparator(), handler, persisted); + } + + /** + * Create a persisted sorted map + * + * @param comparator + * the key comparator + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + */ + public FileByteDocumentSortedMap(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultByteComparator() : comparator), new ByteDocumentFileHandler(handler), new Factory(), persisted); + } + + /** + * Create an unpersisted sorted map (still in memory) + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + */ + public FileByteDocumentSortedMap(SortedMap map, SortedMapFileHandler handler) { + super(map, new ByteDocumentFileHandler(handler), new Factory()); + } + + /** + * Create a sorted map out of another sorted map. If persist is true, then the map will be directly persisted using the map's iterator which avoids pulling + * all of its entries into memory at once. + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + * @param persist + * boolean flag for persist + * @throws IOException + * for issues with read/write + */ + public FileByteDocumentSortedMap(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + super(map, new ByteDocumentFileHandler(handler), new Factory(), persist); + } + + /** + * This will dump the map to the file, making the map "persisted" + * + * @param handler + * the sorted map file handler + * @throws IOException + * for issues with read/write + */ + public void persist(SortedMapFileHandler handler) throws IOException { + // ensure this handler is wrapped with our handler + super.persist(new ByteDocumentFileHandler(handler)); + } + + /** + * Clone this map + */ + @Override + public FileByteDocumentSortedMap clone() { + return (FileByteDocumentSortedMap) super.clone(); + } + + /** + * A SortedMapfilehandler that can bound the input stream + */ + public static class ByteDocumentFileHandler implements BoundedTypedSortedMapFileHandler { + SortedMapFileHandler delegate; + + public ByteDocumentFileHandler(SortedMapFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedMapInputStream getInputStream() throws IOException { + return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedMapInputStream getInputStream(byte[] start, byte[] end) throws IOException { + return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize(), start, end); + } + + @Override + public SortedMapOutputStream getOutputStream() throws IOException { + return new RFileByteDocumentOutputStream(delegate.getOutputStream()); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + /** + * A factory for these file sorted maps + */ + public static class Factory implements FileSortedMapFactory { + + @Override + public FileByteDocumentSortedMap newInstance(FileSortedMap other) { + return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other); + } + + @Override + public FileByteDocumentSortedMap newInstance(FileSortedMap other, byte[] from, byte[] to) { + return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other, from, to); + } + + @Override + public FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, + SortedMapFileHandler handler, boolean persisted) { + FileByteDocumentSortedMap map = new FileByteDocumentSortedMap(comparator, handler, persisted); + map.setRewriteStrategy(rewriteStrategy); + return map; + } + + @Override + public FileByteDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileByteDocumentSortedMap(map, handler); + } + + @Override + public FileByteDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + return new FileByteDocumentSortedMap(map, handler, persist); + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyDocumentSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyDocumentSortedMap.java new file mode 100644 index 00000000000..6fdb085f8e5 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyDocumentSortedMap.java @@ -0,0 +1,206 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.util.Comparator; +import java.util.SortedMap; + +import org.apache.accumulo.core.data.Key; +import org.apache.log4j.Logger; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.rfile.RFileKeyDocumentInputStream; +import datawave.query.util.sortedmap.rfile.RFileKeyDocumentOutputStream; +import datawave.query.util.sortedset.FileSortedSet; + +/** + * A sorted map that can be persisted into a file and still be read in its persisted state. The map can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying maps iff a comparator is supplied that can handle null values. + * + * The persisted file will contain the serialized entries, followed by the actual size. + * + */ +public class FileKeyDocumentSortedMap extends FileSortedMap { + private static Logger log = Logger.getLogger(FileKeyDocumentSortedMap.class); + + public static class DefaultKeyComparator implements Comparator { + @Override + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); + } + } + + /** + * Create a file sorted map from another one + * + * @param other + * the other sorted map + */ + public FileKeyDocumentSortedMap(FileKeyDocumentSortedMap other) { + super(other); + } + + /** + * Create a file sorted submap from another one + * + * @param other + * the other sorted map + * @param from + * the from key + * @param to + * the to key + */ + public FileKeyDocumentSortedMap(FileKeyDocumentSortedMap other, Key from, Key to) { + super(other, from, to); + } + + /** + * Create a persisted sorted map + * + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyDocumentSortedMap(SortedMapFileHandler handler, boolean persisted) { + this(new DefaultKeyComparator(), handler, persisted); + } + + /** + * Create a persisted sorted map + * + * @param comparator + * the key comparator + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyDocumentSortedMap(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultKeyComparator() : comparator), new KeyDocumentFileHandler(handler), new Factory(), persisted); + } + + /** + * Create an unpersisted sorted map (still in memory) + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + */ + public FileKeyDocumentSortedMap(SortedMap map, SortedMapFileHandler handler) { + super(map, new KeyDocumentFileHandler(handler), new Factory()); + } + + /** + * Create a sorted map out of another sorted map. If persist is true, then the map will be directly persisted using the map's iterator which avoids pulling + * all of its entries into memory at once. + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + * @param persist + * boolean flag for persist + * @throws IOException + * for issues with read/write + */ + public FileKeyDocumentSortedMap(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + super(map, new KeyDocumentFileHandler(handler), new Factory(), persist); + } + + /** + * This will dump the map to the file, making the map "persisted" + * + * @param handler + * the sorted map file handler + * @throws IOException + * for issues with read/write + */ + public void persist(SortedMapFileHandler handler) throws IOException { + // ensure this handler is wrapped with our handler + super.persist(new KeyDocumentFileHandler(handler)); + } + + /** + * Clone this map + */ + @Override + public FileKeyDocumentSortedMap clone() { + return (FileKeyDocumentSortedMap) super.clone(); + } + + /** + * A SortedMapfilehandler that can bound the input stream + */ + public static class KeyDocumentFileHandler implements BoundedTypedSortedMapFileHandler { + SortedMapFileHandler delegate; + + public KeyDocumentFileHandler(SortedMapFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedMapInputStream getInputStream() throws IOException { + return new RFileKeyDocumentInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedMapInputStream getInputStream(Key start, Key end) throws IOException { + return new RFileKeyDocumentInputStream(delegate.getInputStream(), delegate.getSize(), start, end); + } + + @Override + public SortedMapOutputStream getOutputStream() throws IOException { + return new RFileKeyDocumentOutputStream(delegate.getOutputStream()); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + /** + * A factory for these file sorted maps + */ + public static class Factory implements FileSortedMapFactory { + + @Override + public FileKeyDocumentSortedMap newInstance(FileSortedMap other) { + return new FileKeyDocumentSortedMap((FileKeyDocumentSortedMap) other); + } + + @Override + public FileKeyDocumentSortedMap newInstance(FileSortedMap other, Key from, Key to) { + return new FileKeyDocumentSortedMap((FileKeyDocumentSortedMap) other, from, to); + } + + @Override + public FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, + boolean persisted) { + FileKeyDocumentSortedMap map = new FileKeyDocumentSortedMap(comparator, handler, persisted); + map.setRewriteStrategy(rewriteStrategy); + return map; + } + + @Override + public FileKeyDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileKeyDocumentSortedMap(map, handler); + } + + @Override + public FileKeyDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + return new FileKeyDocumentSortedMap(map, handler, persist); + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyValueSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyValueSortedMap.java new file mode 100644 index 00000000000..976b5d75d47 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyValueSortedMap.java @@ -0,0 +1,206 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.util.Comparator; +import java.util.SortedMap; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.log4j.Logger; + +import datawave.query.util.sortedmap.rfile.RFileKeyValueInputStream; +import datawave.query.util.sortedmap.rfile.RFileKeyValueOutputStream; +import datawave.query.util.sortedset.FileSortedSet; + +/** + * A sorted map that can be persisted into a file and still be read in its persisted state. The map can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying maps iff a comparator is supplied that can handle null values. + * + * The persisted file will contain the serialized entries, followed by the actual size. + * + */ +public class FileKeyValueSortedMap extends FileSortedMap { + private static Logger log = Logger.getLogger(FileKeyValueSortedMap.class); + + public static class DefaultKeyComparator implements Comparator { + @Override + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); + } + } + + /** + * Create a file sorted map from another one + * + * @param other + * the other sorted map + */ + public FileKeyValueSortedMap(FileKeyValueSortedMap other) { + super(other); + } + + /** + * Create a file sorted submap from another one + * + * @param other + * the other sorted map + * @param from + * the from key + * @param to + * the to key + */ + public FileKeyValueSortedMap(FileKeyValueSortedMap other, Key from, Key to) { + super(other, from, to); + } + + /** + * Create a persisted sorted map + * + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyValueSortedMap(SortedMapFileHandler handler, boolean persisted) { + this(new DefaultKeyComparator(), handler, persisted); + } + + /** + * Create a persisted sorted map + * + * @param comparator + * the key comparator + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyValueSortedMap(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { + super(((comparator == null) ? new DefaultKeyComparator() : comparator), new KeyValueFileHandler(handler), new Factory(), persisted); + } + + /** + * Create an unpersisted sorted map (still in memory) + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + */ + public FileKeyValueSortedMap(SortedMap map, SortedMapFileHandler handler) { + super(map, new KeyValueFileHandler(handler), new Factory()); + } + + /** + * Create a sorted map out of another sorted map. If persist is true, then the map will be directly persisted using the map's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + * @param persist + * boolean flag for persist + * @throws IOException + * for issues with read/write + */ + public FileKeyValueSortedMap(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + super(map, new KeyValueFileHandler(handler), new Factory(), persist); + } + + /** + * This will dump the map to the file, making the map "persisted" + * + * @param handler + * the sorted map file handler + * @throws IOException + * for issues with read/write + */ + public void persist(SortedMapFileHandler handler) throws IOException { + // ensure this handler is wrapped with our handler + super.persist(new KeyValueFileHandler(handler)); + } + + /** + * Clone this map + */ + @Override + public FileKeyValueSortedMap clone() { + return (FileKeyValueSortedMap) super.clone(); + } + + /** + * A SortedMapfilehandler that can bound the input stream + */ + public static class KeyValueFileHandler implements BoundedTypedSortedMapFileHandler { + SortedMapFileHandler delegate; + + public KeyValueFileHandler(SortedMapFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedMapInputStream getInputStream() throws IOException { + return new RFileKeyValueInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedMapInputStream getInputStream(Key start, Key end) throws IOException { + return new RFileKeyValueInputStream(delegate.getInputStream(), delegate.getSize(), start, end); + } + + @Override + public SortedMapOutputStream getOutputStream() throws IOException { + return new RFileKeyValueOutputStream(delegate.getOutputStream()); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + /** + * A factory for these file sorted maps + */ + public static class Factory implements FileSortedMapFactory { + + @Override + public FileKeyValueSortedMap newInstance(FileSortedMap other) { + return new FileKeyValueSortedMap((FileKeyValueSortedMap) other); + } + + @Override + public FileKeyValueSortedMap newInstance(FileSortedMap other, Key from, Key to) { + return new FileKeyValueSortedMap((FileKeyValueSortedMap) other, from, to); + } + + @Override + public FileKeyValueSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStategy, SortedMapFileHandler handler, + boolean persisted) { + FileKeyValueSortedMap map = new FileKeyValueSortedMap(comparator, handler, persisted); + map.setRewriteStrategy(rewriteStategy); + return map; + } + + @Override + public FileKeyValueSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileKeyValueSortedMap(map, handler); + } + + @Override + public FileKeyValueSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + return new FileKeyValueSortedMap(map, handler, persist); + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSerializableSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSerializableSortedMap.java new file mode 100644 index 00000000000..c61edd025da --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSerializableSortedMap.java @@ -0,0 +1,289 @@ +package datawave.query.util.sortedmap; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Comparator; +import java.util.Map; +import java.util.SortedMap; + +import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; +import org.apache.log4j.Logger; + +import datawave.query.util.sortedset.FileSortedSet; + +/** + * A sorted map that can be persisted into a file and still be read in its persisted state. The map can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying maps iff a comparator is supplied that can handle null values. + * + * The persisted file will contain the serialized entries, followed by the actual size. + * + * @param + * key of the map + * @param + * value of the map + */ +public class FileSerializableSortedMap extends FileSortedMap { + private static Logger log = Logger.getLogger(FileSerializableSortedMap.class); + + /** + * Create a file sorted map from another one + * + * @param other + * the other sorted map + */ + public FileSerializableSortedMap(FileSerializableSortedMap other) { + super(other); + } + + /** + * Create a file sorted submap from another one + * + * @param other + * the other sorted map + * @param from + * the from file + * @param to + * the to file + */ + public FileSerializableSortedMap(FileSerializableSortedMap other, K from, K to) { + super(other, from, to); + } + + /** + * Create a persisted sorted map + * + * @param handler + * a file handler + * @param persisted + * persisted boolean flag + */ + public FileSerializableSortedMap(TypedSortedMapFileHandler handler, boolean persisted) { + super(handler, new Factory(), persisted); + } + + /** + * Create a persistede sorted map + * + * @param comparator + * a comparator + * @param handler + * a file handler + * @param persisted + * persisted boolean flag + */ + public FileSerializableSortedMap(Comparator comparator, TypedSortedMapFileHandler handler, boolean persisted) { + super(comparator, handler, new Factory(), persisted); + } + + /** + * Create an unpersisted sorted map (still in memory) + * + * @param map + * a sorted map + * @param handler + * a file handler + */ + public FileSerializableSortedMap(SortedMap map, TypedSortedMapFileHandler handler) { + super(map, handler, new Factory()); + } + + /** + * Create an sorted map out of another sorted map. If persist is true, then the map will be directly persisted using the map's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param map + * a sorted map + * @param handler + * a file handler + * @param persist + * a persist flag + * @throws IOException + * for issues with read/write + */ + public FileSerializableSortedMap(SortedMap map, TypedSortedMapFileHandler handler, boolean persist) throws IOException { + super(map, handler, new Factory(), persist); + } + + /** + * Persist a map using the specified handler + * + * @param handler + * a file handler + * @throws IOException + * for issues with read/write + */ + @Override + public void persist(SortedMapFileHandler handler) throws IOException { + super.persist(new SerializableFileHandler(handler)); + } + + @Override + public FileSerializableSortedMap clone() { + return (FileSerializableSortedMap) super.clone(); + } + + /** + * A SortedMapfilehandler that can handler serializable objects + */ + public static class SerializableFileHandler implements TypedSortedMapFileHandler { + SortedMapFileHandler delegate; + + public SerializableFileHandler(SortedMapFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedMapInputStream getInputStream() throws IOException { + return new SerializableInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedMapOutputStream getOutputStream() throws IOException { + return new SerializableOutputStream(delegate.getOutputStream()); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + public static class SerializableInputStream implements SortedMapInputStream { + private final InputStream stream; + private ObjectInputStream delegate; + private final long length; + + public SerializableInputStream(InputStream stream, long length) throws IOException { + this.stream = stream; + this.length = length; + } + + private ObjectInputStream getDelegate() throws IOException { + if (delegate == null) { + this.delegate = new ObjectInputStream(stream); + } + return delegate; + } + + @Override + public Map.Entry readObject() throws IOException { + try { + K key = (K) getDelegate().readObject(); + V value = (V) getDelegate().readObject(); + return new UnmodifiableMapEntry<>(key, value); + } catch (IOException ioe) { + return null; + } catch (ClassNotFoundException nnfe) { + return null; + } + } + + @Override + public int readSize() throws IOException { + long bytesToSkip = length - 4; + long total = 0; + long cur = 0; + + while ((total < bytesToSkip) && ((cur = stream.skip(bytesToSkip - total)) > 0)) { + total += cur; + } + + byte[] buffer = new byte[4]; + stream.read(buffer); + + // read the 4 bytes of an integer in a deterministic order0 + return ((buffer[3] & 0xFF)) + ((buffer[2] & 0xFF) << 8) + ((buffer[1] & 0xFF) << 16) + ((buffer[0]) << 24); + } + + @Override + public void close() { + try { + if (delegate != null) { + delegate.close(); + } else { + stream.close(); + } + } catch (Exception e) { + log.error("Failed to close input stream", e); + } + } + } + + public static class SerializableOutputStream implements FileSortedMap.SortedMapOutputStream { + private ObjectOutputStream delegate; + + public SerializableOutputStream(OutputStream stream) throws IOException { + delegate = new ObjectOutputStream(stream); + } + + @Override + public void writeObject(K key, V value) throws IOException { + delegate.writeObject(key); + delegate.writeObject(value); + } + + @Override + public void writeSize(int size) throws IOException { + // write the 4 bytes of a integer in a deterministic order0 + delegate.write((size >>> 24) & 0xFF); + delegate.write((size >>> 16) & 0xFF); + delegate.write((size >>> 8) & 0xFF); + delegate.write((size >>> 0) & 0xFF); + } + + @Override + public void close() throws IOException { + delegate.close(); + } + } + + /** + * A factory for this map + */ + public static class Factory implements FileSortedMapFactory { + + @Override + public FileSerializableSortedMap newInstance(FileSortedMap other) { + return new FileSerializableSortedMap((FileSerializableSortedMap) other); + } + + @Override + public FileSerializableSortedMap newInstance(FileSortedMap other, K from, K to) { + return new FileSerializableSortedMap((FileSerializableSortedMap) other, from, to); + } + + @Override + public FileSerializableSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, + boolean persisted) { + FileSerializableSortedMap map = new FileSerializableSortedMap(comparator, new SerializableFileHandler(handler), persisted); + map.setRewriteStrategy(rewriteStrategy); + return map; + } + + @Override + public FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileSerializableSortedMap(map, new SerializableFileHandler(handler)); + } + + @Override + public FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + return new FileSerializableSortedMap(map, new SerializableFileHandler(handler), persist); + } + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSortedMap.java new file mode 100644 index 00000000000..d29c2e37150 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSortedMap.java @@ -0,0 +1,1073 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.AbstractCollection; +import java.util.AbstractSet; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.commons.collections.IteratorUtils; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.apache.log4j.Logger; + +import datawave.query.util.sortedset.FileSortedSet; +import datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; + +/** + * A sorted map that can be persisted into a file and still be read in its persisted state. The map can always be re-loaded and then all operations will work as + * expected. This class will not support null values. + * + * The persisted file will contain the serialized entries, followed by the actual size. + * + * A RewriteStrategy can be supplied that will determine whether a value gets replaced when putting a key,value pair. + * + * @param + * key of the map + * @param + * value of the map + */ +public abstract class FileSortedMap implements SortedMap, Cloneable, RewritableSortedMap { + private static final Logger log = Logger.getLogger(FileSortedMap.class); + protected boolean persisted; + protected K[] range; + protected SortedMap map; + protected RewriteStrategy rewriteStrategy = null; + + // The file handler that handles the underlying io + protected TypedSortedMapFileHandler handler; + // The sort map factory + protected FileSortedMapFactory factory; + + /** + * A class that represents a null object within the map + */ + public static class NullObject implements Serializable { + private static final long serialVersionUID = -5528112099317370355L; + } + + /** + * Create a file sorted map from another one + * + * @param other + * the other sorted map + */ + public FileSortedMap(FileSortedMap other) { + this.handler = other.handler; + this.factory = other.factory; + this.map = new TreeMap<>(other.map); + this.persisted = other.persisted; + this.range = other.range; + this.rewriteStrategy = other.rewriteStrategy; + } + + /** + * Create a file sorted submap from another one + * + * @param other + * the other sorted map + * @param from + * the from key + * @param to + * the to key + */ + public FileSortedMap(FileSortedMap other, K from, K to) { + this(other); + if (from != null || to != null) { + if (persisted) { + this.range = (K[]) new Object[] {getStart(from), getEnd(to)}; + } else if (to == null) { + this.map = this.map.tailMap(from); + } else if (from == null) { + this.map = this.map.headMap(to); + } else { + this.map = this.map.subMap(from, to); + } + } + } + + /** + * Create a persisted sorted map + * + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + * @param factory + * the sorted map factory + */ + public FileSortedMap(TypedSortedMapFileHandler handler, FileSortedMapFactory factory, boolean persisted) { + this.handler = handler; + this.factory = factory; + this.map = new TreeMap<>(); + this.persisted = persisted; + } + + /** + * Create a persisted sorted map + * + * @param comparator + * the key comparator + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + * @param factory + * the sorted map factory + */ + public FileSortedMap(Comparator comparator, TypedSortedMapFileHandler handler, FileSortedMapFactory factory, boolean persisted) { + this.handler = handler; + this.factory = factory; + this.map = new TreeMap<>(comparator); + this.persisted = persisted; + } + + /** + * Create an unpersisted sorted map (still in memory) + * + * @param map + * a sorted map + * @param handler + * the sorted map file handler + * @param factory + * the sorted map factory + */ + public FileSortedMap(SortedMap map, TypedSortedMapFileHandler handler, FileSortedMapFactory factory) { + this.handler = handler; + this.factory = factory; + this.map = new TreeMap<>(map); + this.persisted = false; + if (map instanceof RewritableSortedMap) { + setRewriteStrategy(((RewritableSortedMap) map).getRewriteStrategy()); + } + } + + /** + * Create a sorted map out of another sorted map. If persist is true, then the map will be directly persisted using the map's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param map + * a sorted map + * @param handler + * the sorted map file handler + * @param factory + * the sorted map factory + * @param persist + * the persist boolean flag + * @throws IOException + * for issues with read/write + */ + public FileSortedMap(SortedMap map, TypedSortedMapFileHandler handler, FileSortedMapFactory factory, boolean persist) throws IOException { + this.handler = handler; + this.factory = factory; + if (!persist) { + this.map = new TreeMap<>(map); + this.persisted = false; + } else { + this.map = new TreeMap<>(map.comparator()); + persist(map, handler); + persisted = true; + } + if (map instanceof RewritableSortedMap) { + setRewriteStrategy(((RewritableSortedMap) map).getRewriteStrategy()); + } + } + + @Override + public RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; + } + + @Override + public void setRewriteStrategy(RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = rewriteStrategy; + } + + /** + * This will revert this map to whatever contents are in the underlying file, making the map "persisted". This is intended to be used following a load + * command when no changes were actually made the the map If the persist options included verification, then the files will be verified prior to unloading. + * + * @throws IOException + * for issues with read/write + */ + public void unload() throws IOException { + if (!persisted) { + verifyPersistance(handler, this.map.size(), Collections.emptyList()); + this.map.clear(); + persisted = true; + } + } + + /** + * This will dump the map to the file, making the map "persisted" + * + * @throws IOException + * for issues with read/write + */ + public void persist() throws IOException { + persist(this.handler); + } + + /** + * This will dump the map to the file, making the map "persisted" + * + * @param handler + * the handler + * @throws IOException + * for issues with read/write + */ + public void persist(TypedSortedMapFileHandler handler) throws IOException { + if (!persisted) { + persist(this.map, handler); + this.map.clear(); + persisted = true; + } + } + + /** + * This will dump the map to a file, making the map "persisted" The implementation is expected to wrap the handler with a TypedSortedMapFileHandler and the + * call persist(TypedSortedMapFileHandler handler) + * + * @param handler + * the sorted map file handler + * @throws IOException + * for issues with read/write + */ + public abstract void persist(SortedMapFileHandler handler) throws IOException; + + /** + * Persist the supplied map to a file as defined by this classes sorted map file handler. + * + * @param map + * the map + * @param handler + * the handler + * @throws IOException + * for issues with read/write + * + */ + private void persist(SortedMap map, TypedSortedMapFileHandler handler) throws IOException { + if (log.isDebugEnabled()) { + log.debug("Persisting " + handler); + } + + long start = System.currentTimeMillis(); + try { + // assign the passed in file handler + // if we can't persist, we will remap to null + this.handler = handler; + + int actualSize = 0; + FileSortedSet.PersistOptions persistOptions = handler.getPersistOptions(); + List> mapToVerify = new ArrayList<>(); + try (SortedMapOutputStream stream = handler.getOutputStream()) { + for (Entry t : map.entrySet()) { + stream.writeObject(t.getKey(), t.getValue()); + if (persistOptions.isVerifyElements() && mapToVerify.size() < persistOptions.getNumElementsToVerify()) { + mapToVerify.add(t); + } + actualSize++; + } + stream.writeSize(actualSize); + } + // now verify the written file + verifyPersistance(handler, actualSize, mapToVerify); + + } catch (IOException e) { + handler.deleteFile(); + this.handler = null; + throw e; + } + + if (log.isDebugEnabled()) { + long delta = System.currentTimeMillis() - start; + log.debug("Persisting " + handler + " took " + delta + "ms"); + } + } + + private void verifyPersistance(TypedSortedMapFileHandler handler, int size, List> mapToVerify) throws IOException { + // verify we wrote at least the size.... + if (handler.getSize() == 0) { + throw new IOException("Failed to verify file existence"); + } + FileSortedSet.PersistOptions persistOptions = handler.getPersistOptions(); + // now verify the first n objects were written correctly + if (persistOptions.isVerifyElements() && !mapToVerify.isEmpty()) { + try (SortedMapInputStream inStream = handler.getInputStream()) { + int count = 0; + for (Map.Entry t : mapToVerify) { + count++; + Map.Entry input = inStream.readObject(); + if (!equals(t, input)) { + throw new IOException("Failed to verify element " + count + " was written"); + } + } + } + } + + // now verify the size was written at the end + if (persistOptions.isVerifySize()) { + if (readSize() != size) { + throw new IOException("Failed to verify file size was written"); + } + } + } + + /** + * Read the size from the file which is in the last 4 bytes. + * + * @return the size (in terms of objects) + * @throws IOException + * for issues with read/write + */ + private int readSize() throws IOException { + try (SortedMapInputStream inStream = handler.getInputStream()) { + return inStream.readSize(); + } + } + + /** + * This will read the file into an in-memory map, making this file "unpersisted" + * + * @throws IOException + * for issues with read/write + * @throws ClassNotFoundException + * if the class is not found + */ + public void load() throws IOException, ClassNotFoundException { + if (persisted) { + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry obj = stream.readObject(); + while (obj != null) { + map.put(obj.getKey(), obj.getValue()); + obj = stream.readObject(); + } + } + persisted = false; + } + } + + protected Map.Entry readObject(ObjectInputStream stream) { + try { + K key = (K) stream.readObject(); + V value = (V) stream.readObject(); + return new UnmodifiableMapEntry(key, value); + } catch (Exception E) { + return null; + } + } + + protected void writeObject(ObjectOutputStream stream, K key, V value) throws IOException { + stream.writeObject(key); + stream.writeObject(value); + } + + /* + * Is this map persisted? + */ + public boolean isPersisted() { + return persisted; + } + + /** + * Get the size of the map. Note if the map has been persisted, then this may be an upper bound on the size. + * + * @return the size upper bound + */ + @Override + public int size() { + if (persisted) { + if (isSubmap()) { + throw new IllegalStateException("Unable to determine size of a submap of a persisted map. Please call load() first."); + } + try { + return readSize(); + } catch (Exception e) { + throw new IllegalStateException("Unable to get size from file", e); + } + } else { + return map.size(); + } + } + + @Override + public boolean isEmpty() { + // must attempt to read the first element to be sure if persisted + try { + firstKey(); + return false; + } catch (NoSuchElementException e) { + return true; + } + } + + @SuppressWarnings("unchecked") + @Override + public boolean containsKey(Object o) { + if (persisted) { + K t = (K) o; + K start = getStart(); + K end = getEnd(); + if ((start != null) && (compare(t, start) < 0)) { + return false; + } + if ((end != null) && (compare(t, end) >= 0)) { + return false; + } + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(t, end)) { + Map.Entry next = stream.readObject(); + return (next != null && equals(next.getKey(), t)); + } catch (Exception e) { + return false; + } + } else { + return map.containsKey(o); + } + } + + @Override + public boolean containsValue(Object o) { + if (persisted) { + V t = (V) o; + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry next = stream.readObject(); + while (next != null) { + if (next.getValue().equals(t)) { + return true; + } + next = stream.readObject(); + } + } catch (Exception e) { + return false; + } + return false; + } else { + return map.containsValue(o); + } + } + + @Override + public V get(Object key) { + if (persisted) { + K t = (K) key; + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry next = stream.readObject(); + while (next != null) { + if (equals(next.getKey(), t)) { + return next.getValue(); + } + next = stream.readObject(); + } + } catch (Exception e) { + return null; + } + return null; + } else { + return map.get(key); + } + } + + @Override + public V put(K key, V value) { + if (persisted) { + throw new IllegalStateException("Cannot add an element to a persisted FileSortedMap. Please call load() first."); + } else { + V previous = map.get(key); + if ((previous == null) || (rewriteStrategy == null) || (rewriteStrategy.rewrite(key, previous, value))) { + map.put(key, value); + } + return previous; + } + } + + @Override + public V remove(Object o) { + if (persisted) { + throw new IllegalStateException("Cannot remove an element to a persisted FileSortedMap. Please call load() first."); + } else { + return map.remove(o); + } + } + + @Override + public void putAll(Map m) { + for (Entry entry : m.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + @Override + public void clear() { + if (persisted) { + handler.deleteFile(); + persisted = false; + } else { + map.clear(); + } + } + + @Override + public Comparator comparator() { + return map.comparator(); + } + + @Override + public SortedMap subMap(K fromElement, K toElement) { + return factory.newInstance(this, getStart(fromElement), getEnd(toElement)); + } + + @Override + public SortedMap headMap(K toElement) { + return factory.newInstance(this, getStart(null), getEnd(toElement)); + } + + @Override + public SortedMap tailMap(K fromElement) { + return factory.newInstance(this, getStart(fromElement), getEnd(null)); + } + + @Override + public K firstKey() { + if (persisted) { + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry first = stream.readObject(); + return first.getKey(); + } catch (Exception e) { + throw new IllegalStateException(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR, e)); + } + } else if (!map.isEmpty()) { + return map.firstKey(); + } + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR)); + } + + @Override + public K lastKey() { + if (persisted) { + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry last = stream.readObject(); + Map.Entry next = stream.readObject(); + while (next != null) { + last = next; + next = stream.readObject(); + } + return last.getKey(); + } catch (Exception e) { + throw new IllegalStateException(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR, e)); + } + } else if (!map.isEmpty()) { + return map.lastKey(); + } + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR)); + } + + private Iterator> iterator() { + if (persisted) { + return new FileIterator(); + } else { + return map.entrySet().iterator(); + } + } + + @Override + public Set keySet() { + return new AbstractSet() { + + @Override + public Iterator iterator() { + return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), o -> ((Map.Entry) o).getKey()); + } + + @Override + public int size() { + return FileSortedMap.this.size(); + } + }; + } + + @Override + public Collection values() { + return new AbstractCollection() { + + @Override + public Iterator iterator() { + return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), o -> ((Map.Entry) o).getValue()); + } + + @Override + public int size() { + return FileSortedMap.this.size(); + } + }; + } + + @Override + public Set> entrySet() { + return new AbstractSet>() { + + @Override + public Iterator> iterator() { + return FileSortedMap.this.iterator(); + } + + @Override + public int size() { + return FileSortedMap.this.size(); + } + }; + } + + @Override + public String toString() { + return persisted ? handler.toString() : map.toString(); + } + + /** + * Extending classes must implement cloneable + * + * @return A clone + */ + public FileSortedMap clone() { + return factory.newInstance(this); + } + + /* Some utilities */ + private boolean equals(Map.Entry o1, Map.Entry o2) { + if (o1 == null) { + return o2 == null; + } else if (o2 == null) { + return false; + } else { + return equals(o1.getKey(), o2.getKey()) && o1.getValue().equals(o2.getValue()); + } + } + + private boolean equals(K o1, K o2) { + if (o1 == null) { + return o2 == null; + } else if (o2 == null) { + return false; + } else { + if (map.comparator() == null) { + return o1.equals(o2); + } else { + return map.comparator().compare(o1, o2) == 0; + } + } + } + + private K getStart() { + return (isSubmap() ? range[0] : null); + } + + private K getStart(K from) { + K start = getStart(); + if (start == null) { + return from; + } else if (from == null) { + return start; + } else if (compare(start, from) > 0) { + return start; + } else { + return from; + } + } + + private K getEnd() { + return (isSubmap() ? range[1] : null); + } + + private K getEnd(K to) { + K end = getEnd(); + if (end == null) { + return to; + } else if (to == null) { + return end; + } else if (compare(end, to) < 0) { + return end; + } else { + return to; + } + } + + private boolean isSubmap() { + return (range != null); + } + + private int compare(K a, K b) { + return (this.map.comparator() != null) ? this.map.comparator().compare(a, b) : ((Comparable) a).compareTo(b); + } + + public BoundedTypedSortedMapFileHandler getBoundedFileHandler() { + return new DefaultBoundedTypedSortedMapFileHandler(); + } + + /** + * This is the iterator for a persisted FileSortedMap + */ + protected class FileIterator implements Iterator> { + private SortedMapInputStream stream; + private Map.Entry next; + + public FileIterator() { + try { + this.stream = getBoundedFileHandler().getInputStream(getStart(), getEnd()); + next = stream.readObject(); + if (next == null) { + cleanup(); + } + } catch (Exception e) { + cleanup(); + throw new IllegalStateException("Unable to read file", e); + } + } + + public void cleanup() { + if (stream != null) { + try { + stream.close(); + } catch (Exception e) { + // we tried... + } + stream = null; + } + } + + @Override + public boolean hasNext() { + return (next != null); + } + + @Override + public Map.Entry next() { + if (!hasNext()) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + try { + Map.Entry rtrn = next; + next = stream.readObject(); + if (next == null) { + cleanup(); + } + return rtrn; + } catch (Exception e) { + cleanup(); + throw new IllegalStateException("Unable to get next element from file", e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Iterator.remove() not supported on a persisted map."); + } + + @Override + protected void finalize() throws Throwable { + cleanup(); + super.finalize(); + } + } + + /** + * An interface for a sorted map factory + * + * @param + * key of the map + * @param + * value of the map + */ + public interface FileSortedMapFactory { + /** + * factory method + * + * @param other + * the other factory + * @return a new instance + */ + FileSortedMap newInstance(FileSortedMap other); + + /** + * factory method + * + * @param other + * the other factory + * @param from + * from instance + * @param to + * to instance + * @return a new instance + */ + FileSortedMap newInstance(FileSortedMap other, K from, K to); + + /** + * Factory method + * + * @param comparator + * the key comparator + * @param rewriteStrategy + * the rewrite strategy + * @param handler + * the sorted map file handler + * @param persisted + * a persisted boolean flag + * @return a new instance + */ + FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, boolean persisted); + + /** + * Create an unpersisted sorted map (still in memory) + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + * @return a new instance + */ + FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler); + + /** + * factory method + * + * @param map + * the sorted map + * @param handler + * the sorted map file handler + * @param persist + * a persisted boolean flag + * @return a new instance + * @throws IOException + * for problems with read/write + */ + FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException; + } + + /** + * A sorted map input stream + * + * @param + * key of the map + * @param + * value of the map + */ + public interface SortedMapInputStream extends AutoCloseable { + Map.Entry readObject() throws IOException; + + int readSize() throws IOException; + + void close(); + } + + /** + * A sorted map output stream + * + * @param + * key of the map + * @param + * value of the map + */ + public interface SortedMapOutputStream extends AutoCloseable { + void writeObject(K key, V value) throws IOException; + + void writeSize(int size) throws IOException; + + void close() throws IOException; + } + + /** + * A factory that will provide the input stream and output stream to the same underlying file. + * + */ + public interface SortedMapFileHandler { + /** + * Return the input stream + * + * @return the input stream + * @throws IOException + * for problems with read/write + */ + InputStream getInputStream() throws IOException; + + /** + * Return the output stream + * + * @return the sorted map output stream + * @throws IOException + * for problems with read/write + */ + OutputStream getOutputStream() throws IOException; + + /** + * Get the persistent verification options + * + * @return the persistent verification options + */ + FileSortedSet.PersistOptions getPersistOptions(); + + long getSize(); + + void deleteFile(); + } + + /** + * A factory that will provide the input stream and output stream to the same underlying file. + * + */ + public interface TypedSortedMapFileHandler { + /** + * Return the input stream + * + * @return the input stream + * @throws IOException + * for problems with read/write + */ + SortedMapInputStream getInputStream() throws IOException; + + /** + * Return the output stream + * + * @return the sorted map output stream + * @throws IOException + * for problems with read/write + */ + SortedMapOutputStream getOutputStream() throws IOException; + + /** + * Get the persistent verification options + * + * @return persistent verification options + */ + FileSortedSet.PersistOptions getPersistOptions(); + + long getSize(); + + void deleteFile(); + } + + /** + * A factory that will provide the input stream and output stream to the same underlying file. An additional input stream method allows for creating a + * stream submap. + * + */ + public interface BoundedTypedSortedMapFileHandler extends TypedSortedMapFileHandler { + /** + * Return the input stream + * + * @return the input stream + * @param start + * start point + * @param end + * end point + * @throws IOException + * for problems with read/write + */ + SortedMapInputStream getInputStream(K start, K end) throws IOException; + } + + /** + * A default implementation for a bounded typed sorted map + */ + public class DefaultBoundedTypedSortedMapFileHandler implements BoundedTypedSortedMapFileHandler { + @Override + public SortedMapInputStream getInputStream(K start, K end) throws IOException { + if (handler instanceof BoundedTypedSortedMapFileHandler) { + return ((BoundedTypedSortedMapFileHandler) handler).getInputStream(start, end); + } else { + return new BoundedInputStream(handler.getInputStream(), start, end); + } + } + + @Override + public SortedMapInputStream getInputStream() throws IOException { + return handler.getInputStream(); + } + + @Override + public SortedMapOutputStream getOutputStream() throws IOException { + return handler.getOutputStream(); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return handler.getPersistOptions(); + } + + @Override + public long getSize() { + return handler.getSize(); + } + + @Override + public void deleteFile() { + handler.deleteFile(); + } + } + + /** + * An input stream that supports bounding the objects. Used when the underlying stream does not already support bounding. + */ + public class BoundedInputStream implements SortedMapInputStream { + private final SortedMapInputStream delegate; + private final K from; + private final K to; + + public BoundedInputStream(SortedMapInputStream stream, K from, K to) { + this.delegate = stream; + this.from = from; + this.to = to; + } + + @Override + public Map.Entry readObject() throws IOException { + Map.Entry o = delegate.readObject(); + while ((o != null) && (from != null) && (compare(o.getKey(), from) < 0)) { + o = delegate.readObject(); + } + if (o == null || (to != null && compare(o.getKey(), to) >= 0)) { + return null; + } else { + return o; + } + } + + @Override + public int readSize() throws IOException { + return delegate.readSize(); + } + + @Override + public void close() { + delegate.close(); + } + } + + public interface RewriteStrategy { + /** + * Determine if the object should be rewritten + * + * @param key + * The key + * @param original + * The original value + * @param update + * The updated value + * @return true of the original should be replaced with the update + */ + boolean rewrite(K key, V original, V update); + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/HdfsBackedSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/HdfsBackedSortedMap.java new file mode 100644 index 00000000000..215aeb490b0 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/HdfsBackedSortedMap.java @@ -0,0 +1,301 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.MalformedURLException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FsStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.Logger; + +import datawave.query.iterator.ivarator.IvaratorCacheDir; +import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.util.sortedset.FileSortedSet; + +/** + * Sorted map backed by HDFS + * + * @param + * key of the map + * @param + * value of the map + */ +public class HdfsBackedSortedMap extends BufferedFileBackedSortedMap { + private static final Logger log = Logger.getLogger(HdfsBackedSortedMap.class); + private static final String FILENAME_PREFIX = "SortedMapFile."; + + public static class Builder,K,V> extends BufferedFileBackedSortedMap.Builder { + private List ivaratorCacheDirs; + private String uniqueSubPath; + private FileSortedSet.PersistOptions persistOptions; + + public Builder() { + // change the default buffer persist threshold + withBufferPersistThreshold(10_000); + } + + @Override + @SuppressWarnings("unchecked") + protected B self() { + return (B) this; + } + + public B withIvaratorCacheDirs(List ivaratorCacheDirs) { + this.ivaratorCacheDirs = ivaratorCacheDirs; + return self(); + } + + public B withUniqueSubPath(String uniqueSubPath) { + this.uniqueSubPath = uniqueSubPath; + return self(); + } + + public B withPersistOptions(FileSortedSet.PersistOptions persistOptions) { + this.persistOptions = persistOptions; + return self(); + } + + public HdfsBackedSortedMap build() throws IOException { + return new HdfsBackedSortedMap<>(this); + } + } + + public static HdfsBackedSortedMap.Builder builder() { + return new HdfsBackedSortedMap.Builder<>(); + } + + protected HdfsBackedSortedMap(HdfsBackedSortedMap other) { + super(other); + } + + protected HdfsBackedSortedMap(Builder builder) throws IOException { + super(builder); + List factories = createFileHandlerFactories(builder.ivaratorCacheDirs, builder.uniqueSubPath, builder.persistOptions); + // update the parent handler factories (list of SortedMapFileHandlerFactory) + this.handlerFactories = (List) factories; + // for each of the handler factories, check to see if there are any existing files we should load + for (SortedMapHdfsFileHandlerFactory handlerFactory : factories) { + SortedMapHdfsFileHandlerFactory hdfsHandlerFactory = (SortedMapHdfsFileHandlerFactory) handlerFactory; + FileSystem fs = hdfsHandlerFactory.getFs(); + int count = 0; + + // if the directory already exists, load up this sorted map with any existing files + if (fs.exists(hdfsHandlerFactory.getUniqueDir())) { + FileStatus[] files = fs.listStatus(hdfsHandlerFactory.getUniqueDir()); + if (files != null) { + for (FileStatus file : files) { + if (!file.isDir() && file.getPath().getName().startsWith(FILENAME_PREFIX)) { + count++; + addMap(mapFactory.newInstance(comparator, getRewriteStrategy(), + new SortedMapHdfsFileHandler(fs, file.getPath(), builder.persistOptions), true)); + } + } + } + + hdfsHandlerFactory.mapFileCount(count); + } + } + } + + private static List createFileHandlerFactories(List ivaratorCacheDirs, String uniqueSubPath, + FileSortedSet.PersistOptions persistOptions) { + List fileHandlerFactories = new ArrayList<>(); + for (IvaratorCacheDir ivaratorCacheDir : ivaratorCacheDirs) { + fileHandlerFactories.add(new SortedMapHdfsFileHandlerFactory(ivaratorCacheDir, uniqueSubPath, persistOptions)); + } + return fileHandlerFactories; + } + + @Override + public void clear() { + // This will be a new ArrayList<>() containing the same FileSortedMaps + List> SortedMaps = super.getMaps(); + // Clear will call clear on each of the FileSortedMaps, clear the container, and null the buffer + super.clear(); + // We should still be able to access the FileSortedMap objects to get their handler because we + // have a copy of the object in 'SortedMaps' + for (FileSortedMap fss : SortedMaps) { + if (fss.isPersisted() && fss.handler instanceof SortedMapHdfsFileHandler) { + ((SortedMapHdfsFileHandler) fss.handler).deleteFile(); + } + } + } + + public static class SortedMapHdfsFileHandlerFactory implements SortedMapFileHandlerFactory { + final private IvaratorCacheDir ivaratorCacheDir; + private String uniqueSubPath; + private int fileCount = 0; + private FileSortedSet.PersistOptions persistOptions; + + public SortedMapHdfsFileHandlerFactory(IvaratorCacheDir ivaratorCacheDir, String uniqueSubPath, FileSortedSet.PersistOptions persistOptions) { + this.ivaratorCacheDir = ivaratorCacheDir; + this.uniqueSubPath = uniqueSubPath; + this.persistOptions = persistOptions; + } + + public IvaratorCacheDir getIvaratorCacheDir() { + return ivaratorCacheDir; + } + + public FileSystem getFs() { + return ivaratorCacheDir.getFs(); + } + + public Path getUniqueDir() { + return new Path(ivaratorCacheDir.getPathURI(), uniqueSubPath); + } + + public int getFileCount() { + return fileCount; + } + + void mapFileCount(int count) { + this.fileCount = count; + } + + public boolean isValid() { + FsStatus fsStatus = null; + try { + fsStatus = ivaratorCacheDir.getFs().getStatus(); + } catch (IOException e) { + log.warn("Unable to determine status of the filesystem: " + ivaratorCacheDir.getFs()); + } + + // determine whether this fs is a good candidate + if (fsStatus != null) { + long availableStorageMiB = fsStatus.getRemaining() / 0x100000L; + double availableStoragePercent = (double) fsStatus.getRemaining() / fsStatus.getCapacity(); + + // if we are using less than our storage limit, the cache dir is valid + return availableStorageMiB >= ivaratorCacheDir.getConfig().getMinAvailableStorageMiB() + && availableStoragePercent >= ivaratorCacheDir.getConfig().getMinAvailableStoragePercent(); + } + + return false; + } + + @Override + public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { + FileSystem fs = getFs(); + Path uniqueDir = getUniqueDir(); + + // Lazily create the required ivarator cache dirs. + ensureDirsCreated(); + + // generate a unique file name + fileCount++; + Path file = new Path(uniqueDir, FILENAME_PREFIX + fileCount + '.' + System.currentTimeMillis()); + return new SortedMapHdfsFileHandler(fs, file, persistOptions); + } + + private void ensureDirsCreated() throws IOException { + IvaratorCacheDirConfig config = ivaratorCacheDir.getConfig(); + if (config.isValid()) { + ensureCreation(new Path(ivaratorCacheDir.getPathURI())); + ensureCreation(getUniqueDir()); + } else { + throw new IOException("Unable to create Ivarator Cache Dir for invalid config: " + config); + } + } + + private void ensureCreation(Path path) throws IOException { + try { + FileSystem fs = getFs(); + if (!fs.exists(path)) { + // Attempt to create the required directory if it does not exist. + if (!fs.mkdirs(path)) { + throw new IOException("Unable to mkdirs: fs.mkdir(" + path + ")->false"); + } + } + } catch (MalformedURLException e) { + throw new IOException("Unable to load hadoop configuration", e); + } catch (Exception e) { + log.warn("Unable to create directory [" + path + "] in file system [" + getFs() + "]", e); + throw new IOException("Unable to create directory [" + path + "] in file system [" + getFs() + "]", e); + } + } + + @Override + public String toString() { + return getUniqueDir() + " (fileCount=" + fileCount + ')'; + } + + } + + public static class SortedMapHdfsFileHandler implements FileSortedMap.SortedMapFileHandler { + private FileSystem fs; + private Path file; + private FileSortedSet.PersistOptions persistOptions; + + public SortedMapHdfsFileHandler(FileSystem fs, Path file, FileSortedSet.PersistOptions persistOptions) { + this.fs = fs; + this.file = file; + this.persistOptions = persistOptions; + } + + private String getScheme() { + String scheme = file.toUri().getScheme(); + if (scheme == null) { + scheme = fs.getScheme(); + } + return scheme; + } + + @Override + public InputStream getInputStream() throws IOException { + if (log.isDebugEnabled()) { + log.debug("Reading " + file); + } + return fs.open(file); + } + + @Override + public OutputStream getOutputStream() throws IOException { + if (log.isDebugEnabled()) { + log.debug("Creating " + file); + } + return fs.create(file); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return persistOptions; + } + + @Override + public long getSize() { + try { + FileStatus status = fs.getFileStatus(file); + return status.getLen(); + } catch (Exception e) { + log.warn("Failed to verify file " + file, e); + return -1; + } + } + + @Override + public void deleteFile() { + try { + if (log.isDebugEnabled()) { + log.debug("Deleting " + file); + } + if (!fs.delete(file, true)) { + log.error("Failed to delete file " + file + ": delete returned false"); + } + } catch (IOException e) { + log.error("Failed to delete file " + file, e); + } + } + + @Override + public String toString() { + return file.toString(); + } + + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiMapBackedSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiMapBackedSortedMap.java new file mode 100644 index 00000000000..e174e5def30 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiMapBackedSortedMap.java @@ -0,0 +1,407 @@ +package datawave.query.util.sortedmap; + +import java.util.AbstractMap; +import java.util.AbstractSet; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; +import org.apache.commons.lang3.builder.EqualsBuilder; + +import com.google.common.collect.Iterators; + +import datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; + +/* + * This is a sorted map that is backed by multiple underlying sorted maps. It is assumed that the underlying + * sorted maps contain the same type of underlying value, and they use the same comparator. The rewrite + * strategy will be used if the underlying sorted maps are RewritableSortedMap implementations. + * @param + * key of the map + * @param + * value of the map + */ +public class MultiMapBackedSortedMap extends AbstractMap implements RewritableSortedMap { + protected List> maps = new ArrayList<>(); + protected Comparator comparator = null; + protected FileSortedMap.RewriteStrategy rewriteStrategy = null; + + public MultiMapBackedSortedMap() {} + + public MultiMapBackedSortedMap(List> maps) { + for (SortedMap map : maps) { + addMap(map); + } + } + + public void addMap(SortedMap map) { + if (maps.isEmpty()) { + updateConfiguration(map); + } else { + verifyConfiguration(map); + } + maps.add(map); + } + + private void updateConfiguration(SortedMap map) { + comparator = getComparator(map); + rewriteStrategy = getRewriteStrategy(map); + } + + private void verifyConfiguration(SortedMap map) { + if (!(new EqualsBuilder().append(getClass(comparator), getClass(getComparator(map))) + .append(getClass(rewriteStrategy), getClass(getRewriteStrategy(map))).isEquals())) { + throw new IllegalArgumentException("map being added does not match the comparator and rewriteStrategy of the existing maps"); + } + } + + private Class getClass(Object obj) { + return (obj == null ? null : obj.getClass()); + } + + private FileSortedMap.RewriteStrategy getRewriteStrategy(SortedMap map) { + if (map instanceof RewritableSortedMap) { + return ((RewritableSortedMap) map).getRewriteStrategy(); + } + return null; + } + + private Comparator getComparator(SortedMap map) { + return (Comparator) (map.comparator()); + } + + /** + * Get the underlying maps + * + * @return the maps + */ + public List> getMaps() { + return maps; + } + + /** + * Return the size of this map. NOTE that this is somewhat expensive as we require iterating over the maps to determine the true value (see + * MergeSortIterator); + */ + @Override + public int size() { + return Iterators.size(iterator()); + } + + @Override + public boolean isEmpty() { + if (maps == null) { + return true; + } + for (SortedMap map : maps) { + if (map != null && !map.isEmpty()) { + return false; + } + } + return true; + } + + @Override + public boolean containsKey(Object o) { + for (SortedMap map : maps) { + if (map.containsKey(o)) { + return true; + } + } + return false; + } + + protected Iterator> iterator() { + return new MergeSortIterator(); + } + + @Override + public V remove(Object o) { + V value = null; + for (SortedMap map : maps) { + V testValue = map.remove(o); + if (testValue != null) { + if (value != null) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { + value = testValue; + } + } else { + value = testValue; + } + } + } + return value; + } + + @Override + public void clear() { + for (SortedMap map : this.maps) { + try { + map.clear(); + } catch (Exception e) { + // error clearing sorted map + // possibility of FileNotFoundException, etc being + // caught and re-thrown as an exception + } + } + this.maps.clear(); + } + + @Override + public Set> entrySet() { + return new AbstractSet<>() { + + @Override + public Iterator> iterator() { + return MultiMapBackedSortedMap.this.iterator(); + } + + @Override + public int size() { + return MultiMapBackedSortedMap.this.size(); + } + }; + } + + @Override + public Comparator comparator() { + return comparator; + } + + @Override + public RewritableSortedMap subMap(K fromElement, K toElement) { + MultiMapBackedSortedMap submap = new MultiMapBackedSortedMap<>(); + submap.setRewriteStrategy(rewriteStrategy); + for (SortedMap map : maps) { + submap.addMap(map.subMap(fromElement, toElement)); + } + return submap; + } + + @Override + public RewritableSortedMap headMap(K toElement) { + MultiMapBackedSortedMap submap = new MultiMapBackedSortedMap<>(); + submap.setRewriteStrategy(rewriteStrategy); + for (SortedMap map : maps) { + submap.addMap(map.headMap(toElement)); + } + return submap; + } + + @Override + public RewritableSortedMap tailMap(K fromElement) { + MultiMapBackedSortedMap submap = new MultiMapBackedSortedMap<>(); + submap.setRewriteStrategy(rewriteStrategy); + for (SortedMap map : maps) { + submap.addMap(map.tailMap(fromElement)); + } + return submap; + } + + @Override + public K firstKey() throws NoSuchElementException { + if (maps == null || maps.isEmpty()) { + throw new NoSuchElementException("No elements in input maps"); + } + SortedSet firstSet = new TreeSet<>(comparator()); + for (SortedMap map : maps) { + if (map != null && !map.isEmpty()) { + K s = map.firstKey(); + firstSet.add(s); + } + } + if (firstSet.isEmpty()) { + throw new NoSuchElementException("No elements in input maps"); + } + return firstSet.first(); + } + + @Override + public K lastKey() throws NoSuchElementException { + if (maps == null || maps.isEmpty()) { + throw new NoSuchElementException("No elements in input maps"); + } + SortedSet lastSet = new TreeSet<>(comparator()); + for (SortedMap map : maps) { + if (map != null && !map.isEmpty()) { + K s = map.lastKey(); + lastSet.add(s); + } + } + if (lastSet.isEmpty()) { + throw new NoSuchElementException("No elements in input maps"); + } + return lastSet.last(); + } + + @Override + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; + } + + @Override + public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = rewriteStrategy; + } + + @Override + public V get(Object o) { + V value = null; + for (SortedMap map : maps) { + V testValue = map.get(o); + if (testValue != null) { + if (value != null) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { + value = testValue; + } + } else { + value = testValue; + } + } + } + return value; + } + + /** + * This is an iterator that will return a sorted map of items (no dups) from an underlying map of sorted maps. + */ + public class MergeSortIterator implements Iterator> { + + // this is the entire set of iterators + private List>> iterators = new ArrayList<>(); + // this is the list of the last key from each of the iterators available to use + private List lastList = new ArrayList<>(); + // booleans denoting if an iterator has been completely used up + private boolean[] finished = null; + // This map holds the key/values to be returned next. + private SortedMap map = null; + private boolean populated = false; + private K nextKey = null; + private V nextValue = null; + // This is the set of iterators that contributed to the last value returned + private List>> nextIterators = new ArrayList<>(); + + public MergeSortIterator() { + for (SortedMap map : maps) { + Iterator> it = map.entrySet().iterator(); + iterators.add(it); + nextIterators.add(it); + lastList.add(null); + } + this.map = new TreeMap(comparator); + this.finished = new boolean[iterators.size()]; + } + + @Override + public boolean hasNext() { + if (!map.isEmpty()) { + return true; + } + for (Iterator> it : nextIterators) { + if (it != null && it.hasNext()) { + return true; + } + } + return false; + } + + @Override + public Entry next() { + populate(); + if (!populated) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + return new UnmodifiableMapEntry<>(nextKey, nextValue); + } + + @Override + public void remove() { + if (!populated) { + throw new IllegalStateException(); + } + Exception e = null; + for (Iterator> it : nextIterators) { + if (it != null) { + try { + it.remove(); + } catch (UnsupportedOperationException uoe) { + e = uoe; + } + } + } + populated = false; + if (e != null) { + throw new UnsupportedOperationException("One or more of the underlying sets does not support this operation", e); + } + } + + /* Some utility methods */ + private boolean equals(K o1, K o2) { + if (o1 == null) { + return o2 == null; + } else if (o2 == null) { + return false; + } else { + if (map.comparator() == null) { + return o1.equals(o2); + } else { + return map.comparator().compare(o1, o2) == 0; + } + } + } + + private void populate() { + populated = false; + + // update the last value for those iterators contributing to + // the last returned value + for (int i = 0; i < nextIterators.size(); i++) { + if (nextIterators.get(i) != null) { + Iterator> it = nextIterators.get(i); + if (it.hasNext()) { + Entry val = it.next(); + // remember the last key returned + lastList.set(i, val.getKey()); + if ((rewriteStrategy == null) || (!map.containsKey(val.getKey())) + || (rewriteStrategy.rewrite(val.getKey(), map.get(val.getKey()), val.getValue()))) { + // update the map if the rewrite policy allows (or a new key) + map.put(val.getKey(), val.getValue()); + } + } else { + // remember that we are done with this iterator + lastList.set(i, null); + finished[i] = true; + } + } + } + + if (!map.isEmpty()) { + // now get the next key/value from the map + nextKey = map.firstKey(); + nextValue = map.remove(nextKey); + // and update the list of iterators that contributed to this next key + for (int i = 0; i < iterators.size(); i++) { + if (!finished[i] && equals(nextKey, lastList.get(i))) { + nextIterators.set(i, iterators.get(i)); + } else { + // if the iterator is finished, or did not contribute to the value being returned + // then null it out since the value returned is already in the map to compare + // on the next round + nextIterators.set(i, null); + } + } + populated = true; + } + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/RewritableSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/RewritableSortedMap.java new file mode 100644 index 00000000000..68063a1855e --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/RewritableSortedMap.java @@ -0,0 +1,19 @@ +package datawave.query.util.sortedmap; + +import java.util.SortedMap; + +/** + * A rewritable sorted map which will replace the value for a key dependent on a RewriteStrategy + * + * @param + * key of the map + * @param + * value of the map + */ +public interface RewritableSortedMap extends SortedMap { + + FileSortedMap.RewriteStrategy getRewriteStrategy(); + + void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy); + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedMapTempFileHandler.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedMapTempFileHandler.java new file mode 100644 index 00000000000..ef3815c3c6c --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedMapTempFileHandler.java @@ -0,0 +1,63 @@ +package datawave.query.util.sortedmap; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import datawave.query.util.sortedset.FileSortedSet; + +/** + * A sorted set file handler factory that uses temporary local based files. + */ +public class SortedMapTempFileHandler implements FileSortedMap.SortedMapFileHandler { + private final FileSystem fs; + private final File file; + private final Path path; + + public SortedMapTempFileHandler() throws IOException { + this.file = File.createTempFile("SortedSet", ".bin"); + this.file.deleteOnExit(); + this.path = new Path(file.toURI()); + Configuration conf = new Configuration(); + this.fs = path.getFileSystem(conf); + } + + public File getFile() { + return file; + } + + @Override + public InputStream getInputStream() throws IOException { + return fs.open(path); + } + + public OutputStream getOutputStream() throws IOException { + return fs.create(path, true); + } + + @Override + public FileSortedSet.PersistOptions getPersistOptions() { + return new FileSortedSet.PersistOptions(true, false); + } + + @Override + public long getSize() { + return (file.exists() ? file.length() : -1); + } + + @Override + public void deleteFile() { + this.file.delete(); + } + + @Override + public String toString() { + return file.toString(); + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumentTransforms.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumentTransforms.java new file mode 100644 index 00000000000..e5eb3524aef --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumentTransforms.java @@ -0,0 +1,83 @@ +package datawave.query.util.sortedmap.rfile; + +import java.io.ByteArrayInputStream; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; + +import datawave.query.attributes.Attribute; +import datawave.query.attributes.Document; +import datawave.query.function.deserializer.DocumentDeserializer; +import datawave.query.function.deserializer.KryoDocumentDeserializer; +import datawave.query.function.serializer.DocumentSerializer; +import datawave.query.function.serializer.KryoDocumentSerializer; + +public class KeyValueByteDocumentTransforms { + + private static KryoDocumentSerializer serializer = new KryoDocumentSerializer(false, true); + private static KryoDocumentDeserializer deserializer = new KryoDocumentDeserializer(); + + public static byte[] keyToByte(Key key) { + if (key == null) { + return null; + } + return key.getRow().getBytes(); + } + + public static Key byteToKey(byte[] bytes) { + if (bytes == null) { + return null; + } + return new Key(bytes); + } + + public static Value documentToValue(Document doc) { + if (doc == null) { + return null; + } + byte[] document; + synchronized (serializer) { + document = serializer.serialize(doc); + } + return new Value(document); + } + + public static Document valueToDocument(Value value) { + if (value == null) { + return null; + } + synchronized (deserializer) { + return deserializer.deserialize(new ByteArrayInputStream(value.get())); + } + } + + public static Map.Entry keyValueToByteDocument(Map.Entry keyValue) { + if (keyValue == null) { + return null; + } + return new UnmodifiableMapEntry(keyToByte(keyValue.getKey()), valueToDocument(keyValue.getValue())); + } + + public static Map.Entry byteDocumentToKeyValue(Map.Entry byteKey) { + if (byteKey == null) { + return null; + } + return new UnmodifiableMapEntry(byteToKey(byteKey.getKey()), documentToValue(byteKey.getValue())); + } + + public static Map.Entry keyValueToKeyDocument(Map.Entry keyValue) { + if (keyValue == null) { + return null; + } + return new UnmodifiableMapEntry(keyValue.getKey(), valueToDocument(keyValue.getValue())); + } + + public static Map.Entry KeyDocumentToKeyValue(Map.Entry byteKey) { + if (byteKey == null) { + return null; + } + return new UnmodifiableMapEntry(byteKey.getKey(), documentToValue(byteKey.getValue())); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentInputStream.java new file mode 100644 index 00000000000..0df7adb2b48 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentInputStream.java @@ -0,0 +1,30 @@ +package datawave.query.util.sortedmap.rfile; + +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.byteToKey; +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.keyValueToByteDocument; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.FileSortedMap; + +public class RFileByteDocumentInputStream extends RFileKeyValueInputStreamBase implements FileSortedMap.SortedMapInputStream { + + public RFileByteDocumentInputStream(InputStream inputStream, long length) throws IOException { + super(inputStream, length); + } + + public RFileByteDocumentInputStream(InputStream inputStream, long length, byte[] start, byte[] end) throws IOException { + super(inputStream, length, byteToKey(start), byteToKey(end)); + } + + @Override + public Map.Entry readObject() throws IOException { + return keyValueToByteDocument(readKeyValue()); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentOutputStream.java new file mode 100644 index 00000000000..774593be151 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentOutputStream.java @@ -0,0 +1,26 @@ +package datawave.query.util.sortedmap.rfile; + +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.byteToKey; +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.documentToValue; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.FileSortedMap; + +public class RFileByteDocumentOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedMapOutputStream { + public RFileByteDocumentOutputStream(OutputStream stream) throws IOException { + super(stream); + } + + @Override + public void writeObject(byte[] k, Document v) throws IOException { + writeKeyValue(byteToKey(k), documentToValue(v)); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentInputStream.java new file mode 100644 index 00000000000..1475c0bbc74 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentInputStream.java @@ -0,0 +1,28 @@ +package datawave.query.util.sortedmap.rfile; + +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.keyValueToKeyDocument; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.FileSortedMap; + +public class RFileKeyDocumentInputStream extends RFileKeyValueInputStreamBase implements FileSortedMap.SortedMapInputStream { + + public RFileKeyDocumentInputStream(InputStream inputStream, long length) throws IOException { + super(inputStream, length); + } + + public RFileKeyDocumentInputStream(InputStream inputStream, long length, Key start, Key end) throws IOException { + super(inputStream, length, start, end); + } + + @Override + public Map.Entry readObject() throws IOException { + return keyValueToKeyDocument(super.readKeyValue()); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentOutputStream.java new file mode 100644 index 00000000000..8a5b8f6043b --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyDocumentOutputStream.java @@ -0,0 +1,22 @@ +package datawave.query.util.sortedmap.rfile; + +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.documentToValue; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.accumulo.core.data.Key; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.FileSortedMap; + +public class RFileKeyDocumentOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedMapOutputStream { + public RFileKeyDocumentOutputStream(OutputStream stream) throws IOException { + super(stream); + } + + @Override + public void writeObject(Key k, Document v) throws IOException { + writeKeyValue(k, documentToValue(v)); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStream.java new file mode 100644 index 00000000000..e241d652fcd --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStream.java @@ -0,0 +1,26 @@ +package datawave.query.util.sortedmap.rfile; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import datawave.query.util.sortedmap.FileSortedMap; + +public class RFileKeyValueInputStream extends RFileKeyValueInputStreamBase implements FileSortedMap.SortedMapInputStream { + + public RFileKeyValueInputStream(InputStream inputStream, long length) throws IOException { + super(inputStream, length); + } + + public RFileKeyValueInputStream(InputStream inputStream, long length, Key start, Key end) throws IOException { + super(inputStream, length, start, end); + } + + @Override + public Map.Entry readObject() throws IOException { + return super.readKeyValue(); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStreamBase.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStreamBase.java new file mode 100644 index 00000000000..653cb7de40f --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStreamBase.java @@ -0,0 +1,79 @@ +package datawave.query.util.sortedmap.rfile; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.Map; + +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.rfile.RFile; +import org.apache.accumulo.core.client.rfile.RFileSource; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; + +public abstract class RFileKeyValueInputStreamBase { + private final InputStream inputStream; + private final long length; + private Key start; + private Key end; + private Scanner reader; + private Iterator> iterator; + private int size = -1; + private static final Range ALL = new Range(); + + public RFileKeyValueInputStreamBase(InputStream inputStream, long length) throws IOException { + this.inputStream = inputStream; + this.length = length; + } + + public RFileKeyValueInputStreamBase(InputStream inputStream, long length, Key start, Key end) throws IOException { + this(inputStream, length); + this.start = start; + this.end = end; + } + + private Iterator> keyValueIterator() { + if (iterator == null) { + Range r = ALL; + if (start != null || end != null) { + r = new Range(start, true, end, false); + } + reader = RFile.newScanner().from(new RFileSource(inputStream, length)).withBounds(r).withoutSystemIterators().build(); + iterator = reader.iterator(); + } + return iterator; + } + + public Map.Entry readKeyValue() throws IOException { + if (keyValueIterator().hasNext()) { + Map.Entry next = keyValueIterator().next(); + if (RFileKeyValueOutputStreamBase.SizeKeyUtil.isSizeKey(next.getKey())) { + size = RFileKeyValueOutputStreamBase.SizeKeyUtil.getSize(next.getKey()); + next = null; + } + return next; + } + return null; + } + + public int readSize() throws IOException { + if (size < 0) { + if (iterator != null) { + throw new IllegalStateException("Cannot read size from undetermined location in stream"); + } + reader = RFile.newScanner().from(new RFileSource(inputStream, length)).withBounds(new Range(RFileKeyValueOutputStreamBase.SizeKeyUtil.SIZE_ROW)) + .build(); + iterator = reader.iterator(); + size = RFileKeyValueOutputStreamBase.SizeKeyUtil.getSize(iterator.next().getKey()); + } + return size; + } + + public void close() { + if (reader != null) { + reader.close(); + reader = null; + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStream.java new file mode 100644 index 00000000000..c42d045fab5 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStream.java @@ -0,0 +1,20 @@ +package datawave.query.util.sortedmap.rfile; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import datawave.query.util.sortedmap.FileSortedMap; + +public class RFileKeyValueOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedMapOutputStream { + public RFileKeyValueOutputStream(OutputStream stream) throws IOException { + super(stream); + } + + @Override + public void writeObject(Key k, Value v) throws IOException { + writeKeyValue(k, v); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStreamBase.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStreamBase.java new file mode 100644 index 00000000000..ca4b8845beb --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStreamBase.java @@ -0,0 +1,54 @@ +package datawave.query.util.sortedmap.rfile; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.accumulo.core.client.rfile.RFile; +import org.apache.accumulo.core.client.rfile.RFileWriter; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.hadoop.io.Text; + +public abstract class RFileKeyValueOutputStreamBase { + private final OutputStream outputStream; + private RFileWriter writer; + private static final Value EMPTY_VALUE = new Value(new byte[0]); + + public RFileKeyValueOutputStreamBase(OutputStream outputStream) throws IOException { + this.outputStream = outputStream; + this.writer = RFile.newWriter().to(outputStream).withVisibilityCacheSize(10).build(); + } + + public void writeKeyValue(Key k, Value v) throws IOException { + writer.append(k, v); + } + + public void writeSize(int size) throws IOException { + writer.append(SizeKeyUtil.getKey(size), EMPTY_VALUE); + } + + public void close() throws IOException { + if (writer != null) { + writer.close(); + writer = null; + } + } + + public static class SizeKeyUtil { + private static final char MAX_UNICODE = (char) Character.MAX_CODE_POINT; + public static final Text SIZE_ROW = new Text(MAX_UNICODE + "_SIZE_" + MAX_UNICODE); + + public static Key getKey(int size) { + return new Key(SIZE_ROW, new Text(Integer.toString(size))); + } + + public static boolean isSizeKey(Key key) { + return key.getRow().equals(SIZE_ROW); + } + + public static int getSize(Key key) { + return Integer.parseInt(key.getColumnFamily().toString()); + } + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java b/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java index 83a9087f083..42e442f4276 100644 --- a/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java @@ -1,5 +1,6 @@ package datawave.query; +import java.net.URL; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -152,6 +153,10 @@ public void setup() { TimeZone.setDefault(TimeZone.getTimeZone("GMT")); logic.setFullTableScanEnabled(true); + // setup the hadoop configuration + URL hadoopConfig = this.getClass().getResource("/testhadoop.config"); + logic.setHdfsSiteConfigURLs(hadoopConfig.toExternalForm()); + logic.setQueryExecutionForPageTimeout(300000000000000L); deserializer = new KryoDocumentDeserializer(); } @@ -182,7 +187,10 @@ protected void runTestQueryWithUniqueness(Set> expected, String quer TransformIterator iter = new DatawaveTransformIterator(logic.iterator(), transformer); List eventList = new ArrayList<>(); while (iter.hasNext()) { - eventList.add(iter.next()); + Object o = iter.next(); + if (o != null) { + eventList.add(o); + } } BaseQueryResponse response = transformer.createResponse(eventList); @@ -195,6 +203,9 @@ protected void runTestQueryWithUniqueness(Set> expected, String quer Assert.assertTrue(response instanceof DefaultEventQueryResponse); DefaultEventQueryResponse eventQueryResponse = (DefaultEventQueryResponse) response; + // copy expected set to avoid modifying parameter passed in + expected = new HashSet<>(expected); + for (EventBase event : eventQueryResponse.getEvents()) { boolean found = false; for (Iterator> it = expected.iterator(); it.hasNext();) { @@ -206,9 +217,9 @@ protected void runTestQueryWithUniqueness(Set> expected, String quer break; } } - Assert.assertTrue(found); + Assert.assertTrue("Failed to find " + event.getMetadata().getInternalId() + " in expected results", found); } - Assert.assertTrue(expected.isEmpty()); + Assert.assertTrue("Failed to find all expected results. Missing " + expected, expected.isEmpty()); } @Test @@ -226,12 +237,14 @@ public void testUniqueness() throws Exception { extraParameters.put("unique.fields", "DEATH_DATE,$MAGIC"); runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + expected.clear(); expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); extraParameters.put("unique.fields", "$DEATH_DATE,BIRTH_DATE"); runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + expected.clear(); expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); @@ -271,12 +284,14 @@ public void testUniquenessUsingFunction() throws Exception { runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); queryString = "UUID =~ '^[CS].*' && f:unique('DEATH_DATE','$BIRTH_DATE')"; + expected.clear(); expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); queryString = "UUID =~ '^[CS].*' && f:unique('death_date','$birth_date')"; + expected.clear(); expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); @@ -298,12 +313,14 @@ public void testUniquenessUsingLuceneFunction() throws Exception { runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); queryString = "UUID:/^[CS].*/ AND #UNIQUE(DEATH_DATE,$BIRTH_DATE)"; + expected.clear(); expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); queryString = "UUID:/^[CS].*/ AND #UNIQUE(death_date,birth_date)"; + expected.clear(); expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); @@ -355,4 +372,77 @@ public void testUniquenessWithModelAliases() throws Exception { String queryString = "UUID:/^[CS].*/ AND #UNIQUE(BOTH_NULL)"; runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); } + + @Test + public void testRecentUniquenessWithModelAliases() throws Exception { + Map extraParameters = new HashMap<>(); + extraParameters.put("include.grouping.context", "true"); + extraParameters.put("query.syntax", "LUCENE"); + + Set> expected = new HashSet<>(); + expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID, WiseGuysIngest.corleoneUID, WiseGuysIngest.caponeUID)); + Date startDate = format.parse("20091231"); + Date endDate = format.parse("20150101"); + + String queryString = "UUID:/^[CS].*/ AND #MOST_RECENT_UNIQUE(BOTH_NULL)"; + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + } + + @Test + public void testMostRecentUniqueness() throws Exception { + Map extraParameters = new HashMap<>(); + extraParameters.put("include.grouping.context", "true"); + extraParameters.put(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + Date startDate = format.parse("20091231"); + Date endDate = format.parse("20150101"); + + String queryString = "UUID =~ '^[CS].*'"; + + Set> expected = new HashSet<>(); + expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); + extraParameters.put("unique.fields", "DEATH_DATE,$MAGIC"); + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + + expected.clear(); + expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); + extraParameters.put("unique.fields", "death_date,$magic"); + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + + expected.clear(); + expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); + expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); + expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); + extraParameters.put("unique.fields", "$DEATH_DATE,BIRTH_DATE"); + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + + expected.clear(); + expected.add(Sets.newHashSet(WiseGuysIngest.sopranoUID)); + expected.add(Sets.newHashSet(WiseGuysIngest.corleoneUID)); + expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); + extraParameters.put("unique.fields", "death_date,birth_date"); + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + } + + @Test + public void testHannahHypothesis() throws Exception { + Map extraParameters = new HashMap<>(); + Date startDate = format.parse("20091231"); + Date endDate = format.parse("20150101"); + + Set> expected = new HashSet<>(); + expected.add(Sets.newHashSet(WiseGuysIngest.caponeUID)); + extraParameters.put(QueryParameters.MOST_RECENT_UNIQUE, "true"); + extraParameters.put("unique.fields", "DEATH_DATE,$MAGIC"); + String queryString = "UUID =~ '^[CS].*'"; + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + + extraParameters.clear(); + extraParameters.put(QueryParameters.MOST_RECENT_UNIQUE, "true"); + queryString = "UUID =~ '^[CS].*' && f:unique(death_date,magic)"; + runTestQueryWithUniqueness(expected, queryString, startDate, endDate, extraParameters); + + this.getClass().getMethod("testHannahHypothesis").getName().replace("Hypothesis", "Theory"); + } + } diff --git a/warehouse/query-core/src/test/java/datawave/query/attributes/UniqueFieldsTest.java b/warehouse/query-core/src/test/java/datawave/query/attributes/UniqueFieldsTest.java index a4bf1421a30..00850fba9a6 100644 --- a/warehouse/query-core/src/test/java/datawave/query/attributes/UniqueFieldsTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/attributes/UniqueFieldsTest.java @@ -71,7 +71,7 @@ public void testNonEmptyUniqueFieldsToString() { uniqueFields.put("fieldD", UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR); uniqueFields.put("fieldD", UniqueGranularity.TRUNCATE_TEMPORAL_TO_MINUTE); - assertEquals("fieldA[ALL],fieldB[ALL,DAY],fieldC[HOUR],fieldD[HOUR,MINUTE]", uniqueFields.toString()); + assertEquals("FIELDA[ALL],FIELDB[ALL,DAY],FIELDC[HOUR],FIELDD[HOUR,MINUTE]", uniqueFields.toString()); } /** @@ -345,7 +345,7 @@ public void testSerialization() throws JsonProcessingException { UniqueFields uniqueFields = new UniqueFields(sortedFields); String json = objectMapper.writeValueAsString(uniqueFields); - assertEquals("\"fieldA[ALL],fieldB[ALL,DAY],fieldC[HOUR],fieldD[HOUR,MINUTE]\"", json); + assertEquals("\"FIELDA[ALL],FIELDB[ALL,DAY],FIELDC[HOUR],FIELDD[HOUR,MINUTE]\"", json); } /** @@ -387,7 +387,7 @@ public void testValueTransformation() { uniqueFields.put("fieldA", UniqueGranularity.TRUNCATE_TEMPORAL_TO_MINUTE); Set values = Sets.newHashSet("2020-01-12 15:30:45", "nonDateValue"); - SortedSet actual = Sets.newTreeSet(uniqueFields.transformValues("fieldA", values)); + SortedSet actual = Sets.newTreeSet(uniqueFields.transformValues("FIELDA", values)); assertEquals(expected, actual); } @@ -407,8 +407,6 @@ public void testDeconstructIdentifierFields() { uniqueFields.put("$FIELDB", UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR); uniqueFields.put("FIELDC", UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR); - uniqueFields.deconstructIdentifierFields(); - SortedSet actual = Sets.newTreeSet(uniqueFields.getFields()); assertEquals(expected, actual); diff --git a/warehouse/query-core/src/test/java/datawave/query/config/ShardQueryConfigurationTest.java b/warehouse/query-core/src/test/java/datawave/query/config/ShardQueryConfigurationTest.java index ae9f64a1a84..ff6b304069b 100644 --- a/warehouse/query-core/src/test/java/datawave/query/config/ShardQueryConfigurationTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/config/ShardQueryConfigurationTest.java @@ -443,6 +443,8 @@ public void setUp() throws Exception { updatedValues.put("compositeFilterFunctionsEnabled", true); defaultValues.put("uniqueFields", new UniqueFields()); updatedValues.put("uniqueFields", UniqueFields.from("FIELD_U,FIELD_V")); + defaultValues.put("uniqueCacheBufferSize", 100); + updatedValues.put("uniqueCacheBufferSize", 1000); defaultValues.put("cacheModel", false); updatedValues.put("cacheModel", true); defaultValues.put("trackSizes", true); diff --git a/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/QueryOptionsFromQueryVisitorTest.java b/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/QueryOptionsFromQueryVisitorTest.java index 694984bbc54..6a0e4bf1950 100644 --- a/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/QueryOptionsFromQueryVisitorTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/jexl/visitors/QueryOptionsFromQueryVisitorTest.java @@ -143,34 +143,85 @@ public void testUniqueFunction() throws ParseException { // Verify an empty function results in an empty parameter value. assertResult("f:unique_by_day()", ""); assertOption(QueryParameters.UNIQUE_FIELDS, ""); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Verify that fields of no specified granularity are added with the default ALL granularity. assertResult("f:unique('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL],field2[ALL],field3[ALL]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL],FIELD2[ALL],FIELD3[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Verify that fields with DAY granularity are added as such. assertResult("f:unique('field1[DAY]','field2[DAY]','field3[DAY]')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY],field3[DAY]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY],FIELD3[DAY]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Verify that fields with HOUR granularity are added as such. assertResult("f:unique('field1[HOUR]','field2[HOUR]','field3[HOUR]')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[HOUR],field2[HOUR],field3[HOUR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[HOUR],FIELD2[HOUR],FIELD3[HOUR]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Verify that fields with MINUTE granularity are added as such. assertResult("f:unique('field1[MINUTE]','field2[MINUTE]','field3[MINUTE]')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MINUTE],field2[MINUTE],field3[MINUTE]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MINUTE],FIELD2[MINUTE],FIELD3[MINUTE]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Verify that fields from multiple unique functions are merged together. assertResult("f:unique('field1','field2') AND f:unique('field2[DAY]','field3[DAY]') AND f:unique('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL],field2[ALL,DAY],field3[DAY],field4[ALL]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL],FIELD2[ALL,DAY],FIELD3[DAY],FIELD4[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Verify more complex fields with multiple granularity levels are merged together. assertResult("f:unique('field1[DAY]','field2[DAY,HOUR]','field3[HOUR,MINUTE]','field4[ALL,MINUTE]','field5')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY,HOUR],field3[HOUR,MINUTE],field4[ALL,MINUTE],field5[ALL]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY,HOUR],FIELD3[HOUR,MINUTE],FIELD4[ALL,MINUTE],FIELD5[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); // Lucene will parse comma-delimited granularity levels into separate strings. Ensure it still parses correctly. assertResult("f:unique('field1[DAY]','field2[DAY','HOUR]','field3[HOUR','MINUTE]','field4[ALL','MINUTE]','field5')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY,HOUR],field3[HOUR,MINUTE],field4[ALL,MINUTE],field5[ALL]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY,HOUR],FIELD3[HOUR,MINUTE],FIELD4[ALL,MINUTE],FIELD5[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, null); + } + + @Test + public void testMostRecentUniqueFunction() throws ParseException { + // Verify an empty function results in an empty parameter value. + assertResult("f:most_recent_unique_by_day()", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, ""); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Verify that fields of no specified granularity are added with the default ALL granularity. + assertResult("f:most_recent_unique('field1','field2','field3')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL],FIELD2[ALL],FIELD3[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Verify that fields with DAY granularity are added as such. + assertResult("f:most_recent_unique('field1[DAY]','field2[DAY]','field3[DAY]')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY],FIELD3[DAY]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Verify that fields with HOUR granularity are added as such. + assertResult("f:most_recent_unique('field1[HOUR]','field2[HOUR]','field3[HOUR]')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[HOUR],FIELD2[HOUR],FIELD3[HOUR]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Verify that fields with MINUTE granularity are added as such. + assertResult("f:most_recent_unique('field1[MINUTE]','field2[MINUTE]','field3[MINUTE]')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MINUTE],FIELD2[MINUTE],FIELD3[MINUTE]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Verify that fields from multiple unique functions are merged together. + assertResult("f:most_recent_unique('field1','field2') AND f:unique('field2[DAY]','field3[DAY]') AND f:unique('field4')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL],FIELD2[ALL,DAY],FIELD3[DAY],FIELD4[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Verify more complex fields with multiple granularity levels are merged together. + assertResult("f:most_recent_unique('field1[DAY]','field2[DAY,HOUR]','field3[HOUR,MINUTE]','field4[ALL,MINUTE]','field5')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY,HOUR],FIELD3[HOUR,MINUTE],FIELD4[ALL,MINUTE],FIELD5[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); + + // Lucene will parse comma-delimited granularity levels into separate strings. Ensure it still parses correctly. + assertResult("f:most_recent_unique('field1[DAY]','field2[DAY','HOUR]','field3[HOUR','MINUTE]','field4[ALL','MINUTE]','field5')", ""); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY,HOUR],FIELD3[HOUR,MINUTE],FIELD4[ALL,MINUTE],FIELD5[ALL]"); + assertOption(QueryParameters.MOST_RECENT_UNIQUE, "true"); } @Test @@ -181,11 +232,11 @@ public void testUniqueByDay() throws ParseException { // Verify fields are added with the DAY granularity. assertResult("f:unique_by_day('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY],field3[DAY]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[DAY],FIELD2[DAY],FIELD3[DAY]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[HOUR]') AND f:unique_by_day('field1','field2','field3') AND f:unique_by_day('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,DAY],field2[DAY,HOUR],field3[DAY],field4[DAY]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,DAY],FIELD2[DAY,HOUR],FIELD3[DAY],FIELD4[DAY]"); } @Test @@ -196,11 +247,11 @@ public void testUniqueByHour() throws ParseException { // Verify fields are added with the HOUR granularity. assertResult("f:unique_by_hour('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[HOUR],field2[HOUR],field3[HOUR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[HOUR],FIELD2[HOUR],FIELD3[HOUR]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_hour('field1','field2','field3') AND f:unique_by_hour('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,HOUR],field2[DAY,HOUR],field3[HOUR],field4[HOUR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,HOUR],FIELD2[DAY,HOUR],FIELD3[HOUR],FIELD4[HOUR]"); } @Test @@ -211,11 +262,11 @@ public void testUniqueByMonth() throws ParseException { // Verify fields are added with the HOUR granularity. assertResult("f:unique_by_month('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MONTH],field2[MONTH],field3[MONTH]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MONTH],FIELD2[MONTH],FIELD3[MONTH]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_month('field1','field2','field3') AND f:unique_by_month('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,MONTH],field2[DAY,MONTH],field3[MONTH],field4[MONTH]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,MONTH],FIELD2[DAY,MONTH],FIELD3[MONTH],FIELD4[MONTH]"); } @Test @@ -226,11 +277,11 @@ public void testUniqueBySecond() throws ParseException { // Verify fields are added with the HOUR granularity. assertResult("f:unique_by_second('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[SECOND],field2[SECOND],field3[SECOND]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[SECOND],FIELD2[SECOND],FIELD3[SECOND]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_second('field1','field2','field3') AND f:unique_by_second('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,SECOND],field2[DAY,SECOND],field3[SECOND],field4[SECOND]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,SECOND],FIELD2[DAY,SECOND],FIELD3[SECOND],FIELD4[SECOND]"); } @Test @@ -241,11 +292,11 @@ public void testUniqueByMillisecond() throws ParseException { // Verify fields are added with the HOUR granularity. assertResult("f:unique_by_millisecond('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MILLISECOND],field2[MILLISECOND],field3[MILLISECOND]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MILLISECOND],FIELD2[MILLISECOND],FIELD3[MILLISECOND]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_millisecond('field1','field2','field3') AND f:unique_by_millisecond('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,MILLISECOND],field2[DAY,MILLISECOND],field3[MILLISECOND],field4[MILLISECOND]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,MILLISECOND],FIELD2[DAY,MILLISECOND],FIELD3[MILLISECOND],FIELD4[MILLISECOND]"); } @Test @@ -256,11 +307,11 @@ public void testUniqueByYear() throws ParseException { // Verify fields are added with the MINUTE granularity. assertResult("f:unique_by_year('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[YEAR],field2[YEAR],field3[YEAR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[YEAR],FIELD2[YEAR],FIELD3[YEAR]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_year('field1','field2','field3') AND f:unique_by_year('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,YEAR],field2[DAY,YEAR],field3[YEAR],field4[YEAR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,YEAR],FIELD2[DAY,YEAR],FIELD3[YEAR],FIELD4[YEAR]"); } @Test @@ -271,11 +322,11 @@ public void testUniqueByMinute() throws ParseException { // Verify fields are added with the MINUTE granularity. assertResult("f:unique_by_minute('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MINUTE],field2[MINUTE],field3[MINUTE]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MINUTE],FIELD2[MINUTE],FIELD3[MINUTE]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_minute('field1','field2','field3') AND f:unique_by_minute('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,MINUTE],field2[DAY,MINUTE],field3[MINUTE],field4[MINUTE]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,MINUTE],FIELD2[DAY,MINUTE],FIELD3[MINUTE],FIELD4[MINUTE]"); } @Test @@ -286,26 +337,26 @@ public void testUniqueByTenth() throws ParseException { // Verify fields are added with the MINUTE granularity. assertResult("f:unique_by_tenth_of_hour('field1','field2','field3')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[TENTH_OF_HOUR],field2[TENTH_OF_HOUR],field3[TENTH_OF_HOUR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[TENTH_OF_HOUR],FIELD2[TENTH_OF_HOUR],FIELD3[TENTH_OF_HOUR]"); // Verify fields from multiple functions are merged. assertResult("f:unique('field1','field2[DAY]') AND f:unique_by_tenth_of_hour('field1','field2','field3') AND f:unique_by_tenth_of_hour('field4')", ""); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL,TENTH_OF_HOUR],field2[DAY,TENTH_OF_HOUR],field3[TENTH_OF_HOUR],field4[TENTH_OF_HOUR]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[ALL,TENTH_OF_HOUR],FIELD2[DAY,TENTH_OF_HOUR],FIELD3[TENTH_OF_HOUR],FIELD4[TENTH_OF_HOUR]"); } @Test public void testNonFunctionNodesWithJunctions() throws ParseException { // Verify that only the function node is removed. assertResult("f:unique_by_minute('field1') AND FOO == 'bar'", "FOO == 'bar'"); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MINUTE]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MINUTE]"); // Verify that only the function node is removed. assertResult("f:unique_by_minute('field1') AND (FOO == 'bar' AND BAT == 'foo')", "(FOO == 'bar' AND BAT == 'foo')"); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MINUTE]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MINUTE]"); // Verify that only the function node is removed. assertResult("f:unique_by_minute('field1') OR FOO == 'bar'", "FOO == 'bar'"); - assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MINUTE]"); + assertOption(QueryParameters.UNIQUE_FIELDS, "FIELD1[MINUTE]"); // Verify that AND nodes are cleaned up. assertResult("(FOO == 'bar' OR (BAR == 'foo' AND f:groupby('field1','field2')))", "(FOO == 'bar' OR (BAR == 'foo'))"); diff --git a/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java b/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java new file mode 100644 index 00000000000..f309f4a2b9c --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java @@ -0,0 +1,86 @@ +package datawave.query.transformer; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Collections; +import java.util.UUID; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import datawave.microservice.query.QueryImpl; +import datawave.query.attributes.UniqueGranularity; +import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.tables.ShardQueryLogic; +import datawave.query.util.sortedset.FileSortedSet; + +public class UniqueTransformMostRecentTest extends UniqueTransformTest { + + protected ShardQueryLogic logic = new ShardQueryLogic(); + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Before + public void setUp() throws IOException { + uniqueFields.setMostRecent(true); + + // setup the hadoop configuration + URL hadoopConfig = this.getClass().getResource("/testhadoop.config"); + logic.setHdfsSiteConfigURLs(hadoopConfig.toExternalForm()); + + // setup a directory for cache results + File tmpDir = temporaryFolder.newFolder(); + IvaratorCacheDirConfig config = new IvaratorCacheDirConfig(tmpDir.toURI().toString()); + logic.setIvaratorCacheDirConfigs(Collections.singletonList(config)); + + QueryImpl query = new QueryImpl(); + query.setId(UUID.randomUUID()); + logic.getConfig().setQuery(query); + } + + @Override + protected UniqueTransform getUniqueTransform() { + try { + // @formatter:off + return new UniqueTransform.Builder() + .withUniqueFields(uniqueFields) + .withQueryExecutionForPageTimeout(Long.MAX_VALUE) + .withBufferPersistThreshold(logic.getUniqueCacheBufferSize()) + .withIvaratorCacheDirConfigs(logic.getIvaratorCacheDirConfigs()) + .withHdfsSiteConfigURLs(logic.getHdfsSiteConfigURLs()) + .withSubDirectory(logic.getConfig().getQuery().getId().toString()) + .withMaxOpenFiles(logic.getIvaratorMaxOpenFiles()) + .withNumRetries(logic.getIvaratorNumRetries()) + .withPersistOptions(new FileSortedSet.PersistOptions(true, false, 0)) + .build(); + // @formatter:on + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Verify that field matching is case-insensitive. Query: #UNIQUE(attr0, Attr1, ATTR2) + */ + @Test + public void testMostRecentUniqueness() { + givenInputDocument(1).withKeyValue("ATTR0", randomValues.get(0)); + givenInputDocument(2).withKeyValue("ATTR0", randomValues.get(1)).isExpectedToBeUnique(); + givenInputDocument(3).withKeyValue("ATTR0", randomValues.get(0)).isExpectedToBeUnique(); + givenInputDocument(1).withKeyValue("Attr1", randomValues.get(2)); + givenInputDocument(2).withKeyValue("Attr1", randomValues.get(3)).isExpectedToBeUnique(); + givenInputDocument(3).withKeyValue("Attr1", randomValues.get(2)).isExpectedToBeUnique(); + givenInputDocument(1).withKeyValue("attr2", randomValues.get(4)); + givenInputDocument(2).withKeyValue("attr2", randomValues.get(0)).isExpectedToBeUnique(); + givenInputDocument(3).withKeyValue("attr2", randomValues.get(4)).isExpectedToBeUnique(); + + givenValueTransformerForFields(UniqueGranularity.ALL, "attr0", "Attr1", "ATTR2"); + + assertUniqueDocuments(); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformTest.java b/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformTest.java index 3eb407a43a3..60749416648 100644 --- a/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformTest.java @@ -2,6 +2,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; @@ -19,45 +20,51 @@ import java.util.Set; import java.util.Spliterator; import java.util.Spliterators; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.apache.accumulo.core.data.ArrayByteSequence; -import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; import org.apache.commons.collections4.Transformer; import org.apache.commons.collections4.iterators.TransformIterator; import org.apache.commons.lang.RandomStringUtils; -import org.apache.hadoop.io.Text; import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterators; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; +import com.google.common.collect.SortedSetMultimap; import com.google.common.collect.TreeMultimap; +import com.google.common.primitives.Longs; +import datawave.ingest.time.Now; import datawave.query.attributes.Attribute; import datawave.query.attributes.Attributes; import datawave.query.attributes.DiacriticContent; import datawave.query.attributes.Document; +import datawave.query.attributes.DocumentKey; import datawave.query.attributes.TimingMetadata; import datawave.query.attributes.UniqueFields; import datawave.query.attributes.UniqueGranularity; import datawave.query.function.LogTiming; +import datawave.query.iterator.profile.FinalDocumentTrackingIterator; import datawave.query.jexl.JexlASTHelper; public class UniqueTransformTest { - private static final Random random = new Random(1000); - private static final List randomValues = new ArrayList<>(); + protected static final Random random = new Random(1000); + private static final AtomicLong counter = new AtomicLong(); - private final List inputDocuments = new ArrayList<>(); - private final List expectedUniqueDocuments = new ArrayList<>(); - private byte[] expectedOrderedFieldValues = null; - private UniqueFields uniqueFields = new UniqueFields(); + protected static final List randomValues = new ArrayList<>(); + + protected final List inputDocuments = new ArrayList<>(); + protected final List expectedUniqueDocuments = new ArrayList<>(); + protected byte[] expectedOrderedFieldValues = null; + protected UniqueFields uniqueFields = new UniqueFields(); @BeforeClass public static void setup() { @@ -97,7 +104,7 @@ public void testUniquenessWithRandomDocuments() { while (expectedUniqueDocuments > inputDocuments.size() / 2 || expectedUniqueDocuments < 10) { fields.clear(); while (fields.size() < 3) { - fields.add("Attr" + random.nextInt(100)); + fields.add("ATTR" + random.nextInt(100)); } expectedUniqueDocuments = countUniqueness(inputDocuments, fields); } @@ -108,7 +115,7 @@ public void testUniquenessWithRandomDocuments() { assertEquals(expectedUniqueDocuments, uniqueDocuments.size()); } - private int countUniqueness(List input, Set fields) { + protected int countUniqueness(List input, Set fields) { Set uniqueValues = new HashSet<>(); for (Document document : input) { Multimap fieldValues = getFieldValues(document, fields); @@ -117,7 +124,7 @@ private int countUniqueness(List input, Set fields) { return uniqueValues.size(); } - private Multimap getFieldValues(Document document, Set fields) { + protected Multimap getFieldValues(Document document, Set fields) { Multimap values = HashMultimap.create(); for (String docField : document.getDictionary().keySet()) { for (String field : fields) { @@ -134,7 +141,7 @@ private Multimap getFieldValues(Document document, Set fi return values; } - private String getString(Multimap fieldValues) { + protected String getString(Multimap fieldValues) { StringBuilder sb = new StringBuilder(); fieldValues.keySet().stream().sorted().forEach((field) -> { if (sb.length() > 0) { @@ -154,12 +161,12 @@ public void testUniquenessForCaseInsensitivity() { givenInputDocument().withKeyValue("ATTR0", randomValues.get(0)).isExpectedToBeUnique(); givenInputDocument().withKeyValue("ATTR0", randomValues.get(1)).isExpectedToBeUnique(); givenInputDocument().withKeyValue("ATTR0", randomValues.get(0)); - givenInputDocument().withKeyValue("Attr1", randomValues.get(2)).isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr1", randomValues.get(3)).isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr1", randomValues.get(2)); - givenInputDocument().withKeyValue("attr2", randomValues.get(4)).isExpectedToBeUnique(); - givenInputDocument().withKeyValue("attr2", randomValues.get(0)).isExpectedToBeUnique(); - givenInputDocument().withKeyValue("attr2", randomValues.get(4)); + givenInputDocument().withKeyValue("ATTR1", randomValues.get(2)).isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR1", randomValues.get(3)).isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR1", randomValues.get(2)); + givenInputDocument().withKeyValue("ATTR2", randomValues.get(4)).isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR2", randomValues.get(0)).isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR2", randomValues.get(4)); givenValueTransformerForFields(UniqueGranularity.ALL, "attr0", "Attr1", "ATTR2"); @@ -171,11 +178,11 @@ public void testUniquenessForCaseInsensitivity() { */ @Test public void testUniquenessWithValueTransformer_DAY() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 12:40:15"); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 05:04:20"); - givenInputDocument().withKeyValue("Attr0", "2001-03-12 05:04:20").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "nonDateValue").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 12:40:15"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 05:04:20"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-12 05:04:20").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "nonDateValue").isExpectedToBeUnique(); givenValueTransformerForFields(UniqueGranularity.TRUNCATE_TEMPORAL_TO_DAY, "Attr0"); @@ -187,11 +194,11 @@ public void testUniquenessWithValueTransformer_DAY() { */ @Test public void testUniquenessWithValueTransformer_HOUR() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:40:15"); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 05:04:20").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 05:04:30"); - givenInputDocument().withKeyValue("Attr0", "nonDateValue").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:40:15"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 05:04:20").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 05:04:30"); + givenInputDocument().withKeyValue("ATTR0", "nonDateValue").isExpectedToBeUnique(); givenValueTransformerForFields(UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR, "Attr0"); @@ -203,11 +210,11 @@ public void testUniquenessWithValueTransformer_HOUR() { */ @Test public void testUniquenessWithValueTransformer_MINUTE() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:20"); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:04:20").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:04:15"); - givenInputDocument().withKeyValue("Attr0", "nonDateValue").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:20"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:04:20").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:04:15"); + givenInputDocument().withKeyValue("ATTR0", "nonDateValue").isExpectedToBeUnique(); givenValueTransformerForFields(UniqueGranularity.TRUNCATE_TEMPORAL_TO_MINUTE, "Attr0"); @@ -220,18 +227,18 @@ public void testUniquenessWithValueTransformer_MINUTE() { */ @Test public void testUniquenessWithMixedValueTransformersForDifferentFields() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 12:40:15"); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 05:04:20"); - givenInputDocument().withKeyValue("Attr0", "2001-03-12 05:04:20").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr1", "2001-03-10 10:15:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr1", "2001-03-10 10:40:15"); - givenInputDocument().withKeyValue("Attr1", "2001-03-10 05:04:20").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr1", "2001-03-10 05:04:30"); - givenInputDocument().withKeyValue("Attr2", "2001-03-10 10:15:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr2", "2001-03-10 10:15:20"); - givenInputDocument().withKeyValue("Attr2", "2001-03-10 10:04:20").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr2", "2001-03-10 10:04:15"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 12:40:15"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 05:04:20"); + givenInputDocument().withKeyValue("ATTR0", "2001-03-12 05:04:20").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR1", "2001-03-10 10:15:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR1", "2001-03-10 10:40:15"); + givenInputDocument().withKeyValue("ATTR1", "2001-03-10 05:04:20").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR1", "2001-03-10 05:04:30"); + givenInputDocument().withKeyValue("ATTR2", "2001-03-10 10:15:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR2", "2001-03-10 10:15:20"); + givenInputDocument().withKeyValue("ATTR2", "2001-03-10 10:04:20").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR2", "2001-03-10 10:04:15"); givenValueTransformerForFields(UniqueGranularity.TRUNCATE_TEMPORAL_TO_DAY, "Attr0"); givenValueTransformerForFields(UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR, "Attr1"); @@ -246,12 +253,12 @@ public void testUniquenessWithMixedValueTransformersForDifferentFields() { */ @Test public void testThatValueTransformer_ALL_Supersedes_MINUTE() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:01").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:02").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:03").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:04").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:15:04"); - givenInputDocument().withKeyValue("Attr0", "nonDateValue").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:01").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:02").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:03").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:04").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:15:04"); + givenInputDocument().withKeyValue("ATTR0", "nonDateValue").isExpectedToBeUnique(); givenValueTransformersForField("Attr0", UniqueGranularity.ALL, UniqueGranularity.TRUNCATE_TEMPORAL_TO_MINUTE); @@ -264,12 +271,12 @@ public void testThatValueTransformer_ALL_Supersedes_MINUTE() { */ @Test public void testThatValueTransformer_MINUTE_Supersedes_HOUR() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:01:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:02:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:03:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:04:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:04:20"); - givenInputDocument().withKeyValue("Attr0", "nonDateValue").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:01:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:02:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:03:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:04:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:04:20"); + givenInputDocument().withKeyValue("ATTR0", "nonDateValue").isExpectedToBeUnique(); givenValueTransformersForField("Attr0", UniqueGranularity.TRUNCATE_TEMPORAL_TO_MINUTE, UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR); @@ -282,12 +289,12 @@ public void testThatValueTransformer_MINUTE_Supersedes_HOUR() { */ @Test public void testThatValueTransformer_HOUR_Supersedes_DAY() { - givenInputDocument().withKeyValue("Attr0", "2001-03-10 10:01:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 11:01:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 12:01:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 13:01:15").isExpectedToBeUnique(); - givenInputDocument().withKeyValue("Attr0", "2001-03-10 13:20:15"); - givenInputDocument().withKeyValue("Attr0", "nonDateValue").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 10:01:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 11:01:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 12:01:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 13:01:15").isExpectedToBeUnique(); + givenInputDocument().withKeyValue("ATTR0", "2001-03-10 13:20:15"); + givenInputDocument().withKeyValue("ATTR0", "nonDateValue").isExpectedToBeUnique(); givenValueTransformersForField("Attr0", UniqueGranularity.TRUNCATE_TEMPORAL_TO_HOUR, UniqueGranularity.TRUNCATE_TEMPORAL_TO_DAY); @@ -299,7 +306,7 @@ public void testUniquenessWithTimingMetric() { List input = new ArrayList<>(); List expected = new ArrayList<>(); - String MARKER_STRING = "\u2735FinalDocument\u2735"; + String MARKER_STRING = FinalDocumentTrackingIterator.MARKER_TEXT.toString(); TimingMetadata timingMetadata = new TimingMetadata(); timingMetadata.setNextCount(5l); @@ -326,16 +333,16 @@ public void testUniquenessWithTwoGroups() { // @formatter:off givenInputDocument() - .withKeyValue("Attr0.0.0.0", randomValues.get(0)) - .withKeyValue("Attr1.0.1.0", randomValues.get(1)) - .withKeyValue("Attr0.0.0.1", randomValues.get(2)) - .withKeyValue("Attr1.0.1.1", randomValues.get(3)); + .withKeyValue("ATTR0.0.0.0", randomValues.get(0)) + .withKeyValue("ATTR1.0.1.0", randomValues.get(1)) + .withKeyValue("ATTR0.0.0.1", randomValues.get(2)) + .withKeyValue("ATTR1.0.1.1", randomValues.get(3)); expectedOrderedFieldValues = givenExpectedOrderedFieldValues() - .withKeyValue("Attr0", randomValues.get(0)) - .withKeyValue("Attr1", randomValues.get(1)) - .withKeyValue("Attr0", randomValues.get(2)) - .withKeyValue("Attr1", randomValues.get(3)).build(); + .withKeyValue("ATTR0", randomValues.get(0)) + .withKeyValue("ATTR1", randomValues.get(1)) + .withKeyValue("ATTR0", randomValues.get(2)) + .withKeyValue("ATTR1", randomValues.get(3)).build(); // @formatter:on givenValueTransformerForFields(UniqueGranularity.ALL, "Attr0", "Attr1"); @@ -357,18 +364,18 @@ public void testUniquenessWithTwoGroupsAndUngrouped() { // @formatter:off givenInputDocument() - .withKeyValue("Attr0.0.0.0", randomValues.get(0)) - .withKeyValue("Attr1.0.1.0", randomValues.get(1)) - .withKeyValue("Attr0.0.0.1", randomValues.get(2)) - .withKeyValue("Attr1.0.1.1", randomValues.get(3)) - .withKeyValue("Attr3", randomValues.get(4)); + .withKeyValue("ATTR0.0.0.0", randomValues.get(0)) + .withKeyValue("ATTR1.0.1.0", randomValues.get(1)) + .withKeyValue("ATTR0.0.0.1", randomValues.get(2)) + .withKeyValue("ATTR1.0.1.1", randomValues.get(3)) + .withKeyValue("ATTR3", randomValues.get(4)); expectedOrderedFieldValues = givenExpectedOrderedFieldValues() - .withKeyValue("Attr0", randomValues.get(0)) - .withKeyValue("Attr1", randomValues.get(1)) - .withKeyValue("Attr0", randomValues.get(2)) - .withKeyValue("Attr1", randomValues.get(3)) - .withKeyValue("Attr3", randomValues.get(4)).build(); + .withKeyValue("ATTR0", randomValues.get(0)) + .withKeyValue("ATTR1", randomValues.get(1)) + .withKeyValue("ATTR0", randomValues.get(2)) + .withKeyValue("ATTR1", randomValues.get(3)) + .withKeyValue("ATTR3", randomValues.get(4)).build(); // @formatter:on givenValueTransformerForFields(UniqueGranularity.ALL, "Attr0", "Attr1", "Attr3"); @@ -390,18 +397,18 @@ public void testUniquenessWithTwoGroupsAndSeparateGroup() { // @formatter:off givenInputDocument() - .withKeyValue("Attr0.0.0.0", randomValues.get(0)) - .withKeyValue("Attr1.0.1.0", randomValues.get(1)) - .withKeyValue("Attr0.0.0.1", randomValues.get(2)) - .withKeyValue("Attr1.0.1.1", randomValues.get(3)) - .withKeyValue("Attr3.1.0.0", randomValues.get(4)); + .withKeyValue("ATTR0.0.0.0", randomValues.get(0)) + .withKeyValue("ATTR1.0.1.0", randomValues.get(1)) + .withKeyValue("ATTR0.0.0.1", randomValues.get(2)) + .withKeyValue("ATTR1.0.1.1", randomValues.get(3)) + .withKeyValue("ATTR3.1.0.0", randomValues.get(4)); expectedOrderedFieldValues = givenExpectedOrderedFieldValues() - .withKeyValue("Attr0", randomValues.get(0)) - .withKeyValue("Attr1", randomValues.get(1)) - .withKeyValue("Attr0", randomValues.get(2)) - .withKeyValue("Attr1", randomValues.get(3)) - .withKeyValue("Attr3", randomValues.get(4)).build(); + .withKeyValue("ATTR0", randomValues.get(0)) + .withKeyValue("ATTR1", randomValues.get(1)) + .withKeyValue("ATTR0", randomValues.get(2)) + .withKeyValue("ATTR1", randomValues.get(3)) + .withKeyValue("ATTR3", randomValues.get(4)).build(); // @formatter:on givenValueTransformerForFields(UniqueGranularity.ALL, "Attr0", "Attr1", "Attr3"); @@ -424,20 +431,20 @@ public void testUniquenessWithTwoGroupsAndSeparateGroups() { // @formatter:off givenInputDocument() - .withKeyValue("Attr0.0.0.0", randomValues.get(0)) - .withKeyValue("Attr1.0.1.0", randomValues.get(1)) - .withKeyValue("Attr0.0.0.1", randomValues.get(2)) - .withKeyValue("Attr1.0.1.1", randomValues.get(3)) - .withKeyValue("Attr3.1.0.0", randomValues.get(4)) - .withKeyValue("Attr3.1.0.1", randomValues.get(0)); + .withKeyValue("ATTR0.0.0.0", randomValues.get(0)) + .withKeyValue("ATTR1.0.1.0", randomValues.get(1)) + .withKeyValue("ATTR0.0.0.1", randomValues.get(2)) + .withKeyValue("ATTR1.0.1.1", randomValues.get(3)) + .withKeyValue("ATTR3.1.0.0", randomValues.get(4)) + .withKeyValue("ATTR3.1.0.1", randomValues.get(0)); expectedOrderedFieldValues = givenExpectedOrderedFieldValues() - .withKeyValue("Attr0", randomValues.get(0)) - .withKeyValue("Attr1", randomValues.get(1)) - .withKeyValue("Attr0", randomValues.get(2)) - .withKeyValue("Attr1", randomValues.get(3)) - .withKeyValue("Attr3", randomValues.get(4)) - .withKeyValue("Attr3", randomValues.get(0)).build(); + .withKeyValue("ATTR0", randomValues.get(0)) + .withKeyValue("ATTR1", randomValues.get(1)) + .withKeyValue("ATTR0", randomValues.get(2)) + .withKeyValue("ATTR1", randomValues.get(3)) + .withKeyValue("ATTR3", randomValues.get(4)) + .withKeyValue("ATTR3", randomValues.get(0)).build(); // @formatter:on givenValueTransformerForFields(UniqueGranularity.ALL, "Attr0", "Attr1", "Attr3"); @@ -458,18 +465,18 @@ public void testUniquenessWithTwoGroupsAndPartialGroups() { // @formatter:off givenInputDocument() - .withKeyValue("Attr0.0.0.0", randomValues.get(0)) - .withKeyValue("Attr1.0.1.0", randomValues.get(1)) - .withKeyValue("Attr0.0.0.1", randomValues.get(2)) - .withKeyValue("Attr3.1.0.0", randomValues.get(4)) - .withKeyValue("Attr3.1.0.1", randomValues.get(0)); + .withKeyValue("ATTR0.0.0.0", randomValues.get(0)) + .withKeyValue("ATTR1.0.1.0", randomValues.get(1)) + .withKeyValue("ATTR0.0.0.1", randomValues.get(2)) + .withKeyValue("ATTR3.1.0.0", randomValues.get(4)) + .withKeyValue("ATTR3.1.0.1", randomValues.get(0)); expectedOrderedFieldValues = givenExpectedOrderedFieldValues() - .withKeyValue("Attr0", randomValues.get(0)) - .withKeyValue("Attr1", randomValues.get(1)) - .withKeyValue("Attr0", randomValues.get(2)) - .withKeyValue("Attr3", randomValues.get(4)) - .withKeyValue("Attr3", randomValues.get(0)).build(); + .withKeyValue("ATTR0", randomValues.get(0)) + .withKeyValue("ATTR1", randomValues.get(1)) + .withKeyValue("ATTR0", randomValues.get(2)) + .withKeyValue("ATTR3", randomValues.get(4)) + .withKeyValue("ATTR3", randomValues.get(0)).build(); // @formatter:on givenValueTransformerForFields(UniqueGranularity.ALL, "Attr0", "Attr1", "Attr3"); @@ -477,27 +484,64 @@ public void testUniquenessWithTwoGroupsAndPartialGroups() { assertOrderedFieldValues(); } - private void assertUniqueDocuments() { + @Test + public void testFinalDocIgnored() { + SortedSetMultimap fieldMap = TreeMultimap.create(); + fieldMap.put("FIELD", UniqueGranularity.ALL); + UniqueFields fields = new UniqueFields(fieldMap); + UniqueTransform transform = new UniqueTransform(fields, 10000000L); + Key key = new Key("shard", "dt\u0000uid", FinalDocumentTrackingIterator.MARKER_TEXT.toString()); + Document doc = new Document(); + Map.Entry entry = new UnmodifiableMapEntry(key, doc); + for (int i = 0; i < 10; i++) { + assertTrue(entry == transform.apply(entry)); + } + } + + @Test + public void testIntermediateIgnored() { + SortedSetMultimap fieldMap = TreeMultimap.create(); + fieldMap.put("FIELD", UniqueGranularity.ALL); + UniqueFields fields = new UniqueFields(fieldMap); + UniqueTransform transform = new UniqueTransform(fields, 10000000L); + Key key = new Key("shard", "dt\u0000uid"); + Document doc = new Document(); + doc.setIntermediateResult(true); + Map.Entry entry = new UnmodifiableMapEntry(key, doc); + for (int i = 0; i < 10; i++) { + assertTrue(entry == transform.apply(entry)); + } + } + + protected void assertUniqueDocuments() { List actual = getUniqueDocumentsWithUpdateConfigCalls(inputDocuments); Collections.sort(expectedUniqueDocuments); Collections.sort(actual); - assertEquals("Unique documents do not match expected", expectedUniqueDocuments, actual); + assertEquals("Unique documents do not match expected", getIds(expectedUniqueDocuments), getIds(actual)); } - private List getUniqueDocuments(List documents) { + protected List getIds(List docs) { + List ids = new ArrayList<>(); + for (Document d : docs) { + ids.add(d.getDictionary().get("RECORD_ID").getData().toString()); + } + return ids; + } + + protected List getUniqueDocuments(List documents) { Transformer> docToEntry = document -> Maps.immutableEntry(document.getMetadata(), document); TransformIterator> inputIterator = new TransformIterator<>(documents.iterator(), docToEntry); UniqueTransform uniqueTransform = getUniqueTransform(); - Iterator> resultIterator = Iterators.transform(inputIterator, uniqueTransform); + Iterator> resultIterator = uniqueTransform.getIterator(inputIterator); return StreamSupport.stream(Spliterators.spliteratorUnknownSize(resultIterator, Spliterator.ORDERED), false).filter(Objects::nonNull) .map(Map.Entry::getValue).collect(Collectors.toList()); } - private List getUniqueDocumentsWithUpdateConfigCalls(List documents) { + protected List getUniqueDocumentsWithUpdateConfigCalls(List documents) { Transformer> docToEntry = document -> Maps.immutableEntry(document.getMetadata(), document); TransformIterator> inputIterator = new TransformIterator<>(documents.iterator(), docToEntry); UniqueTransform uniqueTransform = getUniqueTransform(); - Iterator> resultIterator = Iterators.transform(inputIterator, uniqueTransform); + Iterator> resultIterator = uniqueTransform.getIterator(inputIterator); ArrayList docs = new ArrayList<>(); while (resultIterator.hasNext()) { Map.Entry next = resultIterator.next(); @@ -509,7 +553,7 @@ private List getUniqueDocumentsWithUpdateConfigCalls(List do return docs; } - private void assertOrderedFieldValues() { + protected void assertOrderedFieldValues() { try { UniqueTransform uniqueTransform = getUniqueTransform(); for (Document d : inputDocuments) { @@ -521,53 +565,61 @@ private void assertOrderedFieldValues() { } } - private void givenValueTransformerForFields(UniqueGranularity transformer, String... fields) { + protected void givenValueTransformerForFields(UniqueGranularity transformer, String... fields) { Arrays.stream(fields).forEach((field) -> uniqueFields.put(field, transformer)); } - private void givenValueTransformersForField(String field, UniqueGranularity... transformers) { + protected void givenValueTransformersForField(String field, UniqueGranularity... transformers) { Arrays.stream(transformers).forEach((transformer) -> uniqueFields.put(field, transformer)); } - private UniqueTransform getUniqueTransform() { - return new UniqueTransform(uniqueFields, Long.MAX_VALUE); + protected UniqueTransform getUniqueTransform() { + try { + return new UniqueTransform.Builder().withUniqueFields(uniqueFields).withQueryExecutionForPageTimeout(Long.MAX_VALUE).build(); + } catch (IOException e) { + throw new RuntimeException(e); + } } - private void updateUniqueTransform(UniqueTransform uniqueTransform) { - uniqueTransform.updateConfig(uniqueFields, null); + protected void updateUniqueTransform(UniqueTransform uniqueTransform) { + uniqueTransform.updateConfig(uniqueFields); } - private InputDocumentBuilder givenInputDocument() { - return new InputDocumentBuilder(); + protected InputDocumentBuilder givenInputDocument() { + return new InputDocumentBuilder("", 0); } - private InputDocumentBuilder givenInputDocument(String docKey) { - return new InputDocumentBuilder(docKey); + protected InputDocumentBuilder givenInputDocument(String cq) { + return new InputDocumentBuilder(cq, 0); } - private ExpectedOrderedFieldValuesBuilder givenExpectedOrderedFieldValues() { - return new ExpectedOrderedFieldValuesBuilder(); + protected InputDocumentBuilder givenInputDocument(long ts) { + return new InputDocumentBuilder("", ts); } - private class InputDocumentBuilder { + protected InputDocumentBuilder givenInputDocument(String docKey, long ts) { + return new InputDocumentBuilder(docKey, ts); + } - private final Document document; + protected ExpectedOrderedFieldValuesBuilder givenExpectedOrderedFieldValues() { + return new ExpectedOrderedFieldValuesBuilder(); + } - InputDocumentBuilder() { - this.document = new Document(); - inputDocuments.add(document); - } + protected class InputDocumentBuilder { - @SuppressWarnings({"UnusedReturnValue", "SameParameterValue"}) - InputDocumentBuilder(String docKey) { + private final Document document; - Text MARKER_TEXT = new Text(docKey); - ByteSequence MARKER_SEQUENCE = new ArrayByteSequence(MARKER_TEXT.getBytes(), 0, MARKER_TEXT.getLength()); - byte EMPTY_BYTES[] = new byte[0]; - Key key = new Key(EMPTY_BYTES, EMPTY_BYTES, MARKER_SEQUENCE.subSequence(0, MARKER_SEQUENCE.length()).toArray()); + InputDocumentBuilder(String cq, long ts) { + Key key = new Key("shardid", "datatype\u0000" + getUid(), cq, ts); this.document = new Document(key, true); inputDocuments.add(document); this.document.getMetadata().set(key); + Attribute docKeyAttributes = new DocumentKey(key, true); + this.document.put(Document.DOCKEY_FIELD_NAME, docKeyAttributes); + } + + String getUid() { + return UUID.nameUUIDFromBytes(Longs.toByteArray(counter.incrementAndGet())).toString(); } @SuppressWarnings({"UnusedReturnValue", "SameParameterValue"}) @@ -590,7 +642,7 @@ private String getRandomKey(int index) { if (random.nextBoolean()) { sb.append(JexlASTHelper.IDENTIFIER_PREFIX); } - return sb.append("Attr").append(index).toString(); + return sb.append("ATTR").append(index).toString(); } private String getRandomValue() { @@ -609,7 +661,7 @@ InputDocumentBuilder isExpectedToBeUnique() { } } - private class ExpectedOrderedFieldValuesBuilder { + protected class ExpectedOrderedFieldValuesBuilder { private Multimap fieldValues = TreeMultimap.create(); @@ -624,13 +676,16 @@ public byte[] build() { try { ByteArrayOutputStream bytes = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(bytes); - int count = 0; for (String field : fieldValues.keySet()) { - String separator = "f-" + field + '/' + (count++) + ":"; - for (String value : fieldValues.get(field)) { + String separator = "f-" + field + ":"; + if (fieldValues.isEmpty()) { output.writeUTF(separator); - output.writeUTF(value); - separator = ","; + } else { + for (String value : fieldValues.get(field)) { + output.writeUTF(separator); + output.writeUTF(value); + separator = ","; + } } } output.flush(); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/WiseGuysIngest.java b/warehouse/query-core/src/test/java/datawave/query/util/WiseGuysIngest.java index 9ed431a66d9..45eb41c76a5 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/WiseGuysIngest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/WiseGuysIngest.java @@ -54,10 +54,13 @@ public enum WhatKindaRange { protected static final long timeStamp = 1356998400000L; public static final String corleoneUID = UID.builder().newId("Corleone".getBytes(), (Date) null).toString(); + public static final long corleoneTimeStampDelta = 0; public static final String corleoneChildUID = UID.builder().newId("Corleone".getBytes(), (Date) null, "1").toString(); - public static final String sopranoUID = UID.builder().newId("Soprano".toString().getBytes(), (Date) null).toString(); - public static final String caponeUID = UID.builder().newId("Capone".toString().getBytes(), (Date) null).toString(); - public static final String tattagliaUID = UID.builder().newId("Tattaglia".toString().getBytes(), (Date) null).toString(); + public static final String sopranoUID = UID.builder().newId("Soprano".getBytes(), (Date) null).toString(); + public static final long sopranoTimeStampDelta = 10; + public static final String caponeUID = UID.builder().newId("Capone".getBytes(), (Date) null).toString(); + public static final long caponeTimeStampDelta = 20; + public static final String tattagliaUID = UID.builder().newId("Tattaglia".getBytes(), (Date) null).toString(); protected static String normalizeColVal(Map.Entry colVal) { switch (colVal.getKey()) { @@ -96,81 +99,93 @@ public static void writeItAll(AccumuloClient client, WhatKindaRange range) throw bw = client.createBatchWriter(TableName.SHARD, bwConfig); mutation = new Mutation(shard); - mutation.put(datatype + "\u0000" + corleoneUID, "NOME.0" + "\u0000" + "SANTINO", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "NOME.1" + "\u0000" + "FREDO", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "NOME.2" + "\u0000" + "MICHAEL", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "NOME.3" + "\u0000" + "CONSTANZIA", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "NOME.4" + "\u0000" + "LUCA", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "NOME.5" + "\u0000" + "VINCENT", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.0" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.1" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.2" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.3" + "\u0000" + "FEMALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.4" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.5" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "ETA.0" + "\u0000" + "24", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "ETA.1" + "\u0000" + "22", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "ETA.2" + "\u0000" + "20", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "ETA.3" + "\u0000" + "18", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "ETA.4" + "\u0000" + "40", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "ETA.5" + "\u0000" + "22", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "MAGIC.0" + "\u0000" + "18", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "UUID.0" + "\u0000" + "CORLEONE", columnVisibility, timeStamp, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NOME.0" + "\u0000" + "SANTINO", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NOME.1" + "\u0000" + "FREDO", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NOME.2" + "\u0000" + "MICHAEL", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NOME.3" + "\u0000" + "CONSTANZIA", columnVisibility, timeStamp + corleoneTimeStampDelta, + emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NOME.4" + "\u0000" + "LUCA", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NOME.5" + "\u0000" + "VINCENT", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.0" + "\u0000" + "MALE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.1" + "\u0000" + "MALE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.2" + "\u0000" + "MALE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.3" + "\u0000" + "FEMALE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.4" + "\u0000" + "MALE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GENERE.5" + "\u0000" + "MALE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "ETA.0" + "\u0000" + "24", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "ETA.1" + "\u0000" + "22", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "ETA.2" + "\u0000" + "20", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "ETA.3" + "\u0000" + "18", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "ETA.4" + "\u0000" + "40", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "ETA.5" + "\u0000" + "22", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "MAGIC.0" + "\u0000" + "18", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "UUID.0" + "\u0000" + "CORLEONE", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); // CORLEONE date delta is 70 years - mutation.put(datatype + "\u0000" + corleoneUID, "BIRTH_DATE" + "\u0000" + "1930-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "DEATH_DATE" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "QUOTE" + "\u0000" + "Im gonna make him an offer he cant refuse", columnVisibility, timeStamp, + mutation.put(datatype + "\u0000" + corleoneUID, "BIRTH_DATE" + "\u0000" + "1930-12-28T00:00:05.000Z", columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "DEATH_DATE" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "QUOTE" + "\u0000" + "Im gonna make him an offer he cant refuse", columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "NUMBER" + "\u0000" + "25", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneUID, "GEO" + "\u0000" + "POINT(10 10)", columnVisibility, timeStamp + corleoneTimeStampDelta, + emptyValue); + + mutation.put(datatype + "\u0000" + corleoneChildUID, "UUID.0" + "\u0000" + "ANDOLINI", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "NUMBER" + "\u0000" + "25", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneUID, "GEO" + "\u0000" + "POINT(10 10)", columnVisibility, timeStamp, emptyValue); - - mutation.put(datatype + "\u0000" + corleoneChildUID, "UUID.0" + "\u0000" + "ANDOLINI", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneChildUID, "ETA.0" + "\u0000" + "12", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneChildUID, "BIRTH_DATE" + "\u0000" + "1930-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + corleoneChildUID, "DEATH_DATE" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); - - mutation.put(datatype + "\u0000" + sopranoUID, "NAME.0" + "\u0000" + "ANTHONY", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "NAME.1" + "\u0000" + "MEADOW", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "GENDER.0" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "GENDER.1" + "\u0000" + "FEMALE", columnVisibility, timeStamp, emptyValue); + mutation.put(datatype + "\u0000" + corleoneChildUID, "ETA.0" + "\u0000" + "12", columnVisibility, timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneChildUID, "BIRTH_DATE" + "\u0000" + "1930-12-28T00:00:05.000Z", columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + corleoneChildUID, "DEATH_DATE" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + + mutation.put(datatype + "\u0000" + sopranoUID, "NAME.0" + "\u0000" + "ANTHONY", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "NAME.1" + "\u0000" + "MEADOW", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "GENDER.0" + "\u0000" + "MALE", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "GENDER.1" + "\u0000" + "FEMALE", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); // to test whether singleton values correctly get matched using the function set methods, only add AGE.1 - // mutation.put(datatype + "\u0000" + sopranoUID, "AGE.0" + "\u0000" + "16", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "AGE.0" + "\u0000" + "16", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "AGE.1" + "\u0000" + "18", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "MAGIC.0" + "\u0000" + "18", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "UUID.0" + "\u0000" + "SOPRANO", columnVisibility, timeStamp, emptyValue); + // mutation.put(datatype + "\u0000" + sopranoUID, "AGE.0" + "\u0000" + "16", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "AGE.0" + "\u0000" + "16", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "AGE.1" + "\u0000" + "18", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "MAGIC.0" + "\u0000" + "18", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "UUID.0" + "\u0000" + "SOPRANO", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); // soprano date delta is 50 years - mutation.put(datatype + "\u0000" + sopranoUID, "BIRTH_DATE" + "\u0000" + "1950-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "DEATH_DATE" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "BIRTH_DATE" + "\u0000" + "1950-12-28T00:00:05.000Z", columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "DEATH_DATE" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); mutation.put(datatype + "\u0000" + sopranoUID, "QUOTE" + "\u0000" + "If you can quote the rules then you can obey them", columnVisibility, - timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + sopranoUID, "GEO" + "\u0000" + "POINT(20 20)", columnVisibility, timeStamp, emptyValue); - - mutation.put(datatype + "\u0000" + caponeUID, "NAME.0" + "\u0000" + "ALPHONSE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "NAME.1" + "\u0000" + "FRANK", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "NAME.2" + "\u0000" + "RALPH", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "NAME.3" + "\u0000" + "MICHAEL", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "GENDER.0" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "GENDER.1" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "GENDER.2" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "GENDER.3" + "\u0000" + "MALE", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "AGE.0" + "\u0000" + "30", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "AGE.1" + "\u0000" + "34", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "AGE.2" + "\u0000" + "20", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "AGE.3" + "\u0000" + "40", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "MAGIC.0" + "\u0000" + "18", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "UUID.0" + "\u0000" + "CAPONE", columnVisibility, timeStamp, emptyValue); + timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + sopranoUID, "GEO" + "\u0000" + "POINT(20 20)", columnVisibility, timeStamp + sopranoTimeStampDelta, emptyValue); + + mutation.put(datatype + "\u0000" + caponeUID, "NAME.0" + "\u0000" + "ALPHONSE", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "NAME.1" + "\u0000" + "FRANK", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "NAME.2" + "\u0000" + "RALPH", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "NAME.3" + "\u0000" + "MICHAEL", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "GENDER.0" + "\u0000" + "MALE", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "GENDER.1" + "\u0000" + "MALE", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "GENDER.2" + "\u0000" + "MALE", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "GENDER.3" + "\u0000" + "MALE", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "AGE.0" + "\u0000" + "30", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "AGE.1" + "\u0000" + "34", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "AGE.2" + "\u0000" + "20", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "AGE.3" + "\u0000" + "40", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "MAGIC.0" + "\u0000" + "18", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "UUID.0" + "\u0000" + "CAPONE", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); // capone date delta is 89 or 90 years - mutation.put(datatype + "\u0000" + caponeUID, "BIRTH_DATE.0" + "\u0000" + "1910-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "BIRTH_DATE.0" + "\u0000" + "1910-12-28T00:00:05.000Z", columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); // add a second date to test function taking an Iterable - mutation.put(datatype + "\u0000" + caponeUID, "BIRTH_DATE.1" + "\u0000" + "1911-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "DEATH_DATE.0" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, timeStamp, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "BIRTH_DATE.1" + "\u0000" + "1911-12-28T00:00:05.000Z", columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "DEATH_DATE.0" + "\u0000" + "2000-12-28T00:00:05.000Z", columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); mutation.put(datatype + "\u0000" + caponeUID, "QUOTE" + "\u0000" + "You can get much farther with a kind word and a gun than you can with a kind word alone", columnVisibility, - timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "NUMBER" + "\u0000" + "25", columnVisibility, timeStamp, emptyValue); - mutation.put(datatype + "\u0000" + caponeUID, "GEO" + "\u0000" + "POINT(30 30)", columnVisibility, timeStamp, emptyValue); + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "NUMBER" + "\u0000" + "25", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "GEO" + "\u0000" + "POINT(30 30)", columnVisibility, timeStamp + caponeTimeStampDelta, emptyValue); // second datatype shard data mutation.put(secondDataType + "\u0000" + tattagliaUID, "NAME.0" + "\u0000" + "Philip", columnVisibility, timeStamp, emptyValue); @@ -410,9 +425,10 @@ public static void writeItAll(AccumuloClient client, WhatKindaRange range) throw bw.addMutation(mutation); // add some tokens - addTokens(bw, range, "QUOTE", "Im gonna make him an offer he cant refuse", corleoneUID); - addTokens(bw, range, "QUOTE", "If you can quote the rules then you can obey them", sopranoUID); - addTokens(bw, range, "QUOTE", "You can get much farther with a kind word and a gun than you can with a kind word alone", caponeUID); + addTokens(bw, range, "QUOTE", "Im gonna make him an offer he cant refuse", corleoneUID, corleoneTimeStampDelta); + addTokens(bw, range, "QUOTE", "If you can quote the rules then you can obey them", sopranoUID, sopranoTimeStampDelta); + addTokens(bw, range, "QUOTE", "You can get much farther with a kind word and a gun than you can with a kind word alone", caponeUID, + caponeTimeStampDelta); } finally { if (null != bw) { bw.close(); @@ -637,103 +653,112 @@ public static void writeItAll(AccumuloClient client, WhatKindaRange range) throw // corleones // uuid mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("CORLEONE") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); // uuid mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("ANDOLINI") + "\u0000" + datatype + "\u0000" + corleoneChildUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); // names mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("SANTINO") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); - mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("FREDO") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("FREDO") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("MICHAEL") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("CONSTANZIA") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); - mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("LUCA") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("LUCA") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("VINCENT") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); // genders mutation.put("fi\u0000" + "GENERE", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); mutation.put("fi\u0000" + "GENERE", lcNoDiacriticsType.normalize("FEMALE") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + corleoneTimeStampDelta, emptyValue); // ages - mutation.put("fi\u0000" + "ETA", numberType.normalize("24") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "ETA", numberType.normalize("22") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "ETA", numberType.normalize("20") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "ETA", numberType.normalize("18") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "ETA", numberType.normalize("40") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "ETA", numberType.normalize("12") + "\u0000" + datatype + "\u0000" + corleoneChildUID, columnVisibility, timeStamp, - emptyValue); + mutation.put("fi\u0000" + "ETA", numberType.normalize("24") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "ETA", numberType.normalize("22") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "ETA", numberType.normalize("20") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "ETA", numberType.normalize("18") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "ETA", numberType.normalize("40") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "ETA", numberType.normalize("12") + "\u0000" + datatype + "\u0000" + corleoneChildUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); // geo for (String normalized : ((OneToManyNormalizerType) geoType).normalizeToMany("POINT(10 10)")) { - mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); + mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); } // sopranos // uuid mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("SOPRANO") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + sopranoTimeStampDelta, emptyValue); // names mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("ANTHONY") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp, emptyValue); - mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("MEADOW") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, - emptyValue); + timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("MEADOW") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); // genders - mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, - emptyValue); + mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("FEMALE") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp, emptyValue); + timeStamp + sopranoTimeStampDelta, emptyValue); // ages - mutation.put("fi\u0000" + "AGE", numberType.normalize("16") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "AGE", numberType.normalize("18") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, emptyValue); + mutation.put("fi\u0000" + "AGE", numberType.normalize("16") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "AGE", numberType.normalize("18") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); // geo for (String normalized : ((OneToManyNormalizerType) geoType).normalizeToMany("POINT(20 20)")) { - mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); + mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp + sopranoTimeStampDelta, + emptyValue); } // capones // uuid - mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("CAPONE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); + mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("CAPONE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); // names mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("ALPHONSE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, - timeStamp, emptyValue); - mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("FRANK") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("RALPH") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("MICHAEL") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("FRANK") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("RALPH") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("MICHAEL") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); // genders - mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); - mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, - emptyValue); + mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); // ages - mutation.put("fi\u0000" + "AGE", numberType.normalize("30") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "AGE", numberType.normalize("34") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "AGE", numberType.normalize("20") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "AGE", numberType.normalize("40") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, emptyValue); + mutation.put("fi\u0000" + "AGE", numberType.normalize("30") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "AGE", numberType.normalize("34") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "AGE", numberType.normalize("20") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "AGE", numberType.normalize("40") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, + timeStamp + caponeTimeStampDelta, emptyValue); // geo for (String normalized : ((OneToManyNormalizerType) geoType).normalizeToMany("POINT(30 30)")) { - mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); + mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp + caponeTimeStampDelta, + emptyValue); } // second datatype field index data @@ -751,16 +776,22 @@ public static void writeItAll(AccumuloClient client, WhatKindaRange range) throw emptyValue); // add some index-only fields - mutation.put("fi\u0000" + "LOCATION", "chicago" + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "POSIZIONE", "newyork" + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "LOCATION", "newjersey" + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, emptyValue); - mutation.put("fi\u0000" + "SENTENCE", "11y" + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, emptyValue); + mutation.put("fi\u0000" + "LOCATION", "chicago" + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp + caponeTimeStampDelta, + emptyValue); + mutation.put("fi\u0000" + "POSIZIONE", "newyork" + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, + timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "LOCATION", "newjersey" + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, + timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "SENTENCE", "11y" + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp + caponeTimeStampDelta, + emptyValue); bw.addMutation(mutation); - addFiTfTokens(bw, range, "QUOTE", "Im gonna make him an offer he cant refuse", corleoneUID); - addFiTfTokens(bw, range, "QUOTE", "If you can quote the rules then you can obey them", sopranoUID); - addFiTfTokens(bw, range, "QUOTE", "You can get much farther with a kind word and a gun than you can with a kind word alone", caponeUID); + addFiTfTokens(bw, range, "QUOTE", "Im gonna make him an offer he cant refuse", corleoneUID, corleoneTimeStampDelta); + addFiTfTokens(bw, range, "QUOTE", "If you can quote the rules then you can obey them", sopranoUID, sopranoTimeStampDelta); + addFiTfTokens(bw, range, "QUOTE", "You can get much farther with a kind word and a gun than you can with a kind word alone", caponeUID, + caponeTimeStampDelta); + } finally { if (null != bw) { bw.close(); @@ -1060,31 +1091,33 @@ private static Value getValueForNuthinAndYourHitsForFree() { return new Value(builder.build().toByteArray()); } - private static void addTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid) throws MutationsRejectedException { + private static void addTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid, long timeStampDelta) + throws MutationsRejectedException { Mutation mutation = new Mutation(lcNoDiacriticsType.normalize(phrase)); - mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp, + mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp + timeStampDelta, range == WhatKindaRange.SHARD ? getValueForNuthinAndYourHitsForFree() : getValueForBuilderFor(uid)); bw.addMutation(mutation); String[] tokens = phrase.split(" "); for (String token : tokens) { mutation = new Mutation(lcNoDiacriticsType.normalize(token)); - mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp, + mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp + timeStampDelta, range == WhatKindaRange.SHARD ? getValueForNuthinAndYourHitsForFree() : getValueForBuilderFor(uid)); bw.addMutation(mutation); } } - private static void addFiTfTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid) throws MutationsRejectedException { + private static void addFiTfTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid, long timeStampDelta) + throws MutationsRejectedException { Mutation fi = new Mutation(shard); - fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(phrase) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, timeStamp, - emptyValue); + fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(phrase) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, + timeStamp + timeStampDelta, emptyValue); OffsetQueue tokenOffsetCache = new BoundedOffsetQueue<>(500); int i = 0; String[] tokens = phrase.split(" "); for (String token : tokens) { - fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(token) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, timeStamp, - emptyValue); + fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(token) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, + timeStamp + timeStampDelta, emptyValue); tokenOffsetCache.addOffset(new TermAndZone(token, field.toUpperCase()), i); i++; @@ -1097,7 +1130,7 @@ private static void addFiTfTokens(BatchWriter bw, WhatKindaRange range, String f } Value value = new Value(builder.build().toByteArray()); fi.put("tf", datatype + "\u0000" + uid + "\u0000" + lcNoDiacriticsType.normalize(nfv.getIndexedFieldValue()) + "\u0000" + nfv.getIndexedFieldName(), - columnVisibility, timeStamp, value); + columnVisibility, timeStamp + timeStampDelta, value); } bw.addMutation(fi); } diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java new file mode 100644 index 00000000000..07b99fdef6e --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java @@ -0,0 +1,71 @@ +package datawave.query.util.sortedmap; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; + +import datawave.data.type.LcNoDiacriticsType; +import datawave.query.attributes.Document; +import datawave.query.composite.CompositeMetadata; +import datawave.query.predicate.EventDataQueryFieldFilter; +import datawave.query.util.TypeMetadata; +import datawave.query.util.sortedmap.rfile.KeyValueByteDocumenTransformsTest; +import datawave.query.util.sortedset.ByteArrayComparator; + +public class BufferedFileBackedByteDocumentSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { + + private Comparator keyComparator = new ByteArrayComparator(); + + private FileSortedMap.RewriteStrategy keyValueComparator = new FileSortedMap.RewriteStrategy<>() { + @Override + public boolean rewrite(byte[] key, Document original, Document update) { + long ts1 = original.get(Document.DOCKEY_FIELD_NAME).getTimestamp(); + long ts2 = update.get(Document.DOCKEY_FIELD_NAME).getTimestamp(); + return (ts2 > ts1); + } + }; + + @Override + protected void testEquality(Map.Entry expected, Map.Entry value) { + testEquality(expected.getKey(), value.getKey()); + KeyValueByteDocumenTransformsTest.assertDocumentEquals(expected.getValue(), value.getValue()); + } + + @Override + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return keyValueComparator; + } + + @Override + public byte[] createKey(byte[] values) { + return values; + } + + @Override + public Document createValue(byte[] values) { + Key docKey = new Key("20200101_1", "datatype\u0000uid", "", values[0]); + Key attrKey = new Key("20200101_1", "datatype\u0000uid", "FIELD\u0000VALUE", values[0]); + List> attrs = new ArrayList<>(); + attrs.add(new UnmodifiableMapEntry(attrKey, new Value())); + Document doc = new Document(docKey, Collections.singleton(docKey), false, attrs.iterator(), + new TypeMetadata().put("FIELD", "datatype", LcNoDiacriticsType.class.getName()), new CompositeMetadata(), true, true, + new EventDataQueryFieldFilter()); + return doc; + } + + @Override + public Comparator getComparator() { + return keyComparator; + } + + @Override + public FileSortedMap.FileSortedMapFactory getFactory() { + return new FileByteDocumentSortedMap.Factory(); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyDocumentSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyDocumentSortedMapTest.java new file mode 100644 index 00000000000..8e3086d6d0b --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyDocumentSortedMapTest.java @@ -0,0 +1,75 @@ +package datawave.query.util.sortedmap; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; + +import datawave.data.type.LcNoDiacriticsType; +import datawave.query.attributes.Document; +import datawave.query.composite.CompositeMetadata; +import datawave.query.predicate.EventDataQueryFieldFilter; +import datawave.query.util.TypeMetadata; +import datawave.query.util.sortedmap.rfile.KeyValueByteDocumenTransformsTest; + +public class BufferedFileBackedKeyDocumentSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { + + private Comparator keyComparator = new Comparator<>() { + @Override + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); + } + }; + + private FileSortedMap.RewriteStrategy keyValueComparator = new FileSortedMap.RewriteStrategy<>() { + @Override + public boolean rewrite(Key key, Document original, Document update) { + long ts1 = original.get(Document.DOCKEY_FIELD_NAME).getTimestamp(); + long ts2 = update.get(Document.DOCKEY_FIELD_NAME).getTimestamp(); + return (ts2 > ts1); + } + }; + + @Override + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return keyValueComparator; + } + + @Override + public Key createKey(byte[] values) { + return new Key(values); + } + + @Override + public Document createValue(byte[] values) { + Key docKey = new Key("20200101_1", "datatype\u0000uid", "", values[0]); + Key attrKey = new Key("20200101_1", "datatype\u0000uid", "FIELD\u0000VALUE", values[0]); + List> attrs = new ArrayList<>(); + attrs.add(new UnmodifiableMapEntry(attrKey, new Value())); + Document doc = new Document(docKey, Collections.singleton(docKey), false, attrs.iterator(), + new TypeMetadata().put("FIELD", "datatype", LcNoDiacriticsType.class.getName()), new CompositeMetadata(), true, true, + new EventDataQueryFieldFilter()); + return doc; + } + + @Override + protected void testEquality(Map.Entry expected, Map.Entry value) { + testEquality(expected.getKey(), value.getKey()); + KeyValueByteDocumenTransformsTest.assertDocumentEquals(expected.getValue(), value.getValue()); + } + + @Override + public Comparator getComparator() { + return keyComparator; + } + + @Override + public FileSortedMap.FileSortedMapFactory getFactory() { + return new FileKeyDocumentSortedMap.Factory(); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java new file mode 100644 index 00000000000..f72407a337f --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java @@ -0,0 +1,58 @@ +package datawave.query.util.sortedmap; + +import static org.junit.Assert.assertEquals; + +import java.util.Comparator; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +public class BufferedFileBackedKeyValueSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { + + private Comparator keyComparator = new Comparator<>() { + @Override + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); + } + }; + + private FileSortedMap.RewriteStrategy keyValueComparator = new FileSortedMap.RewriteStrategy<>() { + @Override + public boolean rewrite(Key key, Value original, Value update) { + return original.compareTo(update) < 0; + } + }; + + @Override + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return keyValueComparator; + } + + @Override + public Key createKey(byte[] values) { + return new Key(values); + } + + @Override + public Value createValue(byte[] values) { + return new Value(values); + } + + @Override + public void testEquality(Map.Entry expected, Map.Entry value) { + assertEquals(expected.getKey(), value.getKey()); + assertEquals(expected.getValue(), value.getValue()); + } + + @Override + public Comparator getComparator() { + return keyComparator; + } + + @Override + public FileSortedMap.FileSortedMapFactory getFactory() { + return new FileKeyValueSortedMap.Factory(); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java new file mode 100644 index 00000000000..42b8c6b443c --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java @@ -0,0 +1,98 @@ +package datawave.query.util.sortedmap; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; + +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.Test; + +public abstract class BufferedFileBackedRewritableSortedMapTest extends BufferedFileBackedSortedMapTest { + + /** + * Create a key given the specified value. This key should sort in the same way the underlying byte array will sort against other byte array. + * + * @param values + * @return The key + */ + public abstract K createKey(byte[] values); + + /** + * Create a value given the specified value. + * + * @param values + * @return The value + */ + public abstract V createValue(byte[] values); + + /** + * Get a rewrite strategy. This strategy should allow rewrites if the value is smaller. + * + * @return the rewrite strategy appropriate for key and value types + */ + @Override + public abstract FileSortedMap.RewriteStrategy getRewriteStrategy(); + + @Override + public Map.Entry createData(byte[] values) { + byte[] vbuffer = new byte[values.length]; + Arrays.fill(vbuffer, (byte) (values[0] + 1)); + return new UnmodifiableMapEntry(createKey(values), createValue(vbuffer)); + } + + @Test + public void testRewrite() throws Exception { + // create a new set of data, only half of which has greater Values + Map.Entry[] data2 = new Map.Entry[template.length * 2]; + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + byte[] vbuffer = new byte[buffer.length]; + Arrays.fill(vbuffer, (byte) (template[i] + 1)); + data2[i] = new UnmodifiableMapEntry(createKey(buffer), createValue(vbuffer)); + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + byte[] vbuffer = new byte[buffer.length]; + Arrays.fill(vbuffer, (byte) (template[i] - 1)); + Map.Entry datum = new UnmodifiableMapEntry(createKey(buffer), createValue(vbuffer)); + data2[i + template.length] = datum; + } + + // create a set with the supplied rewrite strategy + map = new BufferedFileBackedSortedMap.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) + .withMaxOpenFiles(7).withNumRetries(2) + .withHandlerFactories(Collections.singletonList(new BufferedFileBackedSortedMap.SortedMapFileHandlerFactory() { + @Override + public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { + SortedMapTempFileHandler fileHandler = new SortedMapTempFileHandler(); + tempFileHandlers.add(fileHandler); + return fileHandler; + } + + @Override + public boolean isValid() { + return true; + } + })).withMapFactory(getFactory()).build(); + + // adding in the data set multiple times to create underlying files with duplicate values making the + // MergeSortIterator's job a little tougher... + for (int d = 0; d < 11; d++) { + addDataRandomly(map, data); + addDataRandomly(map, data2); + } + + // now test the contents making sure we still have a sorted set with the expected values + int index = 0; + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + int dataIndex = sortedOrder[index++]; + Map.Entry expected = (dataIndex < template.length ? data2[dataIndex] : data[dataIndex]); + testEquality(expected, value); + } + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java new file mode 100644 index 00000000000..40e78a805c6 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java @@ -0,0 +1,395 @@ +package datawave.query.util.sortedmap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public abstract class BufferedFileBackedSortedMapTest { + + protected final List tempFileHandlers = new ArrayList<>(); + protected Map.Entry[] data = null; + protected int[] sortedOrder = null; + protected datawave.query.util.sortedmap.BufferedFileBackedSortedMap map = null; + protected final byte[] template = new byte[] {5, 2, 29, 4, 8, 3, 25, 23, 6, 21, 7, 16}; + protected final int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; + + public abstract Map.Entry createData(byte[] values); + + public abstract Comparator getComparator(); + + public abstract datawave.query.util.sortedmap.FileSortedMap.FileSortedMapFactory getFactory(); + + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return null; + } + + protected void testEquality(SortedMap m1, SortedMap m2) { + for (Map.Entry e1 : m1.entrySet()) { + V v2 = m2.get(e1.getKey()); + testEquality(e1, new UnmodifiableMapEntry(e1.getKey(), v2)); + } + } + + protected void testEquality(Map.Entry expected, Map.Entry value) { + testEquality(expected.getKey(), value.getKey()); + assertEquals(expected.getValue(), value.getValue()); + } + + protected void testEquality(K expected, K value) { + if (map.comparator() != null) { + assertEquals(0, map.comparator().compare(expected, value)); + } else { + assertEquals(expected, value); + } + } + + @Before + public void mapUp() throws Exception { + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + Map.Entry datum = createData(buffer); + if (i == 0) { + data = (Map.Entry[]) Array.newInstance(datum.getClass(), template.length * 2); + } + data[i] = datum; + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + Map.Entry datum = createData(buffer); + data[i + template.length] = datum; + } + sortedOrder = new int[data.length]; + for (int i = 0; i < template.length; i++) { + sortedOrder[i * 2] = sortedTemplate[i] + sortedTemplate.length; + sortedOrder[i * 2 + 1] = sortedTemplate[i]; + } + map = new datawave.query.util.sortedmap.BufferedFileBackedSortedMap.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()) + .withBufferPersistThreshold(5).withMaxOpenFiles(7).withNumRetries(2).withHandlerFactories( + Collections.singletonList(new datawave.query.util.sortedmap.BufferedFileBackedSortedMap.SortedMapFileHandlerFactory() { + @Override + public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { + datawave.query.util.sortedmap.SortedMapTempFileHandler fileHandler = new datawave.query.util.sortedmap.SortedMapTempFileHandler(); + tempFileHandlers.add(fileHandler); + return fileHandler; + } + + @Override + public boolean isValid() { + return true; + } + })) + .withMapFactory(getFactory()).build(); + + // adding in the data map multiple times to create underlying files with duplicate values making the + // MergeSortIterator's job a little tougher... + for (int d = 0; d < 11; d++) { + addDataRandomly(map, data); + } + while (map.getMaps().size() <= 7) { + addDataRandomly(map, data); + } + } + + public void addDataRandomly(BufferedFileBackedSortedMap map, Map.Entry[] data) { + Set added = new HashSet<>(); + Random random = new Random(); + // add data.length items randomly + for (int i = 0; i < data.length; i++) { + int index = random.nextInt(data.length); + map.put(data[index].getKey(), data[index].getValue()); + added.add(index); + } + // ensure all missing items are added + for (int i = 0; i < data.length; i++) { + if (!added.contains(i)) { + map.put(data[i].getKey(), data[i].getValue()); + } + } + } + + @After + public void tearDown() throws Exception { + // Delete each sorted map file and its checksum. + for (SortedMapTempFileHandler fileHandler : tempFileHandlers) { + File file = fileHandler.getFile(); + tryDelete(file); + File checksum = new File(file.getParent(), "." + file.getName() + ".crc"); + tryDelete(checksum); + } + tempFileHandlers.clear(); + + data = null; + sortedOrder = null; + map.clear(); + map = null; + } + + private void tryDelete(File file) { + if (file.exists()) { + Assert.assertTrue("Failed to delete file " + file, file.delete()); + } + } + + @Test + public void testSize() { + int expectedSize = data.length; + assertEquals(expectedSize, map.size()); + for (int i = (data.length / 2); i < data.length; i++) { + map.remove(data[i].getKey()); + expectedSize--; + assertEquals(expectedSize, map.size()); + } + for (int i = 0; i < (data.length / 2); i++) { + map.remove(data[i].getKey()); + expectedSize--; + assertEquals(expectedSize, map.size()); + } + assertEquals(0, map.size()); + for (int i = 0; i < data.length; i++) { + map.put(data[i].getKey(), data[i].getValue()); + expectedSize++; + assertEquals(expectedSize, map.size()); + } + } + + @Test + public void testIsEmpty() { + assertFalse(map.isEmpty()); + for (int i = (data.length / 2); i < data.length; i++) { + map.remove(data[i].getKey()); + assertFalse(map.isEmpty()); + } + for (int i = 1; i < (data.length / 2); i++) { + map.remove(data[i].getKey()); + assertFalse(map.isEmpty()); + } + map.remove(data[0].getKey()); + assertTrue(map.isEmpty()); + for (int i = 0; i < data.length; i++) { + map.put(data[i].getKey(), data[i].getValue()); + assertFalse(map.isEmpty()); + } + } + + @Test + public void testClear() { + map.clear(); + assertTrue(map.isEmpty()); + } + + @Test + public void testContainsObject() { + for (int i = (data.length / 2); i < data.length; i++) { + map.remove(data[i].getKey()); + } + for (int i = 1; i < (data.length / 2); i++) { + assertTrue(map.containsKey(data[i].getKey())); + } + for (int i = (data.length / 2); i < data.length; i++) { + assertFalse(map.containsKey(data[i].getKey())); + } + } + + @Test + public void testRemove() { + int expectedSize = data.length; + + assertFalse(map.isPersisted()); + for (int i = 0; i < data.length; i++) { + map.remove(data[i].getKey()); + assertEquals(--expectedSize, map.size()); + } + assertTrue(map.isEmpty()); + } + + @Test + public void testRemovePersisted() throws IOException { + int expectedSize = data.length; + + assertFalse(map.isPersisted()); + map.persist(); + assertTrue(map.isPersisted()); + for (int i = 0; i < data.length; i++) { + map.remove(data[i].getKey()); + assertEquals(--expectedSize, map.size()); + assertTrue(map.isPersisted()); + } + assertTrue(map.isEmpty()); + } + + @Test + public void testIterator() { + int index = 0; + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + Map.Entry expected = data[sortedOrder[index++]]; + testEquality(expected, value); + } + map.clear(); + for (Map.Entry value : map.entrySet()) { + fail(); + } + } + + @Test + public void testIteratorRemovePersisted() throws IOException { + int size = map.size(); + int failCount = 0; + + assertFalse(map.isPersisted()); + map.persist(); + assertTrue(map.isPersisted()); + + // calling iterator() will force persistence + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + assertTrue(map.containsKey(value.getKey())); + try { + it.remove(); + fail("Expected iterator remove to fail with a persisted map"); + } catch (Exception e) { + // expected that some of the underlying FileSortedMaps are persisted and hence the remove will fail + failCount++; + assertTrue(map.containsKey(value.getKey())); + assertEquals(size, map.size()); + } + } + assertEquals(size, failCount); + assertFalse(map.isEmpty()); + } + + @Test + public void testIteratorRemove() { + int size = map.size(); + int failCount = 0; + + assertFalse(map.isPersisted()); + + // calling iterator() will force persistence + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + assertTrue(map.containsKey(value.getKey())); + try { + it.remove(); + fail("Expected iterator remove to fail with a persisted map"); + } catch (Exception e) { + // expected that some of the underlying FileSortedMaps are persisted and hence the remove will fail + failCount++; + assertTrue(map.containsKey(value.getKey())); + assertEquals(size, map.size()); + } + } + assertEquals(size, failCount); + assertFalse(map.isEmpty()); + } + + @Test + public void testComparator() { + final Comparator comparator = map.comparator(); + Map.Entry[] testData = Arrays.copyOf(data, data.length); + Arrays.sort(testData, new Comparator>() { + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return comparator.compare(o1.getKey(), o2.getKey()); + } + }); + int index = 0; + for (Map.Entry value : map.entrySet()) { + Map.Entry expected = data[sortedOrder[index++]]; + testEquality(expected, value); + } + } + + @Test + public void testSubmap() { + int start = sortedOrder.length / 3; + int end = start * 2; + try { + SortedMap submap = map.subMap(data[sortedOrder[start]].getKey(), data[sortedOrder[end]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); + for (int i = start; i < end; i++) { + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); + } + testEquality(expected, submap); + } catch (Exception e) { + // expected + } + } + + @Test + public void testHeadmap() { + int end = sortedOrder.length / 3; + try { + SortedMap submap = map.headMap(data[sortedOrder[end]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); + for (int i = 0; i < end; i++) { + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); + } + testEquality(expected, submap); + } catch (Exception e) { + // expected + } + } + + @Test + public void testTailmap() { + int start = sortedOrder.length / 3; + try { + SortedMap submap = map.tailMap(data[sortedOrder[start]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); + for (int i = start; i < sortedOrder.length; i++) { + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); + } + testEquality(expected, submap); + } catch (Exception e) { + // expected + } + } + + @Test + public void testLastKey() { + Map.Entry expected = data[sortedOrder[data.length - 1]]; + K value = map.lastKey(); + testEquality(expected.getKey(), value); + } + + @Test + public void testFirstKey() { + Map.Entry expected = data[sortedOrder[0]]; + K value = map.firstKey(); + testEquality(expected.getKey(), value); + } + + @Test + public void testCompaction() throws IOException { + assertEquals(8, map.getMaps().size()); + map.persist(); + assertEquals(3, map.getMaps().size()); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java new file mode 100644 index 00000000000..938e61bf21e --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java @@ -0,0 +1,407 @@ +package datawave.query.util.sortedmap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map; +import java.util.Random; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class FileSortedMapTest { + SortedMap data = null; + SortedMap extraData = null; + FileSortedMap map = null; + datawave.query.util.sortedmap.SortedMapTempFileHandler handler = null; + + @Before + public void setUp() throws Exception { + Comparator c = new Comparator() { + @Override + public int compare(Integer o1, Integer o2) { + if (o1 == null) { + return (o2 == null ? 0 : -1); + } else { + return (o2 == null ? 1 : o1.compareTo(o2)); + } + } + }; + handler = new SortedMapTempFileHandler(); + map = new FileSerializableSortedMap<>(c, new FileSerializableSortedMap.SerializableFileHandler(handler), false); + data = new TreeMap<>(c); + Random r = new Random(123948710248L); + // data.add(null); + for (int i = 0; i < 20; i++) { + data.put(r.nextInt(), r.nextInt() + 1); + } + extraData = new TreeMap<>(c); + for (int i = 0; i < 20; i++) { + extraData.put(r.nextInt(), r.nextInt() + 1); + } + // make sure we have no overlap + data.keySet().removeAll(extraData.keySet()); + map.putAll(data); + } + + @After + public void tearDown() { + handler.getFile().delete(); + } + + private void assertSortedMapEquals(SortedMap map1, SortedMap map2) { + assertEquals(map1.size(), map2.size()); + assertTrue(map1.keySet().containsAll(map2.keySet())); + assertTrue(map1.keySet().containsAll(map2.keySet())); + map1.entrySet().stream().forEach(e -> assertEquals(e.getValue(), map2.get(e.getKey()))); + } + + @Test + public void testReadWrite() throws Exception { + assertFalse(map.isPersisted()); + assertSortedMapEquals(data, map); + map.persist(); + assertTrue(map.isPersisted()); + assertTrue(handler.getFile().exists()); + assertSortedMapEquals(data, map); + map.load(); + assertFalse(map.isPersisted()); + assertSortedMapEquals(data, map); + } + + @Test + public void testIsEmpty() throws Exception { + assertFalse(map.isEmpty()); + map.persist(); + assertFalse(map.isEmpty()); + map.clear(); + assertTrue(map.isEmpty()); + map.load(); + assertTrue(map.isEmpty()); + } + + @Test + public void testContains() throws Exception { + SortedMap someData = new TreeMap<>(data); + someData.put(extraData.firstKey(), extraData.get(extraData.firstKey())); + for (Integer i : data.keySet()) { + assertTrue(map.containsKey(i)); + } + for (Integer i : extraData.keySet()) { + assertFalse(map.containsKey(i)); + } + map.persist(); + for (Integer i : data.keySet()) { + assertTrue(map.containsKey(i)); + } + for (Integer i : extraData.keySet()) { + assertFalse(map.containsKey(i)); + } + map.load(); + for (Integer i : data.keySet()) { + assertTrue(map.containsKey(i)); + } + for (Integer i : extraData.keySet()) { + assertFalse(map.containsKey(i)); + } + } + + private void printMap(SortedMap map1, SortedMap map2) { + Iterator> it1 = map1.entrySet().iterator(); + Iterator> it2 = map2.entrySet().iterator(); + while (it1.hasNext() && it2.hasNext()) { + Map.Entry o1 = it1.next(); + Map.Entry o2 = it2.next(); + System.out.println(o1 + " vs " + o2); + } + while (it1.hasNext()) { + Map.Entry o1 = it1.next(); + System.out.println(o1 + " vs (null)"); + } + while (it2.hasNext()) { + Map.Entry o2 = it2.next(); + System.out.println("(null) vs " + o2); + } + } + + @Test + public void testIterator() throws Exception { + SortedMap testData = new TreeMap<>(data); + Iterator> dataIterator = testData.entrySet().iterator(); + Iterator> mapIterator = map.entrySet().iterator(); + while (dataIterator.hasNext()) { + assertTrue(mapIterator.hasNext()); + assertEquals(dataIterator.next(), mapIterator.next()); + } + assertFalse(mapIterator.hasNext()); + for (Map.Entry i : map.entrySet()) { + assertEquals(testData.remove(i.getKey()), i.getValue()); + } + assertTrue(testData.isEmpty()); + map.persist(); + dataIterator = data.entrySet().iterator(); + mapIterator = map.entrySet().iterator(); + while (dataIterator.hasNext()) { + assertTrue(mapIterator.hasNext()); + assertEquals(dataIterator.next(), mapIterator.next()); + } + assertFalse(mapIterator.hasNext()); + testData.putAll(data); + for (Map.Entry i : map.entrySet()) { + assertEquals(testData.remove(i.getKey()), i.getValue()); + } + assertTrue(testData.isEmpty()); + map.load(); + dataIterator = data.entrySet().iterator(); + mapIterator = map.entrySet().iterator(); + while (dataIterator.hasNext()) { + assertTrue(mapIterator.hasNext()); + assertEquals(dataIterator.next(), mapIterator.next()); + } + assertFalse(mapIterator.hasNext()); + testData.putAll(data); + for (Map.Entry i : map.entrySet()) { + assertEquals(testData.remove(i.getKey()), i.getValue()); + } + assertTrue(testData.isEmpty()); + } + + @Test + public void testPut() throws Exception { + assertNull(map.put(extraData.firstKey(), extraData.get(extraData.firstKey()))); + assertEquals(data.size() + 1, map.size()); + assertTrue(map.containsKey(extraData.firstKey())); + assertEquals(map.remove(extraData.firstKey()), extraData.get(extraData.firstKey())); + assertEquals(data.size(), map.size()); + assertFalse(map.containsKey(extraData.firstKey())); + map.persist(); + try { + map.put(extraData.firstKey(), extraData.get(extraData.firstKey())); + fail("Expected persisted map.add to fail"); + } catch (Exception e) { + // expected + } + map.load(); + assertEquals(data.size(), map.size()); + assertFalse(map.containsKey(extraData.firstKey())); + } + + @Test + public void testPutAll() throws Exception { + map.putAll(extraData); + assertEquals(data.size() + extraData.size(), map.size()); + assertTrue(map.entrySet().containsAll(extraData.entrySet())); + assertTrue(map.keySet().removeAll(extraData.keySet())); + assertEquals(data.size(), map.size()); + assertFalse(map.containsKey(extraData.firstKey())); + map.persist(); + try { + map.putAll(extraData); + fail("Expected persisted map.addAll to fail"); + } catch (Exception e) { + // expected + } + map.load(); + assertEquals(data.size(), map.size()); + assertFalse(map.containsKey(extraData.firstKey())); + } + + @Test + public void testRetainAll() throws Exception { + SortedMap someData = new TreeMap<>(data); + someData.remove(data.firstKey()); + someData.remove(data.lastKey()); + someData.put(extraData.firstKey(), extraData.get(extraData.firstKey())); + someData.put(extraData.lastKey(), extraData.get(extraData.lastKey())); + assertFalse(map.keySet().retainAll(data.keySet())); + assertEquals(someData.size(), map.size()); + assertTrue(map.keySet().retainAll(someData.keySet())); + assertEquals(data.size() - 2, map.size()); + assertFalse(map.keySet().containsAll(data.keySet())); + assertFalse(map.keySet().containsAll(someData.keySet())); + assertFalse(map.containsKey(data.lastKey())); + assertTrue(map.keySet().retainAll(extraData.keySet())); + assertTrue(map.isEmpty()); + + map.putAll(data); + map.persist(); + try { + map.keySet().retainAll(someData.keySet()); + fail("Expected persisted map.retainAll to fail"); + } catch (Exception e) { + // expected + } + + map.load(); + assertEquals(data.size(), map.size()); + assertTrue(map.keySet().containsAll(data.keySet())); + } + + @Test + public void testRemoveAll() throws Exception { + SortedMap someData = new TreeMap<>(data); + someData.remove(data.firstKey()); + someData.remove(data.lastKey()); + someData.put(extraData.firstKey(), extraData.get(extraData.firstKey())); + someData.put(extraData.lastKey(), extraData.get(extraData.lastKey())); + assertFalse(map.keySet().removeAll(extraData.keySet())); + assertEquals(someData.size(), map.size()); + assertTrue(map.keySet().removeAll(someData.keySet())); + assertEquals(2, map.size()); + assertFalse(map.keySet().containsAll(data.keySet())); + assertFalse(map.keySet().containsAll(someData.keySet())); + assertTrue(map.keySet().contains(data.firstKey())); + assertTrue(map.keySet().contains(data.lastKey())); + assertTrue(map.keySet().removeAll(data.keySet())); + assertTrue(map.isEmpty()); + + map.putAll(data); + map.persist(); + try { + map.keySet().removeAll(someData.keySet()); + fail("Expected persisted map.retainAll to fail"); + } catch (Exception e) { + // expected + } + + map.load(); + assertEquals(data.size(), map.size()); + assertTrue(map.keySet().containsAll(data.keySet())); + } + + @Test + public void testClear() throws Exception { + map.clear(); + assertTrue(map.isEmpty()); + map.putAll(data); + map.persist(); + map.clear(); + assertTrue(map.isEmpty()); + map.load(); + assertTrue(map.isEmpty()); + } + + @Test + public void testNoComparator() throws Exception { + assertNotNull(map.comparator()); + map.persist(); + assertNotNull(map.comparator()); + map.load(); + assertNotNull(map.comparator()); + SortedMap tempData = new TreeMap<>(); + tempData.putAll(data); + + map = new FileSerializableSortedMap<>(tempData, new FileSerializableSortedMap.SerializableFileHandler(handler), false); + + assertNull(map.comparator()); + assertSortedMapEquals(tempData, map); + for (Integer i : map.keySet()) { + assertEquals(tempData.firstKey(), i); + tempData.remove(tempData.firstKey()); + } + tempData.putAll(data); + assertSortedMapEquals(tempData, map); + map.persist(); + assertNull(map.comparator()); + map.load(); + assertNull(map.comparator()); + + for (Integer i : map.keySet()) { + assertEquals(tempData.firstKey(), i); + tempData.remove(tempData.firstKey()); + } + } + + @Test + public void testSubmap() throws Exception { + Integer fromElement = null; + Integer toElement = null; + int index = 0; + for (Integer i : data.keySet()) { + if (index == (data.size() / 3)) { + fromElement = i; + } else if (index == data.size() * 2 / 3) { + toElement = i; + break; + } + index++; + } + SortedMap submap = map.subMap(fromElement, toElement); + assertSortedMapEquals(data.subMap(fromElement, toElement), submap); + map.persist(); + map.subMap(fromElement, toElement); + assertSortedMapEquals(data.subMap(fromElement, toElement), submap); + map.load(); + submap = map.subMap(fromElement, toElement); + assertSortedMapEquals(data.subMap(fromElement, toElement), submap); + } + + @Test + public void testHeadmap() throws Exception { + Integer toElement = null; + int index = 0; + for (Integer i : data.keySet()) { + if (index == data.size() * 2 / 3) { + toElement = i; + break; + } + index++; + } + SortedMap submap = map.headMap(toElement); + assertSortedMapEquals(data.headMap(toElement), submap); + map.persist(); + map.headMap(toElement); + assertSortedMapEquals(data.headMap(toElement), submap); + map.load(); + submap = map.headMap(toElement); + assertSortedMapEquals(data.headMap(toElement), submap); + } + + @Test + public void testTailmap() throws Exception { + Integer fromElement = null; + int index = 0; + for (Integer i : data.keySet()) { + if (index == (data.size() / 3)) { + fromElement = i; + break; + } + index++; + } + SortedMap submap = map.tailMap(fromElement); + assertSortedMapEquals(data.tailMap(fromElement), submap); + map.persist(); + map.tailMap(fromElement); + assertSortedMapEquals(data.tailMap(fromElement), submap); + map.load(); + submap = map.tailMap(fromElement); + assertSortedMapEquals(data.tailMap(fromElement), submap); + } + + @Test + public void testFirstKey() throws Exception { + assertEquals(data.firstKey(), map.firstKey()); + map.persist(); + assertEquals(data.firstKey(), map.firstKey()); + map.load(); + assertEquals(data.firstKey(), map.firstKey()); + } + + @Test + public void testLast() throws Exception { + assertEquals(data.lastKey(), map.lastKey()); + map.persist(); + assertEquals(data.lastKey(), map.lastKey()); + map.load(); + assertEquals(data.lastKey(), map.lastKey()); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java new file mode 100644 index 00000000000..9aa0fbcf5fc --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java @@ -0,0 +1,254 @@ +package datawave.query.util.sortedmap; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FsStatus; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import datawave.query.iterator.ivarator.IvaratorCacheDir; +import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.util.sortedset.FileSortedSet; +import datawave.query.util.sortedset.HdfsBackedSortedSet; + +public class HdfsBackedSortedMapTest { + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void persistReloadTest() throws Exception { + File tempDir = temporaryFolder.newFolder(); + + File smallDir = new File(tempDir, "small"); + Assert.assertTrue(smallDir.mkdirs()); + + File largeDir = new File(tempDir, "large"); + Assert.assertTrue(largeDir.mkdirs()); + + LocalFileSystem fs = new LocalFileSystem(); + fs.initialize(tempDir.toURI(), new Configuration()); + + FsStatus fsStatus = fs.getStatus(); + + // set the min remaining MB to something which will cause the 'small' directiory to be skipped + long minRemainingMB = (fsStatus.getRemaining() / 0x100000L) + 4096l; + + List ivaratorCacheDirs = new ArrayList<>(); + ivaratorCacheDirs + .add(new IvaratorCacheDir(new IvaratorCacheDirConfig(smallDir.toURI().toString(), 0, minRemainingMB), fs, smallDir.toURI().toString())); + ivaratorCacheDirs.add(new IvaratorCacheDir(new IvaratorCacheDirConfig(largeDir.toURI().toString()), fs, largeDir.toURI().toString())); + + String uniquePath = "blah"; + + // @formatter:off + @SuppressWarnings("unchecked") + datawave.query.util.sortedset.HdfsBackedSortedSet sortedSet = (datawave.query.util.sortedset.HdfsBackedSortedSet) datawave.query.util.sortedset.HdfsBackedSortedSet.builder() + .withIvaratorCacheDirs(ivaratorCacheDirs) + .withUniqueSubPath(uniquePath) + .withMaxOpenFiles(9999) + .withNumRetries(2) + .withPersistOptions(new datawave.query.util.sortedset.FileSortedSet.PersistOptions()) + .build(); + // @formatter:on + + // Add an entry to the sorted set + String someTestString = "some test string"; + sortedSet.add(someTestString); + + // persist the sorted set + sortedSet.persist(); + + Path smallPath = new Path(smallDir.toURI().toString()); + Path smallSubPath = new Path(smallPath, uniquePath); + Path largePath = new Path(largeDir.toURI().toString()); + Path largeSubPath = new Path(largePath, uniquePath); + + // ensure that data was written to the large folder, not the small folder + Assert.assertFalse(fs.exists(smallSubPath)); + Assert.assertEquals(0, fs.listStatus(smallPath).length); + Assert.assertTrue(fs.exists(largeSubPath)); + + FileStatus[] fileStatuses = fs.listStatus(largeSubPath); + Assert.assertEquals(1, fileStatuses.length); + Assert.assertTrue(fileStatuses[0].getPath().getName().startsWith("SortedSet")); + + // Now make sure reloading an ivarator cache dir works + // @formatter:off + @SuppressWarnings("unchecked") + datawave.query.util.sortedset.HdfsBackedSortedSet reloadedSortedSet = (datawave.query.util.sortedset.HdfsBackedSortedSet) datawave.query.util.sortedset.HdfsBackedSortedSet.builder() + .withIvaratorCacheDirs(ivaratorCacheDirs) + .withUniqueSubPath(uniquePath) + .withMaxOpenFiles(9999) + .withNumRetries(2) + .withPersistOptions(new datawave.query.util.sortedset.FileSortedSet.PersistOptions()) + .build(); + // @formatter:on + + Assert.assertEquals(1, reloadedSortedSet.size()); + Assert.assertEquals(someTestString, reloadedSortedSet.first()); + } + + @Test + public void persistCompactReloadTest() throws Exception { + File tempDir = temporaryFolder.newFolder(); + + File[] dirs = new File[] {new File(tempDir, "first"), new File(tempDir, "second"), new File(tempDir, "third")}; + + for (File dir : dirs) + Assert.assertTrue(dir.mkdirs()); + + String uniquePath = "blah"; + + Path[] paths = Arrays.stream(dirs).map(dir -> new Path(dir.toURI().toString())).toArray(Path[]::new); + Path[] subPaths = Arrays.stream(paths).map(path -> new Path(path, uniquePath)).toArray(Path[]::new); + + LocalFileSystem fs = new LocalFileSystem(); + fs.initialize(tempDir.toURI(), new Configuration()); + + // set the min remaining percent to something which will cause the second directory to be skipped + double minRemainingPercent = 1.0; + + List ivaratorCacheDirs = new ArrayList<>(); + for (File dir : dirs) { + if (dir.getName().equalsIgnoreCase("second")) + ivaratorCacheDirs.add( + new IvaratorCacheDir(new IvaratorCacheDirConfig(dir.toURI().toString(), 0, minRemainingPercent), fs, dir.toURI().toString())); + else + ivaratorCacheDirs.add(new IvaratorCacheDir(new IvaratorCacheDirConfig(dir.toURI().toString(), 1), fs, dir.toURI().toString())); + } + + // @formatter:off + @SuppressWarnings("unchecked") + datawave.query.util.sortedset.HdfsBackedSortedSet firstSortedSet = (datawave.query.util.sortedset.HdfsBackedSortedSet) datawave.query.util.sortedset.HdfsBackedSortedSet.builder() + .withIvaratorCacheDirs(Collections.singletonList(ivaratorCacheDirs.get(0))) + .withUniqueSubPath(uniquePath) + .withMaxOpenFiles(9999) + .withNumRetries(2) + .withPersistOptions(new datawave.query.util.sortedset.FileSortedSet.PersistOptions()) + .build(); + // @formatter:on + + // Add an entry to the first sorted set + String someTestString = "some test string"; + firstSortedSet.add(someTestString); + + // persist the sorted set + firstSortedSet.persist(); + + // @formatter:off + @SuppressWarnings("unchecked") + datawave.query.util.sortedset.HdfsBackedSortedSet thirdSortedSet = (datawave.query.util.sortedset.HdfsBackedSortedSet) datawave.query.util.sortedset.HdfsBackedSortedSet.builder() + .withIvaratorCacheDirs(Collections.singletonList(ivaratorCacheDirs.get(2))) + .withUniqueSubPath(uniquePath) + .withMaxOpenFiles(9999) + .withNumRetries(2) + .withPersistOptions(new datawave.query.util.sortedset.FileSortedSet.PersistOptions()) + .build(); + // @formatter:on + + // Add an entry to the third sorted set + String anotherTestString = "another test string"; + thirdSortedSet.add(anotherTestString); + + // persist the sorted set + thirdSortedSet.persist(); + + // ensure that data was written to the first and third folders + Assert.assertTrue(fs.exists(subPaths[0])); + Assert.assertTrue(fs.exists(subPaths[2])); + + // ensure that data was not written to the second folder + Assert.assertFalse(fs.exists(subPaths[1])); + Assert.assertEquals(0, fs.listStatus(paths[1]).length); + + // ensure that 1 file was written to the first folder + FileStatus[] fileStatuses = fs.listStatus(subPaths[0]); + Assert.assertEquals(1, fileStatuses.length); + Assert.assertTrue(fileStatuses[0].getPath().getName().startsWith("SortedSet")); + + // ensure that 1 file was written to the third folder + fileStatuses = fs.listStatus(subPaths[2]); + Assert.assertEquals(1, fileStatuses.length); + Assert.assertTrue(fileStatuses[0].getPath().getName().startsWith("SortedSet")); + + // Now make sure reloading an ivarator cache dir works, and set maxOpenFiles to 1 so that we compact during the next persist + // @formatter:off + @SuppressWarnings("unchecked") + datawave.query.util.sortedset.HdfsBackedSortedSet reloadedSortedSet = (datawave.query.util.sortedset.HdfsBackedSortedSet) datawave.query.util.sortedset.HdfsBackedSortedSet.builder() + .withIvaratorCacheDirs(ivaratorCacheDirs) + .withUniqueSubPath(uniquePath) + .withMaxOpenFiles(1) + .withNumRetries(2) + .withPersistOptions(new datawave.query.util.sortedset.FileSortedSet.PersistOptions()) + .build(); + // @formatter:on + + // Ensure that we have 2 entries total + Assert.assertEquals(2, reloadedSortedSet.size()); + + // This is what we expect to be loaded by the set + List results = new ArrayList<>(); + results.add(someTestString); + results.add(anotherTestString); + + // for each result we find, remove it from the results list and ensure that the list is empty when we're done + reloadedSortedSet.iterator().forEachRemaining(results::remove); + Assert.assertTrue(results.isEmpty()); + + // Finally, add an entry to the reloaded sorted set + String lastTestString = "last test string"; + reloadedSortedSet.add(lastTestString); + + // persist the sorted set (this should cause a compaction down to 1 file) + reloadedSortedSet.persist(); + + // ensure that data was not written to the second folder + Assert.assertFalse(fs.exists(subPaths[1])); + Assert.assertEquals(0, fs.listStatus(paths[1]).length); + + // ensure that while the folder still exists, data no longer exists for the third folder + Assert.assertTrue(fs.exists(subPaths[2])); + Assert.assertEquals(0, fs.listStatus(subPaths[2]).length); + + // ensure that all data exists in the first folder + fileStatuses = fs.listStatus(subPaths[0]); + Assert.assertEquals(1, fileStatuses.length); + Assert.assertTrue(fileStatuses[0].getPath().getName().startsWith("SortedSet")); + + // Finally, make sure that the compacted data can be reloaded + // @formatter:off + @SuppressWarnings("unchecked") + datawave.query.util.sortedset.HdfsBackedSortedSet compactedSortedSet = (datawave.query.util.sortedset.HdfsBackedSortedSet) HdfsBackedSortedSet.builder() + .withIvaratorCacheDirs(ivaratorCacheDirs) + .withUniqueSubPath(uniquePath) + .withMaxOpenFiles(9999) + .withNumRetries(2) + .withPersistOptions(new FileSortedSet.PersistOptions()) + .build(); + // @formatter:on + + // Ensure that we have 3 entries total + Assert.assertEquals(3, compactedSortedSet.size()); + + // This is what we expect to be loaded by the set + results.clear(); + results.add(someTestString); + results.add(anotherTestString); + results.add(lastTestString); + + // for each result we find, remove it from the results list and ensure that the list is empty when we're done + compactedSortedSet.iterator().forEachRemaining(results::remove); + Assert.assertTrue(results.isEmpty()); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MergeSortIteratorTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MergeSortIteratorTest.java new file mode 100644 index 00000000000..b0aee4bda10 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MergeSortIteratorTest.java @@ -0,0 +1,241 @@ +package datawave.query.util.sortedmap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.junit.Test; + +public class MergeSortIteratorTest { + + @Test + public void testIteration() { + SortedMap set1 = new TreeMap<>(); + SortedMap set2 = new TreeMap<>(); + SortedMap set3 = new TreeMap<>(); + + set1.put(1, 1); + set1.put(3, 1); + set1.put(4, 1); + set1.put(5, 1); + set1.put(6, 1); + set1.put(10, 1); + + set2.put(1, 1); + set2.put(2, 1); + set2.put(5, 1); + set2.put(20, 1); + + set3.put(2, 1); + set3.put(5, 1); + set3.put(6, 1); + set3.put(30, 1); + + List expected = new ArrayList<>(); + expected.add(1); + expected.add(2); + expected.add(3); + expected.add(4); + expected.add(5); + expected.add(6); + expected.add(10); + expected.add(20); + expected.add(30); + + List> col = new ArrayList<>(); + col.add(set1); + col.add(set2); + col.add(set3); + List results = new ArrayList<>(); + + Iterator it = new MultiMapBackedSortedMap(col).keySet().iterator(); + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + while (it.hasNext()) { + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + Integer next = it.next(); + results.add(next); + assertTrue(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); + it.remove(); + assertFalse(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + } + assertEquals(expected, results); + assertTrue(set1.isEmpty() && set2.isEmpty() && set3.isEmpty()); + } + + @Test + public void testIterationSansHasNext() { + SortedMap set1 = new TreeMap<>(); + SortedMap set2 = new TreeMap<>(); + SortedMap set3 = new TreeMap<>(); + + set1.put(1, 1); + set1.put(3, 1); + set1.put(4, 1); + set1.put(5, 1); + set1.put(6, 1); + set1.put(10, 1); + + set2.put(1, 1); + set2.put(2, 1); + set2.put(5, 1); + set2.put(20, 1); + + set3.put(2, 1); + set3.put(5, 1); + set3.put(6, 1); + set3.put(30, 1); + + List expected = new ArrayList<>(); + expected.add(1); + expected.add(2); + expected.add(3); + expected.add(4); + expected.add(5); + expected.add(6); + expected.add(10); + expected.add(20); + expected.add(30); + + List> col = new ArrayList<>(); + col.add(set1); + col.add(set2); + col.add(set3); + List results = new ArrayList<>(); + Iterator it = new MultiMapBackedSortedMap(col).keySet().iterator(); + while (true) { + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + Integer next; + try { + next = it.next(); + } catch (NoSuchElementException nsee) { + break; + } + results.add(next); + assertTrue(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); + it.remove(); + assertFalse(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + } + assertEquals(expected, results); + assertTrue(set1.isEmpty() && set2.isEmpty() && set3.isEmpty()); + } + + @Test + public void testIterationSansWithNulls() { + Comparator c = new Comparator() { + + @Override + public int compare(Integer o1, Integer o2) { + if (o1 == null) { + return (o2 == null ? 0 : -1); + } else { + return (o2 == null ? 1 : o1.compareTo(o2)); + } + } + }; + + SortedMap set1 = new TreeMap<>(c); + SortedMap set2 = new TreeMap<>(c); + SortedMap set3 = new TreeMap<>(c); + + set1.put(1, 1); + set1.put(3, 1); + set1.put(4, 1); + set1.put(5, 1); + set1.put(6, 1); + set1.put(10, 1); + + set2.put(null, 1); + set2.put(1, 1); + set2.put(2, 1); + set2.put(5, 1); + set2.put(20, 1); + + set3.put(null, 1); + set3.put(2, 1); + set3.put(5, 1); + set3.put(6, 1); + set3.put(30, 1); + + List expected = new ArrayList<>(); + expected.add(null); + expected.add(1); + expected.add(2); + expected.add(3); + expected.add(4); + expected.add(5); + expected.add(6); + expected.add(10); + expected.add(20); + expected.add(30); + + List> col = new ArrayList<>(); + col.add(set1); + col.add(set2); + col.add(set3); + List results = new ArrayList<>(); + Iterator it = new MultiMapBackedSortedMap(col).keySet().iterator(); + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + while (it.hasNext()) { + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + Integer next = it.next(); + results.add(next); + assertTrue(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); + it.remove(); + assertFalse(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); + try { + it.remove(); + fail("Expected remove to fail"); + } catch (Exception e) { + // expected + } + } + assertEquals(expected, results); + assertTrue(set1.isEmpty() && set2.isEmpty() && set3.isEmpty()); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedKeyValueSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedKeyValueSortedMapTest.java new file mode 100644 index 00000000000..9eb9c4bcaac --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedKeyValueSortedMapTest.java @@ -0,0 +1,53 @@ +package datawave.query.util.sortedmap; + +import static org.junit.Assert.assertEquals; + +import java.util.Comparator; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +public class MultiMapBackedKeyValueSortedMapTest extends MultiMapBackedRewritableSortedMapTest { + + private Comparator keyComparator = new Comparator<>() { + @Override + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); + } + }; + + private FileSortedMap.RewriteStrategy keyValueComparator = new FileSortedMap.RewriteStrategy<>() { + @Override + public boolean rewrite(Key key, Value original, Value update) { + return original.compareTo(update) < 0; + } + }; + + @Override + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return keyValueComparator; + } + + @Override + public Key createKey(byte[] values) { + return new Key(values); + } + + @Override + public Value createValue(byte[] values) { + return new Value(values); + } + + @Override + public void testEquality(Map.Entry expected, Map.Entry value) { + assertEquals(expected.getKey(), value.getKey()); + assertEquals(expected.getValue(), value.getValue()); + } + + @Override + public Comparator getComparator() { + return keyComparator; + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedRewritableSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedRewritableSortedMapTest.java new file mode 100644 index 00000000000..377878a31c3 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedRewritableSortedMapTest.java @@ -0,0 +1,89 @@ +package datawave.query.util.sortedmap; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.Test; + +public abstract class MultiMapBackedRewritableSortedMapTest extends MultiMapBackedSortedMapTest { + + /** + * Create a key given the specified value. This key should sort in the same way the underlying byte array will sort against other byte array. + * + * @param values + * @return The key + */ + public abstract K createKey(byte[] values); + + /** + * Create a value given the specified value. + * + * @param values + * @return The value + */ + public abstract V createValue(byte[] values); + + /** + * Get a rewrite strategy. This strategy should allow rewrites if the value is smaller. + * + * @return the rewrite strategy appropriate for key and value types + */ + @Override + public abstract FileSortedMap.RewriteStrategy getRewriteStrategy(); + + @Override + public Map.Entry createData(byte[] values) { + byte[] vbuffer = new byte[values.length]; + Arrays.fill(vbuffer, (byte) (values[0] + 1)); + return new UnmodifiableMapEntry(createKey(values), createValue(vbuffer)); + } + + @Test + public void testRewrite() throws Exception { + // create a new set of data, only half of which has greater Values + Map.Entry[] data2 = new Map.Entry[template.length * 2]; + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + byte[] vbuffer = new byte[buffer.length]; + Arrays.fill(vbuffer, (byte) (template[i] + 1)); + data2[i] = new UnmodifiableMapEntry(createKey(buffer), createValue(vbuffer)); + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + byte[] vbuffer = new byte[buffer.length]; + Arrays.fill(vbuffer, (byte) (template[i] - 1)); + Map.Entry datum = new UnmodifiableMapEntry(createKey(buffer), createValue(vbuffer)); + data2[i + template.length] = datum; + } + + // create a new map with the supplied rewrite strategy + maps = new ArrayList<>(); + for (int i = 0; i < 7; i++) { + maps.add(new TreeMap(getComparator())); + } + map = new MultiMapBackedSortedMap(maps); + map.setRewriteStrategy(getRewriteStrategy()); + + // adding in the data set multiple times to create underlying files with duplicate values making the + // MergeSortIterator's job a little tougher... + for (int d = 0; d < 11; d++) { + addDataRandomly(maps, data); + addDataRandomly(maps, data2); + } + + // now test the contents making sure we still have a sorted set with the expected values + int index = 0; + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + int dataIndex = sortedOrder[index++]; + Map.Entry expected = (dataIndex < template.length ? data2[dataIndex] : data[dataIndex]); + testEquality(expected, value); + } + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedSortedMapTest.java new file mode 100644 index 00000000000..f171947e3e5 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiMapBackedSortedMapTest.java @@ -0,0 +1,327 @@ +package datawave.query.util.sortedmap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.File; +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public abstract class MultiMapBackedSortedMapTest { + + protected final List tempFileHandlers = new ArrayList<>(); + protected Map.Entry[] data = null; + protected int[] sortedOrder = null; + protected MultiMapBackedSortedMap map = null; + protected List maps = null; + protected final byte[] template = new byte[] {5, 2, 29, 4, 8, 3, 25, 23, 6, 21, 7, 16}; + protected final int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; + + public abstract Map.Entry createData(byte[] values); + + public abstract Comparator getComparator(); + + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return null; + } + + protected void testEquality(SortedMap m1, SortedMap m2) { + for (Map.Entry e1 : m1.entrySet()) { + V v2 = m2.get(e1.getKey()); + testEquality(e1, new UnmodifiableMapEntry(e1.getKey(), v2)); + } + } + + protected void testEquality(Map.Entry expected, Map.Entry value) { + testEquality(expected.getKey(), value.getKey()); + assertEquals(expected.getValue(), value.getValue()); + } + + protected void testEquality(K expected, K value) { + if (map.comparator() != null) { + assertEquals(0, map.comparator().compare(expected, value)); + } else { + assertEquals(expected, value); + } + } + + @Before + public void mapUp() throws Exception { + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + Map.Entry datum = createData(buffer); + if (i == 0) { + data = (Map.Entry[]) Array.newInstance(datum.getClass(), template.length * 2); + } + data[i] = datum; + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + Map.Entry datum = createData(buffer); + data[i + template.length] = datum; + } + sortedOrder = new int[data.length]; + for (int i = 0; i < template.length; i++) { + sortedOrder[i * 2] = sortedTemplate[i] + sortedTemplate.length; + sortedOrder[i * 2 + 1] = sortedTemplate[i]; + } + maps = new ArrayList<>(); + for (int i = 0; i < 7; i++) { + maps.add(new TreeMap(getComparator())); + } + map = new MultiMapBackedSortedMap(maps); + map.setRewriteStrategy(getRewriteStrategy()); + + // adding in the data map multiple times to create duplicate values across the underlying maps + // MergeSortIterator's job a little tougher... + for (int i = 0; i < 4; i++) { + for (int d = 0; d < 11; d++) { + addDataRandomly(maps, data); + } + } + } + + public void putRandomly(List maps, K key, V value) { + Random random = new Random(); + int mapIndex = random.nextInt(maps.size()); + maps.get(mapIndex).put(key, value); + } + + public void addDataRandomly(List maps, Map.Entry[] data) { + Set added = new HashSet<>(); + Random random = new Random(); + // add data.length items randomly + for (int i = 0; i < data.length; i++) { + int index = random.nextInt(data.length); + putRandomly(maps, data[index].getKey(), data[index].getValue()); + added.add(index); + } + // ensure all missing items are added + for (int i = 0; i < data.length; i++) { + if (!added.contains(i)) { + putRandomly(maps, data[i].getKey(), data[i].getValue()); + } + } + } + + @After + public void tearDown() throws Exception { + // Delete each sorted map file and its checksum. + for (SortedMapTempFileHandler fileHandler : tempFileHandlers) { + File file = fileHandler.getFile(); + tryDelete(file); + File checksum = new File(file.getParent(), "." + file.getName() + ".crc"); + tryDelete(checksum); + } + tempFileHandlers.clear(); + + data = null; + sortedOrder = null; + map.clear(); + map = null; + } + + private void tryDelete(File file) { + if (file.exists()) { + Assert.assertTrue("Failed to delete file " + file, file.delete()); + } + } + + @Test + public void testSize() { + int expectedSize = data.length; + assertEquals(expectedSize, map.size()); + for (int i = (data.length / 2); i < data.length; i++) { + map.remove(data[i].getKey()); + expectedSize--; + assertEquals(expectedSize, map.size()); + } + for (int i = 0; i < (data.length / 2); i++) { + map.remove(data[i].getKey()); + expectedSize--; + assertEquals(expectedSize, map.size()); + } + assertEquals(0, map.size()); + for (int i = 0; i < data.length; i++) { + putRandomly(maps, data[i].getKey(), data[i].getValue()); + expectedSize++; + assertEquals(expectedSize, map.size()); + } + } + + @Test + public void testIsEmpty() { + assertFalse(map.isEmpty()); + for (int i = (data.length / 2); i < data.length; i++) { + map.remove(data[i].getKey()); + assertFalse(map.isEmpty()); + } + for (int i = 1; i < (data.length / 2); i++) { + map.remove(data[i].getKey()); + assertFalse(map.isEmpty()); + } + map.remove(data[0].getKey()); + assertTrue(map.isEmpty()); + for (int i = 0; i < data.length; i++) { + putRandomly(maps, data[i].getKey(), data[i].getValue()); + assertFalse(map.isEmpty()); + } + } + + @Test + public void testClear() { + map.clear(); + assertTrue(map.isEmpty()); + } + + @Test + public void testContainsObject() { + for (int i = (data.length / 2); i < data.length; i++) { + map.remove(data[i].getKey()); + } + for (int i = 1; i < (data.length / 2); i++) { + assertTrue(map.containsKey(data[i].getKey())); + } + for (int i = (data.length / 2); i < data.length; i++) { + assertFalse(map.containsKey(data[i].getKey())); + } + } + + @Test + public void testRemove() { + int expectedSize = data.length; + + for (int i = 0; i < data.length; i++) { + map.remove(data[i].getKey()); + assertEquals(--expectedSize, map.size()); + } + assertTrue(map.isEmpty()); + } + + @Test + public void testIterator() { + int index = 0; + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + Map.Entry expected = data[sortedOrder[index++]]; + testEquality(expected, value); + } + map.clear(); + for (Map.Entry value : map.entrySet()) { + fail(); + } + } + + @Test + public void testIteratorRemove() { + int size = map.size(); + + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { + Map.Entry value = it.next(); + assertTrue(map.containsKey(value.getKey())); + it.remove(); + size--; + assertFalse(map.containsKey(value.getKey())); + assertEquals(size, map.size()); + } + assertTrue(map.isEmpty()); + } + + @Test + public void testComparator() { + final Comparator comparator = map.comparator(); + Map.Entry[] testData = Arrays.copyOf(data, data.length); + Arrays.sort(testData, new Comparator>() { + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return comparator.compare(o1.getKey(), o2.getKey()); + } + }); + int index = 0; + for (Map.Entry value : map.entrySet()) { + Map.Entry expected = data[sortedOrder[index++]]; + testEquality(expected, value); + } + } + + @Test + public void testSubmap() { + int start = sortedOrder.length / 3; + int end = start * 2; + try { + SortedMap submap = map.subMap(data[sortedOrder[start]].getKey(), data[sortedOrder[end]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); + for (int i = start; i < end; i++) { + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); + } + testEquality(expected, submap); + } catch (Exception e) { + // expected + } + } + + @Test + public void testHeadmap() { + int end = sortedOrder.length / 3; + try { + SortedMap submap = map.headMap(data[sortedOrder[end]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); + for (int i = 0; i < end; i++) { + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); + } + testEquality(expected, submap); + } catch (Exception e) { + // expected + } + } + + @Test + public void testTailmap() { + int start = sortedOrder.length / 3; + try { + SortedMap submap = map.tailMap(data[sortedOrder[start]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); + for (int i = start; i < sortedOrder.length; i++) { + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); + } + testEquality(expected, submap); + } catch (Exception e) { + // expected + } + } + + @Test + public void testLastKey() { + Map.Entry expected = data[sortedOrder[data.length - 1]]; + K value = map.lastKey(); + testEquality(expected.getKey(), value); + } + + @Test + public void testFirstKey() { + Map.Entry expected = data[sortedOrder[0]]; + K value = map.firstKey(); + testEquality(expected.getKey(), value); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumenTransformsTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumenTransformsTest.java new file mode 100644 index 00000000000..e4ac44d3bf2 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumenTransformsTest.java @@ -0,0 +1,107 @@ +package datawave.query.util.sortedmap.rfile; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.Test; + +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + +import datawave.data.type.LcNoDiacriticsType; +import datawave.query.attributes.Attribute; +import datawave.query.attributes.Attributes; +import datawave.query.attributes.Document; +import datawave.query.composite.CompositeMetadata; +import datawave.query.predicate.EventDataQueryFieldFilter; +import datawave.query.util.TypeMetadata; + +public class KeyValueByteDocumenTransformsTest { + + protected final byte[] template = new byte[] {5, 2, 29, 4, 8, 3, 25, 23, 6, 21, 7, 16}; + + @Test + public void testDocumentTransforms() { + List docs = createDocuments(); + for (Document d : docs) { + Value v = KeyValueByteDocumentTransforms.documentToValue(d); + Document d2 = KeyValueByteDocumentTransforms.valueToDocument(v); + assertDocumentEquals(d, d2); + } + } + + @Test + public void testByteTransforms() { + List docs = createByteArrays(); + for (byte[] d : docs) { + Key k = KeyValueByteDocumentTransforms.byteToKey(d); + byte[] d2 = KeyValueByteDocumentTransforms.keyToByte(k); + assertArrayEquals(d, d2); + } + } + + public List createByteArrays() { + List docs = new ArrayList<>(); + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + docs.add(buffer); + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + docs.add(buffer); + } + return docs; + } + + public List createDocuments() { + List docs = new ArrayList<>(); + for (byte[] buffer : createByteArrays()) { + docs.add(createDocument(buffer)); + } + return docs; + } + + public Document createDocument(byte[] values) { + Key docKey = new Key("20200101_1", "datatype\u0000uid", "", values[0]); + Key attrKey = new Key("20200101_1", "datatype\u0000uid", "FIELD\u0000VALUE", values[0]); + List> attrs = new ArrayList<>(); + attrs.add(new UnmodifiableMapEntry(attrKey, new Value())); + Document doc = new Document(docKey, Collections.singleton(docKey), false, attrs.iterator(), + new TypeMetadata().put("FIELD", "datatype", LcNoDiacriticsType.class.getName()), new CompositeMetadata(), true, true, + new EventDataQueryFieldFilter()); + return doc; + } + + public static void assertDocumentEquals(Document d, Document d2) { + // a document comparison that does not include comparing metadata as that does not survive the serialization process + assertEquals(getDictionary(d), getDictionary(d2)); + } + + public static Multimap getDictionary(Document d) { + Multimap map = HashMultimap.create(); + for (Map.Entry>> e : d.entrySet()) { + String key = e.getKey(); + Attribute a = e.getValue(); + if (a instanceof Attributes) { + for (Attribute a2 : ((Attributes) a).getAttributes()) { + map.put(key, String.valueOf(a2.getData())); + } + } else { + map.put(key, String.valueOf(a.getData())); + } + } + return map; + } + +} diff --git a/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml b/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml index ed6d1e2345c..c86a2ee30ef 100644 --- a/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml +++ b/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml @@ -28,11 +28,23 @@ - + + + + + + + + + + + + + diff --git a/web-services/cached-results/src/main/java/datawave/webservice/results/cached/CachedRunningQuery.java b/web-services/cached-results/src/main/java/datawave/webservice/results/cached/CachedRunningQuery.java index b2a4658acd8..da9e2cd1ffe 100644 --- a/web-services/cached-results/src/main/java/datawave/webservice/results/cached/CachedRunningQuery.java +++ b/web-services/cached-results/src/main/java/datawave/webservice/results/cached/CachedRunningQuery.java @@ -306,7 +306,7 @@ public CachedRunningQuery(Query query, QueryLogic queryLogic, String queryId, public CachedRunningQuery(Connection connection, Query query, QueryLogic queryLogic, String queryId, String alias, String user, String view, String fields, String conditions, String grouping, String order, int pagesize, Set variableFields, Set fixedFieldsInEvent, - QueryMetricFactory metricFactory) throws SQLException { + QueryMetricFactory metricFactory) throws SQLException, QueryException { super(metricFactory); this.variableFields.clear(); @@ -574,7 +574,7 @@ private List getViewColumnNames(Connection connection, String view) thro return columns; } - public void activate(Connection connection, QueryLogic queryLogic) throws SQLException { + public void activate(Connection connection, QueryLogic queryLogic) throws SQLException, QueryException { this.connection = connection; this.transformer = queryLogic.getEnrichedTransformer(this.query); diff --git a/web-services/deploy/configuration/src/main/resources/datawave/query/QueryLogicFactory.xml b/web-services/deploy/configuration/src/main/resources/datawave/query/QueryLogicFactory.xml index 08cc4b74187..ad53f615676 100644 --- a/web-services/deploy/configuration/src/main/resources/datawave/query/QueryLogicFactory.xml +++ b/web-services/deploy/configuration/src/main/resources/datawave/query/QueryLogicFactory.xml @@ -38,15 +38,23 @@ - + - + + + + + + + + +