From 53866324e072e3c61768ccf09111c20bc40e4e55 Mon Sep 17 00:00:00 2001 From: Ivan Bella Date: Wed, 10 Jul 2024 20:35:54 +0000 Subject: [PATCH 1/5] Initial refactoring of the mostRecentUnique functionality to avoid the ivarator paths --- .../map/BulkResultsFileOutputMapper.java | 1 + .../input/microservice/QueryLogicFactory.xml | 16 + .../input/webservice/QueryLogicFactory.xml | 12 +- .../microservice/QueryLogicFactory.xml | 16 + ...DatawaveFieldIndexCachingIteratorJexl.java | 7 +- .../iterators/filesystem/FileSystemCache.java | 6 +- .../java/datawave/query/QueryParameters.java | 1 + .../query/attributes/UniqueFields.java | 139 +- .../query/common/grouping/GroupFields.java | 2 +- .../query/config/ShardQueryConfiguration.java | 22 +- .../query/iterator/QueryIterator.java | 19 +- .../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 | 59 +- .../tables/async/event/VisitorFunction.java | 2 +- .../query/transformer/DocumentTransform.java | 2 +- .../query/transformer/GroupingTransform.java | 5 +- .../query/transformer/UniqueTransform.java | 440 +++++-- .../BufferedFileBackedSortedMap.java | 622 +++++++++ .../sortedmap/FileByteDocumentSortedMap.java | 234 ++++ .../util/sortedmap/FileKeySortedMap.java | 218 ++++ .../util/sortedmap/FileKeyValueSortedMap.java | 233 ++++ .../sortedmap/FileSerializableSortedMap.java | 299 +++++ .../query/util/sortedmap/FileSortedMap.java | 1135 +++++++++++++++++ .../util/sortedmap/HdfsBackedSortedMap.java | 292 +++++ .../sortedmap/MultiSetBackedSortedMap.java | 347 +++++ .../util/sortedmap/SortedByteMapBuffer.java | 463 +++++++ .../sortedmap/SortedMapTempFileHandler.java | 64 + .../rfile/KeyValueByteDocumentTransforms.java | 63 + .../rfile/RFileByteDocumentInputStream.java | 32 + .../rfile/RFileByteDocumentOutputStream.java | 21 + .../sortedmap/rfile/RFileKeyInputStream.java | 26 + .../sortedmap/rfile/RFileKeyOutputStream.java | 18 + .../rfile/RFileKeyValueInputStream.java | 28 + .../rfile/RFileKeyValueInputStreamBase.java | 87 ++ .../rfile/RFileKeyValueOutputStream.java | 20 + .../rfile/RFileKeyValueOutputStreamBase.java | 55 + .../test/java/datawave/query/UniqueTest.java | 96 +- .../query/attributes/UniqueFieldsTest.java | 8 +- .../config/ShardQueryConfigurationTest.java | 2 + .../QueryOptionsFromQueryVisitorTest.java | 103 +- .../UniqueTransformMostRecentTest.java | 89 ++ .../transformer/UniqueTransformTest.java | 353 ++--- .../datawave/query/util/WiseGuysIngest.java | 318 ++--- ...feredFileBackedByteArraySortedSetTest.java | 25 + ...edFileBackedByteDocumentSortedSetTest.java | 84 ++ .../BufferedFileBackedKeySortedSetTest.java | 26 + ...fferedFileBackedKeyValueSortedSetTest.java | 63 + ...eredFileBackedRewritableSortedSetTest.java | 111 ++ .../BufferedFileBackedSortedSetTest.java | 436 +++++++ .../util/sortedmap/FileSortedSetTest.java | 450 +++++++ .../sortedmap/HdfsBackedSortedSetTest.java | 253 ++++ .../util/sortedmap/MergeSortIteratorTest.java | 242 ++++ .../MultiSetBackedSortedSetTest.java | 596 +++++++++ .../sortedmap/RewritableSortedSetTest.java | 283 ++++ .../sortedmap/SortedByteSetBufferTest.java | 577 +++++++++ .../datawave/query/QueryLogicFactory.xml | 14 +- .../results/cached/CachedRunningQuery.java | 4 +- .../datawave/query/QueryLogicFactory.xml | 12 +- 75 files changed, 8949 insertions(+), 525 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/FileKeySortedMap.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/MultiSetBackedSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedByteMapBuffer.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/RFileKeyInputStream.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.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/BufferedFileBackedByteArraySortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.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/MultiSetBackedSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.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/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/warehouse/query-core/src/main/java/datawave/core/iterators/DatawaveFieldIndexCachingIteratorJexl.java b/warehouse/query-core/src/main/java/datawave/core/iterators/DatawaveFieldIndexCachingIteratorJexl.java index 1dcc248d6f4..a6d588d2a50 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 @@ -1296,8 +1296,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 ef80c579bb2..60180bfcb38 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 @@ -24,8 +24,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 01ae45c84e0..9299c563089 100644 --- a/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java +++ b/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java @@ -179,6 +179,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/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 6dcadee1bea..82fdb658534 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 @@ -354,6 +354,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; @@ -681,7 +683,9 @@ public ShardQueryConfiguration(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())); @@ -1477,6 +1481,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; } @@ -1832,11 +1844,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() { @@ -2854,6 +2862,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() && @@ -3120,6 +3129,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 92b118ce09e..23954939d0c 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 @@ -459,8 +459,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 @@ -1540,11 +1539,23 @@ public Comparator getValueComparator(Tuple3 hitsOnlySet = new HashSet<>(); @@ -506,6 +510,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; @@ -958,6 +963,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; } @@ -1103,7 +1116,7 @@ public UniqueFields getUniqueFields() { } public void setUniqueFields(UniqueFields uniqueFields) { - this.uniqueFields = uniqueFields; + this.uniqueFields = uniqueFields.clone(); } public Set getHitsOnlySet() { @@ -1582,6 +1595,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 b11b96ee5b0..3c7fd8ce575 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 @@ -532,6 +532,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 2370a8e809b..630baa049b0 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 @@ -96,6 +96,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; @@ -272,7 +273,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) + '(' @@ -609,7 +612,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; } @@ -632,7 +639,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; } @@ -649,7 +660,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()); @@ -657,10 +668,29 @@ 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( + isIvaratorPersistVerify(), + isIvaratorPersistVerify(), + getIvaratorPersistVerifyCount())) + .build()); + // @formatter:on + } catch (IOException ioe) { + throw new QueryException("Unable to create a unique transform", ioe); + } } } @@ -907,11 +937,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)) { @@ -1909,6 +1946,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 75eacf4688e..43f5b58b734 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 @@ -415,7 +415,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..8bb9167f0ff 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 @@ -31,7 +31,7 @@ public interface DocumentTransform extends Function,Map. class DefaultDocumentTransform implements DocumentTransform { protected Query settings; protected MarkingFunctions markingFunctions; - protected long queryExecutionForPageStartTime; + protected long queryExecutionForPageStartTime = System.currentTimeMillis(); @Override public void initialize(Query settings, MarkingFunctions markingFunctions) { 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..d68ae473dbc 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,25 @@ 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.commons.collections4.keyvalue.UnmodifiableMapEntry; +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 +30,37 @@ 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.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.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.FileByteDocumentSortedSet; +import datawave.query.util.sortedset.FileKeyValueSortedSet; +import datawave.query.util.sortedset.FileSortedSet; +import datawave.query.util.sortedset.HdfsBackedSortedSet; +import datawave.query.util.sortedset.RewritableSortedSetImpl; /** * 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 HdfsBackedSortedSet> set; + private HdfsBackedSortedSet> 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,7 +81,6 @@ 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()); @@ -74,58 +88,32 @@ public UniqueTransform(UniqueFields uniqueFields, long queryExecutionForPageTime } /** - * 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()); - } - } - } - 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) { + // 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"); + this.bloom = BloomFilter.create(new ByteFunnel(), 500000, 1e-15); + 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 +131,65 @@ public Entry apply(@Nullable Entry keyDocumentEntry) return keyDocumentEntry; } + if (keyDocumentEntry.getValue().isIntermediateResult()) { + return keyDocumentEntry; + } + try { - if (isDuplicate(keyDocumentEntry.getValue())) { - keyDocumentEntry = null; - } else { + if (set != null) { + byte[] signature = getBytes(keyDocumentEntry.getValue()); + synchronized (set) { + this.set.add(new UnmodifiableMapEntry(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 (set != null) { + synchronized (set) { + 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 : set) { + returnSet.add(new UnmodifiableMapEntry<>(getDocKey(entry.getValue()), entry.getValue())); + } + setIterator = returnSet.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 +238,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 +331,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 +356,217 @@ 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 RewritableSortedSetImpl.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 Comparator<>() { + private Comparator comparator = new ByteArrayComparator(); + + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return comparator.compare(o1.getKey(), o2.getKey()); + } + }; + + keyValueComparator = (original, update) -> { + int comparison = keyComparator.compare(original, update); + if (comparison == 0) { + long ts1 = getTimestamp(original.getValue()); + long ts2 = getTimestamp(update.getValue()); + return (ts2 > ts1); + } + return comparison < 0; + }; + } + + /** + * 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.set = (HdfsBackedSortedSet>) HdfsBackedSortedSet.builder() + .withComparator(keyComparator) + .withRewriteStrategy(keyValueComparator) + .withBufferPersistThreshold(bufferPersistThreshold) + .withIvaratorCacheDirs(getIvaratorCacheDirs(ivaratorCacheDirConfigs, hdfsSiteConfigURLs, subDirectory)) + .withUniqueSubPath("byUniqueKey") + .withMaxOpenFiles(maxOpenFiles) + .withNumRetries(numRetries) + .withPersistOptions(persistOptions) + .withSetFactory(new FileByteDocumentSortedSet.Factory()) + .build(); + + transform.returnSet = (HdfsBackedSortedSet>) HdfsBackedSortedSet.builder() + .withBufferPersistThreshold(bufferPersistThreshold) + .withIvaratorCacheDirs(getIvaratorCacheDirs(ivaratorCacheDirConfigs, hdfsSiteConfigURLs, subDirectory)) + .withUniqueSubPath("byDocKey") + .withMaxOpenFiles(maxOpenFiles) + .withNumRetries(numRetries) + .withPersistOptions(persistOptions) + .withSetFactory(new FileKeyValueSortedSet.Factory()) + .build(); + // @formatter:on + } + + 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..9e49197b9de --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMap.java @@ -0,0 +1,622 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.FileSortedSet.SortedSetFileHandler; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.SortedSet; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * This is a sorted set 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 subset operations will work up until any buffer has been flushed to disk. + * After that, those operations will not work as specified by the underlying FileSortedSet. + * + * @param + * type of the set + */ +public class BufferedFileBackedSortedMap implements RewritableSortedSet { + 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 MultiSetBackedSortedMap set = new MultiSetBackedSortedMap<>(); + protected int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; + protected FileSortedMap buffer = null; + protected FileSortedMap.FileSortedSetFactory setFactory = null; + protected final Comparator comparator; + protected final 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 SortedSetFileHandlers + * + * + * + */ + public interface SortedSetFileHandlerFactory { + SortedSetFileHandler createHandler() throws IOException; + + boolean isValid(); + } + + public static class Builder,E> { + private int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; + private FileSortedMap.FileSortedSetFactory setFactory = new FileSerializableSortedMap.Factory(); + private Comparator comparator; + private 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 withSetFactory(FileSortedMap.FileSortedSetFactory setFactory) { + this.setFactory = (FileSortedMap.FileSortedSetFactory) setFactory; + return self(); + } + + @SuppressWarnings("unchecked") + public B withComparator(Comparator comparator) { + this.comparator = (Comparator) comparator; + return self(); + } + + @SuppressWarnings("unchecked") + public B withRewriteStrategy(RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = (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.setFactory = other.setFactory; + this.bufferPersistThreshold = other.bufferPersistThreshold; + this.numRetries = other.numRetries; + this.maxOpenFiles = other.maxOpenFiles; + for (SortedSet subSet : other.set.getSets()) { + FileSortedMap clone = ((FileSortedMap) subSet).clone(); + this.set.addSet(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.setFactory = builder.setFactory; + this.bufferPersistThreshold = builder.bufferPersistThreshold; + this.numRetries = builder.numRetries; + this.maxOpenFiles = builder.maxOpenFiles; + } + + private SortedSetFileHandler createFileHandler(SortedSetFileHandlerFactory 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 set + for (int i = 0; i < handlerFactories.size() && !buffer.isPersisted(); i++) { + SortedSetFileHandlerFactory handlerFactory = handlerFactories.get(i); + SortedSetFileHandler 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 set using the file handler: " + handler, cause); + + // if this was an hdfs file handler, decrement the count + if (handlerFactory instanceof HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory) { + HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory) handlerFactory); + hdfsHandlerFactory.setFileCount(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 set using the configured handler factories."); + + buffer = null; + compact(maxOpenFiles); + } + } + + protected List> getSets() { + List> sets = new ArrayList<>(); + for (SortedSet subSet : set.getSets()) { + sets.add((FileSortedMap) subSet); + } + return sets; + } + + protected void addSet(FileSortedMap subSet) { + set.addSet(subSet); + size += subSet.size(); + } + + public boolean hasPersistedData() { + for (SortedSet subSet : set.getSets()) { + if (((FileSortedMap) subSet).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 = set.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 contains(Object o) { + // try the cheap operation first + if (buffer != null && buffer.contains(o)) { + return true; + } else { + return set.contains(o); + } + } + + @Override + public boolean containsAll(Collection c) { + // try the cheap operation first + if (buffer != null && buffer.containsAll(c)) { + return true; + } else { + return set.containsAll(c); + } + } + + @Override + public Iterator iterator() { + // first lets compact down the sets if needed + try { + // if we have any persisted sets, then ensure we are persisted + if (set.getSets().size() > 1) { + persist(); + } + } catch (IOException ioe) { + throw new RuntimeException("Unable to persist or compact file backed sorted set", ioe); + } + return set.iterator(); + } + + private String printHandlerFactories() { + return String.join(", ", handlerFactories.stream().map(SortedSetFileHandlerFactory::toString).collect(Collectors.toList())); + } + + /** + * If the number of sets 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 sets than we are allowed, then we need to compact this down + if (maxFiles > 0 && set.getSets().size() > maxFiles) { + if (log.isDebugEnabled()) { + log.debug("Compacting [" + printHandlerFactories() + "]"); + } + // create a copy of the set list (sorting below) + List> sets = new ArrayList<>(set.getSets()); + + // calculate the number of sets to compact + int numSets = sets.size(); + int excessSets = numSets - (maxFiles / 2); // those over 50% of maxFiles + int setsPerCompaction = Math.min(excessSets + 1, numSets); // Add in 1 to account for the compacted set being added back in + + // sort the sets by size (compact up smaller sets first) + sets.sort(Comparator.comparing(SortedSet::size).reversed()); + + // newSet will be the final multiset + MultiSetBackedSortedMap newSet = new MultiSetBackedSortedMap<>(); + + // create a set for those sets to be compacted into one file + MultiSetBackedSortedMap setToCompact = new MultiSetBackedSortedMap<>(); + for (int i = 0; i < setsPerCompaction; i++) { + setToCompact.addSet(sets.remove(sets.size() - 1)); + } + + // compact it + if (log.isDebugEnabled()) { + log.debug("Starting compaction for " + setToCompact); + } + long start = System.currentTimeMillis(); + FileSortedMap compaction = compact(setToCompact); + if (log.isDebugEnabled()) { + long delta = System.currentTimeMillis() - start; + log.debug("Compacted " + setToCompact + " -> " + compaction + " in " + delta + "ms"); + } + + // add the compacted set to our final multiset + newSet.addSet(compaction); + + // clear the compactions set to remove the files that were compacted + setToCompact.clear(); + + // now add in the sets we did not compact + for (int i = 0; i < sets.size(); i++) { + newSet.addSet(sets.get(i)); + } + + // and replace our set + this.set = newSet; + } + } + + private FileSortedMap compact(MultiSetBackedSortedMap setToCompact) throws IOException { + FileSortedMap compactedSet = null; + + // go through the handler factories and try to persist the sorted set + for (int i = 0; i < handlerFactories.size() && compactedSet == null; i++) { + SortedSetFileHandlerFactory handlerFactory = handlerFactories.get(i); + SortedSetFileHandler handler = createFileHandler(handlerFactory); + + // if we have a valid handler, try to persist + if (handler != null) { + Exception cause = null; + for (int attempts = 0; attempts <= numRetries && compactedSet == null; attempts++) { + try { + compactedSet = setFactory.newInstance(setToCompact, handlerFactory.createHandler(), true); + } catch (IOException e) { + if (attempts == numRetries) + cause = e; + } + } + + if (compactedSet == null) { + log.warn("Unable to compact the sorted set using the file handler: " + handler, cause); + + // if this was an hdfs file handler, decrement the count + if (handlerFactory instanceof HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory) { + HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory) handlerFactory); + hdfsHandlerFactory.setFileCount(hdfsHandlerFactory.getFileCount() - 1); + } + } + } else { + log.warn("Unable to create a file handler using the handler factory: " + handlerFactory); + } + } + + // if the sorted sets were not compacted, throw an exception + if (compactedSet == null) + throw new IOException("Unable to persist the sorted set using the configured handler factories."); + + return compactedSet; + } + + @Override + public Object[] toArray() { + return set.toArray(); + } + + @Override + public T[] toArray(T[] a) { + return set.toArray(a); + } + + @Override + public boolean add(E e) { + if (buffer == null) { + try { + buffer = setFactory.newInstance(comparator, rewriteStrategy, null, false); + } catch (Exception ex) { + throw new IllegalStateException("Unable to create an underlying FileSortedSet", ex); + } + + set.addSet(buffer); + } + if (buffer.add(e)) { + sizeModified = true; + if (buffer.size() >= bufferPersistThreshold) { + try { + persist(); + } catch (Exception ex) { + throw new IllegalStateException("Unable to persist or compact FileSortedSet", ex); + } + } + return true; + } + return false; + } + + @Override + public boolean addAll(Collection c) { + if (buffer == null) { + try { + buffer = setFactory.newInstance(comparator, rewriteStrategy, null, false); + } catch (Exception ex) { + throw new IllegalStateException("Unable to create an underlying FileSortedSet", ex); + } + set.addSet(buffer); + } + if (buffer.addAll(c)) { + sizeModified = true; + if (buffer.size() >= bufferPersistThreshold) { + try { + persist(); + } catch (Exception ex) { + throw new IllegalStateException("Unable to persist or compact FileSortedSet", ex); + } + } + return true; + } + return false; + } + + @Override + public boolean remove(Object o) { + boolean removed = false; + for (SortedSet subSet : set.getSets()) { + FileSortedMap fileSet = (FileSortedMap) subSet; + if (fileSet.contains(o)) { + if (fileSet.isPersisted()) { + try { + fileSet.load(); + if (fileSet.remove(o)) { + removed = true; + fileSet.persist(); + } else { + fileSet.unload(); + // since we checked for containership first, remove should have returned true + throw new IllegalStateException("FileSet contains object but failed to remove it from persisted set"); + } + } catch (Exception e) { + throw new IllegalStateException("Unable to remove item from underlying files", e); + } + } else { + if (fileSet.remove(o)) { + removed = true; + } else { + // since we checked for containership first, remove should have returned true + throw new IllegalStateException("FileSet contains object but failed to remove it"); + } + } + } + } + if (removed) { + this.sizeModified = true; + } + return removed; + } + + @Override + public boolean retainAll(Collection c) { + boolean modified = false; + for (SortedSet subSet : set.getSets()) { + FileSortedMap fileSet = (FileSortedMap) subSet; + if (fileSet.isPersisted()) { + try { + fileSet.load(); + if (fileSet.retainAll(c)) { + modified = true; + fileSet.persist(); + } else { + fileSet.unload(); + } + } catch (Exception e) { + throw new IllegalStateException("Unable to remove item from underlying files", e); + } + } else { + if (fileSet.retainAll(c)) { + modified = true; + } + } + } + if (modified) { + this.sizeModified = true; + } + return modified; + } + + @Override + public boolean removeAll(Collection c) { + boolean modified = false; + for (SortedSet subSet : set.getSets()) { + FileSortedMap fileSet = (FileSortedMap) subSet; + if (fileSet.isPersisted()) { + try { + fileSet.load(); + if (fileSet.removeAll(c)) { + modified = true; + fileSet.persist(); + } else { + fileSet.unload(); + } + } catch (Exception e) { + throw new IllegalStateException("Unable to remove item from underlying files", e); + } + } else { + if (fileSet.removeAll(c)) { + modified = true; + } + } + } + if (modified) { + this.sizeModified = true; + } + return modified; + } + + @Override + public boolean removeIf(Predicate filter) { + boolean removed = false; + for (SortedSet subSet : set.getSets()) { + FileSortedMap fileSet = (FileSortedMap) subSet; + if (fileSet.isPersisted()) { + try { + fileSet.load(); + if (fileSet.removeIf(filter)) { + removed = true; + fileSet.persist(); + } else { + fileSet.unload(); + } + } catch (Exception e) { + throw new IllegalStateException("Unable to remove item from underlying files", e); + } + } else { + if (fileSet.removeIf(filter)) { + removed = true; + } + } + } + if (removed) { + this.sizeModified = true; + } + return removed; + } + + @Override + public void clear() { + // This will cause the MultiSetBackedSortedSet to call clear on each Set in its Set of Sets, including the buffer + // It will also call clear on its Set of Sets, emptying the contents + set.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 RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; + } + + @Override + public E get(E e) { + return null; + } + + @Override + public RewritableSortedSet subSet(E fromElement, E toElement) { + return set.subSet(fromElement, toElement); + } + + @Override + public RewritableSortedSet headSet(E toElement) { + return set.headSet(toElement); + } + + @Override + public RewritableSortedSet tailSet(E fromElement) { + return set.tailSet(fromElement); + } + + @Override + public E first() { + return set.first(); + } + + @Override + public E last() { + return set.last(); + } +} 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..ae9750f575e --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileByteDocumentSortedMap.java @@ -0,0 +1,234 @@ +package datawave.query.util.sortedmap; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.rfile.RFileByteDocumentInputStream; +import datawave.query.util.sortedmap.rfile.RFileByteDocumentOutputStream; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Comparator; +import java.util.Map; + +/** + * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying sets 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 DefaultByteDocumentComparator implements Comparator> { + + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return new ByteArrayComparator().compare(o1.getKey(), o2.getKey()); + } + } + + /** + * Create a file sorted set from another one + * + * @param other + * the other sorted set + */ + public FileByteDocumentSortedMap(FileByteDocumentSortedMap other) { + super(other); + } + + /** + * Create a file sorted subset from another one + * + * @param other + * the other sorted set + * @param from + * the from Document + * @param to + * the to Document + */ + public FileByteDocumentSortedMap(FileByteDocumentSortedMap other, Map.Entry from, Map.Entry to) { + super(other, from, to); + } + + /** + * Create a persisted sorted set + * + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileByteDocumentSortedMap(SortedSetFileHandler handler, boolean persisted) { + this(new DefaultByteDocumentComparator(), handler, persisted); + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileByteDocumentSortedMap(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultByteDocumentComparator() : comparator), new ByteDocumentFileHandler(handler), + new Factory(), persisted); + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param rewriteStrategy + * the rewrite strategy + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileByteDocumentSortedMap(Comparator> comparator, RewriteStrategy> rewriteStrategy, + SortedSetFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultByteDocumentComparator() : comparator), rewriteStrategy, new ByteDocumentFileHandler(handler), + new Factory(), persisted); + } + + /** + * Create an unpersisted sorted set (still in memory) + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + */ + public FileByteDocumentSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler) { + super(set, new ByteDocumentFileHandler(handler), new Factory()); + } + + /** + * Create a sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoids pulling + * all of its entries into memory at once. + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + * @param persist + * boolean flag for persist + * @throws IOException + * for issues with read/write + */ + public FileByteDocumentSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) throws IOException { + super(set, new ByteDocumentFileHandler(handler), new Factory(), persist); + } + + /** + * This will dump the set to the file, making the set "persisted" + * + * @param handler + * the sorted set file handler + * @throws IOException + * for issues with read/write + */ + public void persist(SortedSetFileHandler handler) throws IOException { + // ensure this handler is wrapped with our handler + super.persist(new ByteDocumentFileHandler(handler)); + } + + /** + * Clone this set + */ + @Override + public FileByteDocumentSortedMap clone() { + return (FileByteDocumentSortedMap) super.clone(); + } + + /** + * A sortedsetfilehandler that can bound the input stream + */ + public static class ByteDocumentFileHandler implements BoundedTypedSortedSetFileHandler> { + SortedSetFileHandler delegate; + + public ByteDocumentFileHandler(SortedSetFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedSetInputStream> getInputStream() throws IOException { + return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedSetInputStream> getInputStream(Map.Entry start, Map.Entry end) + throws IOException { + return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize(), start, end); + } + + @Override + public SortedSetOutputStream getOutputStream() throws IOException { + return new RFileByteDocumentOutputStream(delegate.getOutputStream()); + } + + @Override + public PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + /** + * A factory for these file sorted sets + */ + public static class Factory implements FileSortedSetFactory> { + + @Override + public FileByteDocumentSortedMap newInstance(FileSortedMap> other) { + return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other); + } + + @Override + public FileByteDocumentSortedMap newInstance(FileSortedMap> other, Map.Entry from, + Map.Entry to) { + return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other, from, to); + } + + @Override + public FileByteDocumentSortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + return new FileByteDocumentSortedMap(handler, persisted); + } + + @Override + public FileByteDocumentSortedMap newInstance(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { + return new FileByteDocumentSortedMap(comparator, handler, persisted); + } + + @Override + public FileSortedMap> newInstance(Comparator> comparator, + RewriteStrategy> rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { + return new FileByteDocumentSortedMap(comparator, rewriteStrategy, handler, persisted); + } + + @Override + public FileByteDocumentSortedMap newInstance(RewritableSortedSet> set, SortedSetFileHandler handler) { + return new FileByteDocumentSortedMap(set, handler); + } + + @Override + public FileByteDocumentSortedMap newInstance(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) + throws IOException { + return new FileByteDocumentSortedMap(set, handler, persist); + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java new file mode 100644 index 00000000000..80b7483d013 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java @@ -0,0 +1,218 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedmap.rfile.RFileKeyInputStream; +import datawave.query.util.sortedmap.rfile.RFileKeyOutputStream; +import org.apache.accumulo.core.data.Key; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Comparator; + +/** + * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying sets 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 FileKeySortedMap extends FileSortedMap { + private static Logger log = Logger.getLogger(FileKeySortedMap.class); + + /** + * Create a file sorted set from another one + * + * @param other + * the other sorted set + */ + public FileKeySortedMap(FileKeySortedMap other) { + super(other); + } + + /** + * Create a file sorted subset from another one + * + * @param other + * the other sorted set + * @param from + * the from key + * @param to + * the to key + */ + public FileKeySortedMap(FileKeySortedMap other, Key from, Key to) { + super(other, from, to); + } + + /** + * Create a persisted sorted set + * + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeySortedMap(SortedSetFileHandler handler, boolean persisted) { + super(new KeyFileHandler(handler), new Factory(), persisted); + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeySortedMap(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + super(comparator, new KeyFileHandler(handler), new Factory(), persisted); + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param rewriteStrategy + * rewrite strategy + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeySortedMap(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { + super(comparator, rewriteStrategy, new KeyFileHandler(handler), new Factory(), persisted); + } + + /** + * Create an unpersisted sorted set (still in memory) + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + */ + public FileKeySortedMap(RewritableSortedSet set, SortedSetFileHandler handler) { + super(set, new KeyFileHandler(handler), new Factory()); + } + + /** + * Create an sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + * @param persist + * boolean flag for persist + * @throws IOException + * for issues with read/write + */ + public FileKeySortedMap(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { + super(set, new KeyFileHandler(handler), new Factory(), persist); + } + + /** + * This will dump the set to the file, making the set "persisted" + * + * @param handler + * the sorted set file handler + * @throws IOException + * for issues with read/write + */ + public void persist(SortedSetFileHandler handler) throws IOException { + // ensure this handler is wrapped with our handler + super.persist(new KeyFileHandler(handler)); + } + + /** + * Clone this set + */ + @Override + public FileKeySortedMap clone() { + return (FileKeySortedMap) super.clone(); + } + + /** + * A sortedsetfilehandler that can bound the input stream + */ + public static class KeyFileHandler implements BoundedTypedSortedSetFileHandler { + SortedSetFileHandler delegate; + + public KeyFileHandler(SortedSetFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedSetInputStream getInputStream() throws IOException { + return new RFileKeyInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedSetInputStream getInputStream(Key start, Key end) throws IOException { + return new RFileKeyInputStream(delegate.getInputStream(), delegate.getSize(), start, end); + } + + @Override + public SortedSetOutputStream getOutputStream() throws IOException { + return new RFileKeyOutputStream(delegate.getOutputStream()); + } + + @Override + public PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + /** + * A factory for these file sorted sets + */ + public static class Factory implements FileSortedSetFactory { + + @Override + public FileKeySortedMap newInstance(FileSortedMap other) { + return new FileKeySortedMap((FileKeySortedMap) other); + } + + @Override + public FileKeySortedMap newInstance(FileSortedMap other, Key from, Key to) { + return new FileKeySortedMap((FileKeySortedMap) other, from, to); + } + + @Override + public FileKeySortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + return new FileKeySortedMap(handler, persisted); + } + + @Override + public FileKeySortedMap newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + return new FileKeySortedMap(comparator, handler, persisted); + } + + @Override + public FileKeySortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { + return new FileKeySortedMap(comparator, rewriteStrategy, handler, persisted); + } + + @Override + public FileKeySortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler) { + return new FileKeySortedMap(set, handler); + } + + @Override + public FileKeySortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { + return new FileKeySortedMap(set, 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..bab39c1e5fc --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyValueSortedMap.java @@ -0,0 +1,233 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedmap.rfile.RFileKeyValueInputStream; +import datawave.query.util.sortedmap.rfile.RFileKeyValueOutputStream; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Comparator; +import java.util.Map; + +/** + * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying sets 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 DefaultKeyValueComparator implements Comparator> { + + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return o1.getKey().compareTo(o2.getKey()); + } + } + + /** + * Create a file sorted set from another one + * + * @param other + * the other sorted set + */ + public FileKeyValueSortedMap(FileKeyValueSortedMap other) { + super(other); + } + + /** + * Create a file sorted subset from another one + * + * @param other + * the other sorted set + * @param from + * the from key + * @param to + * the to key + */ + public FileKeyValueSortedMap(FileKeyValueSortedMap other, Map.Entry from, Map.Entry to) { + super(other, from, to); + } + + /** + * Create a persisted sorted set + * + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyValueSortedMap(SortedSetFileHandler handler, boolean persisted) { + this(new DefaultKeyValueComparator(), handler, persisted); + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyValueSortedMap(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultKeyValueComparator() : comparator), new KeyValueFileHandler(handler), new Factory(), + persisted); + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param rewriteStrategy + * rewrite strategy + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + */ + public FileKeyValueSortedMap(Comparator> comparator, RewriteStrategy> rewriteStrategy, + SortedSetFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultKeyValueComparator() : comparator), rewriteStrategy, new KeyValueFileHandler(handler), + new Factory(), persisted); + } + + /** + * Create an unpersisted sorted set (still in memory) + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + */ + public FileKeyValueSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler) { + super(set, new KeyValueFileHandler(handler), new Factory()); + } + + /** + * Create an sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + * @param persist + * boolean flag for persist + * @throws IOException + * for issues with read/write + */ + public FileKeyValueSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) throws IOException { + super(set, new KeyValueFileHandler(handler), new Factory(), persist); + } + + /** + * This will dump the set to the file, making the set "persisted" + * + * @param handler + * the sorted set file handler + * @throws IOException + * for issues with read/write + */ + public void persist(SortedSetFileHandler handler) throws IOException { + // ensure this handler is wrapped with our handler + super.persist(new KeyValueFileHandler(handler)); + } + + /** + * Clone this set + */ + @Override + public FileKeyValueSortedMap clone() { + return (FileKeyValueSortedMap) super.clone(); + } + + /** + * A sortedsetfilehandler that can bound the input stream + */ + public static class KeyValueFileHandler implements BoundedTypedSortedSetFileHandler> { + SortedSetFileHandler delegate; + + public KeyValueFileHandler(SortedSetFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedSetInputStream> getInputStream() throws IOException { + return new RFileKeyValueInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedSetInputStream> getInputStream(Map.Entry start, Map.Entry end) throws IOException { + return new RFileKeyValueInputStream(delegate.getInputStream(), delegate.getSize(), start, end); + } + + @Override + public SortedSetOutputStream getOutputStream() throws IOException { + return new RFileKeyValueOutputStream(delegate.getOutputStream()); + } + + @Override + public PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + /** + * A factory for these file sorted sets + */ + public static class Factory implements FileSortedSetFactory> { + + @Override + public FileKeyValueSortedMap newInstance(FileSortedMap> other) { + return new FileKeyValueSortedMap((FileKeyValueSortedMap) other); + } + + @Override + public FileKeyValueSortedMap newInstance(FileSortedMap> other, Map.Entry from, Map.Entry to) { + return new FileKeyValueSortedMap((FileKeyValueSortedMap) other, from, to); + } + + @Override + public FileKeyValueSortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + return new FileKeyValueSortedMap(handler, persisted); + } + + @Override + public FileKeyValueSortedMap newInstance(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { + return new FileKeyValueSortedMap(comparator, handler, persisted); + } + + @Override + public FileKeyValueSortedMap newInstance(Comparator> comparator, RewriteStrategy> rewriteStategy, + SortedSetFileHandler handler, boolean persisted) { + return new FileKeyValueSortedMap(comparator, rewriteStategy, handler, persisted); + } + + @Override + public FileKeyValueSortedMap newInstance(RewritableSortedSet> set, SortedSetFileHandler handler) { + return new FileKeyValueSortedMap(set, handler); + } + + @Override + public FileKeyValueSortedMap newInstance(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) + throws IOException { + return new FileKeyValueSortedMap(set, 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..780fc5208fd --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSerializableSortedMap.java @@ -0,0 +1,299 @@ +package datawave.query.util.sortedmap; + +import org.apache.accumulo.core.data.Key; +import org.apache.log4j.Logger; + +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; + +/** + * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as + * expected. This will support null contained in the underlying sets 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 FileSerializableSortedMap extends FileSortedMap { + private static Logger log = Logger.getLogger(FileSerializableSortedMap.class); + + /** + * Create a file sorted set from another one + * + * @param other + * the other sorted set + */ + public FileSerializableSortedMap(FileSerializableSortedMap other) { + super(other); + } + + /** + * Create a file sorted subset from another one + * + * @param other + * the other sorted set + * @param from + * the from file + * @param to + * the to file + */ + public FileSerializableSortedMap(FileSerializableSortedMap other, E from, E to) { + super(other, from, to); + } + + /** + * Create a persisted sorted set + * + * @param handler + * a file handler + * @param persisted + * persisted boolean flag + */ + public FileSerializableSortedMap(TypedSortedSetFileHandler handler, boolean persisted) { + super(handler, new Factory(), persisted); + } + + /** + * Create a persistede sorted set + * + * @param comparator + * a comparator + * @param handler + * a file handler + * @param persisted + * persisted boolean flag + */ + public FileSerializableSortedMap(Comparator comparator, TypedSortedSetFileHandler handler, boolean persisted) { + super(comparator, handler, new Factory(), persisted); + } + + /** + * Create a persistede sorted set + * + * @param comparator + * a comparator + * @param rewriteStrategy + * rewrite strategy + * @param handler + * a file handler + * @param persisted + * persisted boolean flag + */ + public FileSerializableSortedMap(Comparator comparator, RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, boolean persisted) { + super(comparator, rewriteStrategy, handler, new Factory(), persisted); + } + + /** + * Create an unpersisted sorted set (still in memory) + * + * @param set + * a sorted set + * @param handler + * a file handler + */ + public FileSerializableSortedMap(RewritableSortedSet set, TypedSortedSetFileHandler handler) { + super(set, handler, new Factory()); + } + + /** + * Create an sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param set + * a sorted set + * @param handler + * a file handler + * @param persist + * a persist flag + * @throws IOException + * for issues with read/write + */ + public FileSerializableSortedMap(RewritableSortedSet set, TypedSortedSetFileHandler handler, boolean persist) throws IOException { + super(set, handler, new Factory(), persist); + } + + /** + * Persist a set using the specified handler + * + * @param handler + * a file handler + * @throws IOException + * for issues with read/write + */ + @Override + public void persist(SortedSetFileHandler handler) throws IOException { + super.persist(new SerializableFileHandler(handler)); + } + + @Override + public FileSerializableSortedMap clone() { + return (FileSerializableSortedMap) super.clone(); + } + + /** + * A sortedsetfilehandler that can handler serializable objects + */ + public static class SerializableFileHandler implements TypedSortedSetFileHandler { + SortedSetFileHandler delegate; + + public SerializableFileHandler(SortedSetFileHandler handler) { + this.delegate = handler; + } + + @Override + public SortedSetInputStream getInputStream() throws IOException { + return new SerializableInputStream(delegate.getInputStream(), delegate.getSize()); + } + + @Override + public SortedSetOutputStream getOutputStream() throws IOException { + return new SerializableOutputStream(delegate.getOutputStream()); + } + + @Override + public PersistOptions getPersistOptions() { + return delegate.getPersistOptions(); + } + + @Override + public long getSize() { + return delegate.getSize(); + } + + @Override + public void deleteFile() { + delegate.deleteFile(); + } + } + + public static class SerializableInputStream implements SortedSetInputStream { + 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 E readObject() throws IOException { + try { + return (E) getDelegate().readObject(); + } 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); + + 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 SortedSetOutputStream { + private ObjectOutputStream delegate; + + public SerializableOutputStream(OutputStream stream) throws IOException { + delegate = new ObjectOutputStream(stream); + } + + @Override + public void writeObject(E obj) throws IOException { + delegate.writeObject(obj); + } + + @Override + public void writeSize(int size) throws IOException { + 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 set + */ + public static class Factory implements FileSortedSetFactory { + + @Override + public FileSerializableSortedMap newInstance(FileSortedMap other) { + return new FileSerializableSortedMap((FileSerializableSortedMap) other); + } + + @Override + public FileSerializableSortedMap newInstance(FileSortedMap other, E from, E to) { + return new FileSerializableSortedMap((FileSerializableSortedMap) other, from, to); + } + + @Override + public FileSerializableSortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + return new FileSerializableSortedMap(new SerializableFileHandler(handler), persisted); + } + + @Override + public FileSerializableSortedMap newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + return new FileSerializableSortedMap(comparator, new SerializableFileHandler(handler), persisted); + } + + @Override + public FileSerializableSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, + boolean persisted) { + return new FileSerializableSortedMap(comparator, rewriteStrategy, new SerializableFileHandler(handler), persisted); + } + + @Override + public FileSerializableSortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler) { + return new FileSerializableSortedMap(set, new SerializableFileHandler(handler)); + } + + @Override + public FileSerializableSortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { + return new FileSerializableSortedMap(set, 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..589ba57e90b --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileSortedMap.java @@ -0,0 +1,1135 @@ +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.lang.reflect.Array; +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.NoSuchElementException; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import org.apache.log4j.Logger; + +import datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; + +/** + * A sorted set that can be persisted into a file and still be read in its persisted state. The set 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. + * + * @param + * type of set + */ +public abstract class FileSortedMap extends RewritableSortedMapImpl implements SortedSet, Cloneable { + private static Logger log = Logger.getLogger(FileSortedMap.class); + protected boolean persisted = false; + protected E[] range; + // The file handler that handles the underlying io + public TypedSortedSetFileHandler handler; + // The sort set factory + public FileSortedSetFactory factory; + + /** + * Create a file sorted set from another one + * + * @param other + * the other sorted set + */ + public FileSortedMap(FileSortedMap other) { + super(other); + this.handler = other.handler; + this.factory = other.factory; + this.persisted = other.persisted; + this.range = other.range; + } + + /** + * Create a file sorted subset from another one + * + * @param other + * the other sorted set + * @param from + * the from key + * @param to + * the to key + */ + public FileSortedMap(FileSortedMap other, E from, E to) { + this(other); + if (from != null || to != null) { + if (persisted) { + this.range = (E[]) new Object[] {getStart(from), getEnd(to)}; + } else if (to == null) { + this.set = this.set.tailMap(from, true); + } else if (from == null) { + this.set = this.set.headMap(to, false); + } else { + this.set = this.set.subMap(from, true, to, false); + } + } + } + + /** + * Create a persisted sorted set + * + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @param factory + * the sorted set factory + */ + public FileSortedMap(TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { + this.handler = handler; + this.factory = factory; + this.set = new TreeMap<>(); + this.persisted = persisted; + } + + /** + * Create a persisted sorted set + * + * @param rewriteStrategy + * the item rewrite strategy + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @param factory + * the sorted set factory + */ + public FileSortedMap(RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { + super(rewriteStrategy); + this.handler = handler; + this.factory = factory; + this.set = new TreeMap<>(); + this.persisted = persisted; + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param rewriteStrategy + * the item rewrite strategy + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @param factory + * the sorted set factory + */ + public FileSortedMap(Comparator comparator, RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, + boolean persisted) { + super(comparator, rewriteStrategy); + this.handler = handler; + this.factory = factory; + this.set = new TreeMap<>(comparator); + this.persisted = persisted; + } + + /** + * Create a persisted sorted set + * + * @param comparator + * the key comparator + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @param factory + * the sorted set factory + */ + public FileSortedMap(Comparator comparator, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { + super(comparator); + this.handler = handler; + this.factory = factory; + this.set = new TreeMap<>(comparator); + this.persisted = persisted; + } + + /** + * Create an unpersisted sorted set (still in memory) + * + * @param set + * a sorted set + * @param handler + * the sorted set file handler + * @param factory + * the sorted set factory + */ + public FileSortedMap(RewritableSortedMap set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory) { + this.handler = handler; + this.factory = factory; + this.set = set.stream().collect(Collectors.toMap(value -> value, value -> value, (l, r) -> l, () -> new TreeMap<>(set.comparator()))); + this.rewriteStrategy = set.getRewriteStrategy(); + this.persisted = false; + } + + /** + * Create a sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * all of its entries into memory at once. + * + * @param set + * a sorted set + * @param handler + * the sorted set file handler + * @param factory + * the sorted set factory + * @param persist + * the persist boolean flag + * @throws IOException + * for issues with read/write + */ + public FileSortedMap(RewritableSortedMap set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persist) throws IOException { + this(set, handler, factory); + if (persist) { + persist(set, handler); + persisted = true; + } + } + + /** + * This will revert this set to whatever contents are in the underlying file, making the set "persisted". This is intended to be used following a load + * command when no changes were actually made the the set 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.set.size(), Collections.emptyList()); + this.set.clear(); + persisted = true; + } + } + + /** + * This will dump the set to the file, making the set "persisted" + * + * @throws IOException + * for issues with read/write + */ + public void persist() throws IOException { + persist(this.handler); + } + + /** + * This will dump the set to the file, making the set "persisted" + * + * @param handler + * the handler + * @throws IOException + * for issues with read/write + */ + public void persist(TypedSortedSetFileHandler handler) throws IOException { + if (!persisted) { + persist(this.set.navigableKeySet(), handler); + this.set.clear(); + persisted = true; + } + } + + /** + * This will dump the set to a file, making the set "persisted" The implementation is expected to wrap the handler with a TypedSortedSetFileHandler and the + * call persist(TypedSortedSetFileHandler handler) + * + * @param handler + * the sorted set file handler + * @throws IOException + * for issues with read/write + */ + public abstract void persist(SortedSetFileHandler handler) throws IOException; + + /** + * Persist the supplied set to a file as defined by this classes sorted set file handler. + * + * @param set + * the set + * @param handler + * the handler + * @throws IOException + * for issues with read/write + * + */ + private void persist(SortedSet set, TypedSortedSetFileHandler 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 reset to null + this.handler = handler; + + int actualSize = 0; + PersistOptions persistOptions = handler.getPersistOptions(); + List setToVerify = new ArrayList<>(); + try (SortedSetOutputStream stream = handler.getOutputStream()) { + for (E t : set) { + stream.writeObject(t); + if (persistOptions.isVerifyElements() && setToVerify.size() < persistOptions.getNumElementsToVerify()) { + setToVerify.add(t); + } + actualSize++; + } + stream.writeSize(actualSize); + } + // now verify the written file + verifyPersistance(handler, actualSize, setToVerify); + + } 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(TypedSortedSetFileHandler handler, int size, List setToVerify) throws IOException { + // verify we wrote at least the size.... + if (handler.getSize() == 0) { + throw new IOException("Failed to verify file existence"); + } + PersistOptions persistOptions = handler.getPersistOptions(); + // now verify the first n objects were written correctly + if (persistOptions.isVerifyElements() && !setToVerify.isEmpty()) { + try (SortedSetInputStream inStream = handler.getInputStream()) { + int count = 0; + for (E t : setToVerify) { + count++; + E 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 (SortedSetInputStream inStream = handler.getInputStream()) { + return inStream.readSize(); + } + } + + /** + * This will read the file into an in-memory set, 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 (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + E obj = stream.readObject(); + while (obj != null) { + super.add(obj); + obj = stream.readObject(); + } + } + persisted = false; + } + } + + protected E readObject(ObjectInputStream stream) { + try { + return (E) stream.readObject(); + } catch (Exception E) { + return null; + } + } + + protected void writeObject(ObjectOutputStream stream, E obj) throws IOException { + stream.writeObject(obj); + } + + /* + * Is this set persisted? + */ + public boolean isPersisted() { + return persisted; + } + + /** + * Get the size of the set. Note if the set 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 (isSubset()) { + throw new IllegalStateException("Unable to determine size of a subset of a persisted set. Please call load() first."); + } + try { + return readSize(); + } catch (Exception e) { + throw new IllegalStateException("Unable to get size from file", e); + } + } else { + return super.size(); + } + } + + @Override + public boolean isEmpty() { + // must attempt to read the first element to be sure if persisted + try { + first(); + return false; + } catch (NoSuchElementException e) { + return true; + } + } + + @SuppressWarnings("unchecked") + @Override + public boolean contains(Object o) { + if (persisted) { + E t = (E) o; + for (E next : this) { + if (equals(next, t)) { + return true; + } + } + return false; + } else { + return super.contains(o); + } + } + + @Override + public Iterator iterator() { + if (persisted) { + return new FileIterator(); + } else { + return super.iterator(); + } + } + + @Override + public Object[] toArray() { + if (persisted) { + try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Object[] data = new Object[readSize()]; + int index = 0; + E obj = stream.readObject(); + while (obj != null) { + data[index++] = obj; + obj = stream.readObject(); + } + if (index < data.length) { + Object[] dataCpy = new Object[index]; + System.arraycopy(data, 0, dataCpy, 0, index); + data = dataCpy; + } + return data; + } catch (IOException e) { + throw new IllegalStateException("Unable to read file into a complete set", e); + } + } else { + return super.toArray(); + } + } + + @SuppressWarnings({"unchecked"}) + @Override + public T[] toArray(T[] a) { + if (persisted) { + try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + T[] data = a; + int index = 0; + T obj = (T) stream.readObject(); + while (obj != null) { + if (index > data.length) { + T[] dataCpy = (T[]) (Array.newInstance(a.getClass().getComponentType(), data.length + (data.length / 2))); + System.arraycopy(data, 0, dataCpy, 0, data.length); + data = dataCpy; + } + data[index++] = obj; + obj = (T) stream.readObject(); + } + // if not resized + if (data == a) { + // ensure extra elements are set to null + for (; index < data.length; index++) { + data[index] = null; + } + } else if (index < data.length) { + T[] dataCpy = (T[]) (Array.newInstance(a.getClass().getComponentType(), index)); + System.arraycopy(data, 0, dataCpy, 0, index); + data = dataCpy; + } + return data; + } catch (IOException e) { + throw new IllegalStateException("Unable to read file into a complete set", e); + } + } else { + return super.toArray(a); + } + } + + @Override + public boolean add(E e) { + if (persisted) { + throw new IllegalStateException("Cannot add an element to a persisted FileSortedSet. Please call load() first."); + } + return super.add(e); + } + + @Override + public boolean remove(Object o) { + if (persisted) { + throw new IllegalStateException("Cannot remove an element to a persisted FileSortedSet. Please call load() first."); + } else { + return (super.remove(o)); + } + } + + @SuppressWarnings("unchecked") + @Override + public boolean containsAll(Collection c) { + if (c.isEmpty()) { + return true; + } + if (persisted) { + try { + SortedSet all = new TreeSet<>(set.comparator()); + for (Object o : c) { + all.add((E) o); + } + if (all.isEmpty()) { + return true; + } + try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + E obj = stream.readObject(); + while (obj != null) { + if (all.remove(obj)) { + if (all.isEmpty()) { + return true; + } + } + obj = stream.readObject(); + } + } + } catch (Exception e) { + throw new IllegalStateException("Unable to read file into a complete set", e); + } + return false; + } else { + return super.containsAll(c); + } + } + + @Override + public boolean addAll(Collection c) { + if (persisted) { + throw new IllegalStateException("Unable to add to a persisted FileSortedSet. Please call load() first."); + } else { + return super.addAll(c); + } + } + + @Override + public boolean retainAll(Collection c) { + if (persisted) { + throw new IllegalStateException("Unable to modify a persisted FileSortedSet. Please call load() first."); + } else { + return super.retainAll(c); + } + } + + @Override + public boolean removeAll(Collection c) { + if (persisted) { + throw new IllegalStateException("Unable to remove from a persisted FileSortedSet. Please call load() first."); + } else { + return super.removeAll(c); + } + } + + @Override + public boolean removeIf(Predicate filter) { + if (persisted) { + throw new IllegalStateException("Unable to remove from a persisted FileSortedSet. Please call load() first."); + } else { + return super.removeIf(filter); + } + } + + @Override + public void clear() { + if (persisted) { + handler.deleteFile(); + persisted = false; + } else { + super.clear(); + } + } + + @Override + public Comparator comparator() { + return super.comparator(); + } + + @Override + public RewritableSortedMap subSet(E fromElement, E toElement) { + return factory.newInstance(this, fromElement, toElement); + } + + @Override + public RewritableSortedMap headSet(E toElement) { + return factory.newInstance(this, null, toElement); + } + + @Override + public RewritableSortedMap tailSet(E fromElement) { + return factory.newInstance(this, fromElement, null); + } + + @Override + public E first() { + if (persisted) { + try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + E object = stream.readObject(); + if (object == null) { + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR)); + } else { + return object; + } + } catch (Exception e) { + throw new IllegalStateException(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR, e)); + } + } else { + return super.first(); + } + } + + @Override + public E last() { + if (persisted) { + boolean gotLast = false; + E last = null; + try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + last = stream.readObject(); + E next = stream.readObject(); + while (next != null) { + last = next; + gotLast = true; + next = stream.readObject(); + } + } catch (Exception e) { + throw new IllegalStateException(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR, e)); + } + if (gotLast) { + return last; + } else { + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR)); + } + } else { + return super.last(); + } + } + + @Override + public String toString() { + if (persisted) { + return handler.toString(); + } else { + return super.toString(); + } + } + + /** + * Extending classes must implement cloneable + * + * @return A clone + */ + public FileSortedMap clone() { + return factory.newInstance(this); + } + + /* Some utilities */ + private boolean equals(E o1, E o2) { + if (o1 == null) { + return o2 == null; + } else if (o2 == null) { + return false; + } else { + if (set.comparator() == null) { + return o1.equals(o2); + } else { + return set.comparator().compare(o1, o2) == 0; + } + } + } + + private E getStart() { + return (isSubset() ? range[0] : null); + } + + private E getStart(E from) { + E 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 E getEnd() { + return (isSubset() ? range[1] : null); + } + + private E getEnd(E to) { + E 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 isSubset() { + return (range != null); + } + + private int compare(E a, E b) { + return (this.set.comparator() != null) ? this.set.comparator().compare(a, b) : ((Comparable) a).compareTo(b); + } + + public BoundedTypedSortedSetFileHandler getBoundedFileHandler() { + return new DefaultBoundedTypedSortedSetFileHandler(); + } + + /** + * This is the iterator for a persisted FileSortedSet + */ + protected class FileIterator implements Iterator { + private SortedSetInputStream stream; + private E 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 E next() { + if (!hasNext()) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + try { + E 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."); + } + + @Override + protected void finalize() throws Throwable { + cleanup(); + super.finalize(); + } + + } + + /** + * An interface for a sorted set factory + * + * @param + * type of the factory + */ + public interface FileSortedSetFactory { + /** + * 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, E from, E to); + + /** + * factory method + * + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @return a new instance + */ + FileSortedMap newInstance(SortedSetFileHandler handler, boolean persisted); + + /** + * Factory method + * + * @param comparator + * the key comparator + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @return a new instance + */ + FileSortedMap newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted); + + /** + * Factory method + * + * @param comparator + * the key comparator + * @param rewriteStrategy + * the collision rewrite strategy + * @param handler + * the sorted set file handler + * @param persisted + * a persisted boolean flag + * @return a new instance + */ + FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted); + + /** + * Create an unpersisted sorted set (still in memory) + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + * @return a new instance + */ + FileSortedMap newInstance(RewritableSortedMap set, SortedSetFileHandler handler); + + /** + * factory method + * + * @param set + * the sorted set + * @param handler + * the sorted set file handler + * @param persist + * a persisted boolean flag + * @return a new instance + * @throws IOException + * for problems with read/write + */ + FileSortedMap newInstance(RewritableSortedMap set, SortedSetFileHandler handler, boolean persist) throws IOException; + } + + /** + * A sorted set input stream + * + * @param + * type of the stream + */ + public interface SortedSetInputStream extends AutoCloseable { + E readObject() throws IOException; + + int readSize() throws IOException; + + void close(); + } + + /** + * A sorted set output stream + * + * @param + * type of the stream + */ + public interface SortedSetOutputStream extends AutoCloseable { + void writeObject(E obj) 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 SortedSetFileHandler { + /** + * 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 set output stream + * @throws IOException + * for problems with read/write + */ + OutputStream getOutputStream() throws IOException; + + /** + * Get the persistent verification options + * + * @return the persistent verification options + */ + PersistOptions getPersistOptions(); + + long getSize(); + + void deleteFile(); + } + + /** + * A factory that will provide the input stream and output stream to the same underlying file. + * + */ + public interface TypedSortedSetFileHandler { + /** + * Return the input stream + * + * @return the input stream + * @throws IOException + * for problems with read/write + */ + SortedSetInputStream getInputStream() throws IOException; + + /** + * Return the output stream + * + * @return the sorted set output stream + * @throws IOException + * for problems with read/write + */ + SortedSetOutputStream getOutputStream() throws IOException; + + /** + * Get the persistent verification options + * + * @return persistent verification options + */ + 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 subset. + * + */ + public interface BoundedTypedSortedSetFileHandler extends TypedSortedSetFileHandler { + /** + * Return the input stream + * + * @return the input stream + * @param start + * start point + * @param end + * end point + * @throws IOException + * for problems with read/write + */ + SortedSetInputStream getInputStream(E start, E end) throws IOException; + } + + /** + * A default implementation for a bounded typed sorted set + */ + public class DefaultBoundedTypedSortedSetFileHandler implements BoundedTypedSortedSetFileHandler { + @Override + public SortedSetInputStream getInputStream(E start, E end) throws IOException { + if (handler instanceof FileSortedMap.BoundedTypedSortedSetFileHandler) { + return ((BoundedTypedSortedSetFileHandler) handler).getInputStream(start, end); + } else { + return new BoundedInputStream(handler.getInputStream(), start, end); + } + } + + @Override + public SortedSetInputStream getInputStream() throws IOException { + return handler.getInputStream(); + } + + @Override + public SortedSetOutputStream getOutputStream() throws IOException { + return handler.getOutputStream(); + } + + @Override + public 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 SortedSetInputStream { + private final SortedSetInputStream delegate; + private final E from; + private final E to; + + public BoundedInputStream(SortedSetInputStream stream, E from, E to) { + this.delegate = stream; + this.from = from; + this.to = to; + } + + @Override + public E readObject() throws IOException { + E o = delegate.readObject(); + while ((o != null) && (from != null) && (compare(o, from) < 0)) { + o = delegate.readObject(); + } + if (o == null || (to != null && compare(o, to) >= 0)) { + return null; + } else { + return o; + } + } + + @Override + public int readSize() throws IOException { + return delegate.readSize(); + } + + @Override + public void close() { + delegate.close(); + } + } + + public static class PersistOptions { + private boolean verifySize = true; + private boolean verifyElements = true; + private int numElementsToVerify = 100; + + public PersistOptions() {} + + public PersistOptions(boolean verifySize, boolean verifyElements) { + this.verifySize = verifySize; + this.verifyElements = verifyElements; + } + + public PersistOptions(boolean verifySize, boolean verifyElements, int numElementsToVerify) { + this(verifySize, verifyElements); + this.numElementsToVerify = numElementsToVerify; + } + + public boolean isVerifySize() { + return verifySize; + } + + public boolean isVerifyElements() { + return verifyElements; + } + + public int getNumElementsToVerify() { + return numElementsToVerify; + } + } +} 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..934505e5a32 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/HdfsBackedSortedMap.java @@ -0,0 +1,292 @@ +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; + +public class HdfsBackedSortedMap extends BufferedFileBackedSortedMap { + private static final Logger log = Logger.getLogger(HdfsBackedSortedMap.class); + private static final String FILENAME_PREFIX = "SortedSetFile."; + + public static class Builder,E> extends BufferedFileBackedSortedMap.Builder { + private List ivaratorCacheDirs; + private String uniqueSubPath; + private FileSortedMap.PersistOptions persistOptions; + + public Builder() { + // change the default buffer persist threshold + withBufferPersistThreshold(10000); + } + + @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(FileSortedMap.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); + this.handlerFactories = createFileHandlerFactories(builder.ivaratorCacheDirs, builder.uniqueSubPath, builder.persistOptions); + // for each of the handler factories, check to see if there are any existing files we should load + for (SortedSetFileHandlerFactory handlerFactory : handlerFactories) { + // Note: All of the file handler factories created by 'createFileHandlerFactories' are SortedSetHdfsFileHandlerFactories + if (handlerFactory instanceof SortedSetHdfsFileHandlerFactory) { + SortedSetHdfsFileHandlerFactory hdfsHandlerFactory = (SortedSetHdfsFileHandlerFactory) handlerFactory; + FileSystem fs = hdfsHandlerFactory.getFs(); + int count = 0; + + // if the directory already exists, load up this sorted set 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++; + addSet(setFactory.newInstance(comparator, new SortedSetHdfsFileHandler(fs, file.getPath(), builder.persistOptions), true)); + } + } + } + + hdfsHandlerFactory.setFileCount(count); + } + } + } + } + + private static List createFileHandlerFactories(List ivaratorCacheDirs, String uniqueSubPath, + FileSortedMap.PersistOptions persistOptions) { + List fileHandlerFactories = new ArrayList<>(); + for (IvaratorCacheDir ivaratorCacheDir : ivaratorCacheDirs) { + fileHandlerFactories.add(new SortedSetHdfsFileHandlerFactory(ivaratorCacheDir, uniqueSubPath, persistOptions)); + } + return fileHandlerFactories; + } + + @Override + public void clear() { + // This will be a new ArrayList<>() containing the same FileSortedSets + List> sortedSets = super.getSets(); + // Clear will call clear on each of the FileSortedSets, clear the container, and null the buffer + super.clear(); + // We should still be able to access the FileSortedSet objects to get their handler because we + // have a copy of the object in 'sortedSets' + for (FileSortedMap fss : sortedSets) { + if (fss.isPersisted() && fss.handler instanceof SortedSetHdfsFileHandler) { + ((SortedSetHdfsFileHandler) fss.handler).deleteFile(); + } + } + } + + public static class SortedSetHdfsFileHandlerFactory implements SortedSetFileHandlerFactory { + final private IvaratorCacheDir ivaratorCacheDir; + private String uniqueSubPath; + private int fileCount = 0; + private FileSortedMap.PersistOptions persistOptions; + + public SortedSetHdfsFileHandlerFactory(IvaratorCacheDir ivaratorCacheDir, String uniqueSubPath, FileSortedMap.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 setFileCount(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.SortedSetFileHandler 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 SortedSetHdfsFileHandler(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 SortedSetHdfsFileHandler implements FileSortedMap.SortedSetFileHandler { + private FileSystem fs; + private Path file; + private FileSortedMap.PersistOptions persistOptions; + + public SortedSetHdfsFileHandler(FileSystem fs, Path file, FileSortedMap.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 FileSortedMap.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/MultiSetBackedSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiSetBackedSortedMap.java new file mode 100644 index 00000000000..ee2c83b4da5 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiSetBackedSortedMap.java @@ -0,0 +1,347 @@ +package datawave.query.util.sortedmap; + +import datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; +import org.apache.commons.lang3.builder.EqualsBuilder; + +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.SortedSet; + +/* + * This is a sorted set that is backed by multiple underlying sorted sets. It is assumed that the underlying + * sorted sets contain the same type of underlying value, and they use the same comparator. The rewrite + * strategy will be used if the underlying sorted sets are RewriteableSortedSet implementations. + */ +public class MultiSetBackedSortedMap extends AbstractSet implements RewritableSortedSet { + protected List> sets = new ArrayList<>(); + protected Comparator comparator = null; + protected RewriteStrategy rewriteStrategy = null; + + public MultiSetBackedSortedMap() {} + + public MultiSetBackedSortedMap(List> sets) { + for (SortedSet set : sets) { + addSet(set); + } + } + + public void addSet(SortedSet set) { + if (sets.isEmpty()) { + updateConfiguration(set); + } else { + verifyConfiguration(set); + } + sets.add(set); + } + + private void updateConfiguration(SortedSet set) { + comparator = getComparator(set); + rewriteStrategy = getRewriteStrategy(set); + } + + private void verifyConfiguration(SortedSet set) { + if (!(new EqualsBuilder().append(getClass(comparator), getClass(getComparator(set))) + .append(getClass(rewriteStrategy), getClass(getRewriteStrategy(set))).isEquals())) { + throw new IllegalArgumentException("Set being added does not match the comparator and rewriteStrategy of the existing sets"); + } + } + + private Class getClass(Object obj) { + return (obj == null ? null : obj.getClass()); + } + + private RewriteStrategy getRewriteStrategy(SortedSet set) { + if (set instanceof RewritableSortedSet) { + return ((RewritableSortedSet) set).getRewriteStrategy(); + } + return null; + } + + private Comparator getComparator(SortedSet set) { + return (Comparator) (set.comparator()); + } + + /** + * Get the underlying sets + * + * @return the sets + */ + public List> getSets() { + return sets; + } + + /** + * Return the size of this set. NOTE that this is somewhat expensive as we require iterating over the sets to determine the true value (see + * MergeSortIterator); + */ + @Override + public int size() { + int size = 0; + for (@SuppressWarnings("unused") + E t : this) { + size++; + } + return size; + } + + @Override + public boolean isEmpty() { + if (sets == null) { + return true; + } + for (SortedSet set : sets) { + if (set != null && !set.isEmpty()) { + return false; + } + } + return true; + } + + @Override + public boolean contains(Object o) { + for (SortedSet set : sets) { + if (set.contains(o)) { + return true; + } + } + return false; + } + + @Override + public Iterator iterator() { + return new MergeSortIterator(); + } + + @Override + public boolean add(E e) { + throw new UnsupportedOperationException("Please use addSet to add a sorted set or add this item to one of the existing underlying sets"); + } + + @Override + public boolean remove(Object o) { + boolean removed = false; + for (SortedSet set : sets) { + if (set.remove(o)) { + removed = true; + } + } + return removed; + } + + @Override + public void clear() { + for (SortedSet set : this.sets) { + try { + set.clear(); + } catch (Exception e) { + // error clearing sorted set + // possibility of FileNotFoundException, etc being + // caught and re-thrown as an exception + } + } + this.sets.clear(); + } + + @Override + public Comparator comparator() { + return comparator; + } + + @Override + public RewritableSortedSet subSet(E fromElement, E toElement) { + MultiSetBackedSortedMap subSet = new MultiSetBackedSortedMap<>(); + for (SortedSet set : sets) { + subSet.addSet(set.subSet(fromElement, toElement)); + } + return subSet; + } + + @Override + public RewritableSortedSet headSet(E toElement) { + MultiSetBackedSortedMap subSet = new MultiSetBackedSortedMap<>(); + for (SortedSet set : sets) { + subSet.addSet(set.headSet(toElement)); + } + return subSet; + } + + @Override + public RewritableSortedSet tailSet(E fromElement) { + MultiSetBackedSortedMap subSet = new MultiSetBackedSortedMap<>(); + for (SortedSet set : sets) { + subSet.addSet(set.tailSet(fromElement)); + } + return subSet; + } + + @Override + public E first() throws NoSuchElementException { + if (sets == null || sets.isEmpty()) { + throw new NoSuchElementException("No elements in input sets"); + } + SortedSet firstSet = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); + for (SortedSet set : sets) { + if (set != null && !set.isEmpty()) { + E s = set.first(); + firstSet.add(s); + } + } + if (firstSet.isEmpty()) { + throw new NoSuchElementException("No elements in input sets"); + } + return firstSet.first(); + } + + @Override + public E last() throws NoSuchElementException { + if (sets == null || sets.isEmpty()) { + throw new NoSuchElementException("No elements in input sets"); + } + SortedSet lastSet = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); + for (SortedSet set : sets) { + if (set != null && !set.isEmpty()) { + E s = set.last(); + lastSet.add(s); + } + } + if (lastSet.isEmpty()) { + throw new NoSuchElementException("No elements in input sets"); + } + return lastSet.last(); + } + + @Override + public RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; + } + + @Override + public E get(E e) { + return null; + } + + /** + * This is an iterator that will return a sorted set of items (no dups) from an underlying set of sorted sets. + */ + public class MergeSortIterator implements Iterator { + + private List> iterators = new ArrayList<>(); + private List lastList = new ArrayList<>(); + private boolean[] finished = null; + private RewritableSortedSet set = null; + private boolean populated = false; + private E next = null; + private List> nextIterators = new ArrayList<>(); + + public MergeSortIterator() { + for (SortedSet set : sets) { + Iterator it = set.iterator(); + iterators.add(it); + nextIterators.add(it); + lastList.add(null); + } + this.set = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); + this.finished = new boolean[iterators.size()]; + } + + @Override + public boolean hasNext() { + if (!set.isEmpty()) { + return true; + } + for (Iterator it : nextIterators) { + if (it != null && it.hasNext()) { + return true; + } + } + return false; + } + + @Override + public E next() { + populate(); + if (!populated) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + return next; + } + + @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(E o1, E o2) { + if (o1 == null) { + return o2 == null; + } else if (o2 == null) { + return false; + } else { + if (set.comparator() == null) { + return o1.equals(o2); + } else { + return set.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()) { + E val = it.next(); + lastList.set(i, val); + set.add(val); + } else { + lastList.set(i, null); + finished[i] = true; + } + } + } + + if (!set.isEmpty()) { + next = set.first(); + set.remove(next); + for (int i = 0; i < iterators.size(); i++) { + if (!finished[i] && equals(next, 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 set 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/SortedByteMapBuffer.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedByteMapBuffer.java new file mode 100644 index 00000000000..1b73d3e8d13 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedByteMapBuffer.java @@ -0,0 +1,463 @@ +package datawave.query.util.sortedmap; + +import datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; + +import java.util.AbstractSet; +import java.util.Arrays; +import java.util.Comparator; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.SortedSet; + +/** + * This is an sorted set of byte arrays which keeps one large byte array as the backing store and a separate array of indices and sizes in sorted value order. + * The reason for building this sorted set structure is to minimize memory usage and object creation while maintaining fast add capabilities. + * + */ +public class SortedByteMapBuffer extends AbstractSet implements SortedSet { + public static final int AVERAGE_VALUE_SIZE = 32; + public static final int DEFAULT_BUFFER_SIZE = 64; + + protected byte[] data = null; + protected int[] sortedDataIndicies = null; + protected byte[] sortedDataSizes = null; + protected int size = 0; + protected int bufferSize = 0; + protected int modCount = 0; + + public SortedByteMapBuffer() { + this(DEFAULT_BUFFER_SIZE); + } + + public SortedByteMapBuffer(int capacity) { + this.data = new byte[capacity * AVERAGE_VALUE_SIZE]; + this.sortedDataIndicies = new int[capacity]; + this.sortedDataSizes = new byte[capacity]; + } + + /************************** Overridden methods *************************/ + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return size == 0; + } + + @Override + public boolean contains(Object o) { + if (o instanceof byte[]) { + return binarySearch((byte[]) o) >= 0; + } + return false; + } + + @Override + public Iterator iterator() { + return new SortedByteSetBufferIterator(); + } + + @Override + public boolean add(byte[] e) { + if (e.length > Byte.MAX_VALUE) { + throw new IllegalArgumentException("SortedByteSetBuffer does not support data elements greater than " + Byte.MAX_VALUE + " bytes"); + } + int index = binarySearch(e); + if (index < 0) { + add(-1 - index, e); + return true; + } + return false; + } + + @Override + public boolean remove(Object o) { + if (!(o instanceof byte[])) { + return false; + } + int index = binarySearch((byte[]) o); + if (index >= 0) { + remove(index); + return true; + } + return false; + } + + @Override + public void clear() { + modCount++; + size = 0; + bufferSize = 0; + } + + @Override + public Comparator comparator() { + return new ByteArrayComparator(); + } + + @Override + public SortedSet subSet(byte[] fromElement, byte[] toElement) { + return new SortedByteSubSetBuffer(fromElement, toElement); + } + + @Override + public SortedSet headSet(byte[] toElement) { + return new SortedByteSubSetBuffer(null, toElement); + } + + @Override + public SortedSet tailSet(byte[] fromElement) { + return new SortedByteSubSetBuffer(fromElement, null); + } + + @Override + public byte[] first() { + if (size == 0) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + return get(0); + } + + @Override + public byte[] last() { + if (size == 0) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + return get(size - 1); + } + + /* Other public methods */ + + public byte[] get(int index) { + if (index < 0 || index >= size) { + throw new IndexOutOfBoundsException("index is out of range"); + } + int dataIndex = sortedDataIndicies[index]; + int dataSize = sortedDataSizes[index]; + byte[] term = new byte[dataSize]; + System.arraycopy(data, dataIndex, term, 0, dataSize); + return term; + } + + /* The protected stuff */ + + protected void checkCapacity(int plusSize) { + int plusLen = 1; + int minCapacity = bufferSize + plusSize; + int oldCapacity = data.length; + if (minCapacity > oldCapacity) { + int newCapacity = (oldCapacity * 3) / 2 + 1; + if (newCapacity < minCapacity) { + newCapacity = minCapacity; + } + plusLen = ((newCapacity - oldCapacity) / 32) + 1; + data = Arrays.copyOf(data, newCapacity); + } + int minLen = size + plusLen; + int oldLen = sortedDataIndicies.length; + if (minLen > oldLen) { + int newLen = (oldLen * 3) / 2 + 1; + if (newLen < minLen) { + newLen = minLen; + } + sortedDataIndicies = Arrays.copyOf(sortedDataIndicies, newLen); + sortedDataSizes = Arrays.copyOf(sortedDataSizes, newLen); + } + } + + protected void add(int index, byte[] value) { + modCount++; + checkCapacity(value.length); + int dataIndex = bufferSize; + System.arraycopy(value, 0, data, bufferSize, value.length); + System.arraycopy(sortedDataIndicies, index, sortedDataIndicies, index + 1, size - index); + System.arraycopy(sortedDataSizes, index, sortedDataSizes, index + 1, size - index); + sortedDataIndicies[index] = dataIndex; + sortedDataSizes[index] = (byte) (value.length); + bufferSize += value.length; + size++; + } + + protected void remove(int index) { + modCount++; + int dataIndex = sortedDataIndicies[index]; + int dataSize = sortedDataSizes[index]; + bufferSize -= dataSize; + size--; + System.arraycopy(data, dataIndex + dataSize, data, dataIndex, bufferSize - dataIndex); + System.arraycopy(sortedDataSizes, index + 1, sortedDataSizes, index, size - index); + System.arraycopy(sortedDataIndicies, index + 1, sortedDataIndicies, index, size - index); + for (int i = 0; i < size; i++) { + if (sortedDataIndicies[i] > dataIndex) { + sortedDataIndicies[i] -= dataSize; + } + } + } + + protected static int compare(byte[] data, int dataIndex, int dataSize, byte[] term) { + int minSize = dataSize; + if (term.length < minSize) + minSize = term.length; + int comparison = 0; + for (int i = 0; i < minSize; i++) { + comparison = data[dataIndex + i] - term[i]; + if (comparison != 0) + break; + } + if (comparison == 0) { + if (minSize < dataSize) { + comparison = 1; + } else if (minSize < term.length) { + comparison = -1; + } + } + return comparison; + } + + /** + * A binary search of the byte array based on a sorted index array + * + * @param term + * aterm + * @return location result of the search + */ + protected int binarySearch(byte[] term) { + return binarySearch(term, 0, this.size - 1); + } + + protected int binarySearch(byte[] term, int start, int end) { + while (start <= end) { + int middle = (start + end) >>> 1; + int comparison = compare(data, sortedDataIndicies[middle], sortedDataSizes[middle], term); + + if (comparison < 0) + start = middle + 1; + else if (comparison > 0) + end = middle - 1; + else + return middle; + } + // return a negative index if not found so we know where it should go + return -(start + 1); + } + + protected class SortedByteSetBufferIterator implements Iterator { + protected int index = 0; + protected int end = 0; + protected int expectedModCount = -1; + protected int last = -1; + + public SortedByteSetBufferIterator() { + this(0, size); + } + + public SortedByteSetBufferIterator(int start, int end) { + this.expectedModCount = modCount; + this.index = start; + this.end = end; + } + + final void checkModCount() { + if (modCount != expectedModCount) + throw new ConcurrentModificationException(); + } + + @Override + public boolean hasNext() { + checkModCount(); + return index < end; + } + + @Override + public byte[] next() { + if (!hasNext()) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + checkModCount(); + last = index; + byte[] entry = get(index++); + checkModCount(); + return entry; + } + + @Override + public void remove() { + checkModCount(); + if (last >= 0) { + SortedByteMapBuffer.this.remove(last); + index--; + end--; + expectedModCount = modCount; + } else { + throw new IllegalStateException("next needs to be called first"); + } + } + } + + protected class SortedByteSubSetBuffer extends AbstractSet implements SortedSet { + protected byte[] from; + protected byte[] to; + protected int expectedModCount = -1; + protected int[] range = null; + + public SortedByteSubSetBuffer(byte[] from, byte[] to) { + if (from != null && to != null && comparator().compare(from, to) > 0) { + throw new IllegalArgumentException("The start is greater than the end"); + } + this.from = from; + this.to = to; + } + + @Override + public Comparator comparator() { + return SortedByteMapBuffer.this.comparator(); + } + + @Override + public SortedSet subSet(byte[] fromElement, byte[] toElement) { + if ((from != null && comparator().compare(fromElement, from) < 0) || (to != null && comparator().compare(to, toElement) < 0)) { + throw new IllegalArgumentException("Cannot create subset outside of the range of this subset"); + } + return SortedByteMapBuffer.this.subSet(fromElement, toElement); + } + + @Override + public SortedSet headSet(byte[] toElement) { + return subSet(from, toElement); + } + + @Override + public SortedSet tailSet(byte[] fromElement) { + return subSet(fromElement, to); + } + + @Override + public byte[] first() { + int[] range = getRange(); + if (range == null) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + return get(range[0]); + } + + @Override + public byte[] last() { + int[] range = getRange(); + if (range == null) { + QueryException qe = new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR); + throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); + } + return get(range[1]); + } + + @Override + public Iterator iterator() { + int[] range = getRange(); + if (range == null) { + return new SortedByteSetBufferIterator(0, 0); + } else { + return new SortedByteSetBufferIterator(range[0], range[1]); + } + } + + @Override + public int size() { + int[] range = getRange(); + if (range == null) { + return 0; + } else { + return range[1] - range[0] + 1; + } + } + + @Override + public boolean contains(Object o) { + if (!(o instanceof byte[])) { + return false; + } + int[] range = getRange(); + if (range == null) { + return false; + } + boolean contains = (binarySearch((byte[]) o, range[0], range[1]) >= 0); + checkModCount(); + return contains; + } + + @Override + public boolean add(byte[] e) { + if ((from != null && comparator().compare(e, from) < 0) || (to != null && comparator().compare(e, to) >= 0)) { + throw new IllegalArgumentException("Cannot add element outside of subset range"); + } + return SortedByteMapBuffer.this.add(e); + } + + @Override + public boolean remove(Object o) { + if (contains(o)) { + return SortedByteMapBuffer.this.remove(o); + } + return false; + } + + /*** + * Get the range of elements in the SortedByteSetBuffer + * + * @return int[] {firstIndex, lastIndex} + */ + protected int[] getRange() { + if (expectedModCount != modCount) { + expectedModCount = modCount; + if (SortedByteMapBuffer.this.isEmpty()) { + range = null; + } else { + + // find the first entry + int start = (from == null ? 0 : binarySearch(from)); + if (start < 0) { + start = -1 - start; + } + + // if the start is past the end, then we have no range + if (start == SortedByteMapBuffer.this.size()) { + range = null; + } else { + + // find the last entry + int end = (to == null ? SortedByteMapBuffer.this.size() : binarySearch(to)); + if (end < 0) { + end = -1 - end; + } + // since the end is exclusive, go to the previous element + end--; + + // if the start is not before the end, then no range + if (start >= end) { + range = null; + } else { + range = new int[] {start, end}; + } + } + } + checkModCount(); + } + return range; + } + + final void checkModCount() { + if (modCount != expectedModCount) + throw new ConcurrentModificationException(); + } + + } + +} 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..7c259128312 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedMapTempFileHandler.java @@ -0,0 +1,64 @@ +package datawave.query.util.sortedmap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A sorted set file handler factory that uses temporary local based files. + * + * + * + */ +public class SortedTempFileHandler implements FileSortedMap.SortedSetFileHandler { + private final FileSystem fs; + private final File file; + private final Path path; + + public SortedSetTempFileHandler() 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 FileSortedMap.PersistOptions getPersistOptions() { + return new FileSortedMap.PersistOptions(); + } + + @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..c568993e9fb --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumentTransforms.java @@ -0,0 +1,63 @@ +package datawave.query.util.sortedmap.rfile; + +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; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.Map; + +public class KeyValueByteDocumentTransforms { + + 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) throws IOException { + if (doc == null) { + return null; + } + DocumentSerializer serializer = new KryoDocumentSerializer(false, true); + byte[] document = serializer.serialize(doc); + return new Value(document); + } + + public static Document valueToDocument(Value value) throws IOException { + if (value == null) { + return null; + } + DocumentDeserializer deserializer = new KryoDocumentDeserializer(); + Document document = deserializer.deserialize(new ByteArrayInputStream(value.get())); + return document; + } + + public static Map.Entry keyValueToByteDocument(Map.Entry keyValue) throws IOException { + if (keyValue == null) { + return null; + } + return new UnmodifiableMapEntry(keyToByte(keyValue.getKey()), valueToDocument(keyValue.getValue())); + } + + public static Map.Entry byteDocumentToKeyValue(Map.Entry byteKey) throws IOException { + if (byteKey == null) { + return null; + } + return new UnmodifiableMapEntry(byteToKey(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..94c284f797a --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentInputStream.java @@ -0,0 +1,32 @@ +package datawave.query.util.sortedmap.rfile; + +import datawave.query.attributes.Document; +import org.apache.accumulo.core.data.Key; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; +import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.keyValueToByteDocument; + +public class RFileByteDocumentInputStream extends RFileKeyValueInputStreamBase> { + + public RFileByteDocumentInputStream(InputStream inputStream, long length) throws IOException { + super(inputStream, length); + } + + public RFileByteDocumentInputStream(InputStream inputStream, long length, Key start, Key end) throws IOException { + super(inputStream, length, start, end); + } + + public RFileByteDocumentInputStream(InputStream inputStream, long length, Map.Entry start, Map.Entry end) + throws IOException { + super(inputStream, length, byteDocumentToKeyValue(start), byteDocumentToKeyValue(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..20a690c6ea1 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileByteDocumentOutputStream.java @@ -0,0 +1,21 @@ +package datawave.query.util.sortedmap.rfile; + +import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.FileSortedMap; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; + +public class RFileByteDocumentOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedSetOutputStream> { + public RFileByteDocumentOutputStream(OutputStream stream) throws IOException { + super(stream); + } + + @Override + public void writeObject(Map.Entry obj) throws IOException { + writeKeyValue(byteDocumentToKeyValue(obj)); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyInputStream.java new file mode 100644 index 00000000000..0254435ae93 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyInputStream.java @@ -0,0 +1,26 @@ +package datawave.query.util.sortedmap.rfile; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +public class RFileKeyInputStream extends RFileKeyValueInputStreamBase { + + public RFileKeyInputStream(InputStream inputStream, long length) throws IOException { + super(inputStream, length); + } + + public RFileKeyInputStream(InputStream inputStream, long length, Key start, Key end) throws IOException { + super(inputStream, length, start, end); + } + + @Override + public Key readObject() throws IOException { + Map.Entry obj = readKeyValue(); + return (obj == null ? null : obj.getKey()); + } + +} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java new file mode 100644 index 00000000000..cb7416d395b --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java @@ -0,0 +1,18 @@ +package datawave.query.util.sortedmap.rfile; + +import datawave.query.util.sortedmap.FileSortedMap; +import org.apache.accumulo.core.data.Key; + +import java.io.IOException; +import java.io.OutputStream; + +public class RFileKeyOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedSetOutputStream { + public RFileKeyOutputStream(OutputStream stream) throws IOException { + super(stream); + } + + @Override + public void writeObject(Key o) throws IOException { + writeKeyValue(o, EMPTY_VALUE); + } +} 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..3c19feb6e1b --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStream.java @@ -0,0 +1,28 @@ +package datawave.query.util.sortedmap.rfile; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +public class RFileKeyValueInputStream extends RFileKeyValueInputStreamBase> { + + 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); + } + + public RFileKeyValueInputStream(InputStream inputStream, long length, Map.Entry start, Map.Entry 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..12862f683b6 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueInputStreamBase.java @@ -0,0 +1,87 @@ +package datawave.query.util.sortedmap.rfile; + +import datawave.query.util.sortedmap.FileSortedMap; +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; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.Map; + +public abstract class RFileKeyValueInputStreamBase implements FileSortedMap.SortedSetInputStream { + 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; + } + + public RFileKeyValueInputStreamBase(InputStream inputStream, long length, Map.Entry start, Map.Entry end) throws IOException { + this(inputStream, length); + this.start = (start == null ? null : start.getKey()); + this.end = (end == null ? null : end.getKey()); + } + + 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 (RFileKeyOutputStream.SizeKeyUtil.isSizeKey(next.getKey())) { + size = RFileKeyOutputStream.SizeKeyUtil.getSize(next.getKey()); + next = null; + } + return next; + } + return null; + } + + @Override + 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(RFileKeyOutputStream.SizeKeyUtil.SIZE_ROW)).build(); + iterator = reader.iterator(); + size = RFileKeyOutputStream.SizeKeyUtil.getSize(iterator.next().getKey()); + } + return size; + } + + @Override + 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..35a44852974 --- /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 datawave.query.util.sortedmap.FileSortedMap; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +public class RFileKeyValueOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedSetOutputStream> { + public RFileKeyValueOutputStream(OutputStream stream) throws IOException { + super(stream); + } + + @Override + public void writeObject(Map.Entry obj) throws IOException { + writeKeyValue(obj); + } +} 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..fa19ef85986 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyValueOutputStreamBase.java @@ -0,0 +1,55 @@ +package datawave.query.util.sortedmap.rfile; + +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; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +public class RFileKeyValueOutputStreamBase { + private RFileWriter writer; + static final Value EMPTY_VALUE = new Value(new byte[0]); + + public RFileKeyValueOutputStreamBase(OutputStream stream) throws IOException { + super(); + this.writer = RFile.newWriter().to(stream).withVisibilityCacheSize(10).build(); + } + + public void writeKeyValue(Key key, Value value) throws IOException { + writer.append(key, value); + } + + public void writeKeyValue(Map.Entry keyValue) throws IOException { + writer.append(keyValue.getKey(), keyValue.getValue()); + } + + public void writeSize(int i) throws IOException { + writeKeyValue(SizeKeyUtil.getKey(i), EMPTY_VALUE); + } + + public void close() throws IOException { + 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 a780f5be2ca..895a2cbae9f 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 @@ -440,6 +440,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 ff07a788854..afee2b09606 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 @@ -59,34 +59,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 @@ -97,11 +148,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 @@ -112,11 +163,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 @@ -127,11 +178,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 @@ -142,11 +193,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 @@ -157,11 +208,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 @@ -172,11 +223,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 @@ -187,11 +238,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 @@ -202,26 +253,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..5487ae7162c --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java @@ -0,0 +1,89 @@ +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( + logic.isIvaratorPersistVerify(), + logic.isIvaratorPersistVerify(), + logic.getIvaratorPersistVerifyCount())) + .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 ba0354c12e1..8414dbad2f3 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,56 @@ 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); } - private List getUniqueDocuments(List documents) { + 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 +545,7 @@ private List getUniqueDocumentsWithUpdateConfigCalls(List do return docs; } - private void assertOrderedFieldValues() { + protected void assertOrderedFieldValues() { try { UniqueTransform uniqueTransform = getUniqueTransform(); for (Document d : inputDocuments) { @@ -521,53 +557,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); + } + } + + protected void updateUniqueTransform(UniqueTransform uniqueTransform) { + uniqueTransform.updateConfig(uniqueFields); } - private void updateUniqueTransform(UniqueTransform uniqueTransform) { - uniqueTransform.updateConfig(uniqueFields, null); + protected InputDocumentBuilder givenInputDocument() { + return new InputDocumentBuilder("", 0); } - private InputDocumentBuilder givenInputDocument() { - return new InputDocumentBuilder(); + protected InputDocumentBuilder givenInputDocument(String cq) { + return new InputDocumentBuilder(cq, 0); } - private InputDocumentBuilder givenInputDocument(String docKey) { - return new InputDocumentBuilder(docKey); + protected InputDocumentBuilder givenInputDocument(long ts) { + return new InputDocumentBuilder("", ts); } - private ExpectedOrderedFieldValuesBuilder givenExpectedOrderedFieldValues() { + protected InputDocumentBuilder givenInputDocument(String docKey, long ts) { + return new InputDocumentBuilder(docKey, ts); + } + + protected ExpectedOrderedFieldValuesBuilder givenExpectedOrderedFieldValues() { return new ExpectedOrderedFieldValuesBuilder(); } - private class InputDocumentBuilder { + protected class InputDocumentBuilder { private final Document document; - InputDocumentBuilder() { - this.document = new Document(); - inputDocuments.add(document); - } - - @SuppressWarnings({"UnusedReturnValue", "SameParameterValue"}) - InputDocumentBuilder(String docKey) { - - 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 +634,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 +653,7 @@ InputDocumentBuilder isExpectedToBeUnique() { } } - private class ExpectedOrderedFieldValuesBuilder { + protected class ExpectedOrderedFieldValuesBuilder { private Multimap fieldValues = TreeMultimap.create(); @@ -624,13 +668,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 898b6481381..d5c5506aba3 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 @@ -53,9 +53,12 @@ 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 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; protected static String normalizeColVal(Map.Entry colVal) { switch (colVal.getKey()) { @@ -94,81 +97,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); bw.addMutation(mutation); @@ -385,9 +400,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(); @@ -608,116 +624,130 @@ 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); } // 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(); @@ -985,31 +1015,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++; @@ -1022,7 +1054,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/BufferedFileBackedByteArraySortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedSetTest.java new file mode 100644 index 00000000000..3ecf97a7d78 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedSetTest.java @@ -0,0 +1,25 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.FileSerializableSortedSet; +import datawave.query.util.sortedset.FileSortedSet; + +import java.util.Comparator; + +public class BufferedFileBackedByteArraySortedSetTest extends BufferedFileBackedSortedSetTest { + + @Override + public byte[] createData(byte[] values) { + return values; + } + + @Override + public Comparator getComparator() { + return new ByteArrayComparator(); + } + + @Override + public FileSortedMap.FileSortedMapFactory getFactory() { + return new FileSerializableSortedSet.Factory(); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java new file mode 100644 index 00000000000..fc0e9c6992d --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java @@ -0,0 +1,84 @@ +package datawave.query.util.sortedmap; + +import datawave.data.type.LcNoDiacriticsType; +import datawave.query.attributes.Document; +import datawave.query.composite.CompositeMetadata; +import datawave.query.predicate.EventDataQueryFieldFilter; +import datawave.query.predicate.KeyProjection; +import datawave.query.util.TypeMetadata; +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.FileSortedSet; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class BufferedFileBackedByteDocumentSortedSetTest extends BufferedFileBackedRewritableSortedSetTest { + + private Comparator> keyComparator = new Comparator<>() { + private Comparator comparator = new ByteArrayComparator(); + + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return comparator.compare(o1.getKey(), o2.getKey()); + } + }; + + private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator = new RewritableSortedSetImpl.RewriteStrategy<>() { + @Override + public boolean rewrite(Map.Entry original, Map.Entry update) { + int comparison = keyComparator.compare(original, update); + if (comparison == 0) { + long ts1 = original.getValue().get(Document.DOCKEY_FIELD_NAME).getTimestamp(); + long ts2 = update.getValue().get(Document.DOCKEY_FIELD_NAME).getTimestamp(); + return (ts2 > ts1); + } + return comparison < 0; + } + }; + + @Override + public RewritableSortedSet.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(new KeyProjection())); + return doc; + } + + @Override + public void testFullEquality(Map.Entry expected, Map.Entry value) { + assertEquals(0, keyComparator.compare(expected, value)); + assertEquals(expected.getValue().get(Document.DOCKEY_FIELD_NAME), value.getValue().get(Document.DOCKEY_FIELD_NAME)); + } + + @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/BufferedFileBackedKeySortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedSetTest.java new file mode 100644 index 00000000000..cd677fd631f --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedSetTest.java @@ -0,0 +1,26 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.FileKeySortedSet; +import datawave.query.util.sortedset.FileSortedSet; +import org.apache.accumulo.core.data.Key; + +import java.util.Comparator; + +public class BufferedFileBackedKeySortedSetTest extends BufferedFileBackedSortedSetTest { + + @Override + public Key createData(byte[] values) { + return new Key(values); + } + + @Override + public Comparator getComparator() { + return null; + } + + @Override + public FileSortedMap.FileSortedMapFactory getFactory() { + return new FileKeySortedSet.Factory(); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java new file mode 100644 index 00000000000..46886abe50a --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java @@ -0,0 +1,63 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.FileSortedSet; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; + +import java.util.Comparator; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class BufferedFileBackedKeyValueSortedSetTest extends BufferedFileBackedRewritableSortedSetTest { + + private Comparator> keyComparator = new Comparator<>() { + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return o1.getKey().compareTo(o2.getKey()); + } + }; + + private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator = new RewritableSortedSetImpl.RewriteStrategy<>() { + @Override + public boolean rewrite(Map.Entry original, Map.Entry update) { + int comparison = original.getKey().compareTo(update.getKey()); + if (comparison == 0) { + comparison = original.getValue().compareTo(update.getValue()); + } + return comparison < 0; + } + }; + + @Override + public RewritableSortedSet.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 testFullEquality(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/BufferedFileBackedRewritableSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedSetTest.java new file mode 100644 index 00000000000..c9ba805aa0f --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedSetTest.java @@ -0,0 +1,111 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.BufferedFileBackedSortedSet; +import datawave.query.util.sortedset.FileSortedSet; +import datawave.query.util.sortedset.SortedSetTempFileHandler; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; + +public abstract class BufferedFileBackedRewritableSortedSetTest extends BufferedFileBackedSortedSetTest> { + + /** + * 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); + + /** + * Test whether the key and value match the expected key and value using junit assertions and hence will throw and exeption if they do not match. + * + * @param expected + * The expected key, value + * @param value + * The key, value being tested + */ + public abstract void testFullEquality(Map.Entry expected, Map.Entry value); + + /** + * 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 RewritableSortedSet.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 + set = new datawave.query.util.sortedset.BufferedFileBackedSortedSet.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) + .withMaxOpenFiles(7).withNumRetries(2) + .withHandlerFactories(Collections.singletonList(new BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { + @Override + public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { + datawave.query.util.sortedset.SortedSetTempFileHandler fileHandler = new SortedSetTempFileHandler(); + tempFileHandlers.add(fileHandler); + return fileHandler; + } + + @Override + public boolean isValid() { + return true; + } + })).withSetFactory(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(set, data); + addDataRandomly(set, data2); + } + + // now test the contents making sure we still have a sorted set with the expected values + int index = 0; + for (Iterator> it = set.iterator(); it.hasNext();) { + Map.Entry value = it.next(); + int dataIndex = sortedOrder[index++]; + Map.Entry expected = (dataIndex < template.length ? data2[dataIndex] : data[dataIndex]); + testFullEquality(expected, value); + } + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java new file mode 100644 index 00000000000..76dfac96f2a --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java @@ -0,0 +1,436 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.BufferedFileBackedSortedSet; +import datawave.query.util.sortedset.FileSortedSet; +import datawave.query.util.sortedset.SortedSetTempFileHandler; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +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.Random; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.function.Predicate; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public abstract class BufferedFileBackedSortedSetTest { + + protected final List tempFileHandlers = new ArrayList<>(); + protected E[] data = null; + protected int[] sortedOrder = null; + protected datawave.query.util.sortedset.BufferedFileBackedSortedSet set = 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 E createData(byte[] values); + + public abstract Comparator getComparator(); + + public abstract datawave.query.util.sortedset.FileSortedSet.FileSortedSetFactory getFactory(); + + public RewritableSortedSet.RewriteStrategy getRewriteStrategy() { + return null; + } + + protected void testEquality(E expected, E value) { + if (set.comparator() != null) { + assertEquals(0, set.comparator().compare(expected, value)); + } else { + assertEquals(expected, value); + } + } + + @Before + public void setUp() throws Exception { + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + E datum = createData(buffer); + if (i == 0) { + data = (E[]) 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]); + E 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]; + } + set = new datawave.query.util.sortedset.BufferedFileBackedSortedSet.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) + .withMaxOpenFiles(7).withNumRetries(2) + .withHandlerFactories(Collections.singletonList(new datawave.query.util.sortedset.BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { + @Override + public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { + datawave.query.util.sortedset.SortedSetTempFileHandler fileHandler = new datawave.query.util.sortedset.SortedSetTempFileHandler(); + tempFileHandlers.add(fileHandler); + return fileHandler; + } + + @Override + public boolean isValid() { + return true; + } + })).withSetFactory(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(set, data); + } + while (set.getSets().size() <= 7) { + addDataRandomly(set, data); + } + } + + public void addDataRandomly(BufferedFileBackedSortedSet set, E[] 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); + set.add(data[index]); + added.add(index); + } + // ensure all missing items are added + for (int i = 0; i < data.length; i++) { + if (!added.contains(i)) { + set.add(data[i]); + } + } + } + + @After + public void tearDown() throws Exception { + // Delete each sorted set file and its checksum. + for (SortedSetTempFileHandler 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; + set.clear(); + set = 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, set.size()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + for (int i = 0; i < (data.length / 2); i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + assertEquals(0, set.size()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + expectedSize++; + assertEquals(expectedSize, set.size()); + } + } + + @Test + public void testIsEmpty() { + assertFalse(set.isEmpty()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + for (int i = 1; i < (data.length / 2); i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + set.remove(data[0]); + assertTrue(set.isEmpty()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + assertFalse(set.isEmpty()); + } + } + + @Test + public void testClear() { + set.clear(); + assertTrue(set.isEmpty()); + } + + @Test + public void testContainsObject() { + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + } + for (int i = 1; i < (data.length / 2); i++) { + assertTrue(set.contains(data[i])); + } + for (int i = (data.length / 2); i < data.length; i++) { + assertFalse(set.contains(data[i])); + } + } + + @Test + public void testRemove() { + int expectedSize = data.length; + + assertFalse(set.isPersisted()); + for (int i = 0; i < data.length; i++) { + set.remove(data[i]); + assertEquals(--expectedSize, set.size()); + } + assertTrue(set.isEmpty()); + } + + @Test + public void testRemovePersisted() throws IOException { + int expectedSize = data.length; + + assertFalse(set.isPersisted()); + set.persist(); + assertTrue(set.isPersisted()); + for (int i = 0; i < data.length; i++) { + set.remove(data[i]); + assertEquals(--expectedSize, set.size()); + assertTrue(set.isPersisted()); + } + assertTrue(set.isEmpty()); + } + + @Test + public void testRemoveIf() { + int expectedSize = data.length; + + assertFalse(set.isPersisted()); + set.removeIf(new Predicate() { + @Override + public boolean test(E bytes) { + return false; + } + }); + assertFalse(set.isPersisted()); + assertEquals(expectedSize, set.size()); + + set.removeIf(new Predicate() { + @Override + public boolean test(E bytes) { + return true; + } + }); + assertFalse(set.isPersisted()); + assertTrue(set.isEmpty()); + } + + @Test + public void testRemoveIfPersisted() throws IOException { + int expectedSize = data.length; + + assertFalse(set.isPersisted()); + set.persist(); + assertTrue(set.isPersisted()); + + set.removeIf(new Predicate() { + @Override + public boolean test(E bytes) { + return false; + } + }); + assertTrue(set.isPersisted()); + assertEquals(expectedSize, set.size()); + + set.removeIf(new Predicate() { + @Override + public boolean test(E bytes) { + return true; + } + }); + assertTrue(set.isPersisted()); + assertTrue(set.isEmpty()); + } + + @Test + public void testRemoveAll() { + int expectedSize = data.length; + + assertFalse(set.isPersisted()); + set.removeAll(Collections.emptySet()); + assertFalse(set.isPersisted()); + assertEquals(expectedSize, set.size()); + + Set dataSet = new TreeSet<>(set.comparator()); + dataSet.addAll(Arrays.asList(data)); + set.removeAll(dataSet); + assertFalse(set.isPersisted()); + assertTrue(set.isEmpty()); + } + + @Test + public void testRemoveAllPersisted() throws IOException { + int expectedSize = data.length; + + assertFalse(set.isPersisted()); + set.persist(); + assertTrue(set.isPersisted()); + set.removeAll(Collections.emptySet()); + assertTrue(set.isPersisted()); + assertEquals(expectedSize, set.size()); + + Set dataSet = new TreeSet<>(set.comparator()); + dataSet.addAll(Arrays.asList(data)); + set.removeAll(dataSet); + assertTrue(set.isPersisted()); + assertTrue(set.isEmpty()); + } + + @Test + public void testIterator() { + int index = 0; + for (Iterator it = set.iterator(); it.hasNext();) { + E value = it.next(); + E expected = data[sortedOrder[index++]]; + testEquality(expected, value); + } + set.clear(); + for (E value : set) { + fail(); + } + } + + @Test + public void testIteratorRemove() { + int size = set.size(); + int failCount = 0; + assertFalse(set.isPersisted()); + // calling iterator() will force persistence + for (Iterator it = set.iterator(); it.hasNext();) { + assertTrue(set.isPersisted()); + E value = it.next(); + assertTrue(set.contains(value)); + try { + it.remove(); + fail("Expected iterator remove to fail with a persisted set"); + } catch (Exception e) { + // expected that some of the underlying FileSortedSets are persisted and hence the remove will fail + failCount++; + assertTrue(set.contains(value)); + assertEquals(size, set.size()); + } + } + assertEquals(size, failCount); + assertFalse(set.isEmpty()); + } + + @Test + public void testComparator() { + Comparator comparator = set.comparator(); + E[] testData = Arrays.copyOf(data, data.length); + Arrays.sort(testData, comparator); + int index = 0; + for (E value : set) { + E expected = data[sortedOrder[index++]]; + testEquality(expected, value); + } + } + + @Test + public void testSubSet() { + int start = sortedOrder.length / 3; + int end = start * 2; + try { + SortedSet subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); + SortedSet expected = new TreeSet<>(set.comparator()); + for (int i = start; i < end; i++) { + expected.add(data[sortedOrder[i]]); + } + assertEquals(expected, subSet); + } catch (Exception e) { + // expected + } + } + + @Test + public void testHeadSet() { + int end = sortedOrder.length / 3; + try { + SortedSet subSet = set.headSet(data[sortedOrder[end]]); + SortedSet expected = new TreeSet<>(set.comparator()); + for (int i = 0; i < end; i++) { + expected.add(data[sortedOrder[i]]); + } + assertEquals(expected, subSet); + } catch (Exception e) { + // expected + } + } + + @Test + public void testTailSet() { + int start = sortedOrder.length / 3; + try { + SortedSet subSet = set.tailSet(data[sortedOrder[start]]); + SortedSet expected = new TreeSet<>(set.comparator()); + for (int i = start; i < sortedOrder.length; i++) { + expected.add(data[sortedOrder[i]]); + } + assertEquals(expected, subSet); + } catch (Exception e) { + // expected + } + } + + @Test + public void testLast() { + E expected = data[sortedOrder[data.length - 1]]; + E value = set.last(); + testEquality(expected, value); + } + + @Test + public void testFirst() { + E expected = data[sortedOrder[0]]; + E value = set.first(); + testEquality(expected, value); + } + + @Test + public void testCompaction() throws IOException { + assertEquals(8, set.getSets().size()); + set.persist(); + assertEquals(3, set.getSets().size()); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedSetTest.java new file mode 100644 index 00000000000..2b2fa6cfaa4 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedSetTest.java @@ -0,0 +1,450 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.FileSerializableSortedSet; +import datawave.query.util.sortedset.FileSortedSet; +import datawave.query.util.sortedset.SortedSetTempFileHandler; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.Random; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.junit.Assert.assertArrayEquals; +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; + +public class FileSortedSetTest { + SortedSet data = null; + SortedSet extraData = null; + FileSortedMap set = null; + datawave.query.util.sortedset.SortedSetTempFileHandler 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 SortedSetTempFileHandler(); + set = new datawave.query.util.sortedset.FileSerializableSortedSet<>(c, new datawave.query.util.sortedset.FileSerializableSortedSet.SerializableFileHandler(handler), false); + data = new TreeSet<>(c); + Random r = new Random(123948710248L); + // data.add(null); + for (int i = 0; i < 20; i++) { + data.add(r.nextInt()); + } + extraData = new TreeSet<>(c); + for (int i = 0; i < 20; i++) { + extraData.add(r.nextInt()); + } + extraData.removeAll(data); + set.addAll(data); + } + + @After + public void tearDown() { + handler.getFile().delete(); + } + + @Test + public void testReadWrite() throws Exception { + assertFalse(set.isPersisted()); + assertEquals(data.size(), set.size()); + assertTrue(set.containsAll(data)); + assertTrue(data.containsAll(set)); + set.persist(); + assertTrue(set.isPersisted()); + assertTrue(handler.getFile().exists()); + assertEquals(data.size(), set.size()); + assertTrue(set.containsAll(data)); + assertTrue(data.containsAll(set)); + set.load(); + assertFalse(set.isPersisted()); + assertEquals(data.size(), set.size()); + assertTrue(set.containsAll(data)); + assertTrue(data.containsAll(set)); + } + + @Test + public void testIsEmpty() throws Exception { + assertFalse(set.isEmpty()); + set.persist(); + assertFalse(set.isEmpty()); + set.clear(); + assertTrue(set.isEmpty()); + set.load(); + assertTrue(set.isEmpty()); + } + + @Test + public void testContains() throws Exception { + SortedSet someData = new TreeSet<>(data); + someData.add(extraData.first()); + assertTrue(set.containsAll(data)); + for (Integer i : data) { + assertTrue(set.contains(i)); + } + assertFalse(set.containsAll(someData)); + for (Integer i : extraData) { + assertFalse(set.contains(i)); + } + set.persist(); + assertTrue(set.containsAll(data)); + for (Integer i : data) { + assertTrue(set.contains(i)); + } + assertFalse(set.containsAll(someData)); + for (Integer i : extraData) { + assertFalse(set.contains(i)); + } + set.load(); + assertTrue(set.containsAll(data)); + for (Integer i : data) { + assertTrue(set.contains(i)); + } + assertFalse(set.containsAll(someData)); + for (Integer i : extraData) { + assertFalse(set.contains(i)); + } + } + + @Test + public void testIterator() throws Exception { + SortedSet testData = new TreeSet<>(data); + Iterator dataIterator = testData.iterator(); + Iterator setIterator = set.iterator(); + while (dataIterator.hasNext()) { + assertTrue(setIterator.hasNext()); + assertEquals(dataIterator.next(), setIterator.next()); + } + assertFalse(setIterator.hasNext()); + for (Integer i : set) { + assertTrue(testData.remove(i)); + } + assertTrue(testData.isEmpty()); + set.persist(); + dataIterator = data.iterator(); + setIterator = set.iterator(); + while (dataIterator.hasNext()) { + assertTrue(setIterator.hasNext()); + assertEquals(dataIterator.next(), setIterator.next()); + } + assertFalse(setIterator.hasNext()); + testData.addAll(data); + for (Integer i : set) { + assertTrue(testData.remove(i)); + } + assertTrue(testData.isEmpty()); + set.load(); + dataIterator = data.iterator(); + setIterator = set.iterator(); + while (dataIterator.hasNext()) { + assertTrue(setIterator.hasNext()); + assertEquals(dataIterator.next(), setIterator.next()); + } + assertFalse(setIterator.hasNext()); + testData.addAll(data); + for (Integer i : set) { + assertTrue(testData.remove(i)); + } + assertTrue(testData.isEmpty()); + } + + @Test + public void testToArray() throws Exception { + Object[] a = set.toArray(); + Object[] d = data.toArray(); + assertArrayEquals(d, a); + set.persist(); + a = set.toArray(); + assertArrayEquals(d, a); + set.load(); + a = set.toArray(); + assertArrayEquals(d, a); + } + + @Test + public void testToArrayTArray() throws Exception { + Integer[] d = data.toArray(new Integer[set.size()]); + + Integer[] a = set.toArray(new Integer[set.size()]); + assertArrayEquals(d, a); + set.persist(); + a = set.toArray(new Integer[set.size()]); + assertArrayEquals(d, a); + set.load(); + a = set.toArray(new Integer[set.size()]); + assertArrayEquals(d, a); + + a = set.toArray(new Integer[set.size()]); + assertArrayEquals(d, a); + set.persist(); + a = set.toArray(new Integer[set.size()]); + assertArrayEquals(d, a); + set.load(); + a = set.toArray(new Integer[set.size()]); + assertArrayEquals(d, a); + + d = data.toArray(new Integer[set.size() * 2]); + a = set.toArray((new Integer[set.size() * 2])); + assertArrayEquals(d, a); + set.persist(); + a = set.toArray(new Integer[set.size() * 2]); + assertArrayEquals(d, a); + set.load(); + a = set.toArray(new Integer[set.size() * 2]); + assertArrayEquals(d, a); + } + + @Test + public void testAdd() throws Exception { + assertTrue(set.add(extraData.first())); + assertEquals(data.size() + 1, set.size()); + assertTrue(set.contains(extraData.first())); + assertTrue(set.remove(extraData.first())); + assertEquals(data.size(), set.size()); + assertFalse(set.contains(extraData.first())); + set.persist(); + try { + set.add(extraData.first()); + fail("Expected persisted set.add to fail"); + } catch (Exception e) { + // expected + } + set.load(); + assertEquals(data.size(), set.size()); + assertFalse(set.contains(extraData.first())); + } + + @Test + public void testAddAll() throws Exception { + assertTrue(set.addAll(extraData)); + assertEquals(data.size() + extraData.size(), set.size()); + assertTrue(set.containsAll(extraData)); + assertTrue(set.removeAll(extraData)); + assertEquals(data.size(), set.size()); + assertFalse(set.contains(extraData.first())); + set.persist(); + try { + set.addAll(extraData); + fail("Expected persisted set.addAll to fail"); + } catch (Exception e) { + // expected + } + set.load(); + assertEquals(data.size(), set.size()); + assertFalse(set.contains(extraData.first())); + } + + @Test + public void testRetainAll() throws Exception { + SortedSet someData = new TreeSet<>(data); + someData.remove(data.first()); + someData.remove(data.last()); + someData.add(extraData.first()); + someData.add(extraData.last()); + assertFalse(set.retainAll(data)); + assertEquals(someData.size(), set.size()); + assertTrue(set.retainAll(someData)); + assertEquals(data.size() - 2, set.size()); + assertFalse(set.containsAll(data)); + assertFalse(set.containsAll(someData)); + assertFalse(set.contains(data.last())); + assertTrue(set.retainAll(extraData)); + assertTrue(set.isEmpty()); + + set.addAll(data); + set.persist(); + try { + set.retainAll(someData); + fail("Expected persisted set.retainAll to fail"); + } catch (Exception e) { + // expected + } + + set.load(); + assertEquals(data.size(), set.size()); + assertTrue(set.containsAll(data)); + } + + @Test + public void testRemoveAll() throws Exception { + SortedSet someData = new TreeSet<>(data); + someData.remove(data.first()); + someData.remove(data.last()); + someData.add(extraData.first()); + someData.add(extraData.last()); + assertFalse(set.removeAll(extraData)); + assertEquals(someData.size(), set.size()); + assertTrue(set.removeAll(someData)); + assertEquals(2, set.size()); + assertFalse(set.containsAll(data)); + assertFalse(set.containsAll(someData)); + assertTrue(set.contains(data.first())); + assertTrue(set.contains(data.last())); + assertTrue(set.removeAll(data)); + assertTrue(set.isEmpty()); + + set.addAll(data); + set.persist(); + try { + set.removeAll(someData); + fail("Expected persisted set.retainAll to fail"); + } catch (Exception e) { + // expected + } + + set.load(); + assertEquals(data.size(), set.size()); + assertTrue(set.containsAll(data)); + } + + @Test + public void testClear() throws Exception { + set.clear(); + assertTrue(set.isEmpty()); + set.addAll(data); + set.persist(); + set.clear(); + assertTrue(set.isEmpty()); + set.load(); + assertTrue(set.isEmpty()); + } + + @Test + public void testNoComparator() throws Exception { + assertNotNull(set.comparator()); + set.persist(); + assertNotNull(set.comparator()); + set.load(); + assertNotNull(set.comparator()); + RewritableSortedSet tempData = new RewritableSortedSetImpl<>(); + for (Integer i : data) { + if (i != null) { + tempData.add(i); + } + } + + set = new datawave.query.util.sortedset.FileSerializableSortedSet<>(tempData, new FileSerializableSortedSet.SerializableFileHandler(handler)); + + assertNull(set.comparator()); + assertEquals(tempData, set); + for (Integer i : set) { + assertEquals(tempData.first(), i); + tempData.remove(tempData.first()); + } + for (Integer i : data) { + if (i != null) { + tempData.add(i); + } + } + assertEquals(tempData, set); + set.persist(); + assertNull(set.comparator()); + set.load(); + assertNull(set.comparator()); + + for (Integer i : data) { + assertEquals(tempData.first(), i); + tempData.remove(tempData.first()); + } + } + + @Test + public void testSubSet() throws Exception { + Integer fromElement = null; + Integer toElement = null; + int index = 0; + for (Integer i : data) { + if (index == (data.size() / 3)) { + fromElement = i; + } else if (index == data.size() * 2 / 3) { + toElement = i; + break; + } + index++; + } + SortedSet subSet = set.subSet(fromElement, toElement); + assertEquals(data.subSet(fromElement, toElement), subSet); + set.persist(); + set.subSet(fromElement, toElement); + assertEquals(data.subSet(fromElement, toElement), subSet); + set.load(); + subSet = set.subSet(fromElement, toElement); + assertEquals(data.subSet(fromElement, toElement), subSet); + } + + @Test + public void testHeadSet() throws Exception { + Integer toElement = null; + int index = 0; + for (Integer i : data) { + if (index == data.size() * 2 / 3) { + toElement = i; + break; + } + index++; + } + SortedSet subSet = set.headSet(toElement); + assertEquals(data.headSet(toElement), subSet); + set.persist(); + set.headSet(toElement); + assertEquals(data.headSet(toElement), subSet); + set.load(); + subSet = set.headSet(toElement); + assertEquals(data.headSet(toElement), subSet); + } + + @Test + public void testTailSet() throws Exception { + Integer fromElement = null; + int index = 0; + for (Integer i : data) { + if (index == (data.size() / 3)) { + fromElement = i; + break; + } + index++; + } + SortedSet subSet = set.tailSet(fromElement); + assertEquals(data.tailSet(fromElement), subSet); + set.persist(); + set.tailSet(fromElement); + assertEquals(data.tailSet(fromElement), subSet); + set.load(); + subSet = set.tailSet(fromElement); + assertEquals(data.tailSet(fromElement), subSet); + } + + @Test + public void testFirst() throws Exception { + assertEquals(data.first(), set.first()); + set.persist(); + assertEquals(data.first(), set.first()); + set.load(); + assertEquals(data.first(), set.first()); + } + + @Test + public void testLast() throws Exception { + assertEquals(data.last(), set.last()); + set.persist(); + assertEquals(data.last(), set.last()); + set.load(); + assertEquals(data.last(), set.last()); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.java new file mode 100644 index 00000000000..092fb1d3722 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.java @@ -0,0 +1,253 @@ +package datawave.query.util.sortedmap; + +import datawave.query.iterator.ivarator.IvaratorCacheDir; +import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.util.sortedset.FileSortedSet; +import datawave.query.util.sortedset.HdfsBackedSortedSet; +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 java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class HdfsBackedSortedSetTest { + + @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 FileSortedMap.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..d97a0ddb0ea --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MergeSortIteratorTest.java @@ -0,0 +1,242 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.MultiSetBackedSortedSet; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class MergeSortIteratorTest { + + @Test + public void testIteration() { + SortedSet set1 = new TreeSet<>(); + SortedSet set2 = new TreeSet<>(); + SortedSet set3 = new TreeSet<>(); + + set1.add(1); + set1.add(3); + set1.add(4); + set1.add(5); + set1.add(6); + set1.add(10); + + set2.add(1); + set2.add(2); + set2.add(5); + set2.add(20); + + set3.add(2); + set3.add(5); + set3.add(6); + set3.add(30); + + 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 datawave.query.util.sortedset.MultiSetBackedSortedSet(col).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.contains(next) || set2.contains(next) || set3.contains(next)); + it.remove(); + assertFalse(set1.contains(next) || set2.contains(next) || set3.contains(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() { + SortedSet set1 = new TreeSet<>(); + SortedSet set2 = new TreeSet<>(); + SortedSet set3 = new TreeSet<>(); + + set1.add(1); + set1.add(3); + set1.add(4); + set1.add(5); + set1.add(6); + set1.add(10); + + set2.add(1); + set2.add(2); + set2.add(5); + set2.add(20); + + set3.add(2); + set3.add(5); + set3.add(6); + set3.add(30); + + 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 datawave.query.util.sortedset.MultiSetBackedSortedSet(col).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.contains(next) || set2.contains(next) || set3.contains(next)); + it.remove(); + assertFalse(set1.contains(next) || set2.contains(next) || set3.contains(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)); + } + } + }; + + SortedSet set1 = new TreeSet<>(c); + SortedSet set2 = new TreeSet<>(c); + SortedSet set3 = new TreeSet<>(c); + + set1.add(1); + set1.add(3); + set1.add(4); + set1.add(5); + set1.add(6); + set1.add(10); + + set2.add(null); + set2.add(1); + set2.add(2); + set2.add(5); + set2.add(20); + + set3.add(null); + set3.add(2); + set3.add(5); + set3.add(6); + set3.add(30); + + 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 MultiSetBackedSortedSet(col).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.contains(next) || set2.contains(next) || set3.contains(next)); + it.remove(); + assertFalse(set1.contains(next) || set2.contains(next) || set3.contains(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/MultiSetBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedSetTest.java new file mode 100644 index 00000000000..15626bbb236 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedSetTest.java @@ -0,0 +1,596 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.MultiSetBackedSortedSet; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Random; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class MultiSetBackedSortedSetTest { + private byte[][] data = null; + private int[] sortedOrder = null; + private datawave.query.util.sortedset.MultiSetBackedSortedSet set = null; + + @Before + public void setUp() throws Exception { + byte[] template = new byte[] {5, 2, 78, 4, 8, 3, 54, 23, 6, 21, 7, 16}; + int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; + data = new byte[template.length * 2][]; + for (int i = 0; i < template.length; i++) { + data[i] = new byte[i + 11]; + Arrays.fill(data[i], template[i]); + } + for (int i = 0; i < template.length; i++) { + data[i + template.length] = new byte[10]; + Arrays.fill(data[i + template.length], template[i]); + } + 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]; + } + + set = new datawave.query.util.sortedset.MultiSetBackedSortedSet<>(); + + // create multiple underlying sets that contain random samplings of the data, ensuring we have complete coverage + boolean[] used = new boolean[data.length]; + int usedCount = 0; + Random random = new Random(2123974611); + while (usedCount < data.length) { + // create a set of 10 elements + TreeSet subset = new TreeSet<>(new datawave.query.util.sortedset.ByteArrayComparator()); + for (int i = 0; i < 10; i++) { + int index = random.nextInt(data.length); + if (!used[index]) { + usedCount++; + used[index] = true; + } + subset.add(data[index]); + } + set.addSet(subset); + } + } + + /** + * throws java.lang.Exception + */ + @After + public void tearDown() throws Exception { + data = null; + sortedOrder = null; + set.clear(); + set = null; + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#size()} + */ + @Test + public void testSize() { + int expectedSize = data.length; + assertEquals(expectedSize, set.size()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + for (int i = 0; i < (data.length / 2); i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + assertEquals(0, set.size()); + for (int i = 0; i < data.length; i++) { + set.getSets().get(0).add(data[i]); + expectedSize++; + assertEquals(expectedSize, set.size()); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#isEmpty()}. + */ + @Test + public void testIsEmpty() { + assertFalse(set.isEmpty()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + for (int i = 1; i < (data.length / 2); i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + set.remove(data[0]); + assertTrue(set.isEmpty()); + for (int i = 0; i < data.length; i++) { + set.getSets().get(0).add(data[i]); + assertFalse(set.isEmpty()); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#clear}. + */ + @Test + public void testClear() { + set.clear(); + assertTrue(set.isEmpty()); + SortedSet newSet = new TreeSet<>(new ByteArrayComparator()); + newSet.add("test".getBytes()); + set.addSet(newSet); + set.clear(); + assertTrue(set.isEmpty()); + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#contains(Object)}. + */ + @Test + public void testContainsObject() { + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + } + for (int i = 0; i < (data.length / 2); i++) { + assertTrue(set.contains(data[i])); + } + for (int i = (data.length / 2); i < data.length; i++) { + assertFalse(set.contains(data[i])); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#iterator()}. + */ + @Test + public void testIterator() { + int index = 0; + for (Iterator it = set.iterator(); it.hasNext();) { + byte[] value = it.next(); + byte[] expected = data[sortedOrder[index++]]; + assertArrayEquals(expected, value); + } + set.clear(); + for (@SuppressWarnings("unused") + byte[] value : set) { + fail(); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#iterator().remove()}. + */ + @Test + public void testIteratorRemove() { + int size = set.size(); + for (Iterator it = set.iterator(); it.hasNext();) { + byte[] value = it.next(); + assertTrue(set.contains(value)); + it.remove(); + assertFalse(set.contains(value)); + size--; + assertEquals(size, set.size()); + } + assertTrue(set.isEmpty()); + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#comparator()}. + */ + @Test + public void testComparator() { + Comparator comparator = set.comparator(); + byte[][] testData = Arrays.copyOf(data, data.length); + Arrays.sort(testData, comparator); + int index = 0; + for (byte[] value : set) { + byte[] expected = data[sortedOrder[index++]]; + assertArrayEquals(expected, value); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#subSet(byte[], byte[])}. + */ + @Test + public void testSubSet() { + int start = sortedOrder.length / 3; + int end = start * 2; + SortedSet subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); + + // verify contents + assertEquals(end - start, subSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify order + assertArrayEquals(data[sortedOrder[start]], subSet.first()); + int index = start; + for (byte[] value : subSet) { + assertArrayEquals(data[sortedOrder[index++]], value); + } + assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); + + // verify add + for (SortedSet subSubSet : ((datawave.query.util.sortedset.MultiSetBackedSortedSet) subSet).getSets()) { + assertTrue(subSubSet.contains(data[sortedOrder[start]]) != subSubSet.add(data[sortedOrder[start]])); + assertTrue(subSubSet.contains(data[sortedOrder[end - 1]]) != subSubSet.add(data[sortedOrder[end - 1]])); + try { + subSubSet.add(data[sortedOrder[start - 1]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + try { + subSubSet.add(data[sortedOrder[end]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + } + byte[] startValue = data[sortedOrder[start]]; + byte[] value = Arrays.copyOf(startValue, startValue.length + 50); + assertTrue(((datawave.query.util.sortedset.MultiSetBackedSortedSet) subSet).getSets().get(0).add(value)); + assertEquals(end - start + 1, subSet.size()); + assertEquals(data.length + 1, set.size()); + assertTrue(subSet.contains(value)); + assertTrue(set.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify remove + assertFalse(subSet.remove(data[sortedOrder[start - 1]])); + assertFalse(subSet.remove(data[sortedOrder[end]])); + assertTrue(subSet.remove(value)); + assertEquals(end - start, subSet.size()); + assertEquals(data.length, set.size()); + assertFalse(subSet.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify subSet + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start - 1]], data[sortedOrder[end]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start]], data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); + assertEquals(end - start - 2, subSubSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < (end - 1)) { + assertTrue(subSubSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSubSet.contains(data[sortedOrder[i]])); + } + } + + // verify tailSet + try { + @SuppressWarnings("unused") + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start - 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); + assertEquals(end - start - 1, subTailSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < end) { + assertTrue(subTailSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subTailSet.contains(data[sortedOrder[i]])); + } + } + + // verify headSet + try { + @SuppressWarnings("unused") + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); + assertEquals(end - start - 1, subHeadSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < (end - 1)) { + assertTrue(subHeadSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subHeadSet.contains(data[sortedOrder[i]])); + } + } + } + + /** + * Test method for {@link datwave.query.util.sortedset.MultiSetBackedSortedSet#headSet(byte[])}. + */ + @Test + public void testHeadSet() { + int end = sortedOrder.length / 3; + int start = 0; + SortedSet subSet = set.headSet(data[sortedOrder[end]]); + + // verify contents + assertEquals(end - start, subSet.size()); + for (int i = 0; i < data.length; i++) { + if (i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + // verify order + assertFalse(subSet.isEmpty()); + assertArrayEquals(data[sortedOrder[start]], subSet.iterator().next()); + assertArrayEquals(data[sortedOrder[start]], subSet.first()); + int index = start; + for (byte[] value : subSet) { + assertArrayEquals(data[sortedOrder[index++]], value); + } + ArrayList list = new ArrayList<>(subSet); + assertArrayEquals(data[sortedOrder[end - 1]], list.get(list.size() - 1)); + assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); + + subSet = set.headSet(data[sortedOrder[end]]); + + // verify add + for (SortedSet subSubSet : ((datawave.query.util.sortedset.MultiSetBackedSortedSet) subSet).getSets()) { + assertTrue(subSubSet.contains(data[sortedOrder[start]]) != subSubSet.add(data[sortedOrder[start]])); + assertTrue(subSubSet.contains(data[sortedOrder[end - 1]]) != subSubSet.add(data[sortedOrder[end - 1]])); + try { + subSubSet.add(data[sortedOrder[end]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + } + byte[] startValue = data[sortedOrder[start]]; + byte[] value = Arrays.copyOf(startValue, startValue.length + 50); + assertTrue(((datawave.query.util.sortedset.MultiSetBackedSortedSet) subSet).getSets().get(0).add(value)); + assertEquals(end - start + 1, subSet.size()); + assertEquals(data.length + 1, set.size()); + assertTrue(subSet.contains(value)); + assertTrue(set.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + // verify remove + assertFalse(subSet.remove(data[sortedOrder[end]])); + assertTrue(subSet.remove(value)); + assertEquals(end - start, subSet.size()); + assertEquals(data.length, set.size()); + assertFalse(subSet.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify subSet + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start]], data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); + assertEquals(end - start - 2, subSubSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < (end - 1)) { + assertTrue(subSubSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSubSet.contains(data[sortedOrder[i]])); + } + } + + // verify tailSet + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); + assertEquals(end - start - 1, subTailSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < end) { + assertTrue(subTailSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subTailSet.contains(data[sortedOrder[i]])); + } + } + + // verify headSet + try { + @SuppressWarnings("unused") + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); + assertEquals(end - start - 1, subHeadSet.size()); + for (int i = 0; i < data.length; i++) { + if (i < (end - 1)) { + assertTrue(subHeadSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subHeadSet.contains(data[sortedOrder[i]])); + } + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.MultiSetBackedSortedSet#tailSet(byte[])}. + */ + @Test + public void testTailSet() { + int start = sortedOrder.length / 3; + int end = sortedOrder.length; + SortedSet subSet = set.tailSet(data[sortedOrder[start]]); + + // verify contents + assertEquals(end - start, subSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify order + assertArrayEquals(data[sortedOrder[start]], subSet.first()); + int index = start; + for (byte[] value : subSet) { + assertArrayEquals(data[sortedOrder[index++]], value); + } + assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); + + // verify add + for (SortedSet subSubSet : ((datawave.query.util.sortedset.MultiSetBackedSortedSet) subSet).getSets()) { + assertTrue(subSubSet.contains(data[sortedOrder[start]]) != subSubSet.add(data[sortedOrder[start]])); + assertTrue(subSubSet.contains(data[sortedOrder[end - 1]]) != subSubSet.add(data[sortedOrder[end - 1]])); + try { + subSubSet.add(data[sortedOrder[start - 1]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + } + byte[] startValue = data[sortedOrder[start]]; + byte[] value = Arrays.copyOf(startValue, startValue.length + 50); + assertTrue(((MultiSetBackedSortedSet) subSet).getSets().get(0).add(value)); + assertEquals(end - start + 1, subSet.size()); + assertEquals(data.length + 1, set.size()); + assertTrue(subSet.contains(value)); + assertTrue(set.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify remove + assertFalse(subSet.remove(data[sortedOrder[start - 1]])); + assertTrue(subSet.remove(value)); + assertEquals(end - start, subSet.size()); + assertEquals(data.length, set.size()); + assertFalse(subSet.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify subSet + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start - 1]], data[sortedOrder[end - 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); + assertEquals(end - start - 2, subSubSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < (end - 1)) { + assertTrue(subSubSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSubSet.contains(data[sortedOrder[i]])); + } + } + + // verify tailSet + try { + @SuppressWarnings("unused") + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start - 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); + assertEquals(end - start - 1, subTailSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < end) { + assertTrue(subTailSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subTailSet.contains(data[sortedOrder[i]])); + } + } + + // verify headSet + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); + assertEquals(end - start - 1, subHeadSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < (end - 1)) { + assertTrue(subHeadSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subHeadSet.contains(data[sortedOrder[i]])); + } + } + } + + /** + * Test method for {@link MultiSetBackedSortedSet#last}. + */ + @Test + public void testLast() { + byte[] expected = data[sortedOrder[data.length - 1]]; + byte[] value = set.last(); + assertArrayEquals(expected, value); + } + + /** + * Test method for {@link MultiSetBackedSortedSet#first()}. + */ + @Test + public void testFirst() { + byte[] expected = data[sortedOrder[0]]; + byte[] value = set.first(); + assertArrayEquals(expected, value); + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java new file mode 100644 index 00000000000..590825d7d6f --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java @@ -0,0 +1,283 @@ +package datawave.query.util.sortedmap; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class RewritableSortedSetTest { + + private Map.Entry[] data = null; + private int[] sortedOrder = null; + private RewritableSortedSetImpl> set = null; + private final byte[] template = new byte[] {5, 2, 29, 4, 8, 3, 25, 23, 6, 21, 7, 16}; + private final int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; + + private Comparator> keyComparator = new Comparator<>() { + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return o1.getKey().compareTo(o2.getKey()); + } + }; + + private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator = new RewritableSortedSetImpl.RewriteStrategy<>() { + @Override + public boolean rewrite(Map.Entry original, Map.Entry update) { + int comparison = original.getKey().compareTo(update.getKey()); + if (comparison == 0) { + comparison = original.getValue().compareTo(update.getValue()); + } + return comparison < 0; + } + }; + + @Before + public void setUp() throws Exception { + data = new Map.Entry[template.length * 2]; + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + byte[] vbuffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + Arrays.fill(vbuffer, (byte) (template[i] + 1)); + data[i] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + byte[] vbuffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + Arrays.fill(vbuffer, (byte) (template[i] + 1)); + data[i + template.length] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); + } + 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]; + } + set = new RewritableSortedSetImpl<>(keyComparator, keyValueComparator); + + // 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(set, data); + } + } + + private void addDataRandomly(RewritableSortedSetImpl> set, Map.Entry[] data) { + Set added = new HashSet<>(); + // add data until all of the entries have been added + Random random = new Random(); + while (added.size() < data.length) { + int i = random.nextInt(data.length); + set.add(data[i]); + added.add(i); + } + } + + @After + public void tearDown() throws Exception { + data = null; + sortedOrder = null; + set.clear(); + set = null; + } + + @Test + public void testSize() { + int expectedSize = data.length; + assertEquals(expectedSize, set.size()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + for (int i = 0; i < (data.length / 2); i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + assertEquals(0, set.size()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + expectedSize++; + assertEquals(expectedSize, set.size()); + } + } + + @Test + public void testIsEmpty() { + assertFalse(set.isEmpty()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + for (int i = 1; i < (data.length / 2); i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + set.remove(data[0]); + assertTrue(set.isEmpty()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + assertFalse(set.isEmpty()); + } + } + + @Test + public void testClear() { + set.clear(); + assertTrue(set.isEmpty()); + } + + @Test + public void testContainsObject() { + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + } + for (int i = 1; i < (data.length / 2); i++) { + assertTrue(set.contains(data[i])); + } + for (int i = (data.length / 2); i < data.length; i++) { + assertFalse(set.contains(data[i])); + } + } + + @Test + public void testIterator() { + int index = 0; + for (Iterator> it = set.iterator(); it.hasNext();) { + Map.Entry value = it.next(); + Map.Entry expected = data[sortedOrder[index++]]; + assertEquals(expected, value); + } + set.clear(); + for (Map.Entry value : set) { + fail(); + } + } + + @Test + public void testIteratorRemove() { + int size = set.size(); + for (Iterator> it = set.iterator(); it.hasNext();) { + Map.Entry value = it.next(); + assertTrue(set.contains(value)); + it.remove(); + size--; + assertEquals(size, set.size()); + } + assertEquals(0, size); + assertTrue(set.isEmpty()); + } + + @Test + public void testSubSet() { + int start = sortedOrder.length / 3; + int end = start * 2; + try { + SortedSet> subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); + SortedSet> expected = new TreeSet<>(); + for (int i = start; i < end; i++) { + expected.add(data[sortedOrder[i]]); + } + assertEquals(expected, subSet); + } catch (Exception e) { + // expected + } + } + + @Test + public void testHeadSet() { + int end = sortedOrder.length / 3; + try { + SortedSet> subSet = set.headSet(data[sortedOrder[end]]); + SortedSet> expected = new TreeSet<>(); + for (int i = 0; i < end; i++) { + expected.add(data[sortedOrder[i]]); + } + assertEquals(expected, subSet); + } catch (Exception e) { + // expected + } + } + + @Test + public void testTailSet() { + int start = sortedOrder.length / 3; + try { + SortedSet> subSet = set.tailSet(data[sortedOrder[start]]); + SortedSet> expected = new TreeSet<>(); + for (int i = start; i < sortedOrder.length; i++) { + expected.add(data[sortedOrder[i]]); + } + assertEquals(expected, subSet); + } catch (Exception e) { + // expected + } + } + + @Test + public void testLast() { + Map.Entry expected = data[sortedOrder[data.length - 1]]; + Map.Entry value = set.last(); + assertEquals(expected, value); + } + + @Test + public void testFirst() { + Map.Entry expected = data[sortedOrder[0]]; + Map.Entry value = set.first(); + assertEquals(expected, value); + } + + @Test + public void testRewrite() { + // create a new set of data, half of which has greater Values and half of which has lesser Values + Map.Entry[] data2 = new Map.Entry[template.length * 2]; + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + byte[] vbuffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + Arrays.fill(vbuffer, (byte) (template[i] + 2)); + data2[i] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 10]; + byte[] vbuffer = new byte[i + 10]; + Arrays.fill(buffer, template[i]); + Arrays.fill(vbuffer, (byte) (template[i] - 1)); + data2[i + template.length] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); + } + + for (int d = 0; d < 11; d++) { + addDataRandomly(set, data2); + } + + // now test the contents + int index = 0; + for (Iterator> it = set.iterator(); it.hasNext();) { + Map.Entry value = it.next(); + int dataIndex = sortedOrder[index++]; + Map.Entry expected = (dataIndex < template.length ? data2[dataIndex] : data[dataIndex]); + assertEquals(expected, value); + } + + } + +} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.java new file mode 100644 index 00000000000..c31b1ead1ac --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.java @@ -0,0 +1,577 @@ +package datawave.query.util.sortedmap; + +import datawave.query.util.sortedset.SortedByteSetBuffer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.SortedSet; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class SortedByteSetBufferTest { + private byte[][] data = null; + private int[] sortedOrder = null; + private datawave.query.util.sortedset.SortedByteSetBuffer set = null; + + @Before + public void setUp() { + byte[] template = new byte[] {5, 2, 78, 4, 8, 3, 54, 23, 6, 21, 7, 16}; + int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; + data = new byte[template.length * 2][]; + for (int i = 0; i < template.length; i++) { + data[i] = new byte[i + 11]; + Arrays.fill(data[i], template[i]); + } + for (int i = 0; i < template.length; i++) { + data[i + template.length] = new byte[10]; + Arrays.fill(data[i + template.length], template[i]); + } + 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]; + } + set = new datawave.query.util.sortedset.SortedByteSetBuffer(5); + Collections.addAll(set, data); + } + + /** + * @throws Exception + */ + @After + public void tearDown() { + data = null; + sortedOrder = null; + set.clear(); + set = null; + } + + /** + * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#size()}. + */ + @Test + public void testSize() { + int expectedSize = data.length; + assertEquals(expectedSize, set.size()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + for (int i = 0; i < (data.length / 2); i++) { + set.remove(data[i]); + expectedSize--; + assertEquals(expectedSize, set.size()); + } + assertEquals(0, set.size()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + expectedSize++; + assertEquals(expectedSize, set.size()); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#isEmpty()}. + */ + @Test + public void testIsEmpty() { + assertFalse(set.isEmpty()); + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + for (int i = 1; i < (data.length / 2); i++) { + set.remove(data[i]); + assertFalse(set.isEmpty()); + } + set.remove(data[0]); + assertTrue(set.isEmpty()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + assertFalse(set.isEmpty()); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#clear()}. + */ + @Test + public void testClear() { + set.clear(); + assertTrue(set.isEmpty()); + for (int i = 0; i < data.length; i++) { + set.add(data[i]); + assertFalse(set.isEmpty()); + } + set.clear(); + assertTrue(set.isEmpty()); + } + + /** + * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#contains(Object)}. + */ + @Test + public void testContainsObject() { + for (int i = (data.length / 2); i < data.length; i++) { + set.remove(data[i]); + } + for (int i = 0; i < (data.length / 2); i++) { + assertTrue(set.contains(data[i])); + } + for (int i = (data.length / 2); i < data.length; i++) { + assertFalse(set.contains(data[i])); + } + } + + /** + * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#iterator()}. + */ + @Test + public void testIterator() { + int index = 0; + for (Iterator it = set.iterator(); it.hasNext();) { + byte[] value = it.next(); + byte[] expected = data[sortedOrder[index++]]; + assertArrayEquals(expected, value); + } + set.clear(); + for (@SuppressWarnings("unused") + byte[] value : set) { + fail(); + } + } + + /** + * Test method fo {@link nsa.datawave.data.SortedByteSetBuffer#iterator().remove()}. + */ + @Test + public void testIteratorRemove() { + int size = set.size(); + for (Iterator it = set.iterator(); it.hasNext();) { + byte[] value = it.next(); + assertTrue(set.contains(value)); + it.remove(); + assertFalse(set.contains((value))); + size--; + assertEquals(size, set.size()); + } + assertTrue(set.isEmpty()); + } + + /** + * Test method for {@link SortedByteSetBuffer#comparator()}. + */ + @Test + public void testComparator() { + Comparator comparator = set.comparator(); + byte[][] testData = Arrays.copyOf(data, data.length); + Arrays.sort(testData, comparator); + int index = 0; + for (byte[] value : set) { + byte[] expected = data[sortedOrder[index++]]; + assertArrayEquals(expected, value); + } + } + + /** + * Test method for {@link SortedByteSetBuffer#subSet(byte[]. byte[]}/ + */ + @Test + public void testSubSet() { + int start = sortedOrder.length / 3; + int end = start * 2; + SortedSet subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); + + // verify contents + assertEquals(end - start, subSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify order + assertArrayEquals(data[sortedOrder[start]], subSet.first()); + int index = start; + for (byte[] value : subSet) { + assertArrayEquals(data[sortedOrder[index++]], value); + } + assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); + + // verify add + assertFalse(subSet.add(data[sortedOrder[start]])); + assertFalse(subSet.add(data[sortedOrder[end - 1]])); + try { + subSet.add(data[sortedOrder[start - 1]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + try { + subSet.add(data[sortedOrder[end]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + byte[] startValue = data[sortedOrder[start]]; + byte[] value = Arrays.copyOf(startValue, startValue.length + 50); + assertTrue(subSet.add(value)); + assertEquals(end - start + 1, subSet.size()); + assertEquals(data.length + 1, set.size()); + assertTrue(subSet.contains(value)); + assertTrue(set.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify remove + assertFalse(subSet.remove(data[sortedOrder[start - 1]])); + assertFalse(subSet.remove(data[sortedOrder[end]])); + assertTrue(subSet.remove(value)); + assertEquals(end - start, subSet.size()); + assertEquals(data.length, set.size()); + assertFalse(subSet.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify subSet + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start - 1]], data[sortedOrder[end]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start]], data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); + assertEquals(end - start - 2, subSubSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < (end - 1)) { + assertTrue(subSubSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSubSet.contains(data[sortedOrder[i]])); + } + } + + // verify tailSet + try { + @SuppressWarnings("unused") + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start - 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); + assertEquals(end - start - 1, subTailSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < end) { + assertTrue(subTailSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subTailSet.contains(data[sortedOrder[i]])); + } + } + + // verify headSet + try { + @SuppressWarnings("unused") + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); + assertEquals(end - start - 1, subHeadSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < (end - 1)) { + assertTrue(subHeadSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subHeadSet.contains(data[sortedOrder[i]])); + } + } + } + + /** + * Test method for {@link SortedByteSetBuffer#headSet(byte[])} + */ + @Test + public void testHeadSet() { + int end = sortedOrder.length / 3; + int start = 0; + SortedSet subSet = set.headSet((data[sortedOrder[end]])); + + // verify contents + assertEquals(end - start, subSet.size()); + for (int i = 0; i < data.length; i++) { + if (i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify order + assertArrayEquals(data[sortedOrder[start]], subSet.first()); + int index = start; + for (byte[] value : subSet) { + assertArrayEquals(data[sortedOrder[index++]], value); + } + assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); + + // verify add + assertFalse(subSet.add(data[sortedOrder[start]])); + assertFalse(subSet.add(data[sortedOrder[end - 1]])); + try { + subSet.add(data[sortedOrder[end]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + byte[] startValue = data[sortedOrder[start]]; + byte[] value = Arrays.copyOf(startValue, startValue.length + 50); + assertTrue(subSet.add(value)); + assertEquals(end - start + 1, subSet.size()); + assertEquals(data.length + 1, set.size()); + assertTrue(subSet.contains(value)); + assertTrue(set.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify remove + assertFalse(subSet.remove(data[sortedOrder[end]])); + assertTrue(subSet.remove(value)); + assertEquals(end - start, subSet.size()); + assertEquals(data.length, set.size()); + assertFalse(subSet.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify subSet + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start]], data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); + assertEquals(end - start - 2, subSubSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < (end - 1)) { + assertTrue(subSubSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSubSet.contains(data[sortedOrder[i]])); + } + } + + // verify tailSet + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); + assertEquals(end - start - 1, subTailSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < end) { + assertTrue(subTailSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subTailSet.contains(data[sortedOrder[i]])); + } + } + + // verify headSet + try { + @SuppressWarnings("unused") + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end + 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); + assertEquals(end - start - 1, subHeadSet.size()); + for (int i = 0; i < data.length; i++) { + if (i < (end - 1)) { + assertTrue(subHeadSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subHeadSet.contains(data[sortedOrder[i]])); + } + } + } + + /** + * Test method for {@link SortedByteSetBuffer#tailSet(byte[])}. + */ + @Test + public void testTailSet() { + int start = sortedOrder.length / 3; + int end = sortedOrder.length; + SortedSet subSet = set.tailSet(data[sortedOrder[start]]); + + // verify contents + assertEquals(end - start, subSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify order + assertArrayEquals(data[sortedOrder[start]], subSet.first()); + int index = start; + for (byte[] value : subSet) { + assertArrayEquals(data[sortedOrder[index++]], value); + } + assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); + + // verify add + assertFalse(subSet.add(data[sortedOrder[start]])); + assertFalse(subSet.add(data[sortedOrder[end - 1]])); + try { + subSet.add(data[sortedOrder[start - 1]]); + fail("Expected to not be able to add something outside the range"); + } catch (IllegalArgumentException iae) { + // ok + } + byte[] startValue = data[sortedOrder[start]]; + byte[] value = Arrays.copyOf(startValue, startValue.length + 50); + assertTrue(subSet.add(value)); + assertEquals(end - start + 1, subSet.size()); + assertEquals(data.length + 1, set.size()); + assertTrue(subSet.contains(value)); + assertTrue(set.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify remove + assertFalse(subSet.remove(data[sortedOrder[start - 1]])); + assertTrue(subSet.remove(value)); + assertEquals(end - start, subSet.size()); + assertEquals(data.length, set.size()); + assertFalse(subSet.contains(value)); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < end) { + assertTrue(subSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSet.contains(data[sortedOrder[i]])); + } + } + + // verify subSet + try { + @SuppressWarnings("unused") + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start - 1]], data[sortedOrder[end - 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); + assertEquals(end - start - 2, subSubSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < (end - 1)) { + assertTrue(subSubSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subSubSet.contains(data[sortedOrder[i]])); + } + } + + // verify tailSet + try { + @SuppressWarnings("unused") + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start - 1]]); + fail("Expected to not be able to create a supper set out of a sub set"); + } catch (IllegalArgumentException iae) { + // ok + } + SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); + assertEquals(end - start - 1, subTailSet.size()); + for (int i = 0; i < data.length; i++) { + if (i > start && i < end) { + assertTrue(subTailSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subTailSet.contains(data[sortedOrder[i]])); + } + } + + // verify headSet + SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); + assertEquals(end - start - 1, subHeadSet.size()); + for (int i = 0; i < data.length; i++) { + if (i >= start && i < (end - 1)) { + assertTrue(subHeadSet.contains(data[sortedOrder[i]])); + } else { + assertFalse(subHeadSet.contains(data[sortedOrder[i]])); + } + } + } + + /** + * Test method for {@link SortedByteSetBuffer#get(int)}. + */ + @Test + public void testGet() { + for (int i = 0; i < data.length; i++) { + byte[] expected = data[sortedOrder[i]]; + byte[] value = set.get(i); + assertArrayEquals(expected, value); + } + } + + /** + * Test method for {@link SortedByteSetBuffer#last()}. + */ + @Test + public void testLast() { + byte[] expected = data[sortedOrder[data.length - 1]]; + byte[] value = set.last(); + assertArrayEquals(expected, value); + } + + /** + * Test method for {@link SortedByteSetBuffer#first()}. + */ + @Test + public void testFirst() { + byte[] expected = data[sortedOrder[0]]; + byte[] value = set.first(); + assertArrayEquals(expected, value); + } +} 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 c7c3d8026c6..6e810c1985a 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 @@ - + - + + + + + + + + + From 3b642b10bb2fdbd536579da26b6c0f2b4c6547a2 Mon Sep 17 00:00:00 2001 From: Ivan Bella Date: Wed, 7 Aug 2024 20:27:45 +0000 Subject: [PATCH 2/5] Initial implementation of a file backed sorted map --- .../query/transformer/UniqueTransform.java | 60 +- .../BufferedFileBackedSortedMap.java | 497 +++++------ .../sortedmap/FileByteDocumentSortedMap.java | 142 ++-- .../util/sortedmap/FileKeySortedMap.java | 218 ----- .../util/sortedmap/FileKeyValueSortedMap.java | 136 ++- .../sortedmap/FileSerializableSortedMap.java | 126 ++- .../query/util/sortedmap/FileSortedMap.java | 773 ++++++++---------- .../util/sortedmap/HdfsBackedSortedMap.java | 75 +- .../sortedmap/MultiMapBackedSortedMap.java | 378 +++++++++ .../sortedmap/MultiSetBackedSortedMap.java | 347 -------- .../util/sortedmap/RewritableSortedMap.java | 11 + .../util/sortedmap/SortedByteMapBuffer.java | 463 ----------- .../sortedmap/SortedMapTempFileHandler.java | 12 +- .../rfile/KeyValueByteDocumentTransforms.java | 14 + .../rfile/RFileByteDocumentInputStream.java | 17 +- .../rfile/RFileByteDocumentOutputStream.java | 12 +- .../sortedmap/rfile/RFileKeyInputStream.java | 26 - .../sortedmap/rfile/RFileKeyOutputStream.java | 18 - .../rfile/RFileKeyValueInputStream.java | 7 +- .../rfile/RFileKeyValueInputStreamBase.java | 19 +- .../rfile/RFileKeyValueOutputStream.java | 7 +- .../rfile/RFileKeyValueOutputStreamBase.java | 31 +- ...eredFileBackedByteArraySortedMapTest.java} | 3 +- ...dFileBackedByteDocumentSortedMapTest.java} | 3 +- ...> BufferedFileBackedKeySortedMapTest.java} | 3 +- ...feredFileBackedKeyValueSortedMapTest.java} | 3 +- ...redFileBackedRewritableSortedMapTest.java} | 3 +- ...a => BufferedFileBackedSortedMapTest.java} | 271 +++--- ...tedSetTest.java => FileSortedMapTest.java} | 12 +- ...Test.java => HdfsBackedSortedMapTest.java} | 3 +- ....java => MultiSetBackedSortedMapTest.java} | 2 +- ...Test.java => RewritableSortedMapTest.java} | 2 +- ...Test.java => SortedByteMapBufferTest.java} | 2 +- 33 files changed, 1425 insertions(+), 2271 deletions(-) delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiMapBackedSortedMap.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiSetBackedSortedMap.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/RewritableSortedMap.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedByteMapBuffer.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyInputStream.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{BufferedFileBackedByteArraySortedSetTest.java => BufferedFileBackedByteArraySortedMapTest.java} (77%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{BufferedFileBackedByteDocumentSortedSetTest.java => BufferedFileBackedByteDocumentSortedMapTest.java} (94%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{BufferedFileBackedKeySortedSetTest.java => BufferedFileBackedKeySortedMapTest.java} (76%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{BufferedFileBackedKeyValueSortedSetTest.java => BufferedFileBackedKeyValueSortedMapTest.java} (91%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{BufferedFileBackedRewritableSortedSetTest.java => BufferedFileBackedRewritableSortedMapTest.java} (96%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{BufferedFileBackedSortedSetTest.java => BufferedFileBackedSortedMapTest.java} (56%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{FileSortedSetTest.java => FileSortedMapTest.java} (97%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{HdfsBackedSortedSetTest.java => HdfsBackedSortedMapTest.java} (99%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{MultiSetBackedSortedSetTest.java => MultiSetBackedSortedMapTest.java} (99%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{RewritableSortedSetTest.java => RewritableSortedMapTest.java} (99%) rename warehouse/query-core/src/test/java/datawave/query/util/sortedmap/{SortedByteSetBufferTest.java => SortedByteMapBufferTest.java} (99%) 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 d68ae473dbc..8eabd56e4b5 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 @@ -17,6 +17,10 @@ import javax.annotation.Nullable; +import datawave.query.util.sortedmap.FileByteDocumentSortedMap; +import datawave.query.util.sortedmap.FileKeyValueSortedMap; +import datawave.query.util.sortedmap.FileSortedMap; +import datawave.query.util.sortedmap.HdfsBackedSortedMap; import org.apache.accumulo.core.data.Key; import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; import org.apache.hadoop.fs.FileSystem; @@ -41,11 +45,8 @@ import datawave.query.iterator.profile.FinalDocumentTrackingIterator; import datawave.query.model.QueryModel; import datawave.query.util.sortedset.ByteArrayComparator; -import datawave.query.util.sortedset.FileByteDocumentSortedSet; -import datawave.query.util.sortedset.FileKeyValueSortedSet; import datawave.query.util.sortedset.FileSortedSet; import datawave.query.util.sortedset.HdfsBackedSortedSet; -import datawave.query.util.sortedset.RewritableSortedSetImpl; /** * 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 @@ -58,8 +59,8 @@ public class UniqueTransform extends DocumentTransform.DefaultDocumentTransform private BloomFilter bloom; private UniqueFields uniqueFields = new UniqueFields(); - private HdfsBackedSortedSet> set; - private HdfsBackedSortedSet> returnSet; + private HdfsBackedSortedMap map; + private HdfsBackedSortedMap returnSet; private Iterator> setIterator; /** @@ -136,10 +137,10 @@ public Entry apply(@Nullable Entry keyDocumentEntry) } try { - if (set != null) { + if (map != null) { byte[] signature = getBytes(keyDocumentEntry.getValue()); - synchronized (set) { - this.set.add(new UnmodifiableMapEntry(signature, keyDocumentEntry.getValue())); + synchronized (map) { + this.map.put(signature, keyDocumentEntry.getValue()); } return null; } else if (!isDuplicate(keyDocumentEntry.getValue())) { @@ -167,8 +168,8 @@ public Entry apply(@Nullable Entry keyDocumentEntry) */ @Override public Map.Entry flush() { - if (set != null) { - synchronized (set) { + if (map != null) { + synchronized (map) { if (setIterator == null) { setupIterator(); } @@ -184,10 +185,10 @@ public Map.Entry flush() { * This will run through the set and create a new set ordered by Key, Document */ private void setupIterator() { - for (Map.Entry entry : set) { - returnSet.add(new UnmodifiableMapEntry<>(getDocKey(entry.getValue()), entry.getValue())); + for (Map.Entry entry : map.entrySet()) { + returnSet.put(getDocKey(entry.getValue()), entry.getValue()); } - setIterator = returnSet.iterator(); + setIterator = returnSet.entrySet().iterator(); } /** @@ -407,8 +408,8 @@ private Map.Entry getNext() { */ public static class Builder { private UniqueFields uniqueFields; - private Comparator> keyComparator; - private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator; + private Comparator keyComparator; + private FileSortedMap.RewriteStrategy keyValueComparator; private QueryModel model; private int bufferPersistThreshold; private List ivaratorCacheDirConfigs; @@ -420,23 +421,12 @@ public static class Builder { private FileSortedSet.PersistOptions persistOptions; public Builder() { - keyComparator = new Comparator<>() { - private Comparator comparator = new ByteArrayComparator(); + keyComparator = new ByteArrayComparator(); - @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return comparator.compare(o1.getKey(), o2.getKey()); - } - }; - - keyValueComparator = (original, update) -> { - int comparison = keyComparator.compare(original, update); - if (comparison == 0) { - long ts1 = getTimestamp(original.getValue()); - long ts2 = getTimestamp(update.getValue()); - return (ts2 > ts1); - } - return comparison < 0; + keyValueComparator = (key, original, update) -> { + long ts1 = getTimestamp(original); + long ts2 = getTimestamp(update); + return (ts2 > ts1); }; } @@ -529,7 +519,7 @@ public UniqueTransform build() throws IOException { if (transform.uniqueFields.isMostRecent()) { // @formatter:off // noinspection unchecked - transform.set = (HdfsBackedSortedSet>) HdfsBackedSortedSet.builder() + transform.map = (HdfsBackedSortedMap) HdfsBackedSortedMap.builder() .withComparator(keyComparator) .withRewriteStrategy(keyValueComparator) .withBufferPersistThreshold(bufferPersistThreshold) @@ -538,17 +528,17 @@ public UniqueTransform build() throws IOException { .withMaxOpenFiles(maxOpenFiles) .withNumRetries(numRetries) .withPersistOptions(persistOptions) - .withSetFactory(new FileByteDocumentSortedSet.Factory()) + .withMapFactory(new FileByteDocumentSortedMap.Factory()) .build(); - transform.returnSet = (HdfsBackedSortedSet>) HdfsBackedSortedSet.builder() + transform.returnSet = (HdfsBackedSortedMap) HdfsBackedSortedMap.builder() .withBufferPersistThreshold(bufferPersistThreshold) .withIvaratorCacheDirs(getIvaratorCacheDirs(ivaratorCacheDirConfigs, hdfsSiteConfigURLs, subDirectory)) .withUniqueSubPath("byDocKey") .withMaxOpenFiles(maxOpenFiles) .withNumRetries(numRetries) .withPersistOptions(persistOptions) - .withSetFactory(new FileKeyValueSortedSet.Factory()) + .withMapFactory(new FileKeyValueSortedMap.Factory()) .build(); // @formatter:on } 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 index 9e49197b9de..82bb76170e0 100644 --- 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 @@ -1,6 +1,6 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.FileSortedSet.SortedSetFileHandler; +import datawave.query.util.sortedmap.FileSortedMap.SortedMapFileHandler; import org.apache.log4j.Logger; import java.io.IOException; @@ -9,56 +9,57 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; -import java.util.SortedSet; -import java.util.function.Predicate; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; import java.util.stream.Collectors; /** - * This is a sorted set 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 subset operations will work up until any buffer has been flushed to disk. - * After that, those operations will not work as specified by the underlying FileSortedSet. + * 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 - * type of the set + * @param + * type of the map */ -public class BufferedFileBackedSortedMap implements RewritableSortedSet { +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 MultiSetBackedSortedMap set = new MultiSetBackedSortedMap<>(); + protected MultiMapBackedSortedMap map = new MultiMapBackedSortedMap<>(); protected int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; - protected FileSortedMap buffer = null; - protected FileSortedMap.FileSortedSetFactory setFactory = null; - protected final Comparator comparator; - protected final RewriteStrategy rewriteStrategy; + 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 List handlerFactories; protected int bufferPersistThreshold = DEFAULT_BUFFER_PERSIST_THRESHOLD; /** - * A factory for SortedSetFileHandlers + * A factory for SortedMapFileHandlers * * * */ - public interface SortedSetFileHandlerFactory { - SortedSetFileHandler createHandler() throws IOException; + public interface SortedMapFileHandlerFactory { + SortedMapFileHandler createHandler() throws IOException; boolean isValid(); } - public static class Builder,E> { + public static class Builder,K,V> { private int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; - private FileSortedMap.FileSortedSetFactory setFactory = new FileSerializableSortedMap.Factory(); - private Comparator comparator; - private RewriteStrategy rewriteStrategy; + 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 List handlerFactories = new ArrayList<>(); private int bufferPersistThreshold = DEFAULT_BUFFER_PERSIST_THRESHOLD; public Builder() {} @@ -74,20 +75,20 @@ public B withMaxOpenFiles(int maxOpenFiles) { } @SuppressWarnings("unchecked") - public B withSetFactory(FileSortedMap.FileSortedSetFactory setFactory) { - this.setFactory = (FileSortedMap.FileSortedSetFactory) setFactory; + public B withMapFactory(FileSortedMap.FileSortedMapFactory mapFactory) { + this.mapFactory = (FileSortedMap.FileSortedMapFactory) mapFactory; return self(); } @SuppressWarnings("unchecked") public B withComparator(Comparator comparator) { - this.comparator = (Comparator) comparator; + this.comparator = (Comparator) comparator; return self(); } @SuppressWarnings("unchecked") - public B withRewriteStrategy(RewriteStrategy rewriteStrategy) { - this.rewriteStrategy = (RewriteStrategy) rewriteStrategy; + public B withRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = (FileSortedMap.RewriteStrategy) rewriteStrategy; return self(); } @@ -96,7 +97,7 @@ public B withNumRetries(int numRetries) { return self(); } - public B withHandlerFactories(List handlerFactories) { + public B withHandlerFactories(List handlerFactories) { this.handlerFactories = handlerFactories; return self(); } @@ -106,26 +107,26 @@ public B withBufferPersistThreshold(int bufferPersistThreshold) { return self(); } - public BufferedFileBackedSortedMap build() throws Exception { + public BufferedFileBackedSortedMap build() throws Exception { return new BufferedFileBackedSortedMap<>(this); } } - public static Builder builder() { + public static Builder builder() { return new Builder<>(); } - protected BufferedFileBackedSortedMap(BufferedFileBackedSortedMap other) { + protected BufferedFileBackedSortedMap(BufferedFileBackedSortedMap other) { this.comparator = other.comparator; this.rewriteStrategy = other.rewriteStrategy; this.handlerFactories = new ArrayList<>(other.handlerFactories); - this.setFactory = other.setFactory; + this.mapFactory = other.mapFactory; this.bufferPersistThreshold = other.bufferPersistThreshold; this.numRetries = other.numRetries; this.maxOpenFiles = other.maxOpenFiles; - for (SortedSet subSet : other.set.getSets()) { - FileSortedMap clone = ((FileSortedMap) subSet).clone(); - this.set.addSet(clone); + for (SortedMap submap : other.map.getMaps()) { + FileSortedMap clone = ((FileSortedMap) submap).clone(); + this.map.addMap(clone); if (!clone.isPersisted()) { this.buffer = clone; } @@ -138,13 +139,13 @@ protected BufferedFileBackedSortedMap(Builder builder) { this.comparator = builder.comparator; this.rewriteStrategy = builder.rewriteStrategy; this.handlerFactories = new ArrayList<>(builder.handlerFactories); - this.setFactory = builder.setFactory; + this.mapFactory = builder.mapFactory; this.bufferPersistThreshold = builder.bufferPersistThreshold; this.numRetries = builder.numRetries; this.maxOpenFiles = builder.maxOpenFiles; } - private SortedSetFileHandler createFileHandler(SortedSetFileHandlerFactory handlerFactory) throws IOException { + private SortedMapFileHandler createFileHandler(SortedMapFileHandlerFactory handlerFactory) throws IOException { if (handlerFactory.isValid()) { try { return handlerFactory.createHandler(); @@ -158,10 +159,10 @@ private SortedSetFileHandler createFileHandler(SortedSetFileHandlerFactory handl public void persist() throws IOException { if (buffer != null) { - // go through the handler factories and try to persist the sorted set + // go through the handler factories and try to persist the sorted map for (int i = 0; i < handlerFactories.size() && !buffer.isPersisted(); i++) { - SortedSetFileHandlerFactory handlerFactory = handlerFactories.get(i); - SortedSetFileHandler handler = createFileHandler(handlerFactory); + SortedMapFileHandlerFactory handlerFactory = handlerFactories.get(i); + SortedMapFileHandler handler = createFileHandler(handlerFactory); // if we have a valid handler, try to persist if (handler != null) { @@ -176,12 +177,12 @@ public void persist() throws IOException { } if (!buffer.isPersisted()) { - log.warn("Unable to persist the sorted set using the file handler: " + handler, cause); + 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.SortedSetHdfsFileHandlerFactory) { - HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory) handlerFactory); - hdfsHandlerFactory.setFileCount(hdfsHandlerFactory.getFileCount() - 1); + if (handlerFactory instanceof HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) { + HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) handlerFactory); + hdfsHandlerFactory.mapFileCount(hdfsHandlerFactory.getFileCount() - 1); } } } else { @@ -191,29 +192,29 @@ public void persist() throws IOException { // if the buffer was not persisted, throw an exception if (!buffer.isPersisted()) - throw new IOException("Unable to persist the sorted set using the configured handler factories."); + throw new IOException("Unable to persist the sorted map using the configured handler factories."); buffer = null; compact(maxOpenFiles); } } - protected List> getSets() { - List> sets = new ArrayList<>(); - for (SortedSet subSet : set.getSets()) { - sets.add((FileSortedMap) subSet); + protected List> getMaps() { + List> maps = new ArrayList<>(); + for (SortedMap submap : map.getMaps()) { + maps.add((FileSortedMap) submap); } - return sets; + return maps; } - protected void addSet(FileSortedMap subSet) { - set.addSet(subSet); - size += subSet.size(); + protected void addMap(FileSortedMap submap) { + map.addMap(submap); + size += submap.size(); } public boolean hasPersistedData() { - for (SortedSet subSet : set.getSets()) { - if (((FileSortedMap) subSet).isPersisted()) { + for (SortedMap submap : map.getMaps()) { + if (((FileSortedMap) submap).isPersisted()) { return true; } } @@ -228,7 +229,7 @@ public boolean isPersisted() { @Override public int size() { if (sizeModified) { - this.size = set.size(); + this.size = map.size(); sizeModified = false; } return this.size; @@ -248,45 +249,39 @@ public boolean isEmpty() { } @Override - public boolean contains(Object o) { + public boolean containsKey(Object o) { // try the cheap operation first - if (buffer != null && buffer.contains(o)) { + if (buffer != null && buffer.containsKey(o)) { return true; } else { - return set.contains(o); + return map.containsKey(o); } } @Override - public boolean containsAll(Collection c) { - // try the cheap operation first - if (buffer != null && buffer.containsAll(c)) { - return true; - } else { - return set.containsAll(c); - } + public boolean containsValue(Object value) { + return false; } - @Override - public Iterator iterator() { - // first lets compact down the sets if needed + protected Iterator> iterator() { + // first lets compact down the maps if needed try { - // if we have any persisted sets, then ensure we are persisted - if (set.getSets().size() > 1) { + // if we have any persisted maps, then ensure we are persisted + if (map.getMaps().size() > 1) { persist(); } } catch (IOException ioe) { - throw new RuntimeException("Unable to persist or compact file backed sorted set", ioe); + throw new RuntimeException("Unable to persist or compact file backed sorted map", ioe); } - return set.iterator(); + return map.iterator(); } private String printHandlerFactories() { - return String.join(", ", handlerFactories.stream().map(SortedSetFileHandlerFactory::toString).collect(Collectors.toList())); + return String.join(", ", handlerFactories.stream().map(SortedMapFileHandlerFactory::toString).collect(Collectors.toList())); } /** - * If the number of sets is over maxFiles, then start compacting those files down. The goal is to get the number of files down around 50% of maxFiles. + * 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 @@ -294,85 +289,85 @@ private String printHandlerFactories() { * for IO Exceptions */ public void compact(int maxFiles) throws IOException { - // if we have more sets than we are allowed, then we need to compact this down - if (maxFiles > 0 && set.getSets().size() > maxFiles) { + // 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 set list (sorting below) - List> sets = new ArrayList<>(set.getSets()); + // create a copy of the map list (sorting below) + List> maps = new ArrayList<>(map.getMaps()); - // calculate the number of sets to compact - int numSets = sets.size(); - int excessSets = numSets - (maxFiles / 2); // those over 50% of maxFiles - int setsPerCompaction = Math.min(excessSets + 1, numSets); // Add in 1 to account for the compacted set being added back in + // 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 sets by size (compact up smaller sets first) - sets.sort(Comparator.comparing(SortedSet::size).reversed()); + // sort the maps by size (compact up smaller maps first) + maps.sort(Comparator.comparing(SortedMap::size).reversed()); - // newSet will be the final multiset - MultiSetBackedSortedMap newSet = new MultiSetBackedSortedMap<>(); + // newmap will be the final multimap + MultiMapBackedSortedMap newmap = new MultiMapBackedSortedMap<>(); - // create a set for those sets to be compacted into one file - MultiSetBackedSortedMap setToCompact = new MultiSetBackedSortedMap<>(); - for (int i = 0; i < setsPerCompaction; i++) { - setToCompact.addSet(sets.remove(sets.size() - 1)); + // 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 " + setToCompact); + log.debug("Starting compaction for " + mapToCompact); } long start = System.currentTimeMillis(); - FileSortedMap compaction = compact(setToCompact); + FileSortedMap compaction = compact(mapToCompact); if (log.isDebugEnabled()) { long delta = System.currentTimeMillis() - start; - log.debug("Compacted " + setToCompact + " -> " + compaction + " in " + delta + "ms"); + log.debug("Compacted " + mapToCompact + " -> " + compaction + " in " + delta + "ms"); } - // add the compacted set to our final multiset - newSet.addSet(compaction); + // add the compacted map to our final multimap + newmap.addMap(compaction); - // clear the compactions set to remove the files that were compacted - setToCompact.clear(); + // clear the compactions map to remove the files that were compacted + mapToCompact.clear(); - // now add in the sets we did not compact - for (int i = 0; i < sets.size(); i++) { - newSet.addSet(sets.get(i)); + // 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 set - this.set = newSet; + // and replace our map + this.map = newmap; } } - private FileSortedMap compact(MultiSetBackedSortedMap setToCompact) throws IOException { - FileSortedMap compactedSet = null; + private FileSortedMap compact(MultiMapBackedSortedMap mapToCompact) throws IOException { + FileSortedMap compactedmap = null; - // go through the handler factories and try to persist the sorted set - for (int i = 0; i < handlerFactories.size() && compactedSet == null; i++) { - SortedSetFileHandlerFactory handlerFactory = handlerFactories.get(i); - SortedSetFileHandler handler = createFileHandler(handlerFactory); + // 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 && compactedSet == null; attempts++) { + for (int attempts = 0; attempts <= numRetries && compactedmap == null; attempts++) { try { - compactedSet = setFactory.newInstance(setToCompact, handlerFactory.createHandler(), true); + compactedmap = mapFactory.newInstance(mapToCompact, handlerFactory.createHandler(), true); } catch (IOException e) { if (attempts == numRetries) cause = e; } } - if (compactedSet == null) { - log.warn("Unable to compact the sorted set using the file handler: " + handler, cause); + 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.SortedSetHdfsFileHandlerFactory) { - HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedSetHdfsFileHandlerFactory) handlerFactory); - hdfsHandlerFactory.setFileCount(hdfsHandlerFactory.getFileCount() - 1); + if (handlerFactory instanceof HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) { + HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory hdfsHandlerFactory = ((HdfsBackedSortedMap.SortedMapHdfsFileHandlerFactory) handlerFactory); + hdfsHandlerFactory.mapFileCount(hdfsHandlerFactory.getFileCount() - 1); } } } else { @@ -380,200 +375,105 @@ private FileSortedMap compact(MultiSetBackedSortedMap setToCompact) throws } } - // if the sorted sets were not compacted, throw an exception - if (compactedSet == null) - throw new IOException("Unable to persist the sorted set using the configured handler factories."); + // 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 compactedSet; + return compactedmap; } @Override - public Object[] toArray() { - return set.toArray(); - } - - @Override - public T[] toArray(T[] a) { - return set.toArray(a); - } - - @Override - public boolean add(E e) { + public V put(K key, V value) { if (buffer == null) { try { - buffer = setFactory.newInstance(comparator, rewriteStrategy, null, false); + buffer = mapFactory.newInstance(comparator, rewriteStrategy, null, false); } catch (Exception ex) { - throw new IllegalStateException("Unable to create an underlying FileSortedSet", ex); + throw new IllegalStateException("Unable to create an underlying FileSortedMap", ex); } - set.addSet(buffer); + map.addMap(buffer); } - if (buffer.add(e)) { + V previous = buffer.put(key, value); + if (previous != null) { sizeModified = true; if (buffer.size() >= bufferPersistThreshold) { try { persist(); } catch (Exception ex) { - throw new IllegalStateException("Unable to persist or compact FileSortedSet", ex); + throw new IllegalStateException("Unable to persist or compact FileSortedMap", ex); } } - return true; + return previous; } - return false; + return null; } @Override - public boolean addAll(Collection c) { + public void putAll(Map c) { if (buffer == null) { try { - buffer = setFactory.newInstance(comparator, rewriteStrategy, null, false); + buffer = mapFactory.newInstance(comparator, rewriteStrategy, null, false); } catch (Exception ex) { - throw new IllegalStateException("Unable to create an underlying FileSortedSet", ex); - } - set.addSet(buffer); - } - if (buffer.addAll(c)) { - sizeModified = true; - if (buffer.size() >= bufferPersistThreshold) { - try { - persist(); - } catch (Exception ex) { - throw new IllegalStateException("Unable to persist or compact FileSortedSet", ex); - } + throw new IllegalStateException("Unable to create an underlying FileSortedMap", ex); } - return true; + map.addMap(buffer); } - return false; - } - - @Override - public boolean remove(Object o) { - boolean removed = false; - for (SortedSet subSet : set.getSets()) { - FileSortedMap fileSet = (FileSortedMap) subSet; - if (fileSet.contains(o)) { - if (fileSet.isPersisted()) { - try { - fileSet.load(); - if (fileSet.remove(o)) { - removed = true; - fileSet.persist(); - } else { - fileSet.unload(); - // since we checked for containership first, remove should have returned true - throw new IllegalStateException("FileSet contains object but failed to remove it from persisted set"); - } - } catch (Exception e) { - throw new IllegalStateException("Unable to remove item from underlying files", e); - } - } else { - if (fileSet.remove(o)) { - removed = true; - } else { - // since we checked for containership first, remove should have returned true - throw new IllegalStateException("FileSet contains object but failed to remove it"); - } - } + buffer.putAll(c); + sizeModified = true; + if (buffer.size() >= bufferPersistThreshold) { + try { + persist(); + } catch (Exception ex) { + throw new IllegalStateException("Unable to persist or compact FileSortedMap", ex); } } - if (removed) { - this.sizeModified = true; - } - return removed; } @Override - public boolean retainAll(Collection c) { - boolean modified = false; - for (SortedSet subSet : set.getSets()) { - FileSortedMap fileSet = (FileSortedMap) subSet; - if (fileSet.isPersisted()) { + public V remove(Object o) { + V value = null; + for (SortedMap map : map.getMaps()) { + FileSortedMap filemap = (FileSortedMap)map; + boolean persist = false; + if (filemap.isPersisted()) { try { - fileSet.load(); - if (fileSet.retainAll(c)) { - modified = true; - fileSet.persist(); - } else { - fileSet.unload(); - } + filemap.load(); + persist = true; } catch (Exception e) { throw new IllegalStateException("Unable to remove item from underlying files", e); } - } else { - if (fileSet.retainAll(c)) { - modified = true; - } } - } - if (modified) { - this.sizeModified = true; - } - return modified; - } - @Override - public boolean removeAll(Collection c) { - boolean modified = false; - for (SortedSet subSet : set.getSets()) { - FileSortedMap fileSet = (FileSortedMap) subSet; - if (fileSet.isPersisted()) { - try { - fileSet.load(); - if (fileSet.removeAll(c)) { - modified = true; - fileSet.persist(); - } else { - fileSet.unload(); + V testValue = map.remove(o); + if (testValue != null) { + if (value != null) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K)o, value, testValue)) { + value = testValue; } - } catch (Exception e) { - throw new IllegalStateException("Unable to remove item from underlying files", e); - } - } else { - if (fileSet.removeAll(c)) { - modified = true; + } else { + value = testValue; } } - } - if (modified) { - this.sizeModified = true; - } - return modified; - } - @Override - public boolean removeIf(Predicate filter) { - boolean removed = false; - for (SortedSet subSet : set.getSets()) { - FileSortedMap fileSet = (FileSortedMap) subSet; - if (fileSet.isPersisted()) { + if (persist) { try { - fileSet.load(); - if (fileSet.removeIf(filter)) { - removed = true; - fileSet.persist(); - } else { - fileSet.unload(); - } + filemap.persist(); } catch (Exception e) { throw new IllegalStateException("Unable to remove item from underlying files", e); } - } else { - if (fileSet.removeIf(filter)) { - removed = true; - } } } - if (removed) { + if (value != null) { this.sizeModified = true; } - return removed; + return value; } @Override public void clear() { - // This will cause the MultiSetBackedSortedSet to call clear on each Set in its Set of Sets, including the buffer - // It will also call clear on its Set of Sets, emptying the contents - set.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; @@ -581,42 +481,95 @@ public void clear() { } @Override - public Comparator comparator() { + public Comparator comparator() { return comparator; } @Override - public RewriteStrategy getRewriteStrategy() { - return rewriteStrategy; + public SortedMap subMap(K fromKey, K toKey) { + return null; + } + + @Override + public SortedMap headMap(K toKey) { + return null; + } + + @Override + public SortedMap tailMap(K fromKey) { + return null; + } + + @Override + public K firstKey() { + return null; } @Override - public E get(E e) { + public K lastKey() { return null; } @Override - public RewritableSortedSet subSet(E fromElement, E toElement) { - return set.subSet(fromElement, toElement); + public Set keySet() { + return null; } @Override - public RewritableSortedSet headSet(E toElement) { - return set.headSet(toElement); + public Collection values() { + return null; } @Override - public RewritableSortedSet tailSet(E fromElement) { - return set.tailSet(fromElement); + public Set> entrySet() { + return null; } @Override - public E first() { - return set.first(); + public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = rewriteStrategy; } @Override - public E last() { - return set.last(); + public FileSortedMap.RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; } + + @Override + public V get(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.get(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); + } + } + } + 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 index ae9750f575e..44c92c5788c 100644 --- 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 @@ -3,143 +3,127 @@ 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; import org.apache.log4j.Logger; import java.io.IOException; import java.util.Comparator; -import java.util.Map; +import java.util.SortedMap; /** - * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as - * expected. This will support null contained in the underlying sets iff a comparator is supplied that can handle null values. + * 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> { +public class FileByteDocumentSortedMap extends FileSortedMap { private static Logger log = Logger.getLogger(FileByteDocumentSortedMap.class); - public final static class DefaultByteDocumentComparator implements Comparator> { + public final static class DefaultByteComparator implements Comparator { @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return new ByteArrayComparator().compare(o1.getKey(), o2.getKey()); + public int compare(byte[] o1, byte[] o2) { + return new ByteArrayComparator().compare(o1, o2); } } /** - * Create a file sorted set from another one + * Create a file sorted map from another one * * @param other - * the other sorted set + * the other sorted map */ public FileByteDocumentSortedMap(FileByteDocumentSortedMap other) { super(other); } /** - * Create a file sorted subset from another one + * Create a file sorted submap from another one * * @param other - * the other sorted set + * the other sorted map * @param from - * the from Document + * the from key * @param to - * the to Document + * the to key */ - public FileByteDocumentSortedMap(FileByteDocumentSortedMap other, Map.Entry from, Map.Entry to) { + public FileByteDocumentSortedMap(FileByteDocumentSortedMap other, byte[] from, byte[] to) { super(other, from, to); } /** - * Create a persisted sorted set + * Create a persisted sorted map * * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag */ - public FileByteDocumentSortedMap(SortedSetFileHandler handler, boolean persisted) { - this(new DefaultByteDocumentComparator(), handler, persisted); + public FileByteDocumentSortedMap(SortedMapFileHandler handler, boolean persisted) { + this(new DefaultByteComparator(), handler, persisted); } /** - * Create a persisted sorted set + * Create a persisted sorted map * * @param comparator * the key comparator * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag */ - public FileByteDocumentSortedMap(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultByteDocumentComparator() : comparator), new ByteDocumentFileHandler(handler), + public FileByteDocumentSortedMap(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { + super((comparator == null ? new DefaultByteComparator() : comparator), new ByteDocumentFileHandler(handler), new Factory(), persisted); } /** - * Create a persisted sorted set + * Create an unpersisted sorted map (still in memory) * - * @param comparator - * the key comparator - * @param rewriteStrategy - * the rewrite strategy - * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag - */ - public FileByteDocumentSortedMap(Comparator> comparator, RewriteStrategy> rewriteStrategy, - SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultByteDocumentComparator() : comparator), rewriteStrategy, new ByteDocumentFileHandler(handler), - new Factory(), persisted); - } - - /** - * Create an unpersisted sorted set (still in memory) - * - * @param set - * the sorted set + * @param map + * the sorted map * @param handler - * the sorted set file handler + * the sorted map file handler */ - public FileByteDocumentSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler) { - super(set, new ByteDocumentFileHandler(handler), new Factory()); + public FileByteDocumentSortedMap(FileSortedMap map, SortedMapFileHandler handler) { + super(map, new ByteDocumentFileHandler(handler), new Factory()); } /** - * Create a sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoids pulling + * 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 set - * the sorted set + * @param map + * the sorted map * @param handler - * the sorted set file handler + * the sorted map file handler * @param persist * boolean flag for persist * @throws IOException * for issues with read/write */ - public FileByteDocumentSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) throws IOException { - super(set, new ByteDocumentFileHandler(handler), new Factory(), persist); + public FileByteDocumentSortedMap(FileSortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + super(map, new ByteDocumentFileHandler(handler), new Factory(), persist); } /** - * This will dump the set to the file, making the set "persisted" + * This will dump the map to the file, making the map "persisted" * * @param handler - * the sorted set file handler + * the sorted map file handler * @throws IOException * for issues with read/write */ - public void persist(SortedSetFileHandler handler) throws IOException { + public void persist(SortedMapFileHandler handler) throws IOException { // ensure this handler is wrapped with our handler super.persist(new ByteDocumentFileHandler(handler)); } /** - * Clone this set + * Clone this map */ @Override public FileByteDocumentSortedMap clone() { @@ -147,33 +131,33 @@ public FileByteDocumentSortedMap clone() { } /** - * A sortedsetfilehandler that can bound the input stream + * A SortedMapfilehandler that can bound the input stream */ - public static class ByteDocumentFileHandler implements BoundedTypedSortedSetFileHandler> { - SortedSetFileHandler delegate; + public static class ByteDocumentFileHandler implements BoundedTypedSortedMapFileHandler { + SortedMapFileHandler delegate; - public ByteDocumentFileHandler(SortedSetFileHandler handler) { + public ByteDocumentFileHandler(SortedMapFileHandler handler) { this.delegate = handler; } @Override - public SortedSetInputStream> getInputStream() throws IOException { + public SortedMapInputStream getInputStream() throws IOException { return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize()); } @Override - public SortedSetInputStream> getInputStream(Map.Entry start, Map.Entry end) + public SortedMapInputStream getInputStream(byte[] start, byte[] end) throws IOException { return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize(), start, end); } @Override - public SortedSetOutputStream getOutputStream() throws IOException { + public SortedMapOutputStream getOutputStream() throws IOException { return new RFileByteDocumentOutputStream(delegate.getOutputStream()); } @Override - public PersistOptions getPersistOptions() { + public FileSortedSet.PersistOptions getPersistOptions() { return delegate.getPersistOptions(); } @@ -189,46 +173,48 @@ public void deleteFile() { } /** - * A factory for these file sorted sets + * A factory for these file sorted maps */ - public static class Factory implements FileSortedSetFactory> { + public static class Factory implements FileSortedMapFactory { @Override - public FileByteDocumentSortedMap newInstance(FileSortedMap> other) { + public FileByteDocumentSortedMap newInstance(FileSortedMap other) { return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other); } @Override - public FileByteDocumentSortedMap newInstance(FileSortedMap> other, Map.Entry from, - Map.Entry to) { + public FileByteDocumentSortedMap newInstance(FileSortedMap other, byte[] from, + byte[] to) { return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other, from, to); } @Override - public FileByteDocumentSortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + public FileByteDocumentSortedMap newInstance(SortedMapFileHandler handler, boolean persisted) { return new FileByteDocumentSortedMap(handler, persisted); } @Override - public FileByteDocumentSortedMap newInstance(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { + public FileByteDocumentSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { return new FileByteDocumentSortedMap(comparator, handler, persisted); } @Override - public FileSortedMap> newInstance(Comparator> comparator, - RewriteStrategy> rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { - return new FileByteDocumentSortedMap(comparator, rewriteStrategy, handler, persisted); + 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(RewritableSortedSet> set, SortedSetFileHandler handler) { - return new FileByteDocumentSortedMap(set, handler); + public FileByteDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileByteDocumentSortedMap((FileSortedMap)map, handler); } @Override - public FileByteDocumentSortedMap newInstance(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) + public FileByteDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { - return new FileByteDocumentSortedMap(set, handler, persist); + return new FileByteDocumentSortedMap((FileSortedMap)map, handler, persist); } } } diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java deleted file mode 100644 index 80b7483d013..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeySortedMap.java +++ /dev/null @@ -1,218 +0,0 @@ -package datawave.query.util.sortedmap; - -import datawave.query.util.sortedmap.rfile.RFileKeyInputStream; -import datawave.query.util.sortedmap.rfile.RFileKeyOutputStream; -import org.apache.accumulo.core.data.Key; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.util.Comparator; - -/** - * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as - * expected. This will support null contained in the underlying sets 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 FileKeySortedMap extends FileSortedMap { - private static Logger log = Logger.getLogger(FileKeySortedMap.class); - - /** - * Create a file sorted set from another one - * - * @param other - * the other sorted set - */ - public FileKeySortedMap(FileKeySortedMap other) { - super(other); - } - - /** - * Create a file sorted subset from another one - * - * @param other - * the other sorted set - * @param from - * the from key - * @param to - * the to key - */ - public FileKeySortedMap(FileKeySortedMap other, Key from, Key to) { - super(other, from, to); - } - - /** - * Create a persisted sorted set - * - * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag - */ - public FileKeySortedMap(SortedSetFileHandler handler, boolean persisted) { - super(new KeyFileHandler(handler), new Factory(), persisted); - } - - /** - * Create a persisted sorted set - * - * @param comparator - * the key comparator - * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag - */ - public FileKeySortedMap(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { - super(comparator, new KeyFileHandler(handler), new Factory(), persisted); - } - - /** - * Create a persisted sorted set - * - * @param comparator - * the key comparator - * @param rewriteStrategy - * rewrite strategy - * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag - */ - public FileKeySortedMap(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { - super(comparator, rewriteStrategy, new KeyFileHandler(handler), new Factory(), persisted); - } - - /** - * Create an unpersisted sorted set (still in memory) - * - * @param set - * the sorted set - * @param handler - * the sorted set file handler - */ - public FileKeySortedMap(RewritableSortedSet set, SortedSetFileHandler handler) { - super(set, new KeyFileHandler(handler), new Factory()); - } - - /** - * Create an sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling - * all of its entries into memory at once. - * - * @param set - * the sorted set - * @param handler - * the sorted set file handler - * @param persist - * boolean flag for persist - * @throws IOException - * for issues with read/write - */ - public FileKeySortedMap(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { - super(set, new KeyFileHandler(handler), new Factory(), persist); - } - - /** - * This will dump the set to the file, making the set "persisted" - * - * @param handler - * the sorted set file handler - * @throws IOException - * for issues with read/write - */ - public void persist(SortedSetFileHandler handler) throws IOException { - // ensure this handler is wrapped with our handler - super.persist(new KeyFileHandler(handler)); - } - - /** - * Clone this set - */ - @Override - public FileKeySortedMap clone() { - return (FileKeySortedMap) super.clone(); - } - - /** - * A sortedsetfilehandler that can bound the input stream - */ - public static class KeyFileHandler implements BoundedTypedSortedSetFileHandler { - SortedSetFileHandler delegate; - - public KeyFileHandler(SortedSetFileHandler handler) { - this.delegate = handler; - } - - @Override - public SortedSetInputStream getInputStream() throws IOException { - return new RFileKeyInputStream(delegate.getInputStream(), delegate.getSize()); - } - - @Override - public SortedSetInputStream getInputStream(Key start, Key end) throws IOException { - return new RFileKeyInputStream(delegate.getInputStream(), delegate.getSize(), start, end); - } - - @Override - public SortedSetOutputStream getOutputStream() throws IOException { - return new RFileKeyOutputStream(delegate.getOutputStream()); - } - - @Override - public PersistOptions getPersistOptions() { - return delegate.getPersistOptions(); - } - - @Override - public long getSize() { - return delegate.getSize(); - } - - @Override - public void deleteFile() { - delegate.deleteFile(); - } - } - - /** - * A factory for these file sorted sets - */ - public static class Factory implements FileSortedSetFactory { - - @Override - public FileKeySortedMap newInstance(FileSortedMap other) { - return new FileKeySortedMap((FileKeySortedMap) other); - } - - @Override - public FileKeySortedMap newInstance(FileSortedMap other, Key from, Key to) { - return new FileKeySortedMap((FileKeySortedMap) other, from, to); - } - - @Override - public FileKeySortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { - return new FileKeySortedMap(handler, persisted); - } - - @Override - public FileKeySortedMap newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { - return new FileKeySortedMap(comparator, handler, persisted); - } - - @Override - public FileKeySortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { - return new FileKeySortedMap(comparator, rewriteStrategy, handler, persisted); - } - - @Override - public FileKeySortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler) { - return new FileKeySortedMap(set, handler); - } - - @Override - public FileKeySortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { - return new FileKeySortedMap(set, 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 index bab39c1e5fc..4db9f5b127f 100644 --- 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 @@ -2,145 +2,127 @@ import datawave.query.util.sortedmap.rfile.RFileKeyValueInputStream; import datawave.query.util.sortedmap.rfile.RFileKeyValueOutputStream; +import datawave.query.util.sortedset.FileSortedSet; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.log4j.Logger; import java.io.IOException; import java.util.Comparator; -import java.util.Map; +import java.util.SortedMap; /** - * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as - * expected. This will support null contained in the underlying sets iff a comparator is supplied that can handle null values. + * 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> { +public class FileKeyValueSortedMap extends FileSortedMap { private static Logger log = Logger.getLogger(FileKeyValueSortedMap.class); - public static class DefaultKeyValueComparator implements Comparator> { - + public static class DefaultKeyComparator implements Comparator { @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return o1.getKey().compareTo(o2.getKey()); + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); } } /** - * Create a file sorted set from another one + * Create a file sorted map from another one * * @param other - * the other sorted set + * the other sorted map */ public FileKeyValueSortedMap(FileKeyValueSortedMap other) { super(other); } /** - * Create a file sorted subset from another one + * Create a file sorted submap from another one * * @param other - * the other sorted set + * the other sorted map * @param from * the from key * @param to * the to key */ - public FileKeyValueSortedMap(FileKeyValueSortedMap other, Map.Entry from, Map.Entry to) { + public FileKeyValueSortedMap(FileKeyValueSortedMap other, Key from, Key to) { super(other, from, to); } /** - * Create a persisted sorted set + * Create a persisted sorted map * * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag */ - public FileKeyValueSortedMap(SortedSetFileHandler handler, boolean persisted) { - this(new DefaultKeyValueComparator(), handler, persisted); + public FileKeyValueSortedMap(SortedMapFileHandler handler, boolean persisted) { + this(new DefaultKeyComparator(), handler, persisted); } /** - * Create a persisted sorted set + * Create a persisted sorted map * * @param comparator * the key comparator * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag */ - public FileKeyValueSortedMap(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultKeyValueComparator() : comparator), new KeyValueFileHandler(handler), new Factory(), + public FileKeyValueSortedMap(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { + super(((comparator == null) ? new DefaultKeyComparator() : comparator), new KeyValueFileHandler(handler), new Factory(), persisted); } /** - * Create a persisted sorted set - * - * @param comparator - * the key comparator - * @param rewriteStrategy - * rewrite strategy - * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag - */ - public FileKeyValueSortedMap(Comparator> comparator, RewriteStrategy> rewriteStrategy, - SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultKeyValueComparator() : comparator), rewriteStrategy, new KeyValueFileHandler(handler), - new Factory(), persisted); - } - - /** - * Create an unpersisted sorted set (still in memory) + * Create an unpersisted sorted map (still in memory) * - * @param set - * the sorted set + * @param map + * the sorted map * @param handler - * the sorted set file handler + * the sorted map file handler */ - public FileKeyValueSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler) { - super(set, new KeyValueFileHandler(handler), new Factory()); + public FileKeyValueSortedMap(SortedMap map, SortedMapFileHandler handler) { + super(map, new KeyValueFileHandler(handler), new Factory()); } /** - * Create an sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * 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 set - * the sorted set + * @param map + * the sorted map * @param handler - * the sorted set file handler + * the sorted map file handler * @param persist * boolean flag for persist * @throws IOException * for issues with read/write */ - public FileKeyValueSortedMap(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) throws IOException { - super(set, new KeyValueFileHandler(handler), new Factory(), persist); + public FileKeyValueSortedMap(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + super(map, new KeyValueFileHandler(handler), new Factory(), persist); } /** - * This will dump the set to the file, making the set "persisted" + * This will dump the map to the file, making the map "persisted" * * @param handler - * the sorted set file handler + * the sorted map file handler * @throws IOException * for issues with read/write */ - public void persist(SortedSetFileHandler handler) throws IOException { + public void persist(SortedMapFileHandler handler) throws IOException { // ensure this handler is wrapped with our handler super.persist(new KeyValueFileHandler(handler)); } /** - * Clone this set + * Clone this map */ @Override public FileKeyValueSortedMap clone() { @@ -148,32 +130,32 @@ public FileKeyValueSortedMap clone() { } /** - * A sortedsetfilehandler that can bound the input stream + * A SortedMapfilehandler that can bound the input stream */ - public static class KeyValueFileHandler implements BoundedTypedSortedSetFileHandler> { - SortedSetFileHandler delegate; + public static class KeyValueFileHandler implements BoundedTypedSortedMapFileHandler { + SortedMapFileHandler delegate; - public KeyValueFileHandler(SortedSetFileHandler handler) { + public KeyValueFileHandler(SortedMapFileHandler handler) { this.delegate = handler; } @Override - public SortedSetInputStream> getInputStream() throws IOException { + public SortedMapInputStream getInputStream() throws IOException { return new RFileKeyValueInputStream(delegate.getInputStream(), delegate.getSize()); } @Override - public SortedSetInputStream> getInputStream(Map.Entry start, Map.Entry end) throws IOException { + public SortedMapInputStream getInputStream(Key start, Key end) throws IOException { return new RFileKeyValueInputStream(delegate.getInputStream(), delegate.getSize(), start, end); } @Override - public SortedSetOutputStream getOutputStream() throws IOException { + public SortedMapOutputStream getOutputStream() throws IOException { return new RFileKeyValueOutputStream(delegate.getOutputStream()); } @Override - public PersistOptions getPersistOptions() { + public FileSortedSet.PersistOptions getPersistOptions() { return delegate.getPersistOptions(); } @@ -189,45 +171,47 @@ public void deleteFile() { } /** - * A factory for these file sorted sets + * A factory for these file sorted maps */ - public static class Factory implements FileSortedSetFactory> { + public static class Factory implements FileSortedMapFactory { @Override - public FileKeyValueSortedMap newInstance(FileSortedMap> other) { + public FileKeyValueSortedMap newInstance(FileSortedMap other) { return new FileKeyValueSortedMap((FileKeyValueSortedMap) other); } @Override - public FileKeyValueSortedMap newInstance(FileSortedMap> other, Map.Entry from, Map.Entry to) { + public FileKeyValueSortedMap newInstance(FileSortedMap other, Key from, Key to) { return new FileKeyValueSortedMap((FileKeyValueSortedMap) other, from, to); } @Override - public FileKeyValueSortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + public FileKeyValueSortedMap newInstance(SortedMapFileHandler handler, boolean persisted) { return new FileKeyValueSortedMap(handler, persisted); } @Override - public FileKeyValueSortedMap newInstance(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { + public FileKeyValueSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { return new FileKeyValueSortedMap(comparator, handler, persisted); } @Override - public FileKeyValueSortedMap newInstance(Comparator> comparator, RewriteStrategy> rewriteStategy, - SortedSetFileHandler handler, boolean persisted) { - return new FileKeyValueSortedMap(comparator, rewriteStategy, handler, persisted); + 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(RewritableSortedSet> set, SortedSetFileHandler handler) { - return new FileKeyValueSortedMap(set, handler); + public FileKeyValueSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileKeyValueSortedMap(map, handler); } @Override - public FileKeyValueSortedMap newInstance(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) + public FileKeyValueSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { - return new FileKeyValueSortedMap(set, handler, persist); + 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 index 780fc5208fd..dfe6854f713 100644 --- 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 @@ -1,6 +1,7 @@ package datawave.query.util.sortedmap; -import org.apache.accumulo.core.data.Key; +import datawave.query.util.sortedset.FileSortedSet; +import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; import org.apache.log4j.Logger; import java.io.IOException; @@ -10,55 +11,57 @@ import java.io.OutputStream; import java.io.Serializable; import java.util.Comparator; +import java.util.Map; +import java.util.SortedMap; /** - * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as - * expected. This will support null contained in the underlying sets iff a comparator is supplied that can handle null values. + * 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 FileSerializableSortedMap extends FileSortedMap { +public class FileSerializableSortedMap extends FileSortedMap { private static Logger log = Logger.getLogger(FileSerializableSortedMap.class); /** - * Create a file sorted set from another one + * Create a file sorted map from another one * * @param other - * the other sorted set + * the other sorted map */ public FileSerializableSortedMap(FileSerializableSortedMap other) { super(other); } /** - * Create a file sorted subset from another one + * Create a file sorted submap from another one * * @param other - * the other sorted set + * the other sorted map * @param from * the from file * @param to * the to file */ - public FileSerializableSortedMap(FileSerializableSortedMap other, E from, E to) { + public FileSerializableSortedMap(FileSerializableSortedMap other, K from, K to) { super(other, from, to); } /** - * Create a persisted sorted set + * Create a persisted sorted map * * @param handler * a file handler * @param persisted * persisted boolean flag */ - public FileSerializableSortedMap(TypedSortedSetFileHandler handler, boolean persisted) { + public FileSerializableSortedMap(TypedSortedMapFileHandler handler, boolean persisted) { super(handler, new Factory(), persisted); } /** - * Create a persistede sorted set + * Create a persistede sorted map * * @param comparator * a comparator @@ -67,44 +70,28 @@ public FileSerializableSortedMap(TypedSortedSetFileHandler handler, boolean pers * @param persisted * persisted boolean flag */ - public FileSerializableSortedMap(Comparator comparator, TypedSortedSetFileHandler handler, boolean persisted) { + public FileSerializableSortedMap(Comparator comparator, TypedSortedMapFileHandler handler, boolean persisted) { super(comparator, handler, new Factory(), persisted); } /** - * Create a persistede sorted set + * Create an unpersisted sorted map (still in memory) * - * @param comparator - * a comparator - * @param rewriteStrategy - * rewrite strategy - * @param handler - * a file handler - * @param persisted - * persisted boolean flag - */ - public FileSerializableSortedMap(Comparator comparator, RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, boolean persisted) { - super(comparator, rewriteStrategy, handler, new Factory(), persisted); - } - - /** - * Create an unpersisted sorted set (still in memory) - * - * @param set - * a sorted set + * @param map + * a sorted map * @param handler * a file handler */ - public FileSerializableSortedMap(RewritableSortedSet set, TypedSortedSetFileHandler handler) { - super(set, handler, new Factory()); + public FileSerializableSortedMap(SortedMap map, TypedSortedMapFileHandler handler) { + super(map, handler, new Factory()); } /** - * Create an sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * 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 set - * a sorted set + * @param map + * a sorted map * @param handler * a file handler * @param persist @@ -112,12 +99,12 @@ public FileSerializableSortedMap(RewritableSortedSet set, TypedSortedSetFileH * @throws IOException * for issues with read/write */ - public FileSerializableSortedMap(RewritableSortedSet set, TypedSortedSetFileHandler handler, boolean persist) throws IOException { - super(set, handler, new Factory(), persist); + public FileSerializableSortedMap(SortedMap map, TypedSortedMapFileHandler handler, boolean persist) throws IOException { + super(map, handler, new Factory(), persist); } /** - * Persist a set using the specified handler + * Persist a map using the specified handler * * @param handler * a file handler @@ -125,37 +112,37 @@ public FileSerializableSortedMap(RewritableSortedSet set, TypedSortedSetFileH * for issues with read/write */ @Override - public void persist(SortedSetFileHandler handler) throws IOException { + public void persist(SortedMapFileHandler handler) throws IOException { super.persist(new SerializableFileHandler(handler)); } @Override - public FileSerializableSortedMap clone() { + public FileSerializableSortedMap clone() { return (FileSerializableSortedMap) super.clone(); } /** - * A sortedsetfilehandler that can handler serializable objects + * A SortedMapfilehandler that can handler serializable objects */ - public static class SerializableFileHandler implements TypedSortedSetFileHandler { - SortedSetFileHandler delegate; + public static class SerializableFileHandler implements TypedSortedMapFileHandler { + SortedMapFileHandler delegate; - public SerializableFileHandler(SortedSetFileHandler handler) { + public SerializableFileHandler(SortedMapFileHandler handler) { this.delegate = handler; } @Override - public SortedSetInputStream getInputStream() throws IOException { + public SortedMapInputStream getInputStream() throws IOException { return new SerializableInputStream(delegate.getInputStream(), delegate.getSize()); } @Override - public SortedSetOutputStream getOutputStream() throws IOException { + public SortedMapOutputStream getOutputStream() throws IOException { return new SerializableOutputStream(delegate.getOutputStream()); } @Override - public PersistOptions getPersistOptions() { + public FileSortedSet.PersistOptions getPersistOptions() { return delegate.getPersistOptions(); } @@ -170,7 +157,7 @@ public void deleteFile() { } } - public static class SerializableInputStream implements SortedSetInputStream { + public static class SerializableInputStream implements SortedMapInputStream { private final InputStream stream; private ObjectInputStream delegate; private final long length; @@ -188,9 +175,11 @@ private ObjectInputStream getDelegate() throws IOException { } @Override - public E readObject() throws IOException { + public Map.Entry readObject() throws IOException { try { - return (E) getDelegate().readObject(); + K key = (K)getDelegate().readObject(); + V value = (V)getDelegate().readObject(); + return new UnmodifiableMapEntry<>(key, value); } catch (IOException ioe) { return null; } catch (ClassNotFoundException nnfe) { @@ -228,7 +217,7 @@ public void close() { } } - public static class SerializableOutputStream implements SortedSetOutputStream { + public static class SerializableOutputStream implements FileSortedMap.SortedMapOutputStream { private ObjectOutputStream delegate; public SerializableOutputStream(OutputStream stream) throws IOException { @@ -236,8 +225,9 @@ public SerializableOutputStream(OutputStream stream) throws IOException { } @Override - public void writeObject(E obj) throws IOException { - delegate.writeObject(obj); + public void writeObject(K key, V value) throws IOException { + delegate.writeObject(key); + delegate.writeObject(value); } @Override @@ -255,44 +245,46 @@ public void close() throws IOException { } /** - * A factory for this set + * A factory for this map */ - public static class Factory implements FileSortedSetFactory { + public static class Factory implements FileSortedMapFactory { @Override - public FileSerializableSortedMap newInstance(FileSortedMap other) { + public FileSerializableSortedMap newInstance(FileSortedMap other) { return new FileSerializableSortedMap((FileSerializableSortedMap) other); } @Override - public FileSerializableSortedMap newInstance(FileSortedMap other, E from, E to) { + public FileSerializableSortedMap newInstance(FileSortedMap other, K from, K to) { return new FileSerializableSortedMap((FileSerializableSortedMap) other, from, to); } @Override - public FileSerializableSortedMap newInstance(SortedSetFileHandler handler, boolean persisted) { + public FileSerializableSortedMap newInstance(SortedMapFileHandler handler, boolean persisted) { return new FileSerializableSortedMap(new SerializableFileHandler(handler), persisted); } @Override - public FileSerializableSortedMap newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + public FileSerializableSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { return new FileSerializableSortedMap(comparator, new SerializableFileHandler(handler), persisted); } @Override - public FileSerializableSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, + public FileSerializableSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, boolean persisted) { - return new FileSerializableSortedMap(comparator, rewriteStrategy, new SerializableFileHandler(handler), persisted); + FileSerializableSortedMap map = new FileSerializableSortedMap(comparator, new SerializableFileHandler(handler), persisted); + map.setRewriteStrategy(rewriteStrategy); + return map; } @Override - public FileSerializableSortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler) { - return new FileSerializableSortedMap(set, new SerializableFileHandler(handler)); + public FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + return new FileSerializableSortedMap(map, new SerializableFileHandler(handler)); } @Override - public FileSerializableSortedMap newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { - return new FileSerializableSortedMap(set, new SerializableFileHandler(handler), persist); + 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 index 589ba57e90b..bf6ede43d25 100644 --- 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 @@ -5,220 +5,207 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.OutputStream; -import java.lang.reflect.Array; +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.SortedSet; +import java.util.Set; +import java.util.SortedMap; import java.util.TreeMap; -import java.util.TreeSet; -import java.util.function.Predicate; -import java.util.stream.Collectors; +import datawave.query.util.sortedset.FileSortedSet; +import org.apache.commons.collections.IteratorUtils; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; import org.apache.log4j.Logger; import datawave.webservice.query.exception.DatawaveErrorCode; import datawave.webservice.query.exception.QueryException; /** - * A sorted set that can be persisted into a file and still be read in its persisted state. The set can always be re-loaded and then all operations will work as + * 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. * - * @param - * type of set + * A RewriteStrategy can be supplied that will determine whether a value gets replaced when putting a key,value pair. + * + * @param + * type of map */ -public abstract class FileSortedMap extends RewritableSortedMapImpl implements SortedSet, Cloneable { - private static Logger log = Logger.getLogger(FileSortedMap.class); - protected boolean persisted = false; - protected E[] range; +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 - public TypedSortedSetFileHandler handler; - // The sort set factory - public FileSortedSetFactory factory; + 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 set from another one + * Create a file sorted map from another one * * @param other - * the other sorted set + * the other sorted map */ - public FileSortedMap(FileSortedMap other) { - super(other); + 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 subset from another one + * Create a file sorted submap from another one * * @param other - * the other sorted set + * the other sorted map * @param from * the from key * @param to * the to key */ - public FileSortedMap(FileSortedMap other, E from, E to) { + public FileSortedMap(FileSortedMap other, K from, K to) { this(other); if (from != null || to != null) { if (persisted) { - this.range = (E[]) new Object[] {getStart(from), getEnd(to)}; + this.range = (K[]) new Object[] {getStart(from), getEnd(to)}; } else if (to == null) { - this.set = this.set.tailMap(from, true); + this.map = this.map.tailMap(from); } else if (from == null) { - this.set = this.set.headMap(to, false); + this.map = this.map.headMap(to); } else { - this.set = this.set.subMap(from, true, to, false); + this.map = this.map.subMap(from, to); } } } /** - * Create a persisted sorted set - * - * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag - * @param factory - * the sorted set factory - */ - public FileSortedMap(TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { - this.handler = handler; - this.factory = factory; - this.set = new TreeMap<>(); - this.persisted = persisted; - } - - /** - * Create a persisted sorted set + * Create a persisted sorted map * - * @param rewriteStrategy - * the item rewrite strategy * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag * @param factory - * the sorted set factory + * the sorted map factory */ - public FileSortedMap(RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { - super(rewriteStrategy); + public FileSortedMap(TypedSortedMapFileHandler handler, FileSortedMapFactory factory, boolean persisted) { this.handler = handler; this.factory = factory; - this.set = new TreeMap<>(); + this.map = new TreeMap<>(); this.persisted = persisted; } /** - * Create a persisted sorted set + * Create a persisted sorted map * * @param comparator * the key comparator - * @param rewriteStrategy - * the item rewrite strategy * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag * @param factory - * the sorted set factory + * the sorted map factory */ - public FileSortedMap(Comparator comparator, RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, - boolean persisted) { - super(comparator, rewriteStrategy); + public FileSortedMap(Comparator comparator, TypedSortedMapFileHandler handler, FileSortedMapFactory factory, boolean persisted) { this.handler = handler; this.factory = factory; - this.set = new TreeMap<>(comparator); + this.map = new TreeMap<>(comparator); this.persisted = persisted; } /** - * Create a persisted sorted set + * Create an unpersisted sorted map (still in memory) * - * @param comparator - * the key comparator + * @param map + * a sorted map * @param handler - * the sorted set file handler - * @param persisted - * a persisted boolean flag + * the sorted map file handler * @param factory - * the sorted set factory + * the sorted map factory */ - public FileSortedMap(Comparator comparator, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { - super(comparator); + public FileSortedMap(SortedMap map, TypedSortedMapFileHandler handler, FileSortedMapFactory factory) { this.handler = handler; this.factory = factory; - this.set = new TreeMap<>(comparator); - this.persisted = persisted; - } - - /** - * Create an unpersisted sorted set (still in memory) - * - * @param set - * a sorted set - * @param handler - * the sorted set file handler - * @param factory - * the sorted set factory - */ - public FileSortedMap(RewritableSortedMap set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory) { - this.handler = handler; - this.factory = factory; - this.set = set.stream().collect(Collectors.toMap(value -> value, value -> value, (l, r) -> l, () -> new TreeMap<>(set.comparator()))); - this.rewriteStrategy = set.getRewriteStrategy(); + this.map = new TreeMap<>(map); this.persisted = false; } /** - * Create a sorted set out of another sorted set. If persist is true, then the set will be directly persisted using the set's iterator which avoid pulling + * 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 set - * a sorted set + * @param map + * a sorted map * @param handler - * the sorted set file handler + * the sorted map file handler * @param factory - * the sorted set factory + * the sorted map factory * @param persist * the persist boolean flag * @throws IOException * for issues with read/write */ - public FileSortedMap(RewritableSortedMap set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persist) throws IOException { - this(set, handler, factory); - if (persist) { - persist(set, handler); + 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; } } + @Override + public RewriteStrategy getRewriteStrategy() { + return rewriteStrategy; + } + + @Override + public void setRewriteStrategy(RewriteStrategy rewriteStrategy) { + this.rewriteStrategy = rewriteStrategy; + } + /** - * This will revert this set to whatever contents are in the underlying file, making the set "persisted". This is intended to be used following a load - * command when no changes were actually made the the set If the persist options included verification, then the files will be verified prior to unloading. + * 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.set.size(), Collections.emptyList()); - this.set.clear(); + verifyPersistance(handler, this.map.size(), Collections.emptyList()); + this.map.clear(); persisted = true; } } /** - * This will dump the set to the file, making the set "persisted" + * This will dump the map to the file, making the map "persisted" * * @throws IOException * for issues with read/write @@ -228,44 +215,44 @@ public void persist() throws IOException { } /** - * This will dump the set to the file, making the set "persisted" + * 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(TypedSortedSetFileHandler handler) throws IOException { + public void persist(TypedSortedMapFileHandler handler) throws IOException { if (!persisted) { - persist(this.set.navigableKeySet(), handler); - this.set.clear(); + persist(this.map, handler); + this.map.clear(); persisted = true; } } /** - * This will dump the set to a file, making the set "persisted" The implementation is expected to wrap the handler with a TypedSortedSetFileHandler and the - * call persist(TypedSortedSetFileHandler handler) + * 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 set file handler + * the sorted map file handler * @throws IOException * for issues with read/write */ - public abstract void persist(SortedSetFileHandler handler) throws IOException; + public abstract void persist(SortedMapFileHandler handler) throws IOException; /** - * Persist the supplied set to a file as defined by this classes sorted set file handler. + * Persist the supplied map to a file as defined by this classes sorted map file handler. * - * @param set - * the set + * @param map + * the map * @param handler * the handler * @throws IOException * for issues with read/write * */ - private void persist(SortedSet set, TypedSortedSetFileHandler handler) throws IOException { + private void persist(SortedMap map, TypedSortedMapFileHandler handler) throws IOException { if (log.isDebugEnabled()) { log.debug("Persisting " + handler); } @@ -273,24 +260,24 @@ private void persist(SortedSet set, TypedSortedSetFileHandler handler) throws long start = System.currentTimeMillis(); try { // assign the passed in file handler - // if we can't persist, we will reset to null + // if we can't persist, we will remap to null this.handler = handler; int actualSize = 0; - PersistOptions persistOptions = handler.getPersistOptions(); - List setToVerify = new ArrayList<>(); - try (SortedSetOutputStream stream = handler.getOutputStream()) { - for (E t : set) { - stream.writeObject(t); - if (persistOptions.isVerifyElements() && setToVerify.size() < persistOptions.getNumElementsToVerify()) { - setToVerify.add(t); + 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, setToVerify); + verifyPersistance(handler, actualSize, mapToVerify); } catch (IOException e) { handler.deleteFile(); @@ -304,19 +291,19 @@ private void persist(SortedSet set, TypedSortedSetFileHandler handler) throws } } - private void verifyPersistance(TypedSortedSetFileHandler handler, int size, List setToVerify) throws IOException { + 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"); } - PersistOptions persistOptions = handler.getPersistOptions(); + FileSortedSet.PersistOptions persistOptions = handler.getPersistOptions(); // now verify the first n objects were written correctly - if (persistOptions.isVerifyElements() && !setToVerify.isEmpty()) { - try (SortedSetInputStream inStream = handler.getInputStream()) { + if (persistOptions.isVerifyElements() && !mapToVerify.isEmpty()) { + try (SortedMapInputStream inStream = handler.getInputStream()) { int count = 0; - for (E t : setToVerify) { + for (Map.Entry t : mapToVerify) { count++; - E input = inStream.readObject(); + Map.Entry input = inStream.readObject(); if (!equals(t, input)) { throw new IOException("Failed to verify element " + count + " was written"); } @@ -340,13 +327,13 @@ private void verifyPersistance(TypedSortedSetFileHandler handler, int size, List * for issues with read/write */ private int readSize() throws IOException { - try (SortedSetInputStream inStream = handler.getInputStream()) { + try (SortedMapInputStream inStream = handler.getInputStream()) { return inStream.readSize(); } } /** - * This will read the file into an in-memory set, making this file "unpersisted" + * This will read the file into an in-memory map, making this file "unpersisted" * * @throws IOException * for issues with read/write @@ -355,10 +342,10 @@ private int readSize() throws IOException { */ public void load() throws IOException, ClassNotFoundException { if (persisted) { - try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - E obj = stream.readObject(); + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry obj = stream.readObject(); while (obj != null) { - super.add(obj); + map.put(obj.getKey(), obj.getValue()); obj = stream.readObject(); } } @@ -366,35 +353,38 @@ public void load() throws IOException, ClassNotFoundException { } } - protected E readObject(ObjectInputStream stream) { + protected Map.Entry readObject(ObjectInputStream stream) { try { - return (E) stream.readObject(); + 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, E obj) throws IOException { - stream.writeObject(obj); + protected void writeObject(ObjectOutputStream stream, K key, V value) throws IOException { + stream.writeObject(key); + stream.writeObject(value); } /* - * Is this set persisted? + * Is this map persisted? */ public boolean isPersisted() { return persisted; } /** - * Get the size of the set. Note if the set has been persisted, then this may be an upper bound on the size. + * 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 (isSubset()) { - throw new IllegalStateException("Unable to determine size of a subset of a persisted set. Please call load() first."); + if (isSubmap()) { + throw new IllegalStateException("Unable to determine size of a submap of a persisted map. Please call load() first."); } try { return readSize(); @@ -402,7 +392,7 @@ public int size() { throw new IllegalStateException("Unable to get size from file", e); } } else { - return super.size(); + return map.size(); } } @@ -410,7 +400,7 @@ public int size() { public boolean isEmpty() { // must attempt to read the first element to be sure if persisted try { - first(); + firstKey(); return false; } catch (NoSuchElementException e) { return true; @@ -419,176 +409,85 @@ public boolean isEmpty() { @SuppressWarnings("unchecked") @Override - public boolean contains(Object o) { + public boolean containsKey(Object o) { if (persisted) { - E t = (E) o; - for (E next : this) { - if (equals(next, t)) { + K t = (K) o; + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry first = stream.readObject(); + if (equals(first.getKey(), t)) { return true; } + } catch (Exception e) { + return false; } return false; } else { - return super.contains(o); - } - } - - @Override - public Iterator iterator() { - if (persisted) { - return new FileIterator(); - } else { - return super.iterator(); - } - } - - @Override - public Object[] toArray() { - if (persisted) { - try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - Object[] data = new Object[readSize()]; - int index = 0; - E obj = stream.readObject(); - while (obj != null) { - data[index++] = obj; - obj = stream.readObject(); - } - if (index < data.length) { - Object[] dataCpy = new Object[index]; - System.arraycopy(data, 0, dataCpy, 0, index); - data = dataCpy; - } - return data; - } catch (IOException e) { - throw new IllegalStateException("Unable to read file into a complete set", e); - } - } else { - return super.toArray(); - } - } - - @SuppressWarnings({"unchecked"}) - @Override - public T[] toArray(T[] a) { - if (persisted) { - try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - T[] data = a; - int index = 0; - T obj = (T) stream.readObject(); - while (obj != null) { - if (index > data.length) { - T[] dataCpy = (T[]) (Array.newInstance(a.getClass().getComponentType(), data.length + (data.length / 2))); - System.arraycopy(data, 0, dataCpy, 0, data.length); - data = dataCpy; - } - data[index++] = obj; - obj = (T) stream.readObject(); - } - // if not resized - if (data == a) { - // ensure extra elements are set to null - for (; index < data.length; index++) { - data[index] = null; - } - } else if (index < data.length) { - T[] dataCpy = (T[]) (Array.newInstance(a.getClass().getComponentType(), index)); - System.arraycopy(data, 0, dataCpy, 0, index); - data = dataCpy; - } - return data; - } catch (IOException e) { - throw new IllegalStateException("Unable to read file into a complete set", e); - } - } else { - return super.toArray(a); - } - } - - @Override - public boolean add(E e) { - if (persisted) { - throw new IllegalStateException("Cannot add an element to a persisted FileSortedSet. Please call load() first."); - } - return super.add(e); - } - - @Override - public boolean remove(Object o) { - if (persisted) { - throw new IllegalStateException("Cannot remove an element to a persisted FileSortedSet. Please call load() first."); - } else { - return (super.remove(o)); + return map.containsKey(o); } } - @SuppressWarnings("unchecked") @Override - public boolean containsAll(Collection c) { - if (c.isEmpty()) { - return true; - } + public boolean containsValue(Object o) { if (persisted) { - try { - SortedSet all = new TreeSet<>(set.comparator()); - for (Object o : c) { - all.add((E) o); - } - if (all.isEmpty()) { + V t = (V) o; + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry first = stream.readObject(); + if (first.getValue().equals(t)) { return true; } - try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - E obj = stream.readObject(); - while (obj != null) { - if (all.remove(obj)) { - if (all.isEmpty()) { - return true; - } - } - obj = stream.readObject(); - } - } } catch (Exception e) { - throw new IllegalStateException("Unable to read file into a complete set", e); + return false; } return false; } else { - return super.containsAll(c); + return map.containsValue(o); } } @Override - public boolean addAll(Collection c) { + public V get(Object key) { if (persisted) { - throw new IllegalStateException("Unable to add to a persisted FileSortedSet. Please call load() first."); + K t = (K) key; + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry first = stream.readObject(); + if (equals(first.getKey(), t)) { + return first.getValue(); + } + } catch (Exception e) { + return null; + } + return null; } else { - return super.addAll(c); + return map.get(key); } } @Override - public boolean retainAll(Collection c) { + public V put(K key, V value) { if (persisted) { - throw new IllegalStateException("Unable to modify a persisted FileSortedSet. Please call load() first."); + throw new IllegalStateException("Cannot add an element to a persisted FileSortedMap. Please call load() first."); } else { - return super.retainAll(c); + V previous = map.get(key); + if ((previous == null) || (rewriteStrategy == null) || (rewriteStrategy.rewrite(key, previous, value))) { + map.put(key, value); + } + return previous; } } @Override - public boolean removeAll(Collection c) { + public V remove(Object o) { if (persisted) { - throw new IllegalStateException("Unable to remove from a persisted FileSortedSet. Please call load() first."); + throw new IllegalStateException("Cannot remove an element to a persisted FileSortedMap. Please call load() first."); } else { - return super.removeAll(c); + return map.remove(o); } } @Override - public boolean removeIf(Predicate filter) { - if (persisted) { - throw new IllegalStateException("Unable to remove from a persisted FileSortedSet. Please call load() first."); - } else { - return super.removeIf(filter); + public void putAll(Map m) { + for (Entry entry : m.entrySet()) { + put(entry.getKey(), entry.getValue()); } } @@ -598,113 +497,168 @@ public void clear() { handler.deleteFile(); persisted = false; } else { - super.clear(); + map.clear(); } } @Override - public Comparator comparator() { - return super.comparator(); + public Comparator comparator() { + return map.comparator(); } @Override - public RewritableSortedMap subSet(E fromElement, E toElement) { + public SortedMap subMap(K fromElement, K toElement) { return factory.newInstance(this, fromElement, toElement); } @Override - public RewritableSortedMap headSet(E toElement) { + public SortedMap headMap(K toElement) { return factory.newInstance(this, null, toElement); } @Override - public RewritableSortedMap tailSet(E fromElement) { + public SortedMap tailMap(K fromElement) { return factory.newInstance(this, fromElement, null); } @Override - public E first() { + public K firstKey() { if (persisted) { - try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - E object = stream.readObject(); - if (object == null) { - throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR)); - } else { - return object; - } + 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 { - return super.first(); + } else if (!map.isEmpty()) { + return map.firstKey(); } + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR)); } @Override - public E last() { + public K lastKey() { if (persisted) { - boolean gotLast = false; - E last = null; - try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - last = stream.readObject(); - E next = stream.readObject(); + try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + Map.Entry last = stream.readObject(); + Map.Entry next = stream.readObject(); while (next != null) { last = next; - gotLast = true; next = stream.readObject(); } + return last.getKey(); } catch (Exception e) { throw new IllegalStateException(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR, e)); } - if (gotLast) { - return last; - } else { - throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR)); - } - } else { - return super.last(); + } else if (!map.isEmpty()) { + return map.lastKey(); } + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR)); } - @Override - public String toString() { + private Iterator> iterator() { if (persisted) { - return handler.toString(); + return new FileIterator(); } else { - return super.toString(); + return map.entrySet().iterator(); } } + @Override + public Set keySet() { + return new AbstractSet() { + + @Override + public Iterator iterator() { + return IteratorUtils.transformedIterator(new FileIterator(), + 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(new FileIterator(), + 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 new FileIterator(); + } + + @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() { + public FileSortedMap clone() { return factory.newInstance(this); } /* Some utilities */ - private boolean equals(E o1, E o2) { + 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 (set.comparator() == null) { + if (map.comparator() == null) { return o1.equals(o2); } else { - return set.comparator().compare(o1, o2) == 0; + return map.comparator().compare(o1, o2) == 0; } } } - private E getStart() { - return (isSubset() ? range[0] : null); + private K getStart() { + return (isSubmap() ? range[0] : null); } - private E getStart(E from) { - E start = getStart(); + private K getStart(K from) { + K start = getStart(); if (start == null) { return from; } else if (from == null) { @@ -716,12 +670,12 @@ private E getStart(E from) { } } - private E getEnd() { - return (isSubset() ? range[1] : null); + private K getEnd() { + return (isSubmap() ? range[1] : null); } - private E getEnd(E to) { - E end = getEnd(); + private K getEnd(K to) { + K end = getEnd(); if (end == null) { return to; } else if (to == null) { @@ -733,24 +687,24 @@ private E getEnd(E to) { } } - private boolean isSubset() { + private boolean isSubmap() { return (range != null); } - private int compare(E a, E b) { - return (this.set.comparator() != null) ? this.set.comparator().compare(a, b) : ((Comparable) a).compareTo(b); + private int compare(K a, K b) { + return (this.map.comparator() != null) ? this.map.comparator().compare(a, b) : ((Comparable) a).compareTo(b); } - public BoundedTypedSortedSetFileHandler getBoundedFileHandler() { - return new DefaultBoundedTypedSortedSetFileHandler(); + public BoundedTypedSortedMapFileHandler getBoundedFileHandler() { + return new DefaultBoundedTypedSortedMapFileHandler(); } /** - * This is the iterator for a persisted FileSortedSet + * This is the iterator for a persisted FileSortedMap */ - protected class FileIterator implements Iterator { - private SortedSetInputStream stream; - private E next; + protected class FileIterator implements Iterator> { + private SortedMapInputStream stream; + private Map.Entry next; public FileIterator() { try { @@ -782,13 +736,13 @@ public boolean hasNext() { } @Override - public E next() { + public Map.Entry next() { if (!hasNext()) { QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); } try { - E rtrn = next; + Map.Entry rtrn = next; next = stream.readObject(); if (next == null) { cleanup(); @@ -814,12 +768,12 @@ protected void finalize() throws Throwable { } /** - * An interface for a sorted set factory + * An interface for a sorted map factory * - * @param + * @param * type of the factory */ - public interface FileSortedSetFactory { + public interface FileSortedMapFactory { /** * factory method * @@ -827,7 +781,7 @@ public interface FileSortedSetFactory { * the other factory * @return a new instance */ - FileSortedMap newInstance(FileSortedMap other); + FileSortedMap newInstance(FileSortedMap other); /** * factory method @@ -840,18 +794,18 @@ public interface FileSortedSetFactory { * to instance * @return a new instance */ - FileSortedMap newInstance(FileSortedMap other, E from, E to); + FileSortedMap newInstance(FileSortedMap other, K from, K to); /** * factory method * * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag * @return a new instance */ - FileSortedMap newInstance(SortedSetFileHandler handler, boolean persisted); + FileSortedMap newInstance(SortedMapFileHandler handler, boolean persisted); /** * Factory method @@ -859,12 +813,12 @@ public interface FileSortedSetFactory { * @param comparator * the key comparator * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag * @return a new instance */ - FileSortedMap newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted); + FileSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted); /** * Factory method @@ -872,50 +826,52 @@ public interface FileSortedSetFactory { * @param comparator * the key comparator * @param rewriteStrategy - * the collision rewrite strategy + * the rewrite strategy * @param handler - * the sorted set file handler + * the sorted map file handler * @param persisted * a persisted boolean flag * @return a new instance */ - FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted); + FileSortedMap newInstance(Comparator comparator, + RewriteStrategy rewriteStrategy, + SortedMapFileHandler handler, boolean persisted); /** - * Create an unpersisted sorted set (still in memory) + * Create an unpersisted sorted map (still in memory) * - * @param set - * the sorted set + * @param map + * the sorted map * @param handler - * the sorted set file handler + * the sorted map file handler * @return a new instance */ - FileSortedMap newInstance(RewritableSortedMap set, SortedSetFileHandler handler); + FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler); /** * factory method * - * @param set - * the sorted set + * @param map + * the sorted map * @param handler - * the sorted set file 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(RewritableSortedMap set, SortedSetFileHandler handler, boolean persist) throws IOException; + FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException; } /** - * A sorted set input stream + * A sorted map input stream * - * @param + * @param * type of the stream */ - public interface SortedSetInputStream extends AutoCloseable { - E readObject() throws IOException; + public interface SortedMapInputStream extends AutoCloseable { + Map.Entry readObject() throws IOException; int readSize() throws IOException; @@ -923,13 +879,13 @@ public interface SortedSetInputStream extends AutoCloseable { } /** - * A sorted set output stream + * A sorted map output stream * - * @param + * @param * type of the stream */ - public interface SortedSetOutputStream extends AutoCloseable { - void writeObject(E obj) throws IOException; + public interface SortedMapOutputStream extends AutoCloseable { + void writeObject(K key, V value) throws IOException; void writeSize(int size) throws IOException; @@ -940,7 +896,7 @@ public interface SortedSetOutputStream extends AutoCloseable { * A factory that will provide the input stream and output stream to the same underlying file. * */ - public interface SortedSetFileHandler { + public interface SortedMapFileHandler { /** * Return the input stream * @@ -953,7 +909,7 @@ public interface SortedSetFileHandler { /** * Return the output stream * - * @return the sorted set output stream + * @return the sorted map output stream * @throws IOException * for problems with read/write */ @@ -964,7 +920,7 @@ public interface SortedSetFileHandler { * * @return the persistent verification options */ - PersistOptions getPersistOptions(); + FileSortedSet.PersistOptions getPersistOptions(); long getSize(); @@ -975,7 +931,7 @@ public interface SortedSetFileHandler { * A factory that will provide the input stream and output stream to the same underlying file. * */ - public interface TypedSortedSetFileHandler { + public interface TypedSortedMapFileHandler { /** * Return the input stream * @@ -983,23 +939,23 @@ public interface TypedSortedSetFileHandler { * @throws IOException * for problems with read/write */ - SortedSetInputStream getInputStream() throws IOException; + SortedMapInputStream getInputStream() throws IOException; /** * Return the output stream * - * @return the sorted set output stream + * @return the sorted map output stream * @throws IOException * for problems with read/write */ - SortedSetOutputStream getOutputStream() throws IOException; + SortedMapOutputStream getOutputStream() throws IOException; /** * Get the persistent verification options * * @return persistent verification options */ - PersistOptions getPersistOptions(); + FileSortedSet.PersistOptions getPersistOptions(); long getSize(); @@ -1008,10 +964,10 @@ public interface TypedSortedSetFileHandler { /** * 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 subset. + * stream submap. * */ - public interface BoundedTypedSortedSetFileHandler extends TypedSortedSetFileHandler { + public interface BoundedTypedSortedMapFileHandler extends TypedSortedMapFileHandler { /** * Return the input stream * @@ -1023,34 +979,34 @@ public interface BoundedTypedSortedSetFileHandler extends TypedSortedSetFileH * @throws IOException * for problems with read/write */ - SortedSetInputStream getInputStream(E start, E end) throws IOException; + SortedMapInputStream getInputStream(K start, K end) throws IOException; } /** - * A default implementation for a bounded typed sorted set + * A default implementation for a bounded typed sorted map */ - public class DefaultBoundedTypedSortedSetFileHandler implements BoundedTypedSortedSetFileHandler { + public class DefaultBoundedTypedSortedMapFileHandler implements BoundedTypedSortedMapFileHandler { @Override - public SortedSetInputStream getInputStream(E start, E end) throws IOException { - if (handler instanceof FileSortedMap.BoundedTypedSortedSetFileHandler) { - return ((BoundedTypedSortedSetFileHandler) handler).getInputStream(start, end); + 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 SortedSetInputStream getInputStream() throws IOException { + public SortedMapInputStream getInputStream() throws IOException { return handler.getInputStream(); } @Override - public SortedSetOutputStream getOutputStream() throws IOException { + public SortedMapOutputStream getOutputStream() throws IOException { return handler.getOutputStream(); } @Override - public PersistOptions getPersistOptions() { + public FileSortedSet.PersistOptions getPersistOptions() { return handler.getPersistOptions(); } @@ -1068,24 +1024,24 @@ public void deleteFile() { /** * An input stream that supports bounding the objects. Used when the underlying stream does not already support bounding. */ - public class BoundedInputStream implements SortedSetInputStream { - private final SortedSetInputStream delegate; - private final E from; - private final E to; + public class BoundedInputStream implements SortedMapInputStream { + private final SortedMapInputStream delegate; + private final K from; + private final K to; - public BoundedInputStream(SortedSetInputStream stream, E from, E to) { + public BoundedInputStream(SortedMapInputStream stream, K from, K to) { this.delegate = stream; this.from = from; this.to = to; } @Override - public E readObject() throws IOException { - E o = delegate.readObject(); - while ((o != null) && (from != null) && (compare(o, from) < 0)) { + 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, to) >= 0)) { + if (o == null || (to != null && compare(o.getKey(), to) >= 0)) { return null; } else { return o; @@ -1103,33 +1059,16 @@ public void close() { } } - public static class PersistOptions { - private boolean verifySize = true; - private boolean verifyElements = true; - private int numElementsToVerify = 100; - - public PersistOptions() {} - - public PersistOptions(boolean verifySize, boolean verifyElements) { - this.verifySize = verifySize; - this.verifyElements = verifyElements; - } - - public PersistOptions(boolean verifySize, boolean verifyElements, int numElementsToVerify) { - this(verifySize, verifyElements); - this.numElementsToVerify = numElementsToVerify; - } - - public boolean isVerifySize() { - return verifySize; - } - - public boolean isVerifyElements() { - return verifyElements; - } - - public int getNumElementsToVerify() { - return numElementsToVerify; - } + 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 index 934505e5a32..92409e3e48a 100644 --- 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 @@ -7,6 +7,7 @@ import java.util.ArrayList; import java.util.List; +import datawave.query.util.sortedset.FileSortedSet; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FsStatus; @@ -16,14 +17,14 @@ import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; -public class HdfsBackedSortedMap extends BufferedFileBackedSortedMap { +public class HdfsBackedSortedMap extends BufferedFileBackedSortedMap { private static final Logger log = Logger.getLogger(HdfsBackedSortedMap.class); - private static final String FILENAME_PREFIX = "SortedSetFile."; + private static final String FILENAME_PREFIX = "SortedMapFile."; - public static class Builder,E> extends BufferedFileBackedSortedMap.Builder { + public static class Builder,K,V> extends BufferedFileBackedSortedMap.Builder { private List ivaratorCacheDirs; private String uniqueSubPath; - private FileSortedMap.PersistOptions persistOptions; + private FileSortedSet.PersistOptions persistOptions; public Builder() { // change the default buffer persist threshold @@ -46,21 +47,21 @@ public B withUniqueSubPath(String uniqueSubPath) { return self(); } - public B withPersistOptions(FileSortedMap.PersistOptions persistOptions) { + public B withPersistOptions(FileSortedSet.PersistOptions persistOptions) { this.persistOptions = persistOptions; return self(); } - public HdfsBackedSortedMap build() throws IOException { + public HdfsBackedSortedMap build() throws IOException { return new HdfsBackedSortedMap<>(this); } } - public static HdfsBackedSortedMap.Builder builder() { + public static HdfsBackedSortedMap.Builder builder() { return new HdfsBackedSortedMap.Builder<>(); } - protected HdfsBackedSortedMap(HdfsBackedSortedMap other) { + protected HdfsBackedSortedMap(HdfsBackedSortedMap other) { super(other); } @@ -68,62 +69,62 @@ protected HdfsBackedSortedMap(Builder builder) throws IOException { super(builder); this.handlerFactories = createFileHandlerFactories(builder.ivaratorCacheDirs, builder.uniqueSubPath, builder.persistOptions); // for each of the handler factories, check to see if there are any existing files we should load - for (SortedSetFileHandlerFactory handlerFactory : handlerFactories) { - // Note: All of the file handler factories created by 'createFileHandlerFactories' are SortedSetHdfsFileHandlerFactories - if (handlerFactory instanceof SortedSetHdfsFileHandlerFactory) { - SortedSetHdfsFileHandlerFactory hdfsHandlerFactory = (SortedSetHdfsFileHandlerFactory) handlerFactory; + for (SortedMapFileHandlerFactory handlerFactory : handlerFactories) { + // Note: All of the file handler factories created by 'createFileHandlerFactories' are SortedMapHdfsFileHandlerFactories + if (handlerFactory instanceof SortedMapHdfsFileHandlerFactory) { + SortedMapHdfsFileHandlerFactory hdfsHandlerFactory = (SortedMapHdfsFileHandlerFactory) handlerFactory; FileSystem fs = hdfsHandlerFactory.getFs(); int count = 0; - // if the directory already exists, load up this sorted set with any existing files + // 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++; - addSet(setFactory.newInstance(comparator, new SortedSetHdfsFileHandler(fs, file.getPath(), builder.persistOptions), true)); + addMap(mapFactory.newInstance(comparator, new SortedMapHdfsFileHandler(fs, file.getPath(), builder.persistOptions), true)); } } } - hdfsHandlerFactory.setFileCount(count); + hdfsHandlerFactory.mapFileCount(count); } } } } - private static List createFileHandlerFactories(List ivaratorCacheDirs, String uniqueSubPath, - FileSortedMap.PersistOptions persistOptions) { - List fileHandlerFactories = new ArrayList<>(); + private static List createFileHandlerFactories(List ivaratorCacheDirs, String uniqueSubPath, + FileSortedSet.PersistOptions persistOptions) { + List fileHandlerFactories = new ArrayList<>(); for (IvaratorCacheDir ivaratorCacheDir : ivaratorCacheDirs) { - fileHandlerFactories.add(new SortedSetHdfsFileHandlerFactory(ivaratorCacheDir, uniqueSubPath, persistOptions)); + fileHandlerFactories.add(new SortedMapHdfsFileHandlerFactory(ivaratorCacheDir, uniqueSubPath, persistOptions)); } return fileHandlerFactories; } @Override public void clear() { - // This will be a new ArrayList<>() containing the same FileSortedSets - List> sortedSets = super.getSets(); - // Clear will call clear on each of the FileSortedSets, clear the container, and null the buffer + // 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 FileSortedSet objects to get their handler because we - // have a copy of the object in 'sortedSets' - for (FileSortedMap fss : sortedSets) { - if (fss.isPersisted() && fss.handler instanceof SortedSetHdfsFileHandler) { - ((SortedSetHdfsFileHandler) fss.handler).deleteFile(); + // 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 SortedSetHdfsFileHandlerFactory implements SortedSetFileHandlerFactory { + public static class SortedMapHdfsFileHandlerFactory implements SortedMapFileHandlerFactory { final private IvaratorCacheDir ivaratorCacheDir; private String uniqueSubPath; private int fileCount = 0; - private FileSortedMap.PersistOptions persistOptions; + private FileSortedSet.PersistOptions persistOptions; - public SortedSetHdfsFileHandlerFactory(IvaratorCacheDir ivaratorCacheDir, String uniqueSubPath, FileSortedMap.PersistOptions persistOptions) { + public SortedMapHdfsFileHandlerFactory(IvaratorCacheDir ivaratorCacheDir, String uniqueSubPath, FileSortedSet.PersistOptions persistOptions) { this.ivaratorCacheDir = ivaratorCacheDir; this.uniqueSubPath = uniqueSubPath; this.persistOptions = persistOptions; @@ -145,7 +146,7 @@ public int getFileCount() { return fileCount; } - void setFileCount(int count) { + void mapFileCount(int count) { this.fileCount = count; } @@ -171,7 +172,7 @@ public boolean isValid() { } @Override - public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { + public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { FileSystem fs = getFs(); Path uniqueDir = getUniqueDir(); @@ -181,7 +182,7 @@ public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { // generate a unique file name fileCount++; Path file = new Path(uniqueDir, FILENAME_PREFIX + fileCount + '.' + System.currentTimeMillis()); - return new SortedSetHdfsFileHandler(fs, file, persistOptions); + return new SortedMapHdfsFileHandler(fs, file, persistOptions); } private void ensureDirsCreated() throws IOException { @@ -218,12 +219,12 @@ public String toString() { } - public static class SortedSetHdfsFileHandler implements FileSortedMap.SortedSetFileHandler { + public static class SortedMapHdfsFileHandler implements FileSortedMap.SortedMapFileHandler { private FileSystem fs; private Path file; - private FileSortedMap.PersistOptions persistOptions; + private FileSortedSet.PersistOptions persistOptions; - public SortedSetHdfsFileHandler(FileSystem fs, Path file, FileSortedMap.PersistOptions persistOptions) { + public SortedMapHdfsFileHandler(FileSystem fs, Path file, FileSortedSet.PersistOptions persistOptions) { this.fs = fs; this.file = file; this.persistOptions = persistOptions; @@ -254,7 +255,7 @@ public OutputStream getOutputStream() throws IOException { } @Override - public FileSortedMap.PersistOptions getPersistOptions() { + public FileSortedSet.PersistOptions getPersistOptions() { return persistOptions; } 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..8d8cf512497 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiMapBackedSortedMap.java @@ -0,0 +1,378 @@ +package datawave.query.util.sortedmap; + +import com.google.common.collect.Iterators; +import datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; +import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; +import org.apache.commons.lang3.builder.EqualsBuilder; + +import java.io.IOException; +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; + +/* + * 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. + */ +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> { + + private List>> iterators = new ArrayList<>(); + private List lastList = new ArrayList<>(); + private boolean[] finished = null; + private SortedMap map = null; + private boolean populated = false; + private K nextKey = null; + private V nextValue = null; + 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() { + throw new UnsupportedOperationException("Merge sort iterator does not support this operation"); + } + + /* 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(); + lastList.set(i, val.getKey()); + if ((rewriteStrategy == null) || + (!map.containsKey(val.getKey())) || + (rewriteStrategy.rewrite(val.getKey(), map.get(val.getKey()), val.getValue()))) { + map.put(val.getKey(), val.getValue()); + } + } else { + lastList.set(i, null); + finished[i] = true; + } + } + } + + if (!map.isEmpty()) { + nextKey = map.firstKey(); + nextValue = map.remove(nextKey); + 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/MultiSetBackedSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiSetBackedSortedMap.java deleted file mode 100644 index ee2c83b4da5..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/MultiSetBackedSortedMap.java +++ /dev/null @@ -1,347 +0,0 @@ -package datawave.query.util.sortedmap; - -import datawave.webservice.query.exception.DatawaveErrorCode; -import datawave.webservice.query.exception.QueryException; -import org.apache.commons.lang3.builder.EqualsBuilder; - -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.SortedSet; - -/* - * This is a sorted set that is backed by multiple underlying sorted sets. It is assumed that the underlying - * sorted sets contain the same type of underlying value, and they use the same comparator. The rewrite - * strategy will be used if the underlying sorted sets are RewriteableSortedSet implementations. - */ -public class MultiSetBackedSortedMap extends AbstractSet implements RewritableSortedSet { - protected List> sets = new ArrayList<>(); - protected Comparator comparator = null; - protected RewriteStrategy rewriteStrategy = null; - - public MultiSetBackedSortedMap() {} - - public MultiSetBackedSortedMap(List> sets) { - for (SortedSet set : sets) { - addSet(set); - } - } - - public void addSet(SortedSet set) { - if (sets.isEmpty()) { - updateConfiguration(set); - } else { - verifyConfiguration(set); - } - sets.add(set); - } - - private void updateConfiguration(SortedSet set) { - comparator = getComparator(set); - rewriteStrategy = getRewriteStrategy(set); - } - - private void verifyConfiguration(SortedSet set) { - if (!(new EqualsBuilder().append(getClass(comparator), getClass(getComparator(set))) - .append(getClass(rewriteStrategy), getClass(getRewriteStrategy(set))).isEquals())) { - throw new IllegalArgumentException("Set being added does not match the comparator and rewriteStrategy of the existing sets"); - } - } - - private Class getClass(Object obj) { - return (obj == null ? null : obj.getClass()); - } - - private RewriteStrategy getRewriteStrategy(SortedSet set) { - if (set instanceof RewritableSortedSet) { - return ((RewritableSortedSet) set).getRewriteStrategy(); - } - return null; - } - - private Comparator getComparator(SortedSet set) { - return (Comparator) (set.comparator()); - } - - /** - * Get the underlying sets - * - * @return the sets - */ - public List> getSets() { - return sets; - } - - /** - * Return the size of this set. NOTE that this is somewhat expensive as we require iterating over the sets to determine the true value (see - * MergeSortIterator); - */ - @Override - public int size() { - int size = 0; - for (@SuppressWarnings("unused") - E t : this) { - size++; - } - return size; - } - - @Override - public boolean isEmpty() { - if (sets == null) { - return true; - } - for (SortedSet set : sets) { - if (set != null && !set.isEmpty()) { - return false; - } - } - return true; - } - - @Override - public boolean contains(Object o) { - for (SortedSet set : sets) { - if (set.contains(o)) { - return true; - } - } - return false; - } - - @Override - public Iterator iterator() { - return new MergeSortIterator(); - } - - @Override - public boolean add(E e) { - throw new UnsupportedOperationException("Please use addSet to add a sorted set or add this item to one of the existing underlying sets"); - } - - @Override - public boolean remove(Object o) { - boolean removed = false; - for (SortedSet set : sets) { - if (set.remove(o)) { - removed = true; - } - } - return removed; - } - - @Override - public void clear() { - for (SortedSet set : this.sets) { - try { - set.clear(); - } catch (Exception e) { - // error clearing sorted set - // possibility of FileNotFoundException, etc being - // caught and re-thrown as an exception - } - } - this.sets.clear(); - } - - @Override - public Comparator comparator() { - return comparator; - } - - @Override - public RewritableSortedSet subSet(E fromElement, E toElement) { - MultiSetBackedSortedMap subSet = new MultiSetBackedSortedMap<>(); - for (SortedSet set : sets) { - subSet.addSet(set.subSet(fromElement, toElement)); - } - return subSet; - } - - @Override - public RewritableSortedSet headSet(E toElement) { - MultiSetBackedSortedMap subSet = new MultiSetBackedSortedMap<>(); - for (SortedSet set : sets) { - subSet.addSet(set.headSet(toElement)); - } - return subSet; - } - - @Override - public RewritableSortedSet tailSet(E fromElement) { - MultiSetBackedSortedMap subSet = new MultiSetBackedSortedMap<>(); - for (SortedSet set : sets) { - subSet.addSet(set.tailSet(fromElement)); - } - return subSet; - } - - @Override - public E first() throws NoSuchElementException { - if (sets == null || sets.isEmpty()) { - throw new NoSuchElementException("No elements in input sets"); - } - SortedSet firstSet = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); - for (SortedSet set : sets) { - if (set != null && !set.isEmpty()) { - E s = set.first(); - firstSet.add(s); - } - } - if (firstSet.isEmpty()) { - throw new NoSuchElementException("No elements in input sets"); - } - return firstSet.first(); - } - - @Override - public E last() throws NoSuchElementException { - if (sets == null || sets.isEmpty()) { - throw new NoSuchElementException("No elements in input sets"); - } - SortedSet lastSet = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); - for (SortedSet set : sets) { - if (set != null && !set.isEmpty()) { - E s = set.last(); - lastSet.add(s); - } - } - if (lastSet.isEmpty()) { - throw new NoSuchElementException("No elements in input sets"); - } - return lastSet.last(); - } - - @Override - public RewriteStrategy getRewriteStrategy() { - return rewriteStrategy; - } - - @Override - public E get(E e) { - return null; - } - - /** - * This is an iterator that will return a sorted set of items (no dups) from an underlying set of sorted sets. - */ - public class MergeSortIterator implements Iterator { - - private List> iterators = new ArrayList<>(); - private List lastList = new ArrayList<>(); - private boolean[] finished = null; - private RewritableSortedSet set = null; - private boolean populated = false; - private E next = null; - private List> nextIterators = new ArrayList<>(); - - public MergeSortIterator() { - for (SortedSet set : sets) { - Iterator it = set.iterator(); - iterators.add(it); - nextIterators.add(it); - lastList.add(null); - } - this.set = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); - this.finished = new boolean[iterators.size()]; - } - - @Override - public boolean hasNext() { - if (!set.isEmpty()) { - return true; - } - for (Iterator it : nextIterators) { - if (it != null && it.hasNext()) { - return true; - } - } - return false; - } - - @Override - public E next() { - populate(); - if (!populated) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } - return next; - } - - @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(E o1, E o2) { - if (o1 == null) { - return o2 == null; - } else if (o2 == null) { - return false; - } else { - if (set.comparator() == null) { - return o1.equals(o2); - } else { - return set.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()) { - E val = it.next(); - lastList.set(i, val); - set.add(val); - } else { - lastList.set(i, null); - finished[i] = true; - } - } - } - - if (!set.isEmpty()) { - next = set.first(); - set.remove(next); - for (int i = 0; i < iterators.size(); i++) { - if (!finished[i] && equals(next, 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 set 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..7dda159e724 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/RewritableSortedMap.java @@ -0,0 +1,11 @@ +package datawave.query.util.sortedmap; + +import java.util.SortedMap; + +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/SortedByteMapBuffer.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedByteMapBuffer.java deleted file mode 100644 index 1b73d3e8d13..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/SortedByteMapBuffer.java +++ /dev/null @@ -1,463 +0,0 @@ -package datawave.query.util.sortedmap; - -import datawave.webservice.query.exception.DatawaveErrorCode; -import datawave.webservice.query.exception.QueryException; - -import java.util.AbstractSet; -import java.util.Arrays; -import java.util.Comparator; -import java.util.ConcurrentModificationException; -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.SortedSet; - -/** - * This is an sorted set of byte arrays which keeps one large byte array as the backing store and a separate array of indices and sizes in sorted value order. - * The reason for building this sorted set structure is to minimize memory usage and object creation while maintaining fast add capabilities. - * - */ -public class SortedByteMapBuffer extends AbstractSet implements SortedSet { - public static final int AVERAGE_VALUE_SIZE = 32; - public static final int DEFAULT_BUFFER_SIZE = 64; - - protected byte[] data = null; - protected int[] sortedDataIndicies = null; - protected byte[] sortedDataSizes = null; - protected int size = 0; - protected int bufferSize = 0; - protected int modCount = 0; - - public SortedByteMapBuffer() { - this(DEFAULT_BUFFER_SIZE); - } - - public SortedByteMapBuffer(int capacity) { - this.data = new byte[capacity * AVERAGE_VALUE_SIZE]; - this.sortedDataIndicies = new int[capacity]; - this.sortedDataSizes = new byte[capacity]; - } - - /************************** Overridden methods *************************/ - - @Override - public int size() { - return size; - } - - @Override - public boolean isEmpty() { - return size == 0; - } - - @Override - public boolean contains(Object o) { - if (o instanceof byte[]) { - return binarySearch((byte[]) o) >= 0; - } - return false; - } - - @Override - public Iterator iterator() { - return new SortedByteSetBufferIterator(); - } - - @Override - public boolean add(byte[] e) { - if (e.length > Byte.MAX_VALUE) { - throw new IllegalArgumentException("SortedByteSetBuffer does not support data elements greater than " + Byte.MAX_VALUE + " bytes"); - } - int index = binarySearch(e); - if (index < 0) { - add(-1 - index, e); - return true; - } - return false; - } - - @Override - public boolean remove(Object o) { - if (!(o instanceof byte[])) { - return false; - } - int index = binarySearch((byte[]) o); - if (index >= 0) { - remove(index); - return true; - } - return false; - } - - @Override - public void clear() { - modCount++; - size = 0; - bufferSize = 0; - } - - @Override - public Comparator comparator() { - return new ByteArrayComparator(); - } - - @Override - public SortedSet subSet(byte[] fromElement, byte[] toElement) { - return new SortedByteSubSetBuffer(fromElement, toElement); - } - - @Override - public SortedSet headSet(byte[] toElement) { - return new SortedByteSubSetBuffer(null, toElement); - } - - @Override - public SortedSet tailSet(byte[] fromElement) { - return new SortedByteSubSetBuffer(fromElement, null); - } - - @Override - public byte[] first() { - if (size == 0) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } - return get(0); - } - - @Override - public byte[] last() { - if (size == 0) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } - return get(size - 1); - } - - /* Other public methods */ - - public byte[] get(int index) { - if (index < 0 || index >= size) { - throw new IndexOutOfBoundsException("index is out of range"); - } - int dataIndex = sortedDataIndicies[index]; - int dataSize = sortedDataSizes[index]; - byte[] term = new byte[dataSize]; - System.arraycopy(data, dataIndex, term, 0, dataSize); - return term; - } - - /* The protected stuff */ - - protected void checkCapacity(int plusSize) { - int plusLen = 1; - int minCapacity = bufferSize + plusSize; - int oldCapacity = data.length; - if (minCapacity > oldCapacity) { - int newCapacity = (oldCapacity * 3) / 2 + 1; - if (newCapacity < minCapacity) { - newCapacity = minCapacity; - } - plusLen = ((newCapacity - oldCapacity) / 32) + 1; - data = Arrays.copyOf(data, newCapacity); - } - int minLen = size + plusLen; - int oldLen = sortedDataIndicies.length; - if (minLen > oldLen) { - int newLen = (oldLen * 3) / 2 + 1; - if (newLen < minLen) { - newLen = minLen; - } - sortedDataIndicies = Arrays.copyOf(sortedDataIndicies, newLen); - sortedDataSizes = Arrays.copyOf(sortedDataSizes, newLen); - } - } - - protected void add(int index, byte[] value) { - modCount++; - checkCapacity(value.length); - int dataIndex = bufferSize; - System.arraycopy(value, 0, data, bufferSize, value.length); - System.arraycopy(sortedDataIndicies, index, sortedDataIndicies, index + 1, size - index); - System.arraycopy(sortedDataSizes, index, sortedDataSizes, index + 1, size - index); - sortedDataIndicies[index] = dataIndex; - sortedDataSizes[index] = (byte) (value.length); - bufferSize += value.length; - size++; - } - - protected void remove(int index) { - modCount++; - int dataIndex = sortedDataIndicies[index]; - int dataSize = sortedDataSizes[index]; - bufferSize -= dataSize; - size--; - System.arraycopy(data, dataIndex + dataSize, data, dataIndex, bufferSize - dataIndex); - System.arraycopy(sortedDataSizes, index + 1, sortedDataSizes, index, size - index); - System.arraycopy(sortedDataIndicies, index + 1, sortedDataIndicies, index, size - index); - for (int i = 0; i < size; i++) { - if (sortedDataIndicies[i] > dataIndex) { - sortedDataIndicies[i] -= dataSize; - } - } - } - - protected static int compare(byte[] data, int dataIndex, int dataSize, byte[] term) { - int minSize = dataSize; - if (term.length < minSize) - minSize = term.length; - int comparison = 0; - for (int i = 0; i < minSize; i++) { - comparison = data[dataIndex + i] - term[i]; - if (comparison != 0) - break; - } - if (comparison == 0) { - if (minSize < dataSize) { - comparison = 1; - } else if (minSize < term.length) { - comparison = -1; - } - } - return comparison; - } - - /** - * A binary search of the byte array based on a sorted index array - * - * @param term - * aterm - * @return location result of the search - */ - protected int binarySearch(byte[] term) { - return binarySearch(term, 0, this.size - 1); - } - - protected int binarySearch(byte[] term, int start, int end) { - while (start <= end) { - int middle = (start + end) >>> 1; - int comparison = compare(data, sortedDataIndicies[middle], sortedDataSizes[middle], term); - - if (comparison < 0) - start = middle + 1; - else if (comparison > 0) - end = middle - 1; - else - return middle; - } - // return a negative index if not found so we know where it should go - return -(start + 1); - } - - protected class SortedByteSetBufferIterator implements Iterator { - protected int index = 0; - protected int end = 0; - protected int expectedModCount = -1; - protected int last = -1; - - public SortedByteSetBufferIterator() { - this(0, size); - } - - public SortedByteSetBufferIterator(int start, int end) { - this.expectedModCount = modCount; - this.index = start; - this.end = end; - } - - final void checkModCount() { - if (modCount != expectedModCount) - throw new ConcurrentModificationException(); - } - - @Override - public boolean hasNext() { - checkModCount(); - return index < end; - } - - @Override - public byte[] next() { - if (!hasNext()) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_NEXT_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } - checkModCount(); - last = index; - byte[] entry = get(index++); - checkModCount(); - return entry; - } - - @Override - public void remove() { - checkModCount(); - if (last >= 0) { - SortedByteMapBuffer.this.remove(last); - index--; - end--; - expectedModCount = modCount; - } else { - throw new IllegalStateException("next needs to be called first"); - } - } - } - - protected class SortedByteSubSetBuffer extends AbstractSet implements SortedSet { - protected byte[] from; - protected byte[] to; - protected int expectedModCount = -1; - protected int[] range = null; - - public SortedByteSubSetBuffer(byte[] from, byte[] to) { - if (from != null && to != null && comparator().compare(from, to) > 0) { - throw new IllegalArgumentException("The start is greater than the end"); - } - this.from = from; - this.to = to; - } - - @Override - public Comparator comparator() { - return SortedByteMapBuffer.this.comparator(); - } - - @Override - public SortedSet subSet(byte[] fromElement, byte[] toElement) { - if ((from != null && comparator().compare(fromElement, from) < 0) || (to != null && comparator().compare(to, toElement) < 0)) { - throw new IllegalArgumentException("Cannot create subset outside of the range of this subset"); - } - return SortedByteMapBuffer.this.subSet(fromElement, toElement); - } - - @Override - public SortedSet headSet(byte[] toElement) { - return subSet(from, toElement); - } - - @Override - public SortedSet tailSet(byte[] fromElement) { - return subSet(fromElement, to); - } - - @Override - public byte[] first() { - int[] range = getRange(); - if (range == null) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } - return get(range[0]); - } - - @Override - public byte[] last() { - int[] range = getRange(); - if (range == null) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } - return get(range[1]); - } - - @Override - public Iterator iterator() { - int[] range = getRange(); - if (range == null) { - return new SortedByteSetBufferIterator(0, 0); - } else { - return new SortedByteSetBufferIterator(range[0], range[1]); - } - } - - @Override - public int size() { - int[] range = getRange(); - if (range == null) { - return 0; - } else { - return range[1] - range[0] + 1; - } - } - - @Override - public boolean contains(Object o) { - if (!(o instanceof byte[])) { - return false; - } - int[] range = getRange(); - if (range == null) { - return false; - } - boolean contains = (binarySearch((byte[]) o, range[0], range[1]) >= 0); - checkModCount(); - return contains; - } - - @Override - public boolean add(byte[] e) { - if ((from != null && comparator().compare(e, from) < 0) || (to != null && comparator().compare(e, to) >= 0)) { - throw new IllegalArgumentException("Cannot add element outside of subset range"); - } - return SortedByteMapBuffer.this.add(e); - } - - @Override - public boolean remove(Object o) { - if (contains(o)) { - return SortedByteMapBuffer.this.remove(o); - } - return false; - } - - /*** - * Get the range of elements in the SortedByteSetBuffer - * - * @return int[] {firstIndex, lastIndex} - */ - protected int[] getRange() { - if (expectedModCount != modCount) { - expectedModCount = modCount; - if (SortedByteMapBuffer.this.isEmpty()) { - range = null; - } else { - - // find the first entry - int start = (from == null ? 0 : binarySearch(from)); - if (start < 0) { - start = -1 - start; - } - - // if the start is past the end, then we have no range - if (start == SortedByteMapBuffer.this.size()) { - range = null; - } else { - - // find the last entry - int end = (to == null ? SortedByteMapBuffer.this.size() : binarySearch(to)); - if (end < 0) { - end = -1 - end; - } - // since the end is exclusive, go to the previous element - end--; - - // if the start is not before the end, then no range - if (start >= end) { - range = null; - } else { - range = new int[] {start, end}; - } - } - } - checkModCount(); - } - return range; - } - - final void checkModCount() { - if (modCount != expectedModCount) - throw new ConcurrentModificationException(); - } - - } - -} 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 index 7c259128312..dc3f9ba2604 100644 --- 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 @@ -1,5 +1,6 @@ package datawave.query.util.sortedmap; +import datawave.query.util.sortedset.FileSortedSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -11,16 +12,13 @@ /** * A sorted set file handler factory that uses temporary local based files. - * - * - * */ -public class SortedTempFileHandler implements FileSortedMap.SortedSetFileHandler { +public class SortedMapTempFileHandler implements FileSortedMap.SortedMapFileHandler { private final FileSystem fs; private final File file; private final Path path; - public SortedSetTempFileHandler() throws IOException { + public SortedMapTempFileHandler() throws IOException { this.file = File.createTempFile("SortedSet", ".bin"); this.file.deleteOnExit(); this.path = new Path(file.toURI()); @@ -42,8 +40,8 @@ public OutputStream getOutputStream() throws IOException { } @Override - public FileSortedMap.PersistOptions getPersistOptions() { - return new FileSortedMap.PersistOptions(); + public FileSortedSet.PersistOptions getPersistOptions() { + return new FileSortedSet.PersistOptions(); } @Override 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 index c568993e9fb..b9d9803cab3 100644 --- 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 @@ -29,6 +29,20 @@ public static Key byteToKey(byte[] bytes) { return new Key(bytes); } + public static Value keyToValue(Key key) { + if (key == null) { + return null; + } + return new Value(key.getRow().getBytes()); + } + + public static Key valueToKey(Value value) { + if (value == null) { + return null; + } + return new Key(value.get()); + } + public static Value documentToValue(Document doc) throws IOException { if (doc == null) { return null; 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 index 94c284f797a..f7d1f9916d3 100644 --- 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 @@ -1,28 +1,25 @@ package datawave.query.util.sortedmap.rfile; import datawave.query.attributes.Document; +import datawave.query.util.sortedmap.FileSortedMap; import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; import java.io.IOException; import java.io.InputStream; import java.util.Map; -import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; -import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.keyValueToByteDocument; +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.byteToKey; +import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.keyValueToByteDocument; -public class RFileByteDocumentInputStream extends RFileKeyValueInputStreamBase> { +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, Key start, Key end) throws IOException { - super(inputStream, length, start, end); - } - - public RFileByteDocumentInputStream(InputStream inputStream, long length, Map.Entry start, Map.Entry end) - throws IOException { - super(inputStream, length, byteDocumentToKeyValue(start), byteDocumentToKeyValue(end)); + public RFileByteDocumentInputStream(InputStream inputStream, long length, byte[] start, byte[] end) throws IOException { + super(inputStream, length, byteToKey(start), byteToKey(end)); } @Override 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 index 20a690c6ea1..f46085fb3a4 100644 --- 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 @@ -2,20 +2,24 @@ import datawave.query.attributes.Document; import datawave.query.util.sortedmap.FileSortedMap; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; import java.io.IOException; import java.io.OutputStream; import java.util.Map; -import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; +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; -public class RFileByteDocumentOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedSetOutputStream> { +public class RFileByteDocumentOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedMapOutputStream { public RFileByteDocumentOutputStream(OutputStream stream) throws IOException { super(stream); } @Override - public void writeObject(Map.Entry obj) throws IOException { - writeKeyValue(byteDocumentToKeyValue(obj)); + 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/RFileKeyInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyInputStream.java deleted file mode 100644 index 0254435ae93..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyInputStream.java +++ /dev/null @@ -1,26 +0,0 @@ -package datawave.query.util.sortedmap.rfile; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Map; - -public class RFileKeyInputStream extends RFileKeyValueInputStreamBase { - - public RFileKeyInputStream(InputStream inputStream, long length) throws IOException { - super(inputStream, length); - } - - public RFileKeyInputStream(InputStream inputStream, long length, Key start, Key end) throws IOException { - super(inputStream, length, start, end); - } - - @Override - public Key readObject() throws IOException { - Map.Entry obj = readKeyValue(); - return (obj == null ? null : obj.getKey()); - } - -} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java deleted file mode 100644 index cb7416d395b..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/rfile/RFileKeyOutputStream.java +++ /dev/null @@ -1,18 +0,0 @@ -package datawave.query.util.sortedmap.rfile; - -import datawave.query.util.sortedmap.FileSortedMap; -import org.apache.accumulo.core.data.Key; - -import java.io.IOException; -import java.io.OutputStream; - -public class RFileKeyOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedSetOutputStream { - public RFileKeyOutputStream(OutputStream stream) throws IOException { - super(stream); - } - - @Override - public void writeObject(Key o) throws IOException { - writeKeyValue(o, EMPTY_VALUE); - } -} 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 index 3c19feb6e1b..edcbc096728 100644 --- 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 @@ -1,5 +1,6 @@ package datawave.query.util.sortedmap.rfile; +import datawave.query.util.sortedmap.FileSortedMap; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; @@ -7,7 +8,7 @@ import java.io.InputStream; import java.util.Map; -public class RFileKeyValueInputStream extends RFileKeyValueInputStreamBase> { +public class RFileKeyValueInputStream extends RFileKeyValueInputStreamBase implements FileSortedMap.SortedMapInputStream { public RFileKeyValueInputStream(InputStream inputStream, long length) throws IOException { super(inputStream, length); @@ -17,10 +18,6 @@ public RFileKeyValueInputStream(InputStream inputStream, long length, Key start, super(inputStream, length, start, end); } - public RFileKeyValueInputStream(InputStream inputStream, long length, Map.Entry start, Map.Entry 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 index 12862f683b6..cf751155efc 100644 --- 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 @@ -1,6 +1,5 @@ package datawave.query.util.sortedmap.rfile; -import datawave.query.util.sortedmap.FileSortedMap; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.rfile.RFile; import org.apache.accumulo.core.client.rfile.RFileSource; @@ -13,7 +12,7 @@ import java.util.Iterator; import java.util.Map; -public abstract class RFileKeyValueInputStreamBase implements FileSortedMap.SortedSetInputStream { +public abstract class RFileKeyValueInputStreamBase { private final InputStream inputStream; private final long length; private Key start; @@ -34,12 +33,6 @@ public RFileKeyValueInputStreamBase(InputStream inputStream, long length, Key st this.end = end; } - public RFileKeyValueInputStreamBase(InputStream inputStream, long length, Map.Entry start, Map.Entry end) throws IOException { - this(inputStream, length); - this.start = (start == null ? null : start.getKey()); - this.end = (end == null ? null : end.getKey()); - } - private Iterator> keyValueIterator() { if (iterator == null) { Range r = ALL; @@ -55,8 +48,8 @@ private Iterator> keyValueIterator() { public Map.Entry readKeyValue() throws IOException { if (keyValueIterator().hasNext()) { Map.Entry next = keyValueIterator().next(); - if (RFileKeyOutputStream.SizeKeyUtil.isSizeKey(next.getKey())) { - size = RFileKeyOutputStream.SizeKeyUtil.getSize(next.getKey()); + if (RFileKeyValueOutputStreamBase.SizeKeyUtil.isSizeKey(next.getKey())) { + size = RFileKeyValueOutputStreamBase.SizeKeyUtil.getSize(next.getKey()); next = null; } return next; @@ -64,20 +57,18 @@ public Map.Entry readKeyValue() throws IOException { return null; } - @Override 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(RFileKeyOutputStream.SizeKeyUtil.SIZE_ROW)).build(); + reader = RFile.newScanner().from(new RFileSource(inputStream, length)).withBounds(new Range(RFileKeyValueOutputStreamBase.SizeKeyUtil.SIZE_ROW)).build(); iterator = reader.iterator(); - size = RFileKeyOutputStream.SizeKeyUtil.getSize(iterator.next().getKey()); + size = RFileKeyValueOutputStreamBase.SizeKeyUtil.getSize(iterator.next().getKey()); } return size; } - @Override public void close() { if (reader != null) { reader.close(); 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 index 35a44852974..ecd438123d1 100644 --- 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 @@ -6,15 +6,14 @@ import java.io.IOException; import java.io.OutputStream; -import java.util.Map; -public class RFileKeyValueOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedSetOutputStream> { +public class RFileKeyValueOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedMapOutputStream { public RFileKeyValueOutputStream(OutputStream stream) throws IOException { super(stream); } @Override - public void writeObject(Map.Entry obj) throws IOException { - writeKeyValue(obj); + 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 index fa19ef85986..ba73fd14426 100644 --- 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 @@ -8,32 +8,30 @@ import java.io.IOException; import java.io.OutputStream; -import java.util.Map; -public class RFileKeyValueOutputStreamBase { +public abstract class RFileKeyValueOutputStreamBase { + private final OutputStream outputStream; private RFileWriter writer; - static final Value EMPTY_VALUE = new Value(new byte[0]); + private static final Value EMPTY_VALUE = new Value(new byte[0]); - public RFileKeyValueOutputStreamBase(OutputStream stream) throws IOException { - super(); - this.writer = RFile.newWriter().to(stream).withVisibilityCacheSize(10).build(); + public RFileKeyValueOutputStreamBase(OutputStream outputStream) throws IOException { + this.outputStream = outputStream; + this.writer = RFile.newWriter().to(outputStream).withVisibilityCacheSize(10).build(); } - public void writeKeyValue(Key key, Value value) throws IOException { - writer.append(key, value); + public void writeKeyValue(Key k, Value v) throws IOException { + writer.append(k, v); } - public void writeKeyValue(Map.Entry keyValue) throws IOException { - writer.append(keyValue.getKey(), keyValue.getValue()); - } - - public void writeSize(int i) throws IOException { - writeKeyValue(SizeKeyUtil.getKey(i), EMPTY_VALUE); + public void writeSize(int size) throws IOException { + writer.append(SizeKeyUtil.getKey(size), EMPTY_VALUE); } public void close() throws IOException { - writer.close(); - writer = null; + if (writer != null) { + writer.close(); + writer = null; + } } public static class SizeKeyUtil { @@ -52,4 +50,5 @@ public static int getSize(Key key) { return Integer.parseInt(key.getColumnFamily().toString()); } } + } diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java similarity index 77% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java index 3ecf97a7d78..c17d5fbede1 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java @@ -2,11 +2,10 @@ import datawave.query.util.sortedset.ByteArrayComparator; import datawave.query.util.sortedset.FileSerializableSortedSet; -import datawave.query.util.sortedset.FileSortedSet; import java.util.Comparator; -public class BufferedFileBackedByteArraySortedSetTest extends BufferedFileBackedSortedSetTest { +public class BufferedFileBackedByteArraySortedMapTest extends BufferedFileBackedSortedMapTest { @Override public byte[] createData(byte[] values) { diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java similarity index 94% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java index fc0e9c6992d..1f23f8b5dc3 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteDocumentSortedMapTest.java @@ -7,7 +7,6 @@ import datawave.query.predicate.KeyProjection; import datawave.query.util.TypeMetadata; import datawave.query.util.sortedset.ByteArrayComparator; -import datawave.query.util.sortedset.FileSortedSet; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; @@ -20,7 +19,7 @@ import static org.junit.Assert.assertEquals; -public class BufferedFileBackedByteDocumentSortedSetTest extends BufferedFileBackedRewritableSortedSetTest { +public class BufferedFileBackedByteDocumentSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { private Comparator> keyComparator = new Comparator<>() { private Comparator comparator = new ByteArrayComparator(); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java similarity index 76% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java index cd677fd631f..f70d1b7a96a 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java @@ -1,12 +1,11 @@ package datawave.query.util.sortedmap; import datawave.query.util.sortedset.FileKeySortedSet; -import datawave.query.util.sortedset.FileSortedSet; import org.apache.accumulo.core.data.Key; import java.util.Comparator; -public class BufferedFileBackedKeySortedSetTest extends BufferedFileBackedSortedSetTest { +public class BufferedFileBackedKeySortedMapTest extends BufferedFileBackedSortedMapTest { @Override public Key createData(byte[] values) { diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java similarity index 91% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java index 46886abe50a..c333f6ce616 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeyValueSortedMapTest.java @@ -1,6 +1,5 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.FileSortedSet; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; @@ -9,7 +8,7 @@ import static org.junit.Assert.assertEquals; -public class BufferedFileBackedKeyValueSortedSetTest extends BufferedFileBackedRewritableSortedSetTest { +public class BufferedFileBackedKeyValueSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { private Comparator> keyComparator = new Comparator<>() { @Override diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java similarity index 96% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java index c9ba805aa0f..4f0721907f0 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedRewritableSortedMapTest.java @@ -1,7 +1,6 @@ package datawave.query.util.sortedmap; import datawave.query.util.sortedset.BufferedFileBackedSortedSet; -import datawave.query.util.sortedset.FileSortedSet; import datawave.query.util.sortedset.SortedSetTempFileHandler; import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; import org.junit.Test; @@ -12,7 +11,7 @@ import java.util.Iterator; import java.util.Map; -public abstract class BufferedFileBackedRewritableSortedSetTest extends BufferedFileBackedSortedSetTest> { +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. diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java similarity index 56% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java index 76dfac96f2a..6c03adeeb4c 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedSortedMapTest.java @@ -1,8 +1,7 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.BufferedFileBackedSortedSet; -import datawave.query.util.sortedset.FileSortedSet; -import datawave.query.util.sortedset.SortedSetTempFileHandler; +import datawave.query.util.sortedmap.BufferedFileBackedSortedMap; +import datawave.query.util.sortedmap.SortedMapTempFileHandler; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -15,13 +14,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Random; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; import java.util.function.Predicate; import static org.junit.Assert.assertEquals; @@ -29,35 +28,35 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public abstract class BufferedFileBackedSortedSetTest { +public abstract class BufferedFileBackedSortedMapTest { - protected final List tempFileHandlers = new ArrayList<>(); - protected E[] data = null; + protected final List tempFileHandlers = new ArrayList<>(); + protected Map.Entry[] data = null; protected int[] sortedOrder = null; - protected datawave.query.util.sortedset.BufferedFileBackedSortedSet set = 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 E createData(byte[] values); - public abstract Comparator getComparator(); + public abstract Comparator getComparator(); - public abstract datawave.query.util.sortedset.FileSortedSet.FileSortedSetFactory getFactory(); + public abstract datawave.query.util.sortedmap.FileSortedMap.FileSortedMapFactory getFactory(); - public RewritableSortedSet.RewriteStrategy getRewriteStrategy() { + public FileSortedMap.RewriteStrategy getRewriteStrategy() { return null; } protected void testEquality(E expected, E value) { - if (set.comparator() != null) { - assertEquals(0, set.comparator().compare(expected, value)); + if (map.comparator() != null) { + assertEquals(0, map.comparator().compare(expected, value)); } else { assertEquals(expected, value); } } @Before - public void setUp() throws Exception { + public void mapUp() throws Exception { for (int i = 0; i < template.length; i++) { byte[] buffer = new byte[i + 11]; Arrays.fill(buffer, template[i]); @@ -78,12 +77,12 @@ public void setUp() throws Exception { sortedOrder[i * 2] = sortedTemplate[i] + sortedTemplate.length; sortedOrder[i * 2 + 1] = sortedTemplate[i]; } - set = new datawave.query.util.sortedset.BufferedFileBackedSortedSet.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) + 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.sortedset.BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { + .withHandlerFactories(Collections.singletonList(new datawave.query.util.sortedmap.BufferedFileBackedSortedMap.SortedMapFileHandlerFactory() { @Override - public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { - datawave.query.util.sortedset.SortedSetTempFileHandler fileHandler = new datawave.query.util.sortedset.SortedSetTempFileHandler(); + public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { + datawave.query.util.sortedmap.SortedMapTempFileHandler fileHandler = new datawave.query.util.sortedmap.SortedMapTempFileHandler(); tempFileHandlers.add(fileHandler); return fileHandler; } @@ -92,39 +91,39 @@ public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { public boolean isValid() { return true; } - })).withSetFactory(getFactory()).build(); + })).withmapFactory(getFactory()).build(); - // adding in the data set multiple times to create underlying files with duplicate values making the + // 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(set, data); + addDataRandomly(map, data); } - while (set.getSets().size() <= 7) { - addDataRandomly(set, data); + while (map.getmaps().size() <= 7) { + addDataRandomly(map, data); } } - public void addDataRandomly(BufferedFileBackedSortedSet set, E[] data) { - Set added = new HashSet<>(); + public void addDataRandomly(BufferedFileBackedSortedMap map, E[] data) { + map added = new Hashmap<>(); Random random = new Random(); // add data.length items randomly for (int i = 0; i < data.length; i++) { int index = random.nextInt(data.length); - set.add(data[index]); + map.add(data[index]); added.add(index); } // ensure all missing items are added for (int i = 0; i < data.length; i++) { if (!added.contains(i)) { - set.add(data[i]); + map.add(data[i]); } } } @After public void tearDown() throws Exception { - // Delete each sorted set file and its checksum. - for (SortedSetTempFileHandler fileHandler : tempFileHandlers) { + // 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"); @@ -134,8 +133,8 @@ public void tearDown() throws Exception { data = null; sortedOrder = null; - set.clear(); - set = null; + map.clear(); + map = null; } private void tryDelete(File file) { @@ -147,60 +146,60 @@ private void tryDelete(File file) { @Test public void testSize() { int expectedSize = data.length; - assertEquals(expectedSize, set.size()); + assertEquals(expectedSize, map.size()); for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); + map.remove(data[i]); expectedSize--; - assertEquals(expectedSize, set.size()); + assertEquals(expectedSize, map.size()); } for (int i = 0; i < (data.length / 2); i++) { - set.remove(data[i]); + map.remove(data[i]); expectedSize--; - assertEquals(expectedSize, set.size()); + assertEquals(expectedSize, map.size()); } - assertEquals(0, set.size()); + assertEquals(0, map.size()); for (int i = 0; i < data.length; i++) { - set.add(data[i]); + map.add(data[i]); expectedSize++; - assertEquals(expectedSize, set.size()); + assertEquals(expectedSize, map.size()); } } @Test public void testIsEmpty() { - assertFalse(set.isEmpty()); + assertFalse(map.isEmpty()); for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - assertFalse(set.isEmpty()); + map.remove(data[i]); + assertFalse(map.isEmpty()); } for (int i = 1; i < (data.length / 2); i++) { - set.remove(data[i]); - assertFalse(set.isEmpty()); + map.remove(data[i]); + assertFalse(map.isEmpty()); } - set.remove(data[0]); - assertTrue(set.isEmpty()); + map.remove(data[0]); + assertTrue(map.isEmpty()); for (int i = 0; i < data.length; i++) { - set.add(data[i]); - assertFalse(set.isEmpty()); + map.add(data[i]); + assertFalse(map.isEmpty()); } } @Test public void testClear() { - set.clear(); - assertTrue(set.isEmpty()); + map.clear(); + assertTrue(map.isEmpty()); } @Test public void testContainsObject() { for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); + map.remove(data[i]); } for (int i = 1; i < (data.length / 2); i++) { - assertTrue(set.contains(data[i])); + assertTrue(map.contains(data[i])); } for (int i = (data.length / 2); i < data.length; i++) { - assertFalse(set.contains(data[i])); + assertFalse(map.contains(data[i])); } } @@ -208,205 +207,205 @@ public void testContainsObject() { public void testRemove() { int expectedSize = data.length; - assertFalse(set.isPersisted()); + assertFalse(map.isPersisted()); for (int i = 0; i < data.length; i++) { - set.remove(data[i]); - assertEquals(--expectedSize, set.size()); + map.remove(data[i]); + assertEquals(--expectedSize, map.size()); } - assertTrue(set.isEmpty()); + assertTrue(map.isEmpty()); } @Test public void testRemovePersisted() throws IOException { int expectedSize = data.length; - assertFalse(set.isPersisted()); - set.persist(); - assertTrue(set.isPersisted()); + assertFalse(map.isPersisted()); + map.persist(); + assertTrue(map.isPersisted()); for (int i = 0; i < data.length; i++) { - set.remove(data[i]); - assertEquals(--expectedSize, set.size()); - assertTrue(set.isPersisted()); + map.remove(data[i]); + assertEquals(--expectedSize, map.size()); + assertTrue(map.isPersisted()); } - assertTrue(set.isEmpty()); + assertTrue(map.isEmpty()); } @Test public void testRemoveIf() { int expectedSize = data.length; - assertFalse(set.isPersisted()); - set.removeIf(new Predicate() { + assertFalse(map.isPersisted()); + map.removeIf(new Predicate() { @Override public boolean test(E bytes) { return false; } }); - assertFalse(set.isPersisted()); - assertEquals(expectedSize, set.size()); + assertFalse(map.isPersisted()); + assertEquals(expectedSize, map.size()); - set.removeIf(new Predicate() { + map.removeIf(new Predicate() { @Override public boolean test(E bytes) { return true; } }); - assertFalse(set.isPersisted()); - assertTrue(set.isEmpty()); + assertFalse(map.isPersisted()); + assertTrue(map.isEmpty()); } @Test public void testRemoveIfPersisted() throws IOException { int expectedSize = data.length; - assertFalse(set.isPersisted()); - set.persist(); - assertTrue(set.isPersisted()); + assertFalse(map.isPersisted()); + map.persist(); + assertTrue(map.isPersisted()); - set.removeIf(new Predicate() { + map.removeIf(new Predicate() { @Override public boolean test(E bytes) { return false; } }); - assertTrue(set.isPersisted()); - assertEquals(expectedSize, set.size()); + assertTrue(map.isPersisted()); + assertEquals(expectedSize, map.size()); - set.removeIf(new Predicate() { + map.removeIf(new Predicate() { @Override public boolean test(E bytes) { return true; } }); - assertTrue(set.isPersisted()); - assertTrue(set.isEmpty()); + assertTrue(map.isPersisted()); + assertTrue(map.isEmpty()); } @Test public void testRemoveAll() { int expectedSize = data.length; - assertFalse(set.isPersisted()); - set.removeAll(Collections.emptySet()); - assertFalse(set.isPersisted()); - assertEquals(expectedSize, set.size()); + assertFalse(map.isPersisted()); + map.removeAll(Collections.emptymap()); + assertFalse(map.isPersisted()); + assertEquals(expectedSize, map.size()); - Set dataSet = new TreeSet<>(set.comparator()); - dataSet.addAll(Arrays.asList(data)); - set.removeAll(dataSet); - assertFalse(set.isPersisted()); - assertTrue(set.isEmpty()); + map datamap = new Treemap<>(map.comparator()); + datamap.addAll(Arrays.asList(data)); + map.removeAll(datamap); + assertFalse(map.isPersisted()); + assertTrue(map.isEmpty()); } @Test public void testRemoveAllPersisted() throws IOException { int expectedSize = data.length; - assertFalse(set.isPersisted()); - set.persist(); - assertTrue(set.isPersisted()); - set.removeAll(Collections.emptySet()); - assertTrue(set.isPersisted()); - assertEquals(expectedSize, set.size()); - - Set dataSet = new TreeSet<>(set.comparator()); - dataSet.addAll(Arrays.asList(data)); - set.removeAll(dataSet); - assertTrue(set.isPersisted()); - assertTrue(set.isEmpty()); + assertFalse(map.isPersisted()); + map.persist(); + assertTrue(map.isPersisted()); + map.removeAll(Collections.emptymap()); + assertTrue(map.isPersisted()); + assertEquals(expectedSize, map.size()); + + map datamap = new Treemap<>(map.comparator()); + datamap.addAll(Arrays.asList(data)); + map.removeAll(datamap); + assertTrue(map.isPersisted()); + assertTrue(map.isEmpty()); } @Test public void testIterator() { int index = 0; - for (Iterator it = set.iterator(); it.hasNext();) { + for (Iterator it = map.iterator(); it.hasNext();) { E value = it.next(); E expected = data[sortedOrder[index++]]; testEquality(expected, value); } - set.clear(); - for (E value : set) { + map.clear(); + for (E value : map) { fail(); } } @Test public void testIteratorRemove() { - int size = set.size(); + int size = map.size(); int failCount = 0; - assertFalse(set.isPersisted()); + assertFalse(map.isPersisted()); // calling iterator() will force persistence - for (Iterator it = set.iterator(); it.hasNext();) { - assertTrue(set.isPersisted()); + for (Iterator it = map.iterator(); it.hasNext();) { + assertTrue(map.isPersisted()); E value = it.next(); - assertTrue(set.contains(value)); + assertTrue(map.contains(value)); try { it.remove(); - fail("Expected iterator remove to fail with a persisted set"); + fail("Expected iterator remove to fail with a persisted map"); } catch (Exception e) { - // expected that some of the underlying FileSortedSets are persisted and hence the remove will fail + // expected that some of the underlying FileSortedMaps are persisted and hence the remove will fail failCount++; - assertTrue(set.contains(value)); - assertEquals(size, set.size()); + assertTrue(map.contains(value)); + assertEquals(size, map.size()); } } assertEquals(size, failCount); - assertFalse(set.isEmpty()); + assertFalse(map.isEmpty()); } @Test public void testComparator() { - Comparator comparator = set.comparator(); + Comparator comparator = map.comparator(); E[] testData = Arrays.copyOf(data, data.length); Arrays.sort(testData, comparator); int index = 0; - for (E value : set) { + for (E value : map) { E expected = data[sortedOrder[index++]]; testEquality(expected, value); } } @Test - public void testSubSet() { + public void testSubmap() { int start = sortedOrder.length / 3; int end = start * 2; try { - SortedSet subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); - SortedSet expected = new TreeSet<>(set.comparator()); + SortedMap submap = map.submap(data[sortedOrder[start]], data[sortedOrder[end]]); + SortedMap expected = new Treemap<>(map.comparator()); for (int i = start; i < end; i++) { expected.add(data[sortedOrder[i]]); } - assertEquals(expected, subSet); + assertEquals(expected, submap); } catch (Exception e) { // expected } } @Test - public void testHeadSet() { + public void testHeadmap() { int end = sortedOrder.length / 3; try { - SortedSet subSet = set.headSet(data[sortedOrder[end]]); - SortedSet expected = new TreeSet<>(set.comparator()); + SortedMap submap = map.headmap(data[sortedOrder[end]]); + SortedMap expected = new Treemap<>(map.comparator()); for (int i = 0; i < end; i++) { expected.add(data[sortedOrder[i]]); } - assertEquals(expected, subSet); + assertEquals(expected, submap); } catch (Exception e) { // expected } } @Test - public void testTailSet() { + public void testTailmap() { int start = sortedOrder.length / 3; try { - SortedSet subSet = set.tailSet(data[sortedOrder[start]]); - SortedSet expected = new TreeSet<>(set.comparator()); + SortedMap submap = map.tailmap(data[sortedOrder[start]]); + SortedMap expected = new Treemap<>(map.comparator()); for (int i = start; i < sortedOrder.length; i++) { expected.add(data[sortedOrder[i]]); } - assertEquals(expected, subSet); + assertEquals(expected, submap); } catch (Exception e) { // expected } @@ -415,22 +414,22 @@ public void testTailSet() { @Test public void testLast() { E expected = data[sortedOrder[data.length - 1]]; - E value = set.last(); + E value = map.last(); testEquality(expected, value); } @Test public void testFirst() { E expected = data[sortedOrder[0]]; - E value = set.first(); + E value = map.first(); testEquality(expected, value); } @Test public void testCompaction() throws IOException { - assertEquals(8, set.getSets().size()); - set.persist(); - assertEquals(3, set.getSets().size()); + 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/FileSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java similarity index 97% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java index 2b2fa6cfaa4..8a0a136727c 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/FileSortedMapTest.java @@ -1,7 +1,6 @@ package datawave.query.util.sortedmap; import datawave.query.util.sortedset.FileSerializableSortedSet; -import datawave.query.util.sortedset.FileSortedSet; import datawave.query.util.sortedset.SortedSetTempFileHandler; import org.junit.After; import org.junit.Before; @@ -10,6 +9,7 @@ import java.util.Comparator; import java.util.Iterator; import java.util.Random; +import java.util.SortedMap; import java.util.SortedSet; import java.util.TreeSet; @@ -21,10 +21,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class FileSortedSetTest { - SortedSet data = null; - SortedSet extraData = null; - FileSortedMap set = null; +public class FileSortedMapTest { + SortedMap data = null; + SortedMap extraData = null; + FileSortedMap set = null; datawave.query.util.sortedset.SortedSetTempFileHandler handler = null; @Before @@ -40,7 +40,7 @@ public int compare(Integer o1, Integer o2) { } }; handler = new SortedSetTempFileHandler(); - set = new datawave.query.util.sortedset.FileSerializableSortedSet<>(c, new datawave.query.util.sortedset.FileSerializableSortedSet.SerializableFileHandler(handler), false); + set = new FileSerializableSortedMap<>(c, new FileSerializableSortedMap.SerializableFileHandler(handler), false); data = new TreeSet<>(c); Random r = new Random(123948710248L); // data.add(null); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java similarity index 99% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java index 092fb1d3722..9b0ef504b45 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/HdfsBackedSortedMapTest.java @@ -2,7 +2,6 @@ import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; -import datawave.query.util.sortedset.FileSortedSet; import datawave.query.util.sortedset.HdfsBackedSortedSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -20,7 +19,7 @@ import java.util.Collections; import java.util.List; -public class HdfsBackedSortedSetTest { +public class HdfsBackedSortedMapTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java similarity index 99% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java index 15626bbb236..fa8b78da640 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class MultiSetBackedSortedSetTest { +public class MultiSetBackedSortedMapTest { private byte[][] data = null; private int[] sortedOrder = null; private datawave.query.util.sortedset.MultiSetBackedSortedSet set = null; diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java similarity index 99% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java index 590825d7d6f..f4bf76ba846 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class RewritableSortedSetTest { +public class RewritableSortedMapTest { private Map.Entry[] data = null; private int[] sortedOrder = null; diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java similarity index 99% rename from warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.java rename to warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java index c31b1ead1ac..288199962a9 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteSetBufferTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java @@ -17,7 +17,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class SortedByteSetBufferTest { +public class SortedByteMapBufferTest { private byte[][] data = null; private int[] sortedOrder = null; private datawave.query.util.sortedset.SortedByteSetBuffer set = null; From 71715dd46c1a7f7e6c1b27d990f9930ae45922d4 Mon Sep 17 00:00:00 2001 From: Ivan Bella Date: Fri, 9 Aug 2024 19:57:56 +0000 Subject: [PATCH 3/5] WOrking through tests --- .../BufferedFileBackedSortedMap.java | 2 +- .../sortedmap/FileSerializableSortedMap.java | 5 +- .../query/util/sortedmap/FileSortedMap.java | 42 +- .../sortedmap/MultiMapBackedSortedMap.java | 18 +- .../BufferedFileBackedSortedMapTest.java | 200 ++---- .../util/sortedmap/FileSortedMapTest.java | 524 ++++++++-------- .../sortedmap/HdfsBackedSortedMapTest.java | 3 +- .../util/sortedmap/MergeSortIteratorTest.java | 153 +++-- .../sortedmap/RewritableSortedMapTest.java | 283 --------- .../sortedmap/SortedByteMapBufferTest.java | 577 ------------------ 10 files changed, 426 insertions(+), 1381 deletions(-) delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java 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 index 82bb76170e0..36bf56f7c63 100644 --- 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 @@ -481,7 +481,7 @@ public void clear() { } @Override - public Comparator comparator() { + public Comparator comparator() { return comparator; } 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 index dfe6854f713..7d4a0de3d60 100644 --- 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 @@ -4,6 +4,7 @@ import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; import org.apache.log4j.Logger; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.ObjectInputStream; @@ -177,8 +178,8 @@ private ObjectInputStream getDelegate() throws IOException { @Override public Map.Entry readObject() throws IOException { try { - K key = (K)getDelegate().readObject(); - V value = (V)getDelegate().readObject(); + K key = (K) getDelegate().readObject(); + V value = (V) getDelegate().readObject(); return new UnmodifiableMapEntry<>(key, value); } catch (IOException ioe) { return null; 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 index bf6ede43d25..e2a107bed51 100644 --- 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 @@ -413,9 +413,12 @@ public boolean containsKey(Object o) { if (persisted) { K t = (K) o; try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - Map.Entry first = stream.readObject(); - if (equals(first.getKey(), t)) { - return true; + Map.Entry next = stream.readObject(); + while (next != null) { + if (equals(next.getKey(), t)) { + return true; + } + next = stream.readObject(); } } catch (Exception e) { return false; @@ -431,9 +434,12 @@ public boolean containsValue(Object o) { if (persisted) { V t = (V) o; try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - Map.Entry first = stream.readObject(); - if (first.getValue().equals(t)) { - return true; + Map.Entry next = stream.readObject(); + while (next != null) { + if (next.getValue().equals(t)) { + return true; + } + next = stream.readObject(); } } catch (Exception e) { return false; @@ -449,9 +455,12 @@ public V get(Object key) { if (persisted) { K t = (K) key; try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { - Map.Entry first = stream.readObject(); - if (equals(first.getKey(), t)) { - return first.getValue(); + Map.Entry next = stream.readObject(); + while (next != null) { + if (equals(next.getKey(), t)) { + return next.getValue(); + } + next = stream.readObject(); } } catch (Exception e) { return null; @@ -508,17 +517,17 @@ public Comparator comparator() { @Override public SortedMap subMap(K fromElement, K toElement) { - return factory.newInstance(this, fromElement, toElement); + return factory.newInstance(this, getStart(fromElement), getEnd(toElement)); } @Override public SortedMap headMap(K toElement) { - return factory.newInstance(this, null, toElement); + return factory.newInstance(this, getStart(null), getEnd(toElement)); } @Override public SortedMap tailMap(K fromElement) { - return factory.newInstance(this, fromElement, null); + return factory.newInstance(this, getStart(fromElement), getEnd(null)); } @Override @@ -570,7 +579,7 @@ public Set keySet() { @Override public Iterator iterator() { - return IteratorUtils.transformedIterator(new FileIterator(), + return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), o -> ((Map.Entry)o).getKey()); } @@ -587,7 +596,7 @@ public Collection values() { @Override public Iterator iterator() { - return IteratorUtils.transformedIterator(new FileIterator(), + return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), o -> ((Map.Entry)o).getValue()); } @@ -604,7 +613,7 @@ public Set> entrySet() { @Override public Iterator> iterator() { - return new FileIterator(); + return FileSortedMap.this.iterator(); } @Override @@ -756,7 +765,7 @@ public Map.Entry next() { @Override public void remove() { - throw new UnsupportedOperationException("Iterator.remove() not supported."); + throw new UnsupportedOperationException("Iterator.remove() not supported on a persisted map."); } @Override @@ -764,7 +773,6 @@ protected void finalize() throws Throwable { cleanup(); super.finalize(); } - } /** 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 index 8d8cf512497..e6180612f95 100644 --- 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 @@ -317,7 +317,23 @@ public Entry next() { @Override public void remove() { - throw new UnsupportedOperationException("Merge sort iterator does not support this operation"); + 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 */ 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 index 6c03adeeb4c..0d384fce9b6 100644 --- 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 @@ -1,7 +1,5 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedmap.BufferedFileBackedSortedMap; -import datawave.query.util.sortedmap.SortedMapTempFileHandler; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -14,14 +12,14 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Random; import java.util.Map; +import java.util.Random; +import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.function.Predicate; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -37,9 +35,9 @@ public abstract class BufferedFileBackedSortedMapTest { 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 E createData(byte[] values); + public abstract Map.Entry createData(byte[] values); - public abstract Comparator getComparator(); + public abstract Comparator getComparator(); public abstract datawave.query.util.sortedmap.FileSortedMap.FileSortedMapFactory getFactory(); @@ -47,7 +45,12 @@ public FileSortedMap.RewriteStrategy getRewriteStrategy() { return null; } - protected void testEquality(E expected, E value) { + 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 { @@ -60,16 +63,16 @@ public void mapUp() throws Exception { for (int i = 0; i < template.length; i++) { byte[] buffer = new byte[i + 11]; Arrays.fill(buffer, template[i]); - E datum = createData(buffer); + Map.Entry datum = createData(buffer); if (i == 0) { - data = (E[]) Array.newInstance(datum.getClass(), template.length * 2); + 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]); - E datum = createData(buffer); + Map.Entry datum = createData(buffer); data[i + template.length] = datum; } sortedOrder = new int[data.length]; @@ -91,31 +94,31 @@ public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { public boolean isValid() { return true; } - })).withmapFactory(getFactory()).build(); + })).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) { + while (map.getMaps().size() <= 7) { addDataRandomly(map, data); } } - public void addDataRandomly(BufferedFileBackedSortedMap map, E[] data) { - map added = new Hashmap<>(); + 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.add(data[index]); + 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.add(data[i]); + map.put(data[i].getKey(), data[i].getValue()); } } } @@ -159,8 +162,7 @@ public void testSize() { } assertEquals(0, map.size()); for (int i = 0; i < data.length; i++) { - map.add(data[i]); - expectedSize++; + map.put(data[i].getKey(), data[i].getValue()); assertEquals(expectedSize, map.size()); } } @@ -179,7 +181,7 @@ public void testIsEmpty() { map.remove(data[0]); assertTrue(map.isEmpty()); for (int i = 0; i < data.length; i++) { - map.add(data[i]); + map.put(data[i].getKey(), data[i].getValue()); assertFalse(map.isEmpty()); } } @@ -196,10 +198,10 @@ public void testContainsObject() { map.remove(data[i]); } for (int i = 1; i < (data.length / 2); i++) { - assertTrue(map.contains(data[i])); + assertTrue(map.containsKey(data[i].getKey())); } for (int i = (data.length / 2); i < data.length; i++) { - assertFalse(map.contains(data[i])); + assertFalse(map.containsKey(data[i].getKey())); } } @@ -230,101 +232,16 @@ public void testRemovePersisted() throws IOException { assertTrue(map.isEmpty()); } - @Test - public void testRemoveIf() { - int expectedSize = data.length; - - assertFalse(map.isPersisted()); - map.removeIf(new Predicate() { - @Override - public boolean test(E bytes) { - return false; - } - }); - assertFalse(map.isPersisted()); - assertEquals(expectedSize, map.size()); - - map.removeIf(new Predicate() { - @Override - public boolean test(E bytes) { - return true; - } - }); - assertFalse(map.isPersisted()); - assertTrue(map.isEmpty()); - } - - @Test - public void testRemoveIfPersisted() throws IOException { - int expectedSize = data.length; - - assertFalse(map.isPersisted()); - map.persist(); - assertTrue(map.isPersisted()); - - map.removeIf(new Predicate() { - @Override - public boolean test(E bytes) { - return false; - } - }); - assertTrue(map.isPersisted()); - assertEquals(expectedSize, map.size()); - - map.removeIf(new Predicate() { - @Override - public boolean test(E bytes) { - return true; - } - }); - assertTrue(map.isPersisted()); - assertTrue(map.isEmpty()); - } - - @Test - public void testRemoveAll() { - int expectedSize = data.length; - - assertFalse(map.isPersisted()); - map.removeAll(Collections.emptymap()); - assertFalse(map.isPersisted()); - assertEquals(expectedSize, map.size()); - - map datamap = new Treemap<>(map.comparator()); - datamap.addAll(Arrays.asList(data)); - map.removeAll(datamap); - assertFalse(map.isPersisted()); - assertTrue(map.isEmpty()); - } - - @Test - public void testRemoveAllPersisted() throws IOException { - int expectedSize = data.length; - - assertFalse(map.isPersisted()); - map.persist(); - assertTrue(map.isPersisted()); - map.removeAll(Collections.emptymap()); - assertTrue(map.isPersisted()); - assertEquals(expectedSize, map.size()); - - map datamap = new Treemap<>(map.comparator()); - datamap.addAll(Arrays.asList(data)); - map.removeAll(datamap); - assertTrue(map.isPersisted()); - assertTrue(map.isEmpty()); - } - @Test public void testIterator() { int index = 0; - for (Iterator it = map.iterator(); it.hasNext();) { - E value = it.next(); - E expected = data[sortedOrder[index++]]; + for (Iterator> it = map.iterator(); it.hasNext();) { + Map.Entry value = it.next(); + Map.Entry expected = data[sortedOrder[index++]]; testEquality(expected, value); } map.clear(); - for (E value : map) { + for (Map.Entry value : map.entrySet()) { fail(); } } @@ -335,17 +252,17 @@ public void testIteratorRemove() { int failCount = 0; assertFalse(map.isPersisted()); // calling iterator() will force persistence - for (Iterator it = map.iterator(); it.hasNext();) { + for (Iterator> it = map.iterator(); it.hasNext();) { assertTrue(map.isPersisted()); - E value = it.next(); - assertTrue(map.contains(value)); + 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.contains(value)); + assertTrue(map.containsKey(value.getKey())); assertEquals(size, map.size()); } } @@ -355,12 +272,17 @@ public void testIteratorRemove() { @Test public void testComparator() { - Comparator comparator = map.comparator(); - E[] testData = Arrays.copyOf(data, data.length); - Arrays.sort(testData, comparator); + 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 (E value : map) { - E expected = data[sortedOrder[index++]]; + for (Map.Entry value : map.entrySet()) { + Map.Entry expected = data[sortedOrder[index++]]; testEquality(expected, value); } } @@ -370,10 +292,10 @@ public void testSubmap() { int start = sortedOrder.length / 3; int end = start * 2; try { - SortedMap submap = map.submap(data[sortedOrder[start]], data[sortedOrder[end]]); - SortedMap expected = new Treemap<>(map.comparator()); + 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.add(data[sortedOrder[i]]); + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); } assertEquals(expected, submap); } catch (Exception e) { @@ -385,10 +307,10 @@ public void testSubmap() { public void testHeadmap() { int end = sortedOrder.length / 3; try { - SortedMap submap = map.headmap(data[sortedOrder[end]]); - SortedMap expected = new Treemap<>(map.comparator()); + SortedMap submap = map.headMap(data[sortedOrder[end]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); for (int i = 0; i < end; i++) { - expected.add(data[sortedOrder[i]]); + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); } assertEquals(expected, submap); } catch (Exception e) { @@ -400,10 +322,10 @@ public void testHeadmap() { public void testTailmap() { int start = sortedOrder.length / 3; try { - SortedMap submap = map.tailmap(data[sortedOrder[start]]); - SortedMap expected = new Treemap<>(map.comparator()); + SortedMap submap = map.tailMap(data[sortedOrder[start]].getKey()); + SortedMap expected = new TreeMap<>(map.comparator()); for (int i = start; i < sortedOrder.length; i++) { - expected.add(data[sortedOrder[i]]); + expected.put(data[sortedOrder[i]].getKey(), data[sortedOrder[i]].getValue()); } assertEquals(expected, submap); } catch (Exception e) { @@ -412,24 +334,24 @@ public void testTailmap() { } @Test - public void testLast() { - E expected = data[sortedOrder[data.length - 1]]; - E value = map.last(); - testEquality(expected, value); + public void testLastKey() { + Map.Entry expected = data[sortedOrder[data.length - 1]]; + K value = map.lastKey(); + testEquality(expected.getKey(), value); } @Test - public void testFirst() { - E expected = data[sortedOrder[0]]; - E value = map.first(); - testEquality(expected, value); + 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()); + assertEquals(8, map.getMaps().size()); map.persist(); - assertEquals(3, map.getmaps().size()); + 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 index 8a0a136727c..cf75fa146aa 100644 --- 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 @@ -1,19 +1,16 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.FileSerializableSortedSet; -import datawave.query.util.sortedset.SortedSetTempFileHandler; import org.junit.After; import org.junit.Before; import org.junit.Test; import java.util.Comparator; import java.util.Iterator; +import java.util.Map; import java.util.Random; import java.util.SortedMap; -import java.util.SortedSet; -import java.util.TreeSet; +import java.util.TreeMap; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -24,8 +21,8 @@ public class FileSortedMapTest { SortedMap data = null; SortedMap extraData = null; - FileSortedMap set = null; - datawave.query.util.sortedset.SortedSetTempFileHandler handler = null; + FileSortedMap map = null; + datawave.query.util.sortedmap.SortedMapTempFileHandler handler = null; @Before public void setUp() throws Exception { @@ -39,20 +36,21 @@ public int compare(Integer o1, Integer o2) { } } }; - handler = new SortedSetTempFileHandler(); - set = new FileSerializableSortedMap<>(c, new FileSerializableSortedMap.SerializableFileHandler(handler), false); - data = new TreeSet<>(c); + 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.add(r.nextInt()); + data.put(r.nextInt(), r.nextInt()+1); } - extraData = new TreeSet<>(c); + extraData = new TreeMap<>(c); for (int i = 0; i < 20; i++) { - extraData.add(r.nextInt()); + extraData.put(r.nextInt(), r.nextInt()+1); } - extraData.removeAll(data); - set.addAll(data); + // make sure we have no overlap + data.keySet().removeAll(extraData.keySet()); + map.putAll(data); } @After @@ -60,316 +58,276 @@ 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(set.isPersisted()); - assertEquals(data.size(), set.size()); - assertTrue(set.containsAll(data)); - assertTrue(data.containsAll(set)); - set.persist(); - assertTrue(set.isPersisted()); + assertFalse(map.isPersisted()); + assertSortedMapEquals(data, map); + map.persist(); + assertTrue(map.isPersisted()); assertTrue(handler.getFile().exists()); - assertEquals(data.size(), set.size()); - assertTrue(set.containsAll(data)); - assertTrue(data.containsAll(set)); - set.load(); - assertFalse(set.isPersisted()); - assertEquals(data.size(), set.size()); - assertTrue(set.containsAll(data)); - assertTrue(data.containsAll(set)); + assertSortedMapEquals(data, map); + map.load(); + assertFalse(map.isPersisted()); + assertSortedMapEquals(data, map); } @Test public void testIsEmpty() throws Exception { - assertFalse(set.isEmpty()); - set.persist(); - assertFalse(set.isEmpty()); - set.clear(); - assertTrue(set.isEmpty()); - set.load(); - assertTrue(set.isEmpty()); + assertFalse(map.isEmpty()); + map.persist(); + assertFalse(map.isEmpty()); + map.clear(); + assertTrue(map.isEmpty()); + map.load(); + assertTrue(map.isEmpty()); } @Test public void testContains() throws Exception { - SortedSet someData = new TreeSet<>(data); - someData.add(extraData.first()); - assertTrue(set.containsAll(data)); - for (Integer i : data) { - assertTrue(set.contains(i)); + 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)); } - assertFalse(set.containsAll(someData)); - for (Integer i : extraData) { - assertFalse(set.contains(i)); + for (Integer i : extraData.keySet()) { + assertFalse(map.containsKey(i)); } - set.persist(); - assertTrue(set.containsAll(data)); - for (Integer i : data) { - assertTrue(set.contains(i)); + map.load(); + for (Integer i : data.keySet()) { + assertTrue(map.containsKey(i)); } - assertFalse(set.containsAll(someData)); - for (Integer i : extraData) { - assertFalse(set.contains(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); } - set.load(); - assertTrue(set.containsAll(data)); - for (Integer i : data) { - assertTrue(set.contains(i)); + while (it1.hasNext()) { + Map.Entry o1 = it1.next(); + System.out.println(o1 + " vs (null)"); } - assertFalse(set.containsAll(someData)); - for (Integer i : extraData) { - assertFalse(set.contains(i)); + while (it2.hasNext()) { + Map.Entry o2 = it2.next(); + System.out.println("(null) vs " + o2); } } @Test public void testIterator() throws Exception { - SortedSet testData = new TreeSet<>(data); - Iterator dataIterator = testData.iterator(); - Iterator setIterator = set.iterator(); + SortedMap testData = new TreeMap<>(data); + Iterator> dataIterator = testData.entrySet().iterator(); + Iterator> mapIterator = map.entrySet().iterator(); while (dataIterator.hasNext()) { - assertTrue(setIterator.hasNext()); - assertEquals(dataIterator.next(), setIterator.next()); + assertTrue(mapIterator.hasNext()); + assertEquals(dataIterator.next(), mapIterator.next()); } - assertFalse(setIterator.hasNext()); - for (Integer i : set) { - assertTrue(testData.remove(i)); + assertFalse(mapIterator.hasNext()); + for (Map.Entry i : map.entrySet()) { + assertEquals(testData.remove(i.getKey()), i.getValue()); } assertTrue(testData.isEmpty()); - set.persist(); - dataIterator = data.iterator(); - setIterator = set.iterator(); + map.persist(); + dataIterator = data.entrySet().iterator(); + mapIterator = map.entrySet().iterator(); while (dataIterator.hasNext()) { - assertTrue(setIterator.hasNext()); - assertEquals(dataIterator.next(), setIterator.next()); + assertTrue(mapIterator.hasNext()); + assertEquals(dataIterator.next(), mapIterator.next()); } - assertFalse(setIterator.hasNext()); - testData.addAll(data); - for (Integer i : set) { - assertTrue(testData.remove(i)); + assertFalse(mapIterator.hasNext()); + testData.putAll(data); + for (Map.Entry i : map.entrySet()) { + assertEquals(testData.remove(i.getKey()), i.getValue()); } assertTrue(testData.isEmpty()); - set.load(); - dataIterator = data.iterator(); - setIterator = set.iterator(); + map.load(); + dataIterator = data.entrySet().iterator(); + mapIterator = map.entrySet().iterator(); while (dataIterator.hasNext()) { - assertTrue(setIterator.hasNext()); - assertEquals(dataIterator.next(), setIterator.next()); + assertTrue(mapIterator.hasNext()); + assertEquals(dataIterator.next(), mapIterator.next()); } - assertFalse(setIterator.hasNext()); - testData.addAll(data); - for (Integer i : set) { - assertTrue(testData.remove(i)); + 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 testToArray() throws Exception { - Object[] a = set.toArray(); - Object[] d = data.toArray(); - assertArrayEquals(d, a); - set.persist(); - a = set.toArray(); - assertArrayEquals(d, a); - set.load(); - a = set.toArray(); - assertArrayEquals(d, a); - } - - @Test - public void testToArrayTArray() throws Exception { - Integer[] d = data.toArray(new Integer[set.size()]); - - Integer[] a = set.toArray(new Integer[set.size()]); - assertArrayEquals(d, a); - set.persist(); - a = set.toArray(new Integer[set.size()]); - assertArrayEquals(d, a); - set.load(); - a = set.toArray(new Integer[set.size()]); - assertArrayEquals(d, a); - - a = set.toArray(new Integer[set.size()]); - assertArrayEquals(d, a); - set.persist(); - a = set.toArray(new Integer[set.size()]); - assertArrayEquals(d, a); - set.load(); - a = set.toArray(new Integer[set.size()]); - assertArrayEquals(d, a); - - d = data.toArray(new Integer[set.size() * 2]); - a = set.toArray((new Integer[set.size() * 2])); - assertArrayEquals(d, a); - set.persist(); - a = set.toArray(new Integer[set.size() * 2]); - assertArrayEquals(d, a); - set.load(); - a = set.toArray(new Integer[set.size() * 2]); - assertArrayEquals(d, a); - } - - @Test - public void testAdd() throws Exception { - assertTrue(set.add(extraData.first())); - assertEquals(data.size() + 1, set.size()); - assertTrue(set.contains(extraData.first())); - assertTrue(set.remove(extraData.first())); - assertEquals(data.size(), set.size()); - assertFalse(set.contains(extraData.first())); - set.persist(); + 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 { - set.add(extraData.first()); - fail("Expected persisted set.add to fail"); + map.put(extraData.firstKey(), extraData.get(extraData.firstKey())); + fail("Expected persisted map.add to fail"); } catch (Exception e) { // expected } - set.load(); - assertEquals(data.size(), set.size()); - assertFalse(set.contains(extraData.first())); + map.load(); + assertEquals(data.size(), map.size()); + assertFalse(map.containsKey(extraData.firstKey())); } @Test - public void testAddAll() throws Exception { - assertTrue(set.addAll(extraData)); - assertEquals(data.size() + extraData.size(), set.size()); - assertTrue(set.containsAll(extraData)); - assertTrue(set.removeAll(extraData)); - assertEquals(data.size(), set.size()); - assertFalse(set.contains(extraData.first())); - set.persist(); + 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 { - set.addAll(extraData); - fail("Expected persisted set.addAll to fail"); + map.putAll(extraData); + fail("Expected persisted map.addAll to fail"); } catch (Exception e) { // expected } - set.load(); - assertEquals(data.size(), set.size()); - assertFalse(set.contains(extraData.first())); + map.load(); + assertEquals(data.size(), map.size()); + assertFalse(map.containsKey(extraData.firstKey())); } @Test public void testRetainAll() throws Exception { - SortedSet someData = new TreeSet<>(data); - someData.remove(data.first()); - someData.remove(data.last()); - someData.add(extraData.first()); - someData.add(extraData.last()); - assertFalse(set.retainAll(data)); - assertEquals(someData.size(), set.size()); - assertTrue(set.retainAll(someData)); - assertEquals(data.size() - 2, set.size()); - assertFalse(set.containsAll(data)); - assertFalse(set.containsAll(someData)); - assertFalse(set.contains(data.last())); - assertTrue(set.retainAll(extraData)); - assertTrue(set.isEmpty()); - - set.addAll(data); - set.persist(); + 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 { - set.retainAll(someData); - fail("Expected persisted set.retainAll to fail"); + map.keySet().retainAll(someData.keySet()); + fail("Expected persisted map.retainAll to fail"); } catch (Exception e) { // expected } - set.load(); - assertEquals(data.size(), set.size()); - assertTrue(set.containsAll(data)); + map.load(); + assertEquals(data.size(), map.size()); + assertTrue(map.keySet().containsAll(data.keySet())); } + @Test public void testRemoveAll() throws Exception { - SortedSet someData = new TreeSet<>(data); - someData.remove(data.first()); - someData.remove(data.last()); - someData.add(extraData.first()); - someData.add(extraData.last()); - assertFalse(set.removeAll(extraData)); - assertEquals(someData.size(), set.size()); - assertTrue(set.removeAll(someData)); - assertEquals(2, set.size()); - assertFalse(set.containsAll(data)); - assertFalse(set.containsAll(someData)); - assertTrue(set.contains(data.first())); - assertTrue(set.contains(data.last())); - assertTrue(set.removeAll(data)); - assertTrue(set.isEmpty()); - - set.addAll(data); - set.persist(); + 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 { - set.removeAll(someData); - fail("Expected persisted set.retainAll to fail"); + map.keySet().removeAll(someData.keySet()); + fail("Expected persisted map.retainAll to fail"); } catch (Exception e) { // expected } - set.load(); - assertEquals(data.size(), set.size()); - assertTrue(set.containsAll(data)); + map.load(); + assertEquals(data.size(), map.size()); + assertTrue(map.keySet().containsAll(data.keySet())); } @Test public void testClear() throws Exception { - set.clear(); - assertTrue(set.isEmpty()); - set.addAll(data); - set.persist(); - set.clear(); - assertTrue(set.isEmpty()); - set.load(); - assertTrue(set.isEmpty()); + 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(set.comparator()); - set.persist(); - assertNotNull(set.comparator()); - set.load(); - assertNotNull(set.comparator()); - RewritableSortedSet tempData = new RewritableSortedSetImpl<>(); - for (Integer i : data) { - if (i != null) { - tempData.add(i); - } + 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()); } - - set = new datawave.query.util.sortedset.FileSerializableSortedSet<>(tempData, new FileSerializableSortedSet.SerializableFileHandler(handler)); - - assertNull(set.comparator()); - assertEquals(tempData, set); - for (Integer i : set) { - assertEquals(tempData.first(), i); - tempData.remove(tempData.first()); - } - for (Integer i : data) { - if (i != null) { - tempData.add(i); - } - } - assertEquals(tempData, set); - set.persist(); - assertNull(set.comparator()); - set.load(); - assertNull(set.comparator()); - - for (Integer i : data) { - assertEquals(tempData.first(), i); - tempData.remove(tempData.first()); + 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 testSubSet() throws Exception { + public void testSubmap() throws Exception { Integer fromElement = null; Integer toElement = null; int index = 0; - for (Integer i : data) { + for (Integer i : data.keySet()) { if (index == (data.size() / 3)) { fromElement = i; } else if (index == data.size() * 2 / 3) { @@ -378,73 +336,73 @@ public void testSubSet() throws Exception { } index++; } - SortedSet subSet = set.subSet(fromElement, toElement); - assertEquals(data.subSet(fromElement, toElement), subSet); - set.persist(); - set.subSet(fromElement, toElement); - assertEquals(data.subSet(fromElement, toElement), subSet); - set.load(); - subSet = set.subSet(fromElement, toElement); - assertEquals(data.subSet(fromElement, toElement), subSet); + 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 testHeadSet() throws Exception { + public void testHeadmap() throws Exception { Integer toElement = null; int index = 0; - for (Integer i : data) { + for (Integer i : data.keySet()) { if (index == data.size() * 2 / 3) { toElement = i; break; } index++; } - SortedSet subSet = set.headSet(toElement); - assertEquals(data.headSet(toElement), subSet); - set.persist(); - set.headSet(toElement); - assertEquals(data.headSet(toElement), subSet); - set.load(); - subSet = set.headSet(toElement); - assertEquals(data.headSet(toElement), subSet); + 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 testTailSet() throws Exception { + public void testTailmap() throws Exception { Integer fromElement = null; int index = 0; - for (Integer i : data) { + for (Integer i : data.keySet()) { if (index == (data.size() / 3)) { fromElement = i; break; } index++; } - SortedSet subSet = set.tailSet(fromElement); - assertEquals(data.tailSet(fromElement), subSet); - set.persist(); - set.tailSet(fromElement); - assertEquals(data.tailSet(fromElement), subSet); - set.load(); - subSet = set.tailSet(fromElement); - assertEquals(data.tailSet(fromElement), subSet); + 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 testFirst() throws Exception { - assertEquals(data.first(), set.first()); - set.persist(); - assertEquals(data.first(), set.first()); - set.load(); - assertEquals(data.first(), set.first()); + 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.last(), set.last()); - set.persist(); - assertEquals(data.last(), set.last()); - set.load(); - assertEquals(data.last(), set.last()); + 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 index 9b0ef504b45..afcb703e8da 100644 --- 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 @@ -2,6 +2,7 @@ import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.util.sortedset.FileSortedSet; import datawave.query.util.sortedset.HdfsBackedSortedSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -232,7 +233,7 @@ public void persistCompactReloadTest() throws Exception { .withUniqueSubPath(uniquePath) .withMaxOpenFiles(9999) .withNumRetries(2) - .withPersistOptions(new FileSortedMap.PersistOptions()) + .withPersistOptions(new FileSortedSet.PersistOptions()) .build(); // @formatter:on 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 index d97a0ddb0ea..9a194e2d6a6 100644 --- 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 @@ -1,6 +1,5 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.MultiSetBackedSortedSet; import org.junit.Test; import java.util.ArrayList; @@ -8,8 +7,8 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; -import java.util.SortedSet; -import java.util.TreeSet; +import java.util.SortedMap; +import java.util.TreeMap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -20,26 +19,26 @@ public class MergeSortIteratorTest { @Test public void testIteration() { - SortedSet set1 = new TreeSet<>(); - SortedSet set2 = new TreeSet<>(); - SortedSet set3 = new TreeSet<>(); - - set1.add(1); - set1.add(3); - set1.add(4); - set1.add(5); - set1.add(6); - set1.add(10); - - set2.add(1); - set2.add(2); - set2.add(5); - set2.add(20); - - set3.add(2); - set3.add(5); - set3.add(6); - set3.add(30); + 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); @@ -52,13 +51,13 @@ public void testIteration() { expected.add(20); expected.add(30); - List> col = new ArrayList<>(); + List> col = new ArrayList<>(); col.add(set1); col.add(set2); col.add(set3); List results = new ArrayList<>(); - Iterator it = new datawave.query.util.sortedset.MultiSetBackedSortedSet(col).iterator(); + Iterator it = new MultiMapBackedSortedMap(col).keySet().iterator(); try { it.remove(); fail("Expected remove to fail"); @@ -74,9 +73,9 @@ public void testIteration() { } Integer next = it.next(); results.add(next); - assertTrue(set1.contains(next) || set2.contains(next) || set3.contains(next)); + assertTrue(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); it.remove(); - assertFalse(set1.contains(next) || set2.contains(next) || set3.contains(next)); + assertFalse(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); try { it.remove(); fail("Expected remove to fail"); @@ -90,26 +89,26 @@ public void testIteration() { @Test public void testIterationSansHasNext() { - SortedSet set1 = new TreeSet<>(); - SortedSet set2 = new TreeSet<>(); - SortedSet set3 = new TreeSet<>(); - - set1.add(1); - set1.add(3); - set1.add(4); - set1.add(5); - set1.add(6); - set1.add(10); - - set2.add(1); - set2.add(2); - set2.add(5); - set2.add(20); - - set3.add(2); - set3.add(5); - set3.add(6); - set3.add(30); + 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); @@ -122,12 +121,12 @@ public void testIterationSansHasNext() { expected.add(20); expected.add(30); - List> col = new ArrayList<>(); + List> col = new ArrayList<>(); col.add(set1); col.add(set2); col.add(set3); List results = new ArrayList<>(); - Iterator it = new datawave.query.util.sortedset.MultiSetBackedSortedSet(col).iterator(); + Iterator it = new MultiMapBackedSortedMap(col).keySet().iterator(); while (true) { try { it.remove(); @@ -142,9 +141,9 @@ public void testIterationSansHasNext() { break; } results.add(next); - assertTrue(set1.contains(next) || set2.contains(next) || set3.contains(next)); + assertTrue(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); it.remove(); - assertFalse(set1.contains(next) || set2.contains(next) || set3.contains(next)); + assertFalse(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); try { it.remove(); fail("Expected remove to fail"); @@ -170,28 +169,28 @@ public int compare(Integer o1, Integer o2) { } }; - SortedSet set1 = new TreeSet<>(c); - SortedSet set2 = new TreeSet<>(c); - SortedSet set3 = new TreeSet<>(c); - - set1.add(1); - set1.add(3); - set1.add(4); - set1.add(5); - set1.add(6); - set1.add(10); - - set2.add(null); - set2.add(1); - set2.add(2); - set2.add(5); - set2.add(20); - - set3.add(null); - set3.add(2); - set3.add(5); - set3.add(6); - set3.add(30); + 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); @@ -205,12 +204,12 @@ public int compare(Integer o1, Integer o2) { expected.add(20); expected.add(30); - List> col = new ArrayList<>(); + List> col = new ArrayList<>(); col.add(set1); col.add(set2); col.add(set3); List results = new ArrayList<>(); - Iterator it = new MultiSetBackedSortedSet(col).iterator(); + Iterator it = new MultiMapBackedSortedMap(col).keySet().iterator(); try { it.remove(); fail("Expected remove to fail"); @@ -226,9 +225,9 @@ public int compare(Integer o1, Integer o2) { } Integer next = it.next(); results.add(next); - assertTrue(set1.contains(next) || set2.contains(next) || set3.contains(next)); + assertTrue(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); it.remove(); - assertFalse(set1.contains(next) || set2.contains(next) || set3.contains(next)); + assertFalse(set1.containsKey(next) || set2.containsKey(next) || set3.containsKey(next)); try { it.remove(); fail("Expected remove to fail"); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java deleted file mode 100644 index f4bf76ba846..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/RewritableSortedMapTest.java +++ /dev/null @@ -1,283 +0,0 @@ -package datawave.query.util.sortedmap; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class RewritableSortedMapTest { - - private Map.Entry[] data = null; - private int[] sortedOrder = null; - private RewritableSortedSetImpl> set = null; - private final byte[] template = new byte[] {5, 2, 29, 4, 8, 3, 25, 23, 6, 21, 7, 16}; - private final int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; - - private Comparator> keyComparator = new Comparator<>() { - @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return o1.getKey().compareTo(o2.getKey()); - } - }; - - private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator = new RewritableSortedSetImpl.RewriteStrategy<>() { - @Override - public boolean rewrite(Map.Entry original, Map.Entry update) { - int comparison = original.getKey().compareTo(update.getKey()); - if (comparison == 0) { - comparison = original.getValue().compareTo(update.getValue()); - } - return comparison < 0; - } - }; - - @Before - public void setUp() throws Exception { - data = new Map.Entry[template.length * 2]; - for (int i = 0; i < template.length; i++) { - byte[] buffer = new byte[i + 11]; - byte[] vbuffer = new byte[i + 11]; - Arrays.fill(buffer, template[i]); - Arrays.fill(vbuffer, (byte) (template[i] + 1)); - data[i] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); - } - for (int i = 0; i < template.length; i++) { - byte[] buffer = new byte[i + 10]; - byte[] vbuffer = new byte[i + 10]; - Arrays.fill(buffer, template[i]); - Arrays.fill(vbuffer, (byte) (template[i] + 1)); - data[i + template.length] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); - } - 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]; - } - set = new RewritableSortedSetImpl<>(keyComparator, keyValueComparator); - - // 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(set, data); - } - } - - private void addDataRandomly(RewritableSortedSetImpl> set, Map.Entry[] data) { - Set added = new HashSet<>(); - // add data until all of the entries have been added - Random random = new Random(); - while (added.size() < data.length) { - int i = random.nextInt(data.length); - set.add(data[i]); - added.add(i); - } - } - - @After - public void tearDown() throws Exception { - data = null; - sortedOrder = null; - set.clear(); - set = null; - } - - @Test - public void testSize() { - int expectedSize = data.length; - assertEquals(expectedSize, set.size()); - for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - expectedSize--; - assertEquals(expectedSize, set.size()); - } - for (int i = 0; i < (data.length / 2); i++) { - set.remove(data[i]); - expectedSize--; - assertEquals(expectedSize, set.size()); - } - assertEquals(0, set.size()); - for (int i = 0; i < data.length; i++) { - set.add(data[i]); - expectedSize++; - assertEquals(expectedSize, set.size()); - } - } - - @Test - public void testIsEmpty() { - assertFalse(set.isEmpty()); - for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - assertFalse(set.isEmpty()); - } - for (int i = 1; i < (data.length / 2); i++) { - set.remove(data[i]); - assertFalse(set.isEmpty()); - } - set.remove(data[0]); - assertTrue(set.isEmpty()); - for (int i = 0; i < data.length; i++) { - set.add(data[i]); - assertFalse(set.isEmpty()); - } - } - - @Test - public void testClear() { - set.clear(); - assertTrue(set.isEmpty()); - } - - @Test - public void testContainsObject() { - for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - } - for (int i = 1; i < (data.length / 2); i++) { - assertTrue(set.contains(data[i])); - } - for (int i = (data.length / 2); i < data.length; i++) { - assertFalse(set.contains(data[i])); - } - } - - @Test - public void testIterator() { - int index = 0; - for (Iterator> it = set.iterator(); it.hasNext();) { - Map.Entry value = it.next(); - Map.Entry expected = data[sortedOrder[index++]]; - assertEquals(expected, value); - } - set.clear(); - for (Map.Entry value : set) { - fail(); - } - } - - @Test - public void testIteratorRemove() { - int size = set.size(); - for (Iterator> it = set.iterator(); it.hasNext();) { - Map.Entry value = it.next(); - assertTrue(set.contains(value)); - it.remove(); - size--; - assertEquals(size, set.size()); - } - assertEquals(0, size); - assertTrue(set.isEmpty()); - } - - @Test - public void testSubSet() { - int start = sortedOrder.length / 3; - int end = start * 2; - try { - SortedSet> subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); - SortedSet> expected = new TreeSet<>(); - for (int i = start; i < end; i++) { - expected.add(data[sortedOrder[i]]); - } - assertEquals(expected, subSet); - } catch (Exception e) { - // expected - } - } - - @Test - public void testHeadSet() { - int end = sortedOrder.length / 3; - try { - SortedSet> subSet = set.headSet(data[sortedOrder[end]]); - SortedSet> expected = new TreeSet<>(); - for (int i = 0; i < end; i++) { - expected.add(data[sortedOrder[i]]); - } - assertEquals(expected, subSet); - } catch (Exception e) { - // expected - } - } - - @Test - public void testTailSet() { - int start = sortedOrder.length / 3; - try { - SortedSet> subSet = set.tailSet(data[sortedOrder[start]]); - SortedSet> expected = new TreeSet<>(); - for (int i = start; i < sortedOrder.length; i++) { - expected.add(data[sortedOrder[i]]); - } - assertEquals(expected, subSet); - } catch (Exception e) { - // expected - } - } - - @Test - public void testLast() { - Map.Entry expected = data[sortedOrder[data.length - 1]]; - Map.Entry value = set.last(); - assertEquals(expected, value); - } - - @Test - public void testFirst() { - Map.Entry expected = data[sortedOrder[0]]; - Map.Entry value = set.first(); - assertEquals(expected, value); - } - - @Test - public void testRewrite() { - // create a new set of data, half of which has greater Values and half of which has lesser Values - Map.Entry[] data2 = new Map.Entry[template.length * 2]; - for (int i = 0; i < template.length; i++) { - byte[] buffer = new byte[i + 11]; - byte[] vbuffer = new byte[i + 11]; - Arrays.fill(buffer, template[i]); - Arrays.fill(vbuffer, (byte) (template[i] + 2)); - data2[i] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); - } - for (int i = 0; i < template.length; i++) { - byte[] buffer = new byte[i + 10]; - byte[] vbuffer = new byte[i + 10]; - Arrays.fill(buffer, template[i]); - Arrays.fill(vbuffer, (byte) (template[i] - 1)); - data2[i + template.length] = new UnmodifiableMapEntry(new Key(buffer), new Value(vbuffer)); - } - - for (int d = 0; d < 11; d++) { - addDataRandomly(set, data2); - } - - // now test the contents - int index = 0; - for (Iterator> it = set.iterator(); it.hasNext();) { - Map.Entry value = it.next(); - int dataIndex = sortedOrder[index++]; - Map.Entry expected = (dataIndex < template.length ? data2[dataIndex] : data[dataIndex]); - assertEquals(expected, value); - } - - } - -} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java deleted file mode 100644 index 288199962a9..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/SortedByteMapBufferTest.java +++ /dev/null @@ -1,577 +0,0 @@ -package datawave.query.util.sortedmap; - -import datawave.query.util.sortedset.SortedByteSetBuffer; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.SortedSet; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class SortedByteMapBufferTest { - private byte[][] data = null; - private int[] sortedOrder = null; - private datawave.query.util.sortedset.SortedByteSetBuffer set = null; - - @Before - public void setUp() { - byte[] template = new byte[] {5, 2, 78, 4, 8, 3, 54, 23, 6, 21, 7, 16}; - int[] sortedTemplate = new int[] {1, 5, 3, 0, 8, 10, 4, 11, 9, 7, 6, 2}; - data = new byte[template.length * 2][]; - for (int i = 0; i < template.length; i++) { - data[i] = new byte[i + 11]; - Arrays.fill(data[i], template[i]); - } - for (int i = 0; i < template.length; i++) { - data[i + template.length] = new byte[10]; - Arrays.fill(data[i + template.length], template[i]); - } - 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]; - } - set = new datawave.query.util.sortedset.SortedByteSetBuffer(5); - Collections.addAll(set, data); - } - - /** - * @throws Exception - */ - @After - public void tearDown() { - data = null; - sortedOrder = null; - set.clear(); - set = null; - } - - /** - * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#size()}. - */ - @Test - public void testSize() { - int expectedSize = data.length; - assertEquals(expectedSize, set.size()); - for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - expectedSize--; - assertEquals(expectedSize, set.size()); - } - for (int i = 0; i < (data.length / 2); i++) { - set.remove(data[i]); - expectedSize--; - assertEquals(expectedSize, set.size()); - } - assertEquals(0, set.size()); - for (int i = 0; i < data.length; i++) { - set.add(data[i]); - expectedSize++; - assertEquals(expectedSize, set.size()); - } - } - - /** - * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#isEmpty()}. - */ - @Test - public void testIsEmpty() { - assertFalse(set.isEmpty()); - for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - assertFalse(set.isEmpty()); - } - for (int i = 1; i < (data.length / 2); i++) { - set.remove(data[i]); - assertFalse(set.isEmpty()); - } - set.remove(data[0]); - assertTrue(set.isEmpty()); - for (int i = 0; i < data.length; i++) { - set.add(data[i]); - assertFalse(set.isEmpty()); - } - } - - /** - * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#clear()}. - */ - @Test - public void testClear() { - set.clear(); - assertTrue(set.isEmpty()); - for (int i = 0; i < data.length; i++) { - set.add(data[i]); - assertFalse(set.isEmpty()); - } - set.clear(); - assertTrue(set.isEmpty()); - } - - /** - * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#contains(Object)}. - */ - @Test - public void testContainsObject() { - for (int i = (data.length / 2); i < data.length; i++) { - set.remove(data[i]); - } - for (int i = 0; i < (data.length / 2); i++) { - assertTrue(set.contains(data[i])); - } - for (int i = (data.length / 2); i < data.length; i++) { - assertFalse(set.contains(data[i])); - } - } - - /** - * Test method for {@link datawave.query.util.sortedset.SortedByteSetBuffer#iterator()}. - */ - @Test - public void testIterator() { - int index = 0; - for (Iterator it = set.iterator(); it.hasNext();) { - byte[] value = it.next(); - byte[] expected = data[sortedOrder[index++]]; - assertArrayEquals(expected, value); - } - set.clear(); - for (@SuppressWarnings("unused") - byte[] value : set) { - fail(); - } - } - - /** - * Test method fo {@link nsa.datawave.data.SortedByteSetBuffer#iterator().remove()}. - */ - @Test - public void testIteratorRemove() { - int size = set.size(); - for (Iterator it = set.iterator(); it.hasNext();) { - byte[] value = it.next(); - assertTrue(set.contains(value)); - it.remove(); - assertFalse(set.contains((value))); - size--; - assertEquals(size, set.size()); - } - assertTrue(set.isEmpty()); - } - - /** - * Test method for {@link SortedByteSetBuffer#comparator()}. - */ - @Test - public void testComparator() { - Comparator comparator = set.comparator(); - byte[][] testData = Arrays.copyOf(data, data.length); - Arrays.sort(testData, comparator); - int index = 0; - for (byte[] value : set) { - byte[] expected = data[sortedOrder[index++]]; - assertArrayEquals(expected, value); - } - } - - /** - * Test method for {@link SortedByteSetBuffer#subSet(byte[]. byte[]}/ - */ - @Test - public void testSubSet() { - int start = sortedOrder.length / 3; - int end = start * 2; - SortedSet subSet = set.subSet(data[sortedOrder[start]], data[sortedOrder[end]]); - - // verify contents - assertEquals(end - start, subSet.size()); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify order - assertArrayEquals(data[sortedOrder[start]], subSet.first()); - int index = start; - for (byte[] value : subSet) { - assertArrayEquals(data[sortedOrder[index++]], value); - } - assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); - - // verify add - assertFalse(subSet.add(data[sortedOrder[start]])); - assertFalse(subSet.add(data[sortedOrder[end - 1]])); - try { - subSet.add(data[sortedOrder[start - 1]]); - fail("Expected to not be able to add something outside the range"); - } catch (IllegalArgumentException iae) { - // ok - } - try { - subSet.add(data[sortedOrder[end]]); - fail("Expected to not be able to add something outside the range"); - } catch (IllegalArgumentException iae) { - // ok - } - byte[] startValue = data[sortedOrder[start]]; - byte[] value = Arrays.copyOf(startValue, startValue.length + 50); - assertTrue(subSet.add(value)); - assertEquals(end - start + 1, subSet.size()); - assertEquals(data.length + 1, set.size()); - assertTrue(subSet.contains(value)); - assertTrue(set.contains(value)); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify remove - assertFalse(subSet.remove(data[sortedOrder[start - 1]])); - assertFalse(subSet.remove(data[sortedOrder[end]])); - assertTrue(subSet.remove(value)); - assertEquals(end - start, subSet.size()); - assertEquals(data.length, set.size()); - assertFalse(subSet.contains(value)); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify subSet - try { - @SuppressWarnings("unused") - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start - 1]], data[sortedOrder[end]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - try { - @SuppressWarnings("unused") - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start]], data[sortedOrder[end + 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); - assertEquals(end - start - 2, subSubSet.size()); - for (int i = 0; i < data.length; i++) { - if (i > start && i < (end - 1)) { - assertTrue(subSubSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSubSet.contains(data[sortedOrder[i]])); - } - } - - // verify tailSet - try { - @SuppressWarnings("unused") - SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start - 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); - assertEquals(end - start - 1, subTailSet.size()); - for (int i = 0; i < data.length; i++) { - if (i > start && i < end) { - assertTrue(subTailSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subTailSet.contains(data[sortedOrder[i]])); - } - } - - // verify headSet - try { - @SuppressWarnings("unused") - SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end + 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); - assertEquals(end - start - 1, subHeadSet.size()); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < (end - 1)) { - assertTrue(subHeadSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subHeadSet.contains(data[sortedOrder[i]])); - } - } - } - - /** - * Test method for {@link SortedByteSetBuffer#headSet(byte[])} - */ - @Test - public void testHeadSet() { - int end = sortedOrder.length / 3; - int start = 0; - SortedSet subSet = set.headSet((data[sortedOrder[end]])); - - // verify contents - assertEquals(end - start, subSet.size()); - for (int i = 0; i < data.length; i++) { - if (i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify order - assertArrayEquals(data[sortedOrder[start]], subSet.first()); - int index = start; - for (byte[] value : subSet) { - assertArrayEquals(data[sortedOrder[index++]], value); - } - assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); - - // verify add - assertFalse(subSet.add(data[sortedOrder[start]])); - assertFalse(subSet.add(data[sortedOrder[end - 1]])); - try { - subSet.add(data[sortedOrder[end]]); - fail("Expected to not be able to add something outside the range"); - } catch (IllegalArgumentException iae) { - // ok - } - byte[] startValue = data[sortedOrder[start]]; - byte[] value = Arrays.copyOf(startValue, startValue.length + 50); - assertTrue(subSet.add(value)); - assertEquals(end - start + 1, subSet.size()); - assertEquals(data.length + 1, set.size()); - assertTrue(subSet.contains(value)); - assertTrue(set.contains(value)); - for (int i = 0; i < data.length; i++) { - if (i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify remove - assertFalse(subSet.remove(data[sortedOrder[end]])); - assertTrue(subSet.remove(value)); - assertEquals(end - start, subSet.size()); - assertEquals(data.length, set.size()); - assertFalse(subSet.contains(value)); - for (int i = 0; i < data.length; i++) { - if (i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify subSet - try { - @SuppressWarnings("unused") - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start]], data[sortedOrder[end + 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); - assertEquals(end - start - 2, subSubSet.size()); - for (int i = 0; i < data.length; i++) { - if (i > start && i < (end - 1)) { - assertTrue(subSubSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSubSet.contains(data[sortedOrder[i]])); - } - } - - // verify tailSet - SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); - assertEquals(end - start - 1, subTailSet.size()); - for (int i = 0; i < data.length; i++) { - if (i > start && i < end) { - assertTrue(subTailSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subTailSet.contains(data[sortedOrder[i]])); - } - } - - // verify headSet - try { - @SuppressWarnings("unused") - SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end + 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); - assertEquals(end - start - 1, subHeadSet.size()); - for (int i = 0; i < data.length; i++) { - if (i < (end - 1)) { - assertTrue(subHeadSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subHeadSet.contains(data[sortedOrder[i]])); - } - } - } - - /** - * Test method for {@link SortedByteSetBuffer#tailSet(byte[])}. - */ - @Test - public void testTailSet() { - int start = sortedOrder.length / 3; - int end = sortedOrder.length; - SortedSet subSet = set.tailSet(data[sortedOrder[start]]); - - // verify contents - assertEquals(end - start, subSet.size()); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify order - assertArrayEquals(data[sortedOrder[start]], subSet.first()); - int index = start; - for (byte[] value : subSet) { - assertArrayEquals(data[sortedOrder[index++]], value); - } - assertArrayEquals(data[sortedOrder[end - 1]], subSet.last()); - - // verify add - assertFalse(subSet.add(data[sortedOrder[start]])); - assertFalse(subSet.add(data[sortedOrder[end - 1]])); - try { - subSet.add(data[sortedOrder[start - 1]]); - fail("Expected to not be able to add something outside the range"); - } catch (IllegalArgumentException iae) { - // ok - } - byte[] startValue = data[sortedOrder[start]]; - byte[] value = Arrays.copyOf(startValue, startValue.length + 50); - assertTrue(subSet.add(value)); - assertEquals(end - start + 1, subSet.size()); - assertEquals(data.length + 1, set.size()); - assertTrue(subSet.contains(value)); - assertTrue(set.contains(value)); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify remove - assertFalse(subSet.remove(data[sortedOrder[start - 1]])); - assertTrue(subSet.remove(value)); - assertEquals(end - start, subSet.size()); - assertEquals(data.length, set.size()); - assertFalse(subSet.contains(value)); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < end) { - assertTrue(subSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSet.contains(data[sortedOrder[i]])); - } - } - - // verify subSet - try { - @SuppressWarnings("unused") - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start - 1]], data[sortedOrder[end - 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subSubSet = subSet.subSet(data[sortedOrder[start + 1]], data[sortedOrder[end - 1]]); - assertEquals(end - start - 2, subSubSet.size()); - for (int i = 0; i < data.length; i++) { - if (i > start && i < (end - 1)) { - assertTrue(subSubSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subSubSet.contains(data[sortedOrder[i]])); - } - } - - // verify tailSet - try { - @SuppressWarnings("unused") - SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start - 1]]); - fail("Expected to not be able to create a supper set out of a sub set"); - } catch (IllegalArgumentException iae) { - // ok - } - SortedSet subTailSet = subSet.tailSet(data[sortedOrder[start + 1]]); - assertEquals(end - start - 1, subTailSet.size()); - for (int i = 0; i < data.length; i++) { - if (i > start && i < end) { - assertTrue(subTailSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subTailSet.contains(data[sortedOrder[i]])); - } - } - - // verify headSet - SortedSet subHeadSet = subSet.headSet(data[sortedOrder[end - 1]]); - assertEquals(end - start - 1, subHeadSet.size()); - for (int i = 0; i < data.length; i++) { - if (i >= start && i < (end - 1)) { - assertTrue(subHeadSet.contains(data[sortedOrder[i]])); - } else { - assertFalse(subHeadSet.contains(data[sortedOrder[i]])); - } - } - } - - /** - * Test method for {@link SortedByteSetBuffer#get(int)}. - */ - @Test - public void testGet() { - for (int i = 0; i < data.length; i++) { - byte[] expected = data[sortedOrder[i]]; - byte[] value = set.get(i); - assertArrayEquals(expected, value); - } - } - - /** - * Test method for {@link SortedByteSetBuffer#last()}. - */ - @Test - public void testLast() { - byte[] expected = data[sortedOrder[data.length - 1]]; - byte[] value = set.last(); - assertArrayEquals(expected, value); - } - - /** - * Test method for {@link SortedByteSetBuffer#first()}. - */ - @Test - public void testFirst() { - byte[] expected = data[sortedOrder[0]]; - byte[] value = set.first(); - assertArrayEquals(expected, value); - } -} From c0cf844d2119d6f7c83ae3092a3cc72342b848bd Mon Sep 17 00:00:00 2001 From: Ivan Bella Date: Fri, 16 Aug 2024 19:17:56 +0000 Subject: [PATCH 4/5] Worked through more tests for the sorted map implementations --- .../datawave/query/attributes/Attribute.java | 23 +++- .../query/attributes/Cardinality.java | 33 +----- .../query/transformer/UniqueTransform.java | 8 +- .../BufferedFileBackedSortedMap.java | 72 +++++++------ .../sortedmap/FileByteDocumentSortedMap.java | 47 +++----- .../util/sortedmap/FileKeyValueSortedMap.java | 33 ++---- .../sortedmap/FileSerializableSortedMap.java | 29 ++--- .../query/util/sortedmap/FileSortedMap.java | 79 ++++++-------- .../util/sortedmap/HdfsBackedSortedMap.java | 5 +- .../sortedmap/MultiMapBackedSortedMap.java | 35 +++--- .../sortedmap/SortedMapTempFileHandler.java | 11 +- .../rfile/KeyValueByteDocumentTransforms.java | 37 +++---- .../rfile/RFileByteDocumentInputStream.java | 13 +-- .../rfile/RFileByteDocumentOutputStream.java | 15 +-- .../rfile/RFileKeyValueInputStream.java | 11 +- .../rfile/RFileKeyValueInputStreamBase.java | 13 +-- .../rfile/RFileKeyValueOutputStream.java | 7 +- .../rfile/RFileKeyValueOutputStreamBase.java | 6 +- ...feredFileBackedByteArraySortedMapTest.java | 24 ----- ...edFileBackedByteDocumentSortedMapTest.java | 54 ++++------ .../BufferedFileBackedKeySortedMapTest.java | 25 ----- ...fferedFileBackedKeyValueSortedMapTest.java | 28 +++-- ...eredFileBackedRewritableSortedMapTest.java | 28 +++-- .../BufferedFileBackedSortedMapTest.java | 70 ++++++------ .../util/sortedmap/FileSortedMapTest.java | 23 ++-- .../sortedmap/HdfsBackedSortedMapTest.java | 19 ++-- .../util/sortedmap/MergeSortIteratorTest.java | 102 +++++++++--------- .../MultiSetBackedSortedMapTest.java | 21 ++-- .../KeyValueByteDocumenTransformsTest.java | 82 ++++++++++++++ 29 files changed, 453 insertions(+), 500 deletions(-) delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java create mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumenTransformsTest.java 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 976f3105940..9a69b417d71 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 @@ -12,6 +12,7 @@ import org.apache.commons.lang.builder.HashCodeBuilder; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.WritableUtils; import org.apache.log4j.Logger; @@ -214,7 +215,25 @@ protected int compareMetadata(Attribute other) { return -1; } } else if (this.isMetadataSet()) { - return this.metadata.compareTo(other.metadata); + // we only need to compare those parts of the metadata that persist through serialization + // return this.metadata.compareTo(other.metadata); + byte[] cvBytes = this.getColumnVisibility().getExpression(); + if (null == cvBytes) { + cvBytes = Constants.EMPTY_BYTES; + } + + byte[] otherCVBytes = other.getColumnVisibility().getExpression(); + if (null == otherCVBytes) { + otherCVBytes = Constants.EMPTY_BYTES; + } + + int result = WritableComparator.compareBytes(cvBytes, 0, cvBytes.length, otherCVBytes, 0, otherCVBytes.length); + + if (result == 0) { + result = new Long(this.getTimestamp()).compareTo(other.getTimestamp()); + } + + return result; } else { return 0; } @@ -232,7 +251,7 @@ public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(145, 11); hcb.append(this.isMetadataSet()); if (isMetadataSet()) { - hcb.append(this.getMetadata()); + hcb.append(this.getMetadata().getColumnVisibility()).append(this.getMetadata().getTimestamp()); } return hcb.toHashCode(); } diff --git a/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java b/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java index e2f91a2d2a0..36d05b2aee8 100644 --- a/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java +++ b/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java @@ -111,41 +111,10 @@ public boolean equals(Object o) { return false; } - @Override - protected int compareMetadata(Attribute other) { - if (this.isMetadataSet() != other.isMetadataSet()) { - if (this.isMetadataSet()) { - return 1; - } else { - return -1; - } - } else if (this.isMetadataSet()) { - byte[] cvBytes = this.getColumnVisibility().getExpression(); - if (null == cvBytes) { - cvBytes = Constants.EMPTY_BYTES; - } - - byte[] otherCVBytes = other.getColumnVisibility().getExpression(); - if (null == otherCVBytes) { - otherCVBytes = Constants.EMPTY_BYTES; - } - - int result = WritableComparator.compareBytes(cvBytes, 0, cvBytes.length, otherCVBytes, 0, otherCVBytes.length); - - if (result == 0) { - result = new Long(this.getTimestamp()).compareTo(other.getTimestamp()); - } - - return result; - } else { - return 0; - } - } - @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(2099, 2129); - hcb.appendSuper(content.hashCode()).append(this.getMetadata().getColumnVisibility()).append(this.getMetadata().getTimestamp()); + hcb.append(super.hashCode()).append(content); return hcb.toHashCode(); } 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 8eabd56e4b5..6913585e07f 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 @@ -17,10 +17,6 @@ import javax.annotation.Nullable; -import datawave.query.util.sortedmap.FileByteDocumentSortedMap; -import datawave.query.util.sortedmap.FileKeyValueSortedMap; -import datawave.query.util.sortedmap.FileSortedMap; -import datawave.query.util.sortedmap.HdfsBackedSortedMap; import org.apache.accumulo.core.data.Key; import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; import org.apache.hadoop.fs.FileSystem; @@ -44,6 +40,10 @@ import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; import datawave.query.iterator.profile.FinalDocumentTrackingIterator; import datawave.query.model.QueryModel; +import datawave.query.util.sortedmap.FileByteDocumentSortedMap; +import datawave.query.util.sortedmap.FileKeyValueSortedMap; +import datawave.query.util.sortedmap.FileSortedMap; +import datawave.query.util.sortedmap.HdfsBackedSortedMap; import datawave.query.util.sortedset.ByteArrayComparator; import datawave.query.util.sortedset.FileSortedSet; import datawave.query.util.sortedset.HdfsBackedSortedSet; 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 index 36bf56f7c63..5606365921b 100644 --- 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 @@ -1,19 +1,19 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedmap.FileSortedMap.SortedMapFileHandler; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; -import java.util.Iterator; 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. @@ -263,19 +263,6 @@ public boolean containsValue(Object value) { return false; } - protected Iterator> iterator() { - // first lets compact down the maps if needed - try { - // if we have any persisted maps, then ensure we are persisted - if (map.getMaps().size() > 1) { - persist(); - } - } catch (IOException ioe) { - throw new RuntimeException("Unable to persist or compact file backed sorted map", ioe); - } - return map.iterator(); - } - private String printHandlerFactories() { return String.join(", ", handlerFactories.stream().map(SortedMapFileHandlerFactory::toString).collect(Collectors.toList())); } @@ -394,8 +381,8 @@ public V put(K key, V value) { map.addMap(buffer); } V previous = buffer.put(key, value); + sizeModified = true; if (previous != null) { - sizeModified = true; if (buffer.size() >= bufferPersistThreshold) { try { persist(); @@ -433,7 +420,7 @@ public void putAll(Map c) { public V remove(Object o) { V value = null; for (SortedMap map : map.getMaps()) { - FileSortedMap filemap = (FileSortedMap)map; + FileSortedMap filemap = (FileSortedMap) map; boolean persist = false; if (filemap.isPersisted()) { try { @@ -447,7 +434,7 @@ public V remove(Object o) { V testValue = map.remove(o); if (testValue != null) { if (value != null) { - if (rewriteStrategy == null || rewriteStrategy.rewrite((K)o, value, testValue)) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { value = testValue; } } else { @@ -486,47 +473,62 @@ public Comparator comparator() { } @Override - public SortedMap subMap(K fromKey, K toKey) { - return null; + public SortedMap subMap(K fromKey, K toKey) { + return map.subMap(fromKey, toKey); } @Override - public SortedMap headMap(K toKey) { - return null; + public SortedMap headMap(K toKey) { + return map.headMap(toKey); } @Override - public SortedMap tailMap(K fromKey) { - return null; + public SortedMap tailMap(K fromKey) { + return map.tailMap(fromKey); } @Override public K firstKey() { - return null; + return map.firstKey(); } @Override public K lastKey() { - return null; + return map.lastKey(); } @Override public Set keySet() { - return null; + persistIfMultipleMaps(); + return map.keySet(); } @Override public Collection values() { - return null; + persistIfMultipleMaps(); + return map.values(); + } + + protected void persistIfMultipleMaps() { + // compact down the sets if needed + try { + // if we have any persisted sets, then ensure we are persisted + if (map.getMaps().size() > 1) { + persist(); + } + } catch (IOException ioe) { + throw new RuntimeException("Unable to persist or compact file backed sorted set", ioe); + } } @Override - public Set> entrySet() { - return null; + public Set> entrySet() { + persistIfMultipleMaps(); + return map.entrySet(); } @Override - public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { this.rewriteStrategy = rewriteStrategy; } @@ -539,7 +541,7 @@ public FileSortedMap.RewriteStrategy getRewriteStrategy() { public V get(Object o) { V value = null; for (SortedMap map : map.getMaps()) { - FileSortedMap filemap = (FileSortedMap)map; + FileSortedMap filemap = (FileSortedMap) map; boolean persist = false; if (filemap.isPersisted()) { try { @@ -553,7 +555,7 @@ public V get(Object o) { V testValue = map.get(o); if (testValue != null) { if (value != null) { - if (rewriteStrategy == null || rewriteStrategy.rewrite((K)o, value, testValue)) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { value = testValue; } } else { 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 index 44c92c5788c..76c34e0d63b 100644 --- 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 @@ -1,15 +1,16 @@ 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; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.util.Comparator; -import java.util.SortedMap; /** * 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 @@ -24,7 +25,7 @@ public class FileByteDocumentSortedMap extends FileSortedMap { public final static class DefaultByteComparator implements Comparator { @Override - public int compare(byte[] o1, byte[] o2) { + public int compare(byte[] o1, byte[] o2) { return new ByteArrayComparator().compare(o1, o2); } } @@ -76,8 +77,7 @@ public FileByteDocumentSortedMap(SortedMapFileHandler handler, boolean 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); + super((comparator == null ? new DefaultByteComparator() : comparator), new ByteDocumentFileHandler(handler), new Factory(), persisted); } /** @@ -88,7 +88,7 @@ public FileByteDocumentSortedMap(Comparator comparator, SortedMapFileHan * @param handler * the sorted map file handler */ - public FileByteDocumentSortedMap(FileSortedMap map, SortedMapFileHandler handler) { + public FileByteDocumentSortedMap(SortedMap map, SortedMapFileHandler handler) { super(map, new ByteDocumentFileHandler(handler), new Factory()); } @@ -105,7 +105,7 @@ public FileByteDocumentSortedMap(FileSortedMap map, SortedMapFi * @throws IOException * for issues with read/write */ - public FileByteDocumentSortedMap(FileSortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { + public FileByteDocumentSortedMap(SortedMap map, SortedMapFileHandler handler, boolean persist) throws IOException { super(map, new ByteDocumentFileHandler(handler), new Factory(), persist); } @@ -146,8 +146,7 @@ public SortedMapInputStream getInputStream() throws IOException } @Override - public SortedMapInputStream getInputStream(byte[] start, byte[] end) - throws IOException { + public SortedMapInputStream getInputStream(byte[] start, byte[] end) throws IOException { return new RFileByteDocumentInputStream(delegate.getInputStream(), delegate.getSize(), start, end); } @@ -183,24 +182,13 @@ public FileByteDocumentSortedMap newInstance(FileSortedMap othe } @Override - public FileByteDocumentSortedMap newInstance(FileSortedMap other, byte[] from, - byte[] to) { + public FileByteDocumentSortedMap newInstance(FileSortedMap other, byte[] from, byte[] to) { return new FileByteDocumentSortedMap((FileByteDocumentSortedMap) other, from, to); } @Override - public FileByteDocumentSortedMap newInstance(SortedMapFileHandler handler, boolean persisted) { - return new FileByteDocumentSortedMap(handler, persisted); - } - - @Override - public FileByteDocumentSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { - return new FileByteDocumentSortedMap(comparator, handler, persisted); - } - - @Override - public FileSortedMap newInstance(Comparator comparator, - RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, boolean persisted) { + public FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, + SortedMapFileHandler handler, boolean persisted) { FileByteDocumentSortedMap map = new FileByteDocumentSortedMap(comparator, handler, persisted); map.setRewriteStrategy(rewriteStrategy); return map; @@ -208,13 +196,12 @@ public FileSortedMap newInstance(Comparator comparator, @Override public FileByteDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { - return new FileByteDocumentSortedMap((FileSortedMap)map, handler); + return new FileByteDocumentSortedMap(map, handler); } @Override - public FileByteDocumentSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) - throws IOException { - return new FileByteDocumentSortedMap((FileSortedMap)map, handler, persist); + 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/FileKeyValueSortedMap.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedmap/FileKeyValueSortedMap.java index 4db9f5b127f..976b5d75d47 100644 --- 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 @@ -1,15 +1,16 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedmap.rfile.RFileKeyValueInputStream; -import datawave.query.util.sortedmap.rfile.RFileKeyValueOutputStream; -import datawave.query.util.sortedset.FileSortedSet; +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 java.io.IOException; -import java.util.Comparator; -import java.util.SortedMap; +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 @@ -75,8 +76,7 @@ public FileKeyValueSortedMap(SortedMapFileHandler handler, boolean 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); + super(((comparator == null) ? new DefaultKeyComparator() : comparator), new KeyValueFileHandler(handler), new Factory(), persisted); } /** @@ -186,18 +186,8 @@ public FileKeyValueSortedMap newInstance(FileSortedMap other, Key fro } @Override - public FileKeyValueSortedMap newInstance(SortedMapFileHandler handler, boolean persisted) { - return new FileKeyValueSortedMap(handler, persisted); - } - - @Override - public FileKeyValueSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { - return new FileKeyValueSortedMap(comparator, handler, persisted); - } - - @Override - public FileKeyValueSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStategy, - SortedMapFileHandler handler, boolean persisted) { + public FileKeyValueSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStategy, SortedMapFileHandler handler, + boolean persisted) { FileKeyValueSortedMap map = new FileKeyValueSortedMap(comparator, handler, persisted); map.setRewriteStrategy(rewriteStategy); return map; @@ -209,8 +199,7 @@ public FileKeyValueSortedMap newInstance(SortedMap map, SortedMapFile } @Override - public FileKeyValueSortedMap newInstance(SortedMap map, SortedMapFileHandler handler, boolean persist) - throws IOException { + 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 index 7d4a0de3d60..d3e6ec42221 100644 --- 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 @@ -1,9 +1,5 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.FileSortedSet; -import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; -import org.apache.log4j.Logger; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; @@ -15,6 +11,11 @@ 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. @@ -22,7 +23,7 @@ * The persisted file will contain the serialized entries, followed by the actual size. * */ -public class FileSerializableSortedMap extends FileSortedMap { +public class FileSerializableSortedMap extends FileSortedMap { private static Logger log = Logger.getLogger(FileSerializableSortedMap.class); /** @@ -248,7 +249,7 @@ public void close() throws IOException { /** * A factory for this map */ - public static class Factory implements FileSortedMapFactory { + public static class Factory implements FileSortedMapFactory { @Override public FileSerializableSortedMap newInstance(FileSortedMap other) { @@ -260,31 +261,21 @@ public FileSerializableSortedMap newInstance(FileSortedMap other, K fr return new FileSerializableSortedMap((FileSerializableSortedMap) other, from, to); } - @Override - public FileSerializableSortedMap newInstance(SortedMapFileHandler handler, boolean persisted) { - return new FileSerializableSortedMap(new SerializableFileHandler(handler), persisted); - } - - @Override - public FileSerializableSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted) { - return new FileSerializableSortedMap(comparator, new SerializableFileHandler(handler), persisted); - } - @Override public FileSerializableSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, - boolean persisted) { + boolean persisted) { FileSerializableSortedMap map = new FileSerializableSortedMap(comparator, new SerializableFileHandler(handler), persisted); map.setRewriteStrategy(rewriteStrategy); return map; } @Override - public FileSortedMap newInstance(SortedMap map, SortedMapFileHandler handler) { + 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 { + 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 index e2a107bed51..fb9697be1c1 100644 --- 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 @@ -20,11 +20,11 @@ import java.util.SortedMap; import java.util.TreeMap; -import datawave.query.util.sortedset.FileSortedSet; 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; @@ -149,6 +149,9 @@ public FileSortedMap(SortedMap map, TypedSortedMapFileHandler handler, File this.factory = factory; this.map = new TreeMap<>(map); this.persisted = false; + if (map instanceof RewritableSortedMap) { + setRewriteStrategy(((RewritableSortedMap) map).getRewriteStrategy()); + } } /** @@ -177,6 +180,9 @@ public FileSortedMap(SortedMap map, TypedSortedMapFileHandler handler, File persist(map, handler); persisted = true; } + if (map instanceof RewritableSortedMap) { + setRewriteStrategy(((RewritableSortedMap) map).getRewriteStrategy()); + } } @Override @@ -355,8 +361,8 @@ public void load() throws IOException, ClassNotFoundException { protected Map.Entry readObject(ObjectInputStream stream) { try { - K key = (K)stream.readObject(); - V value = (V)stream.readObject(); + K key = (K) stream.readObject(); + V value = (V) stream.readObject(); return new UnmodifiableMapEntry(key, value); } catch (Exception E) { return null; @@ -412,18 +418,20 @@ public boolean isEmpty() { public boolean containsKey(Object o) { if (persisted) { K t = (K) o; - try (SortedMapInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { + 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(); - while (next != null) { - if (equals(next.getKey(), t)) { - return true; - } - next = stream.readObject(); - } + return (next != null && equals(next.getKey(), t)); } catch (Exception e) { return false; } - return false; } else { return map.containsKey(o); } @@ -494,8 +502,8 @@ public V remove(Object o) { } @Override - public void putAll(Map m) { - for (Entry entry : m.entrySet()) { + public void putAll(Map m) { + for (Entry entry : m.entrySet()) { put(entry.getKey(), entry.getValue()); } } @@ -579,8 +587,7 @@ public Set keySet() { @Override public Iterator iterator() { - return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), - o -> ((Map.Entry)o).getKey()); + return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), o -> ((Map.Entry) o).getKey()); } @Override @@ -596,8 +603,7 @@ public Collection values() { @Override public Iterator iterator() { - return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), - o -> ((Map.Entry)o).getValue()); + return IteratorUtils.transformedIterator(FileSortedMap.this.iterator(), o -> ((Map.Entry) o).getValue()); } @Override @@ -608,7 +614,7 @@ public int size() { } @Override - public Set> entrySet() { + public Set> entrySet() { return new AbstractSet>() { @Override @@ -804,30 +810,6 @@ public interface FileSortedMapFactory { */ FileSortedMap newInstance(FileSortedMap other, K from, K to); - /** - * factory method - * - * @param handler - * the sorted map file handler - * @param persisted - * a persisted boolean flag - * @return a new instance - */ - FileSortedMap newInstance(SortedMapFileHandler handler, boolean persisted); - - /** - * Factory method - * - * @param comparator - * the key comparator - * @param handler - * the sorted map file handler - * @param persisted - * a persisted boolean flag - * @return a new instance - */ - FileSortedMap newInstance(Comparator comparator, SortedMapFileHandler handler, boolean persisted); - /** * Factory method * @@ -841,9 +823,7 @@ public interface FileSortedMapFactory { * a persisted boolean flag * @return a new instance */ - FileSortedMap newInstance(Comparator comparator, - RewriteStrategy rewriteStrategy, - SortedMapFileHandler handler, boolean persisted); + FileSortedMap newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedMapFileHandler handler, boolean persisted); /** * Create an unpersisted sorted map (still in memory) @@ -1071,9 +1051,12 @@ public interface RewriteStrategy { /** * Determine if the object should be rewritten * - * @param key The key - * @param original The original value - * @param update The updated value + * @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 index 92409e3e48a..ea408385b9a 100644 --- 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 @@ -7,7 +7,6 @@ import java.util.ArrayList; import java.util.List; -import datawave.query.util.sortedset.FileSortedSet; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FsStatus; @@ -16,6 +15,7 @@ import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.util.sortedset.FileSortedSet; public class HdfsBackedSortedMap extends BufferedFileBackedSortedMap { private static final Logger log = Logger.getLogger(HdfsBackedSortedMap.class); @@ -83,7 +83,8 @@ protected HdfsBackedSortedMap(Builder builder) throws IOException { for (FileStatus file : files) { if (!file.isDir() && file.getPath().getName().startsWith(FILENAME_PREFIX)) { count++; - addMap(mapFactory.newInstance(comparator, new SortedMapHdfsFileHandler(fs, file.getPath(), builder.persistOptions), true)); + addMap(mapFactory.newInstance(comparator, getRewriteStrategy(), + new SortedMapHdfsFileHandler(fs, file.getPath(), builder.persistOptions), true)); } } } 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 index e6180612f95..63e5d23e855 100644 --- 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 @@ -1,12 +1,5 @@ package datawave.query.util.sortedmap; -import com.google.common.collect.Iterators; -import datawave.webservice.query.exception.DatawaveErrorCode; -import datawave.webservice.query.exception.QueryException; -import org.apache.commons.collections4.keyvalue.UnmodifiableMapEntry; -import org.apache.commons.lang3.builder.EqualsBuilder; - -import java.io.IOException; import java.util.AbstractMap; import java.util.AbstractSet; import java.util.ArrayList; @@ -20,6 +13,14 @@ 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 @@ -30,8 +31,7 @@ public class MultiMapBackedSortedMap extends AbstractMap implements Re protected Comparator comparator = null; protected FileSortedMap.RewriteStrategy rewriteStrategy = null; - public MultiMapBackedSortedMap() { - } + public MultiMapBackedSortedMap() {} public MultiMapBackedSortedMap(List> maps) { for (SortedMap map : maps) { @@ -127,7 +127,7 @@ public V remove(Object o) { V testValue = map.remove(o); if (testValue != null) { if (value != null) { - if (rewriteStrategy == null || rewriteStrategy.rewrite((K)o, value, testValue)) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { value = testValue; } } else { @@ -153,11 +153,11 @@ public void clear() { } @Override - public Set> entrySet() { + public Set> entrySet() { return new AbstractSet<>() { @Override - public Iterator> iterator() { + public Iterator> iterator() { return MultiMapBackedSortedMap.this.iterator(); } @@ -240,12 +240,12 @@ public K lastKey() throws NoSuchElementException { } @Override - public FileSortedMap.RewriteStrategy getRewriteStrategy() { + public FileSortedMap.RewriteStrategy getRewriteStrategy() { return rewriteStrategy; } @Override - public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { + public void setRewriteStrategy(FileSortedMap.RewriteStrategy rewriteStrategy) { this.rewriteStrategy = rewriteStrategy; } @@ -256,7 +256,7 @@ public V get(Object o) { V testValue = map.get(o); if (testValue != null) { if (value != null) { - if (rewriteStrategy == null || rewriteStrategy.rewrite((K)o, value, testValue)) { + if (rewriteStrategy == null || rewriteStrategy.rewrite((K) o, value, testValue)) { value = testValue; } } else { @@ -362,9 +362,8 @@ private void populate() { if (it.hasNext()) { Entry val = it.next(); lastList.set(i, val.getKey()); - if ((rewriteStrategy == null) || - (!map.containsKey(val.getKey())) || - (rewriteStrategy.rewrite(val.getKey(), map.get(val.getKey()), val.getValue()))) { + if ((rewriteStrategy == null) || (!map.containsKey(val.getKey())) + || (rewriteStrategy.rewrite(val.getKey(), map.get(val.getKey()), val.getValue()))) { map.put(val.getKey(), val.getValue()); } } else { 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 index dc3f9ba2604..684e4b1e2e6 100644 --- 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 @@ -1,15 +1,16 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.FileSortedSet; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - 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. */ 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 index b9d9803cab3..2ecccb59975 100644 --- 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 @@ -1,17 +1,18 @@ 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; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.util.Map; public class KeyValueByteDocumentTransforms { @@ -29,21 +30,7 @@ public static Key byteToKey(byte[] bytes) { return new Key(bytes); } - public static Value keyToValue(Key key) { - if (key == null) { - return null; - } - return new Value(key.getRow().getBytes()); - } - - public static Key valueToKey(Value value) { - if (value == null) { - return null; - } - return new Key(value.get()); - } - - public static Value documentToValue(Document doc) throws IOException { + public static Value documentToValue(Document doc) { if (doc == null) { return null; } @@ -52,7 +39,7 @@ public static Value documentToValue(Document doc) throws IOException { return new Value(document); } - public static Document valueToDocument(Value value) throws IOException { + public static Document valueToDocument(Value value) { if (value == null) { return null; } @@ -61,14 +48,14 @@ public static Document valueToDocument(Value value) throws IOException { return document; } - public static Map.Entry keyValueToByteDocument(Map.Entry keyValue) throws IOException { + 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) throws IOException { + public static Map.Entry byteDocumentToKeyValue(Map.Entry byteKey) { if (byteKey == null) { return null; } 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 index f7d1f9916d3..0df7adb2b48 100644 --- 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 @@ -1,16 +1,17 @@ package datawave.query.util.sortedmap.rfile; -import datawave.query.attributes.Document; -import datawave.query.util.sortedmap.FileSortedMap; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; +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 static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.byteToKey; -import static datawave.query.util.sortedmap.rfile.KeyValueByteDocumentTransforms.keyValueToByteDocument; +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 { 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 index f46085fb3a4..774593be151 100644 --- 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 @@ -1,17 +1,18 @@ package datawave.query.util.sortedmap.rfile; -import datawave.query.attributes.Document; -import datawave.query.util.sortedmap.FileSortedMap; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; +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 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 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 { 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 index edcbc096728..e241d652fcd 100644 --- 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 @@ -1,14 +1,15 @@ package datawave.query.util.sortedmap.rfile; -import datawave.query.util.sortedmap.FileSortedMap; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - import java.io.IOException; import java.io.InputStream; import java.util.Map; -public class RFileKeyValueInputStream extends RFileKeyValueInputStreamBase implements FileSortedMap.SortedMapInputStream { +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); 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 index cf751155efc..653cb7de40f 100644 --- 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 @@ -1,5 +1,10 @@ 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; @@ -7,11 +12,6 @@ import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import java.io.IOException; -import java.io.InputStream; -import java.util.Iterator; -import java.util.Map; - public abstract class RFileKeyValueInputStreamBase { private final InputStream inputStream; private final long length; @@ -62,7 +62,8 @@ public int readSize() throws IOException { 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(); + 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()); } 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 index ecd438123d1..c42d045fab5 100644 --- 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 @@ -1,11 +1,12 @@ package datawave.query.util.sortedmap.rfile; -import datawave.query.util.sortedmap.FileSortedMap; +import java.io.IOException; +import java.io.OutputStream; + import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; -import java.io.IOException; -import java.io.OutputStream; +import datawave.query.util.sortedmap.FileSortedMap; public class RFileKeyValueOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedMap.SortedMapOutputStream { public RFileKeyValueOutputStream(OutputStream stream) throws IOException { 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 index ba73fd14426..ca4b8845beb 100644 --- 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 @@ -1,14 +1,14 @@ 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; -import java.io.IOException; -import java.io.OutputStream; - public abstract class RFileKeyValueOutputStreamBase { private final OutputStream outputStream; private RFileWriter writer; diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java deleted file mode 100644 index c17d5fbede1..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedByteArraySortedMapTest.java +++ /dev/null @@ -1,24 +0,0 @@ -package datawave.query.util.sortedmap; - -import datawave.query.util.sortedset.ByteArrayComparator; -import datawave.query.util.sortedset.FileSerializableSortedSet; - -import java.util.Comparator; - -public class BufferedFileBackedByteArraySortedMapTest extends BufferedFileBackedSortedMapTest { - - @Override - public byte[] createData(byte[] values) { - return values; - } - - @Override - public Comparator getComparator() { - return new ByteArrayComparator(); - } - - @Override - public FileSortedMap.FileSortedMapFactory getFactory() { - return new FileSerializableSortedSet.Factory(); - } -} 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 index 1f23f8b5dc3..0d67f7ebde2 100644 --- 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 @@ -1,15 +1,6 @@ package datawave.query.util.sortedmap; -import datawave.data.type.LcNoDiacriticsType; -import datawave.query.attributes.Document; -import datawave.query.composite.CompositeMetadata; -import datawave.query.predicate.EventDataQueryFieldFilter; -import datawave.query.predicate.KeyProjection; -import datawave.query.util.TypeMetadata; -import datawave.query.util.sortedset.ByteArrayComparator; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; +import static org.junit.Assert.assertEquals; import java.util.ArrayList; import java.util.Collections; @@ -17,34 +8,33 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.assertEquals; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; -public class BufferedFileBackedByteDocumentSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { +import datawave.data.type.LcNoDiacriticsType; +import datawave.query.attributes.Document; +import datawave.query.composite.CompositeMetadata; +import datawave.query.predicate.EventDataQueryFieldFilter; +import datawave.query.predicate.KeyProjection; +import datawave.query.util.TypeMetadata; +import datawave.query.util.sortedset.ByteArrayComparator; - private Comparator> keyComparator = new Comparator<>() { - private Comparator comparator = new ByteArrayComparator(); +public class BufferedFileBackedByteDocumentSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { - @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return comparator.compare(o1.getKey(), o2.getKey()); - } - }; + private Comparator keyComparator = new ByteArrayComparator(); - private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator = new RewritableSortedSetImpl.RewriteStrategy<>() { + private FileSortedMap.RewriteStrategy keyValueComparator = new FileSortedMap.RewriteStrategy<>() { @Override - public boolean rewrite(Map.Entry original, Map.Entry update) { - int comparison = keyComparator.compare(original, update); - if (comparison == 0) { - long ts1 = original.getValue().get(Document.DOCKEY_FIELD_NAME).getTimestamp(); - long ts2 = update.getValue().get(Document.DOCKEY_FIELD_NAME).getTimestamp(); - return (ts2 > ts1); - } - return comparison < 0; + 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 - public RewritableSortedSet.RewriteStrategy> getRewriteStrategy() { + public FileSortedMap.RewriteStrategy getRewriteStrategy() { return keyValueComparator; } @@ -67,17 +57,17 @@ public Document createValue(byte[] values) { @Override public void testFullEquality(Map.Entry expected, Map.Entry value) { - assertEquals(0, keyComparator.compare(expected, value)); + assertEquals(0, keyComparator.compare(expected.getKey(), value.getKey())); assertEquals(expected.getValue().get(Document.DOCKEY_FIELD_NAME), value.getValue().get(Document.DOCKEY_FIELD_NAME)); } @Override - public Comparator> getComparator() { + public Comparator getComparator() { return keyComparator; } @Override - public FileSortedMap.FileSortedMapFactory> getFactory() { + public FileSortedMap.FileSortedMapFactory getFactory() { return new FileByteDocumentSortedMap.Factory(); } } diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java deleted file mode 100644 index f70d1b7a96a..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/BufferedFileBackedKeySortedMapTest.java +++ /dev/null @@ -1,25 +0,0 @@ -package datawave.query.util.sortedmap; - -import datawave.query.util.sortedset.FileKeySortedSet; -import org.apache.accumulo.core.data.Key; - -import java.util.Comparator; - -public class BufferedFileBackedKeySortedMapTest extends BufferedFileBackedSortedMapTest { - - @Override - public Key createData(byte[] values) { - return new Key(values); - } - - @Override - public Comparator getComparator() { - return null; - } - - @Override - public FileSortedMap.FileSortedMapFactory getFactory() { - return new FileKeySortedSet.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 index c333f6ce616..d980d097eda 100644 --- 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 @@ -1,35 +1,31 @@ package datawave.query.util.sortedmap; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; +import static org.junit.Assert.assertEquals; import java.util.Comparator; import java.util.Map; -import static org.junit.Assert.assertEquals; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; public class BufferedFileBackedKeyValueSortedMapTest extends BufferedFileBackedRewritableSortedMapTest { - private Comparator> keyComparator = new Comparator<>() { + private Comparator keyComparator = new Comparator<>() { @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return o1.getKey().compareTo(o2.getKey()); + public int compare(Key o1, Key o2) { + return o1.compareTo(o2); } }; - private RewritableSortedSetImpl.RewriteStrategy> keyValueComparator = new RewritableSortedSetImpl.RewriteStrategy<>() { + private FileSortedMap.RewriteStrategy keyValueComparator = new FileSortedMap.RewriteStrategy<>() { @Override - public boolean rewrite(Map.Entry original, Map.Entry update) { - int comparison = original.getKey().compareTo(update.getKey()); - if (comparison == 0) { - comparison = original.getValue().compareTo(update.getValue()); - } - return comparison < 0; + public boolean rewrite(Key key, Value original, Value update) { + return original.compareTo(update) < 0; } }; @Override - public RewritableSortedSet.RewriteStrategy> getRewriteStrategy() { + public FileSortedMap.RewriteStrategy getRewriteStrategy() { return keyValueComparator; } @@ -50,12 +46,12 @@ public void testFullEquality(Map.Entry expected, Map.Entry } @Override - public Comparator> getComparator() { + public Comparator getComparator() { return keyComparator; } @Override - public FileSortedMap.FileSortedMapFactory> getFactory() { + 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 index 4f0721907f0..97096e17df6 100644 --- 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 @@ -1,17 +1,15 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.BufferedFileBackedSortedSet; -import datawave.query.util.sortedset.SortedSetTempFileHandler; -import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; -import org.junit.Test; - import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.Map; -public abstract class BufferedFileBackedRewritableSortedMapTest extends BufferedFileBackedSortedMapTest> { +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. @@ -45,7 +43,7 @@ public abstract class BufferedFileBackedRewritableSortedMapTest extends Buf * @return the rewrite strategy appropriate for key and value types */ @Override - public abstract RewritableSortedSet.RewriteStrategy> getRewriteStrategy(); + public abstract FileSortedMap.RewriteStrategy getRewriteStrategy(); @Override public Map.Entry createData(byte[] values) { @@ -75,12 +73,12 @@ public void testRewrite() throws Exception { } // create a set with the supplied rewrite strategy - set = new datawave.query.util.sortedset.BufferedFileBackedSortedSet.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) + map = new BufferedFileBackedSortedMap.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) .withMaxOpenFiles(7).withNumRetries(2) - .withHandlerFactories(Collections.singletonList(new BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { + .withHandlerFactories(Collections.singletonList(new BufferedFileBackedSortedMap.SortedMapFileHandlerFactory() { @Override - public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { - datawave.query.util.sortedset.SortedSetTempFileHandler fileHandler = new SortedSetTempFileHandler(); + public FileSortedMap.SortedMapFileHandler createHandler() throws IOException { + SortedMapTempFileHandler fileHandler = new SortedMapTempFileHandler(); tempFileHandlers.add(fileHandler); return fileHandler; } @@ -89,18 +87,18 @@ public FileSortedMap.SortedSetFileHandler createHandler() throws IOException { public boolean isValid() { return true; } - })).withSetFactory(getFactory()).build(); + })).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(set, data); - addDataRandomly(set, data2); + 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 = set.iterator(); it.hasNext();) { + 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]); 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 index 0d384fce9b6..9e7a78a763e 100644 --- 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 @@ -1,9 +1,9 @@ package datawave.query.util.sortedmap; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +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; @@ -21,10 +21,10 @@ import java.util.SortedMap; import java.util.TreeMap; -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 org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; public abstract class BufferedFileBackedSortedMapTest { @@ -80,21 +80,22 @@ public void mapUp() throws Exception { 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(); + 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... @@ -151,18 +152,19 @@ 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]); + map.remove(data[i].getKey()); expectedSize--; assertEquals(expectedSize, map.size()); } for (int i = 0; i < (data.length / 2); i++) { - map.remove(data[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()); } } @@ -171,14 +173,14 @@ public void testSize() { public void testIsEmpty() { assertFalse(map.isEmpty()); for (int i = (data.length / 2); i < data.length; i++) { - map.remove(data[i]); + map.remove(data[i].getKey()); assertFalse(map.isEmpty()); } for (int i = 1; i < (data.length / 2); i++) { - map.remove(data[i]); + map.remove(data[i].getKey()); assertFalse(map.isEmpty()); } - map.remove(data[0]); + map.remove(data[0].getKey()); assertTrue(map.isEmpty()); for (int i = 0; i < data.length; i++) { map.put(data[i].getKey(), data[i].getValue()); @@ -195,7 +197,7 @@ public void testClear() { @Test public void testContainsObject() { for (int i = (data.length / 2); i < data.length; i++) { - map.remove(data[i]); + map.remove(data[i].getKey()); } for (int i = 1; i < (data.length / 2); i++) { assertTrue(map.containsKey(data[i].getKey())); @@ -211,7 +213,7 @@ public void testRemove() { assertFalse(map.isPersisted()); for (int i = 0; i < data.length; i++) { - map.remove(data[i]); + map.remove(data[i].getKey()); assertEquals(--expectedSize, map.size()); } assertTrue(map.isEmpty()); @@ -225,7 +227,7 @@ public void testRemovePersisted() throws IOException { map.persist(); assertTrue(map.isPersisted()); for (int i = 0; i < data.length; i++) { - map.remove(data[i]); + map.remove(data[i].getKey()); assertEquals(--expectedSize, map.size()); assertTrue(map.isPersisted()); } @@ -235,7 +237,7 @@ public void testRemovePersisted() throws IOException { @Test public void testIterator() { int index = 0; - for (Iterator> it = map.iterator(); it.hasNext();) { + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { Map.Entry value = it.next(); Map.Entry expected = data[sortedOrder[index++]]; testEquality(expected, value); @@ -252,7 +254,7 @@ public void testIteratorRemove() { int failCount = 0; assertFalse(map.isPersisted()); // calling iterator() will force persistence - for (Iterator> it = map.iterator(); it.hasNext();) { + for (Iterator> it = map.entrySet().iterator(); it.hasNext();) { assertTrue(map.isPersisted()); Map.Entry value = it.next(); assertTrue(map.containsKey(value.getKey())); @@ -276,7 +278,7 @@ public void testComparator() { Map.Entry[] testData = Arrays.copyOf(data, data.length); Arrays.sort(testData, new Comparator>() { @Override - public int compare(Map.Entry o1, Map.Entry o2) { + public int compare(Map.Entry o1, Map.Entry o2) { return comparator.compare(o1.getKey(), o2.getKey()); } }); 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 index cf75fa146aa..938e61bf21e 100644 --- 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 @@ -1,8 +1,11 @@ package datawave.query.util.sortedmap; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +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; @@ -11,12 +14,9 @@ import java.util.SortedMap; import java.util.TreeMap; -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 org.junit.After; +import org.junit.Before; +import org.junit.Test; public class FileSortedMapTest { SortedMap data = null; @@ -42,11 +42,11 @@ public int compare(Integer o1, Integer o2) { Random r = new Random(123948710248L); // data.add(null); for (int i = 0; i < 20; i++) { - data.put(r.nextInt(), r.nextInt()+1); + 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); + extraData.put(r.nextInt(), r.nextInt() + 1); } // make sure we have no overlap data.keySet().removeAll(extraData.keySet()); @@ -246,7 +246,6 @@ public void testRetainAll() throws Exception { assertTrue(map.keySet().containsAll(data.keySet())); } - @Test public void testRemoveAll() throws Exception { SortedMap someData = new TreeMap<>(data); 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 index afcb703e8da..9aa0fbcf5fc 100644 --- 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 @@ -1,9 +1,11 @@ package datawave.query.util.sortedmap; -import datawave.query.iterator.ivarator.IvaratorCacheDir; -import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; -import datawave.query.util.sortedset.FileSortedSet; -import datawave.query.util.sortedset.HdfsBackedSortedSet; +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; @@ -14,11 +16,10 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; +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 { 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 index 9a194e2d6a6..b0aee4bda10 100644 --- 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 @@ -1,6 +1,9 @@ package datawave.query.util.sortedmap; -import org.junit.Test; +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; @@ -10,10 +13,7 @@ import java.util.SortedMap; import java.util.TreeMap; -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 org.junit.Test; public class MergeSortIteratorTest { @@ -23,22 +23,22 @@ public void testIteration() { 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); + 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); + 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); + set3.put(2, 1); + set3.put(5, 1); + set3.put(6, 1); + set3.put(30, 1); List expected = new ArrayList<>(); expected.add(1); @@ -93,22 +93,22 @@ public void testIterationSansHasNext() { 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); + 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); + 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); + set3.put(2, 1); + set3.put(5, 1); + set3.put(6, 1); + set3.put(30, 1); List expected = new ArrayList<>(); expected.add(1); @@ -173,24 +173,24 @@ public int compare(Integer o1, Integer o2) { 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); + 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); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java index fa8b78da640..1feb5d208ef 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/MultiSetBackedSortedMapTest.java @@ -1,10 +1,10 @@ package datawave.query.util.sortedmap; -import datawave.query.util.sortedset.ByteArrayComparator; -import datawave.query.util.sortedset.MultiSetBackedSortedSet; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import static org.junit.Assert.assertArrayEquals; +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.Arrays; @@ -14,11 +14,12 @@ import java.util.SortedSet; import java.util.TreeSet; -import static org.junit.Assert.assertArrayEquals; -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 org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.MultiSetBackedSortedSet; public class MultiSetBackedSortedMapTest { private byte[][] data = null; 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..7f0b8614a0b --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedmap/rfile/KeyValueByteDocumenTransformsTest.java @@ -0,0 +1,82 @@ +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 datawave.data.type.LcNoDiacriticsType; +import datawave.query.attributes.Document; +import datawave.query.composite.CompositeMetadata; +import datawave.query.predicate.EventDataQueryFieldFilter; +import datawave.query.predicate.KeyProjection; +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); + assertEquals(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(new KeyProjection())); + return doc; + } + +} From c2a9ad4f5650d38fecf6a98f3e413072b6f5fd67 Mon Sep 17 00:00:00 2001 From: Ivan Bella Date: Wed, 21 Aug 2024 22:20:15 +0000 Subject: [PATCH 5/5] Finished working through test cases --- .../datawave/query/attributes/Attribute.java | 28 ++++------------ .../query/attributes/Cardinality.java | 33 ++++++++++++++++++- .../query/attributes/TypeAttribute.java | 2 +- .../iterator/FieldIndexIntegrationTest.java | 6 ++-- 4 files changed, 43 insertions(+), 26 deletions(-) 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 9a69b417d71..3e143f4c02f 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 @@ -12,7 +12,6 @@ import org.apache.commons.lang.builder.HashCodeBuilder; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.WritableUtils; import org.apache.log4j.Logger; @@ -55,6 +54,11 @@ public ColumnVisibility getColumnVisibility() { return Constants.EMPTY_VISIBILITY; } + @Override + public boolean equals(Object o) { + return false; + } + public void setColumnVisibility(ColumnVisibility columnVisibility) { if (isMetadataSet()) { metadata = new Key(metadata.getRow(), metadata.getColumnFamily(), metadata.getColumnQualifier(), columnVisibility, metadata.getTimestamp()); @@ -215,25 +219,7 @@ protected int compareMetadata(Attribute other) { return -1; } } else if (this.isMetadataSet()) { - // we only need to compare those parts of the metadata that persist through serialization - // return this.metadata.compareTo(other.metadata); - byte[] cvBytes = this.getColumnVisibility().getExpression(); - if (null == cvBytes) { - cvBytes = Constants.EMPTY_BYTES; - } - - byte[] otherCVBytes = other.getColumnVisibility().getExpression(); - if (null == otherCVBytes) { - otherCVBytes = Constants.EMPTY_BYTES; - } - - int result = WritableComparator.compareBytes(cvBytes, 0, cvBytes.length, otherCVBytes, 0, otherCVBytes.length); - - if (result == 0) { - result = new Long(this.getTimestamp()).compareTo(other.getTimestamp()); - } - - return result; + return this.metadata.compareTo(other.metadata); } else { return 0; } @@ -251,7 +237,7 @@ public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(145, 11); hcb.append(this.isMetadataSet()); if (isMetadataSet()) { - hcb.append(this.getMetadata().getColumnVisibility()).append(this.getMetadata().getTimestamp()); + hcb.append(this.getMetadata()); } return hcb.toHashCode(); } diff --git a/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java b/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java index 36d05b2aee8..e2f91a2d2a0 100644 --- a/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java +++ b/warehouse/query-core/src/main/java/datawave/query/attributes/Cardinality.java @@ -111,10 +111,41 @@ public boolean equals(Object o) { return false; } + @Override + protected int compareMetadata(Attribute other) { + if (this.isMetadataSet() != other.isMetadataSet()) { + if (this.isMetadataSet()) { + return 1; + } else { + return -1; + } + } else if (this.isMetadataSet()) { + byte[] cvBytes = this.getColumnVisibility().getExpression(); + if (null == cvBytes) { + cvBytes = Constants.EMPTY_BYTES; + } + + byte[] otherCVBytes = other.getColumnVisibility().getExpression(); + if (null == otherCVBytes) { + otherCVBytes = Constants.EMPTY_BYTES; + } + + int result = WritableComparator.compareBytes(cvBytes, 0, cvBytes.length, otherCVBytes, 0, otherCVBytes.length); + + if (result == 0) { + result = new Long(this.getTimestamp()).compareTo(other.getTimestamp()); + } + + return result; + } else { + return 0; + } + } + @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(2099, 2129); - hcb.append(super.hashCode()).append(content); + hcb.appendSuper(content.hashCode()).append(this.getMetadata().getColumnVisibility()).append(this.getMetadata().getTimestamp()); return hcb.toHashCode(); } diff --git a/warehouse/query-core/src/main/java/datawave/query/attributes/TypeAttribute.java b/warehouse/query-core/src/main/java/datawave/query/attributes/TypeAttribute.java index 6dd0c7185bd..28dc1bc1af9 100644 --- a/warehouse/query-core/src/main/java/datawave/query/attributes/TypeAttribute.java +++ b/warehouse/query-core/src/main/java/datawave/query/attributes/TypeAttribute.java @@ -104,7 +104,7 @@ public boolean equals(Object o) { if (o instanceof TypeAttribute) { TypeAttribute other = (TypeAttribute) o; - return this.getType().equals(other.getType()) && (0 == this.compareMetadata(other)); + return this.getType().equals(other.getType()); } return false; diff --git a/warehouse/query-core/src/test/java/datawave/query/iterator/FieldIndexIntegrationTest.java b/warehouse/query-core/src/test/java/datawave/query/iterator/FieldIndexIntegrationTest.java index 25991dffbe7..a0aad76ce16 100644 --- a/warehouse/query-core/src/test/java/datawave/query/iterator/FieldIndexIntegrationTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/iterator/FieldIndexIntegrationTest.java @@ -79,9 +79,9 @@ public class FieldIndexIntegrationTest { put("FIELD_A", 2); put("FIELD_B", 2); put("FIELD_C", 2); - put("TLD_FIELD_A", 6); - put("TLD_FIELD_B", 26); - put("TLD_FIELD_C", 8); + put("TLD_FIELD_A", 4); // 3 record ids, 1 value + put("TLD_FIELD_B", 14); // 13 record ids, 1 value + put("TLD_FIELD_C", 5); // 4 record ids, 1 value put("FIELD_X", 2); put("FIELD_Y", 2); put("FIELD_Z", 2);