From 5f9c92390dcc999cd03590f6a9fe5983255b034b Mon Sep 17 00:00:00 2001 From: Ivan Bella Date: Tue, 9 Jul 2024 19:32:41 +0000 Subject: [PATCH] Task/revert most recent unique (#2464) * Revert "Updated to ensure the most recent unique functions parse correctly." This reverts commit d0aa2660ea6912ebbabbb9b23b0451c6a82f1d4a. * Revert "Fixes to the most-recent-unique functionality (#2392)" This reverts commit 5b4eb71deb27037eef4bae96f39bee6ea87a1761. * Revert "Fixed bug where unique results of null values failed to return (#2354)" This reverts commit 15f08a3d067ea2a8c8801d581eb18b2def4e5fe0. * Revert "Added most recent equivalent unique functions (#2339)" This reverts commit bb8d3531a910b66f1a573e7d51d1a8a4e2b744ea. * Revert "Most recent unique feature (#1991)" This reverts commit a4587afa38a7500588d5eb403e0379e2505d9fb5. * Updated centos dockerfiles to use yum repo mirror * Update quickstart docker image to Rocky Linux 8 (#2454) (#2455) * Update quickstart docker image to Rocky Linux 8 (#2454) * Revert "Updated centos dockerfiles to use yum repo mirror" This reverts commit d41e8861aecab19d5010292cbaa2a3916bc09475. * Added the hdfs sorted set builder mechanism back in --------- Co-authored-by: Whitney O'Meara Co-authored-by: Keith Ratcliffe --- contrib/datawave-quickstart/docker/Dockerfile | 2 +- .../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 | 20 +- .../query/dashboard/DashboardQueryLogic.java | 1 - .../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 ++++-------------- .../BufferedFileBackedSortedSet.java | 72 +-- .../sortedset/FileByteDocumentSortedSet.java | 235 ---------- .../util/sortedset/FileKeySortedSet.java | 42 +- .../util/sortedset/FileKeyValueSortedSet.java | 234 ---------- .../sortedset/FileSerializableSortedSet.java | 34 +- .../query/util/sortedset/FileSortedSet.java | 197 +++----- .../util/sortedset/HdfsBackedSortedSet.java | 69 ++- .../util/sortedset/MergeSortIterator.java | 143 ++++++ .../sortedset/MultiSetBackedSortedSet.java | 208 +-------- ...reamBase.java => RFileKeyInputStream.java} | 29 +- ...eamBase.java => RFileKeyOutputStream.java} | 23 +- .../util/sortedset/RewritableSortedSet.java | 35 -- .../sortedset/RewritableSortedSetImpl.java | 324 ------------- .../rfile/KeyValueByteDocumentTransforms.java | 64 --- .../rfile/RFileByteDocumentInputStream.java | 34 -- .../rfile/RFileByteDocumentOutputStream.java | 21 - .../sortedset/rfile/RFileKeyInputStream.java | 28 -- .../sortedset/rfile/RFileKeyOutputStream.java | 19 - .../rfile/RFileKeyValueInputStream.java | 30 -- .../rfile/RFileKeyValueOutputStream.java | 21 - .../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 | 21 - ...edFileBackedByteDocumentSortedSetTest.java | 89 ---- .../BufferedFileBackedKeySortedSetTest.java | 249 +++++++++- ...fferedFileBackedKeyValueSortedSetTest.java | 62 --- ...eredFileBackedRewritableSortedSetTest.java | 108 ----- .../BufferedFileBackedSortedSetTest.java | 157 +++---- .../util/sortedset/FileSortedSetTest.java | 4 +- .../sortedset/HdfsBackedSortedSetTest.java | 68 +-- .../util/sortedset/MergeSortIteratorTest.java | 8 +- .../MultiSetBackedSortedSetTest.java | 4 +- .../sortedset/RewritableSortedSetTest.java | 284 ----------- .../datawave/query/QueryLogicFactory.xml | 14 +- .../results/cached/CachedRunningQuery.java | 5 +- .../datawave/query/QueryLogicFactory.xml | 12 +- .../map/BulkResultsFileOutputMapper.java | 2 - .../map/BulkResultsTableOutputMapper.java | 1 - .../query/logic/BaseQueryLogic.java | 1 - .../logic/composite/CompositeQueryLogic.java | 1 - .../logic/filtered/FilteredQueryLogic.java | 1 - 81 files changed, 1202 insertions(+), 3792 deletions(-) delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUnique.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByDay.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByHour.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMinute.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMonth.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueBySecond.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByTenthOfHour.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByYear.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileByteDocumentSortedSet.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeyValueSortedSet.java create mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/MergeSortIterator.java rename warehouse/query-core/src/main/java/datawave/query/util/sortedset/{rfile/RFileKeyValueInputStreamBase.java => RFileKeyInputStream.java} (66%) rename warehouse/query-core/src/main/java/datawave/query/util/sortedset/{rfile/RFileKeyValueOutputStreamBase.java => RFileKeyOutputStream.java} (64%) delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSet.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSetImpl.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/KeyValueByteDocumentTransforms.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileByteDocumentInputStream.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileByteDocumentOutputStream.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyInputStream.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyOutputStream.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueInputStream.java delete mode 100644 warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueOutputStream.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteArraySortedSetTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteDocumentSortedSetTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeyValueSortedSetTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedRewritableSortedSetTest.java delete mode 100644 warehouse/query-core/src/test/java/datawave/query/util/sortedset/RewritableSortedSetTest.java diff --git a/contrib/datawave-quickstart/docker/Dockerfile b/contrib/datawave-quickstart/docker/Dockerfile index 15bfe844470..24a59eccce5 100644 --- a/contrib/datawave-quickstart/docker/Dockerfile +++ b/contrib/datawave-quickstart/docker/Dockerfile @@ -1,4 +1,4 @@ -FROM centos:centos7 +FROM rockylinux/rockylinux:8 ARG DATAWAVE_COMMIT_ID ARG DATAWAVE_BRANCH_NAME diff --git a/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml b/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml index e6233a4fac1..c11b1c258c3 100644 --- a/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml +++ b/microservices/configcheck/src/test/resources/input/microservice/QueryLogicFactory.xml @@ -233,23 +233,7 @@ - - - - - - - - - - - - - - - - diff --git a/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml b/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml index 8c40461dcdf..306770277de 100644 --- a/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml +++ b/microservices/configcheck/src/test/resources/input/webservice/QueryLogicFactory.xml @@ -38,23 +38,15 @@ - + + - - - - - - - - - diff --git a/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml b/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml index bb07d23456d..a28dad51d0f 100644 --- a/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml +++ b/microservices/configcheck/src/test/resources/rendered/microservice/QueryLogicFactory.xml @@ -233,23 +233,7 @@ - - - - - - - - - - - - - - - - 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 adfd1785c5e..c846c87cc2f 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 @@ -1297,11 +1297,8 @@ protected void setupRowBasedHdfsBackedSet(String row) throws IOException { this.createdRowDir = false; } - // 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.set = new HdfsBackedSortedSet<>(null, hdfsBackedSetBufferSize, ivaratorCacheDirs, row, maxOpenFiles, numRetries, persistOptions, + new FileKeySortedSet.Factory()); 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 60180bfcb38..ef80c579bb2 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,10 +24,8 @@ public class FileSystemCache { public FileSystemCache(String hdfsSiteConfigs) throws MalformedURLException { conf = new Configuration(); - if (hdfsSiteConfigs != null) { - for (String url : org.apache.commons.lang.StringUtils.split(hdfsSiteConfigs, ',')) { - conf.addResource(new URL(url)); - } + 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 9299c563089..01ae45c84e0 100644 --- a/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java +++ b/warehouse/query-core/src/main/java/datawave/query/QueryParameters.java @@ -179,7 +179,6 @@ 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 0c861f54e6b..0fab5c560ce 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,17 +4,14 @@ 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; @@ -28,11 +25,9 @@ * 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, Cloneable { +public class UniqueFields implements Serializable { - private final TreeMultimap fieldMap = TreeMultimap.create(); - private boolean mostRecent = false; - private static String MOST_RECENT_UNIQUE = "_MOST_RECENT_"; + private Multimap fieldMap; /** * Returns a new {@link UniqueFields} parsed from this string. The provided string is expected to have the format returned by @@ -77,12 +72,8 @@ public static UniqueFields from(String string) { if (nextComma == -1 && nextStartBracket == -1) { String field = string.substring(currentIndex); if (!field.isEmpty()) { - 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); - } + // 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)) { @@ -96,12 +87,8 @@ public static UniqueFields from(String string) { // Add the field with the ALL granularity. String field = string.substring(currentIndex, nextComma); if (!field.isEmpty()) { - 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); - } + // 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 { @@ -113,18 +100,14 @@ public static UniqueFields from(String string) { String field = string.substring(currentIndex, nextStartBracket); int nextEndBracket = string.indexOf(Constants.BRACKET_END, currentIndex); if (!field.isEmpty()) { - if (field.equals(MOST_RECENT_UNIQUE)) { - uniqueFields.setMostRecent(true); + 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 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)); - } + String[] granularities = StringUtils.split(granularityList, Constants.COMMA); + for (String granularity : granularities) { + uniqueFields.put(field, parseGranularity(granularity)); } } } @@ -145,19 +128,24 @@ private static UniqueGranularity parseGranularity(String granularity) { } /** - * Return a clone of this class + * Return a copy of the given {@link UniqueFields}. * + * @param other + * the other instance to copy * @return the copy */ - @Override - public UniqueFields clone() { - UniqueFields newFields = new UniqueFields(); - newFields.fieldMap.putAll(this.fieldMap); - newFields.mostRecent = this.mostRecent; - return newFields; + public static UniqueFields copyOf(UniqueFields other) { + if (other == null) { + return null; + } + UniqueFields uniqueFields = new UniqueFields(); + uniqueFields.fieldMap = TreeMultimap.create(other.fieldMap); + return uniqueFields; } - public UniqueFields() {} + public UniqueFields() { + fieldMap = TreeMultimap.create(); + } /** * Create a new {@link UniqueFields} with the provided map as the underlying field map. @@ -166,24 +154,7 @@ public UniqueFields() {} * the field map to use */ public UniqueFields(SortedSetMultimap 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); + this.fieldMap = fieldMap; } /** @@ -194,9 +165,8 @@ public UniqueFields set(Multimap fields) { * @param uniqueGranularity * the granularity */ - public UniqueFields put(String field, UniqueGranularity uniqueGranularity) { - fieldMap.put(JexlASTHelper.deconstructIdentifier(field).toUpperCase(), uniqueGranularity); - return this; + public void put(String field, UniqueGranularity uniqueGranularity) { + fieldMap.put(field, uniqueGranularity); } /** @@ -205,13 +175,10 @@ public UniqueFields put(String field, UniqueGranularity uniqueGranularity) { * @param fieldMap * the field map to add entries from */ - public UniqueFields putAll(Multimap fieldMap) { + public void putAll(Multimap fieldMap) { if (fieldMap != null) { - for (String field : fieldMap.keySet()) { - this.fieldMap.putAll(JexlASTHelper.deconstructIdentifier(field).toUpperCase(), fieldMap.get(field)); - } + this.fieldMap.putAll(fieldMap); } - return this; } /** @@ -228,12 +195,12 @@ public void replace(String field, String replacement) { } /** - * 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 within this {@link UniqueFields}. Modifications to this set will not modify the fields in this {@link UniqueFields}. * * @return a copy of the fields */ - public NavigableSet getFields() { - return fieldMap.keySet(); + public Set getFields() { + return Sets.newHashSet(fieldMap.keySet()); } /** @@ -241,10 +208,26 @@ public NavigableSet getFields() { * * @return the field map */ - public TreeMultimap getFieldMap() { + public Multimap 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. * @@ -255,11 +238,12 @@ 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)); } } - set(newFieldMap); + this.fieldMap = newFieldMap; } /** @@ -321,10 +305,6 @@ 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. @@ -346,15 +326,6 @@ 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) { @@ -364,12 +335,12 @@ public boolean equals(Object o) { return false; } UniqueFields that = (UniqueFields) o; - return Objects.equals(fieldMap, that.fieldMap) && mostRecent == that.mostRecent; + return Objects.equals(fieldMap, that.fieldMap); } @Override public int hashCode() { - return Objects.hash(fieldMap, mostRecent); + return Objects.hash(fieldMap); } } 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 5f7cd133e2b..d51f64cd65c 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).map(String::toUpperCase).collect(Collectors.toSet()); + return set.stream().map(JexlASTHelper::deconstructIdentifier).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 0ca55e72532..5a48c1ec213 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 @@ -351,8 +351,6 @@ 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; @@ -675,9 +673,7 @@ public ShardQueryConfiguration(ShardQueryConfiguration other) { this.setCompositeFilterFunctionsEnabled(other.isCompositeFilterFunctionsEnabled()); this.setGroupFieldsBatchSize(other.getGroupFieldsBatchSize()); this.setAccrueStats(other.getAccrueStats()); - this.setUniqueFields(other.getUniqueFields()); - log.info("Checkpointing with " + getUniqueFields()); - this.setUniqueCacheBufferSize(other.getUniqueCacheBufferSize()); + this.setUniqueFields(UniqueFields.copyOf(other.getUniqueFields())); this.setCacheModel(other.getCacheModel()); this.setTrackSizes(other.isTrackSizes()); this.setContentFieldNames(null == other.getContentFieldNames() ? null : Lists.newArrayList(other.getContentFieldNames())); @@ -1426,14 +1422,6 @@ 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; } @@ -1794,7 +1782,11 @@ public UniqueFields getUniqueFields() { } public void setUniqueFields(UniqueFields uniqueFields) { - this.uniqueFields = uniqueFields.clone(); + this.uniqueFields = uniqueFields; + // If unique fields are present, make sure they are deconstructed by this point. + if (uniqueFields != null) { + uniqueFields.deconstructIdentifierFields(); + } } public boolean isHitList() { diff --git a/warehouse/query-core/src/main/java/datawave/query/dashboard/DashboardQueryLogic.java b/warehouse/query-core/src/main/java/datawave/query/dashboard/DashboardQueryLogic.java index 5f49ae25fed..9112c8733d9 100644 --- a/warehouse/query-core/src/main/java/datawave/query/dashboard/DashboardQueryLogic.java +++ b/warehouse/query-core/src/main/java/datawave/query/dashboard/DashboardQueryLogic.java @@ -12,7 +12,6 @@ import datawave.webservice.query.cache.ResultsPage; import datawave.webservice.query.dashboard.DashboardFields; import datawave.webservice.query.dashboard.DashboardSummary; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.logic.QueryLogicTransformer; import datawave.webservice.query.logic.ResponseEnricher; import datawave.webservice.query.result.event.EventBase; 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 fff762e7e98..80e60781a69 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,7 +459,8 @@ else if (documentRange != null && (!this.isContainsIndexOnlyTerms() && this.getT // now apply the unique iterator if requested UniqueTransform uniquify = getUniqueTransform(); if (uniquify != null) { - pipelineDocuments = uniquify.getIterator(pipelineDocuments); + // pipelineDocuments = uniquify; + pipelineDocuments = Iterators.filter(pipelineDocuments, uniquify.getUniquePredicate()); } // apply the grouping iterator if requested and if the batch size is greater than zero @@ -1540,23 +1541,11 @@ public Comparator getValueComparator(Tuple3 hitsOnlySet = new HashSet<>(); @@ -510,7 +506,6 @@ 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; @@ -963,14 +958,6 @@ 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; } @@ -1116,7 +1103,7 @@ public UniqueFields getUniqueFields() { } public void setUniqueFields(UniqueFields uniqueFields) { - this.uniqueFields = uniqueFields.clone(); + this.uniqueFields = uniqueFields; } public Set getHitsOnlySet() { @@ -1595,12 +1582,6 @@ 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 c00f60784f0..e8af4ae0931 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')}
  • * */ @@ -61,18 +59,9 @@ 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.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); + 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) { @@ -199,16 +188,7 @@ 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)) { - 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) { + switch (String.valueOf(nsIdentifier.getName())) { case QueryFunctions.OPTIONS_FUNCTION: { List optionsList = new ArrayList<>(); this.visit(node, optionsList); @@ -254,7 +234,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(function)); + updateUniqueFields(node, uniqueFields, optionsMap, UniqueFunction.findByName(nsIdentifier.getName())); 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 deleted file mode 100644 index ed94abec1ae..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUnique.java +++ /dev/null @@ -1,72 +0,0 @@ -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 deleted file mode 100644 index 845bd863ae3..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByDay.java +++ /dev/null @@ -1,22 +0,0 @@ -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 deleted file mode 100644 index c831dac8aec..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByHour.java +++ /dev/null @@ -1,23 +0,0 @@ -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 deleted file mode 100644 index f8b04bc4050..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMinute.java +++ /dev/null @@ -1,23 +0,0 @@ -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 deleted file mode 100644 index 3c611479dd5..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByMonth.java +++ /dev/null @@ -1,23 +0,0 @@ -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 deleted file mode 100644 index 8ff9eedbb45..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueBySecond.java +++ /dev/null @@ -1,23 +0,0 @@ -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 deleted file mode 100644 index 81948a62cb3..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByTenthOfHour.java +++ /dev/null @@ -1,23 +0,0 @@ -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 deleted file mode 100644 index 24d8c8c4471..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/language/functions/jexl/MostRecentUniqueByYear.java +++ /dev/null @@ -1,23 +0,0 @@ -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 c6fd2c0f040..a354b79c5a0 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,9 +1,13 @@ 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 c6f298f0c0b..78afb7e50b1 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,12 +1,16 @@ 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[HOUR])}. + * 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])}. */ 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 b05d880f4fd..90bc82720fc 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,9 +1,13 @@ 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 427567fe16f..ea1db4bdfd4 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,11 +532,6 @@ 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 2a4438cfb72..1294a2f9124 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,14 +62,8 @@ 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 a0c2ca94028..a58126c94cf 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 @@ -83,7 +83,6 @@ 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.common.connection.AccumuloConnectionFactory; import datawave.webservice.common.logging.ThreadConfigurableLogger; @@ -261,9 +260,7 @@ 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) + '(' @@ -590,11 +587,7 @@ protected String getStopwatchHeader(ShardQueryConfiguration config) { @Override public QueryLogicTransformer getTransformer(Query settings) { if (this.transformerInstance != null) { - try { - addConfigBasedTransformers(); - } catch (QueryException e) { - throw new DatawaveFatalQueryException("Unable to configure transformers", e); - } + addConfigBasedTransformers(); return this.transformerInstance; } @@ -615,11 +608,7 @@ public QueryLogicTransformer getTransformer(Query settings) { transformer.setPrimaryToSecondaryFieldMap(primaryToSecondaryFieldMap); transformer.setQm(queryModel); this.transformerInstance = transformer; - try { - addConfigBasedTransformers(); - } catch (QueryException e) { - throw new DatawaveFatalQueryException("Unable to configure transformers", e); - } + addConfigBasedTransformers(); return this.transformerInstance; } @@ -636,7 +625,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() throws QueryException { + private void addConfigBasedTransformers() { if (getConfig() != null) { ((DocumentTransformer) this.transformerInstance).setProjectFields(getConfig().getProjectFields()); ((DocumentTransformer) this.transformerInstance).setDisallowlistedFields(getConfig().getDisallowlistedFields()); @@ -644,29 +633,10 @@ private void addConfigBasedTransformers() throws QueryException { if (getConfig().getUniqueFields() != null && !getConfig().getUniqueFields().isEmpty()) { DocumentTransform alreadyExists = ((DocumentTransformer) this.transformerInstance).containsTransform(UniqueTransform.class); if (alreadyExists != null) { - ((UniqueTransform) alreadyExists).updateConfig(getConfig().getUniqueFields()); + ((UniqueTransform) alreadyExists).updateConfig(getConfig().getUniqueFields(), getQueryModel()); } else { - 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); - } + ((DocumentTransformer) this.transformerInstance) + .addTransform(new UniqueTransform(this, getConfig().getUniqueFields(), this.getQueryExecutionForPageTimeout())); } } @@ -913,18 +883,11 @@ 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()) { - // preserve the most recent flag - uniqueFields.setMostRecent(config.getUniqueFields().isMostRecent()); + this.setUniqueFields(uniqueFields); 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)) { @@ -1922,14 +1885,6 @@ 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 51b4c7484d6..09d5643f9a3 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 && !settings.getOptions().containsKey(QueryOptions.MOST_RECENT_UNIQUE) && !IvaratorRequiredVisitor.isIvaratorRequired(script)) { + if (script != null && !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 98380e10aac..5fe12df407f 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 = System.currentTimeMillis(); + protected long queryExecutionForPageStartTime; @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 c9cfca151db..8327188210f 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,6 +23,7 @@ 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 @@ -90,10 +91,6 @@ 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 d68ae473dbc..85e35fe9f29 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,25 +4,21 @@ 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.commons.collections4.keyvalue.UnmodifiableMapEntry; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; +import org.apache.accumulo.core.data.Value; 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; @@ -30,37 +26,26 @@ import com.google.common.hash.Funnel; import com.google.common.hash.PrimitiveSink; -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.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; +import datawave.query.tables.ShardQueryLogic; +import datawave.webservice.query.logic.BaseQueryLogic; /** * 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 unless mostRecentUnique is specified in which case the most recent instance of an event will be returned. This transform is thread - * safe. + * fields 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 = new UniqueFields(); - private HdfsBackedSortedSet> set; - private HdfsBackedSortedSet> returnSet; - private Iterator> setIterator; + private UniqueFields uniqueFields; + private Multimap modelMapping; /** * 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 @@ -81,6 +66,7 @@ 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()); @@ -88,32 +74,58 @@ public UniqueTransform(UniqueFields uniqueFields, long queryExecutionForPageTime } /** - * Update the configuration of this transform. If the configuration is actually changing, then the bloom filter will be reset as well. + * 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 * + * @param logic + * The query logic from whih to pull the query model * @param uniqueFields - * The new set of unique fields. + * 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. */ - 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()); + 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()); } } } /** - * Add phrase excerpts to the documents from the given iterator. + * Get a predicate that will apply this transform. * - * @param in - * the iterator source - * @return an iterator that will supply the enriched documents + * @return A unique transform predicate */ - public Iterator> getIterator(final Iterator> in) { - return new UniqueTransformIterator(in); + public Predicate> getUniquePredicate() { + return input -> UniqueTransform.this.apply(input) != null; } /** @@ -131,63 +143,25 @@ public Entry apply(@Nullable Entry keyDocumentEntry) return keyDocumentEntry; } - if (keyDocumentEntry.getValue().isIntermediateResult()) { - return keyDocumentEntry; - } - try { - 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())) { + if (isDuplicate(keyDocumentEntry.getValue())) { + keyDocumentEntry = null; + } else { 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(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(); - } - } + long elapsedExecutionTimeForCurrentPage = System.currentTimeMillis() - this.queryExecutionForPageStartTime; + if (elapsedExecutionTimeForCurrentPage > this.queryExecutionForPageTimeout) { + Document intermediateResult = new Document(); + intermediateResult.setIntermediateResult(true); + return Maps.immutableEntry(new Key(), intermediateResult); } - 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(); + return null; } /** @@ -238,90 +212,71 @@ byte[] getBytes(Document document) throws IOException { * if we failed to generate the byte array */ private void outputSortedFieldValues(Document document, DataOutputStream output) throws IOException { - Multimap values = HashMultimap.create(); + int count = 0; + String lastField = ""; + List values = new ArrayList<>(); 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); } } - // 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); - } + dumpValues(count, lastField, values, 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 void dumpValues(String field, Collection values, DataOutputStream output) throws IOException { - String separator = "f-" + field + ":"; + private int dumpValues(int count, String field, List values, DataOutputStream output) throws IOException { if (!values.isEmpty()) { - List valueList = new ArrayList<>(values); - // always output values in sorted order. - Collections.sort(valueList); - for (String value : valueList) { + Collections.sort(values); + String separator = "f-" + field + '/' + (count++) + ":"; + for (String value : values) { output.writeUTF(separator); output.writeUTF(value); separator = ","; } - } 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); + values.clear(); } + return count; } - /** - * 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) { + // Return the set of values for the provided attribute. + private void addValues(final String field, Attribute attribute, List values) { if (attribute instanceof Attributes) { // @formatter:off ((Attributes) attribute).getAttributes().stream() .forEach(a -> addValues(field, a, values)); // @formatter:on } else { - values.put(field, uniqueFields.transformValue(field, String.valueOf(attribute.getData()))); + values.add(uniqueFields.transformValue(field, String.valueOf(attribute.getData()))); } } - /** - * 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 - */ + // Return the query-specified field that the provided document matches, if one exists, or otherwise return null. 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. - * - * @param field - * The field - * @return The field with grouping stripped - */ + // Return the provided field with any grouping context removed. private String getFieldWithoutGrouping(String field) { int index = field.indexOf('.'); if (index < 0) { @@ -331,22 +286,14 @@ private String getFieldWithoutGrouping(String field) { } } - /** - * 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 - */ + // Return whether or not the provided document field is considered a case-insensitive match for the provided field, applying reverse model mappings if + // configured. private boolean isMatchingField(String baseField, String field) { - return baseField.equalsIgnoreCase(field); + baseField = baseField.toUpperCase(); + field = field.toUpperCase(); + return field.equals(baseField) || (modelMapping != null && modelMapping.get(field).contains(baseField)); } - /** - * A funnel to use for the bloom filter - */ public static class ByteFunnel implements Funnel, Serializable { private static final long serialVersionUID = -2126172579955897986L; @@ -356,217 +303,4 @@ 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/sortedset/BufferedFileBackedSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/BufferedFileBackedSortedSet.java index 70669db7124..0a318a44d42 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/BufferedFileBackedSortedSet.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/BufferedFileBackedSortedSet.java @@ -22,24 +22,23 @@ * @param * type of the set */ -public class BufferedFileBackedSortedSet implements RewritableSortedSet { +public class BufferedFileBackedSortedSet implements SortedSet { private static final Logger log = Logger.getLogger(BufferedFileBackedSortedSet.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 MultiSetBackedSortedSet set = new MultiSetBackedSortedSet<>(); - protected int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; + protected int maxOpenFiles = 10000; protected FileSortedSet buffer = null; protected FileSortedSet.FileSortedSetFactory setFactory = null; - protected final Comparator comparator; - protected final RewriteStrategy rewriteStrategy; + protected Comparator comparator = null; protected boolean sizeModified = false; protected int size = 0; - protected int numRetries = DEFAULT_NUM_RETRIES; + protected int numRetries; protected List handlerFactories; - protected int bufferPersistThreshold = DEFAULT_BUFFER_PERSIST_THRESHOLD; + protected int bufferPersistThreshold; /** * A factory for SortedSetFileHandlers @@ -57,7 +56,6 @@ public static class Builder,E> { private int maxOpenFiles = DEFAULT_MAX_OPEN_FILES; private FileSortedSet.FileSortedSetFactory setFactory = new FileSerializableSortedSet.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; @@ -86,12 +84,6 @@ public B withComparator(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(); @@ -118,7 +110,6 @@ public static BufferedFileBackedSortedSet.Builder builder() { protected BufferedFileBackedSortedSet(BufferedFileBackedSortedSet other) { this.comparator = other.comparator; - this.rewriteStrategy = other.rewriteStrategy; this.handlerFactories = new ArrayList<>(other.handlerFactories); this.setFactory = other.setFactory; this.bufferPersistThreshold = other.bufferPersistThreshold; @@ -135,9 +126,40 @@ protected BufferedFileBackedSortedSet(BufferedFileBackedSortedSet other) { this.size = other.size; } + public BufferedFileBackedSortedSet(List handlerFactories) { + this(handlerFactories, new FileSerializableSortedSet.Factory()); + } + + public BufferedFileBackedSortedSet(List handlerFactories, FileSortedSet.FileSortedSetFactory setFactory) { + this(null, DEFAULT_BUFFER_PERSIST_THRESHOLD, DEFAULT_MAX_OPEN_FILES, DEFAULT_NUM_RETRIES, handlerFactories); + } + + public BufferedFileBackedSortedSet(Comparator comparator, List handlerFactories) { + this(comparator, handlerFactories, new FileSerializableSortedSet.Factory()); + } + + public BufferedFileBackedSortedSet(Comparator comparator, List handlerFactories, + FileSortedSet.FileSortedSetFactory setFactory) { + this(comparator, DEFAULT_BUFFER_PERSIST_THRESHOLD, DEFAULT_MAX_OPEN_FILES, DEFAULT_NUM_RETRIES, handlerFactories); + } + + public BufferedFileBackedSortedSet(Comparator comparator, int bufferPersistThreshold, int maxOpenFiles, int numRetries, + List handlerFactories) { + this(comparator, bufferPersistThreshold, maxOpenFiles, numRetries, handlerFactories, new FileSerializableSortedSet.Factory()); + } + + public BufferedFileBackedSortedSet(Comparator comparator, int bufferPersistThreshold, int maxOpenFiles, int numRetries, + List handlerFactories, FileSortedSet.FileSortedSetFactory setFactory) { + this.comparator = comparator; + this.handlerFactories = handlerFactories; + this.setFactory = setFactory; + this.bufferPersistThreshold = bufferPersistThreshold; + this.numRetries = numRetries; + this.maxOpenFiles = maxOpenFiles; + } + protected BufferedFileBackedSortedSet(Builder builder) { this.comparator = builder.comparator; - this.rewriteStrategy = builder.rewriteStrategy; this.handlerFactories = new ArrayList<>(builder.handlerFactories); this.setFactory = builder.setFactory; this.bufferPersistThreshold = builder.bufferPersistThreshold; @@ -402,7 +424,7 @@ public T[] toArray(T[] a) { public boolean add(E e) { if (buffer == null) { try { - buffer = setFactory.newInstance(comparator, rewriteStrategy, null, false); + buffer = setFactory.newInstance(comparator, null, false); } catch (Exception ex) { throw new IllegalStateException("Unable to create an underlying FileSortedSet", ex); } @@ -427,7 +449,7 @@ public boolean add(E e) { public boolean addAll(Collection c) { if (buffer == null) { try { - buffer = setFactory.newInstance(comparator, rewriteStrategy, null, false); + buffer = setFactory.newInstance(comparator, null, false); } catch (Exception ex) { throw new IllegalStateException("Unable to create an underlying FileSortedSet", ex); } @@ -587,27 +609,17 @@ public Comparator comparator() { } @Override - public RewriteStrategy getRewriteStrategy() { - return rewriteStrategy; - } - - @Override - public E get(E e) { - return null; - } - - @Override - public RewritableSortedSet subSet(E fromElement, E toElement) { + public SortedSet subSet(E fromElement, E toElement) { return set.subSet(fromElement, toElement); } @Override - public RewritableSortedSet headSet(E toElement) { + public SortedSet headSet(E toElement) { return set.headSet(toElement); } @Override - public RewritableSortedSet tailSet(E fromElement) { + public SortedSet tailSet(E fromElement) { return set.tailSet(fromElement); } diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileByteDocumentSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileByteDocumentSortedSet.java deleted file mode 100644 index 838a6538d63..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileByteDocumentSortedSet.java +++ /dev/null @@ -1,235 +0,0 @@ -package datawave.query.util.sortedset; - -import java.io.IOException; -import java.util.Comparator; -import java.util.Map; - -import org.apache.log4j.Logger; - -import datawave.query.attributes.Document; -import datawave.query.util.sortedset.rfile.RFileByteDocumentInputStream; -import datawave.query.util.sortedset.rfile.RFileByteDocumentOutputStream; - -/** - * 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 FileByteDocumentSortedSet extends FileSortedSet> { - private static Logger log = Logger.getLogger(FileByteDocumentSortedSet.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 FileByteDocumentSortedSet(FileByteDocumentSortedSet 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 FileByteDocumentSortedSet(FileByteDocumentSortedSet 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 FileByteDocumentSortedSet(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 FileByteDocumentSortedSet(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultByteDocumentComparator() : comparator), new ByteDocumentFileHandler(handler), - new FileByteDocumentSortedSet.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 FileByteDocumentSortedSet(Comparator> comparator, RewriteStrategy> rewriteStrategy, - SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultByteDocumentComparator() : comparator), rewriteStrategy, new ByteDocumentFileHandler(handler), - new FileByteDocumentSortedSet.Factory(), persisted); - } - - /** - * Create an unpersisted sorted set (still in memory) - * - * @param set - * the sorted set - * @param handler - * the sorted set file handler - */ - public FileByteDocumentSortedSet(RewritableSortedSet> set, SortedSetFileHandler handler) { - super(set, new ByteDocumentFileHandler(handler), new FileByteDocumentSortedSet.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 FileByteDocumentSortedSet(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) throws IOException { - super(set, new ByteDocumentFileHandler(handler), new FileByteDocumentSortedSet.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 FileByteDocumentSortedSet clone() { - return (FileByteDocumentSortedSet) 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 FileByteDocumentSortedSet newInstance(FileSortedSet> other) { - return new FileByteDocumentSortedSet((FileByteDocumentSortedSet) other); - } - - @Override - public FileByteDocumentSortedSet newInstance(FileSortedSet> other, Map.Entry from, - Map.Entry to) { - return new FileByteDocumentSortedSet((FileByteDocumentSortedSet) other, from, to); - } - - @Override - public FileByteDocumentSortedSet newInstance(SortedSetFileHandler handler, boolean persisted) { - return new FileByteDocumentSortedSet(handler, persisted); - } - - @Override - public FileByteDocumentSortedSet newInstance(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { - return new FileByteDocumentSortedSet(comparator, handler, persisted); - } - - @Override - public FileSortedSet> newInstance(Comparator> comparator, - RewriteStrategy> rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { - return new FileByteDocumentSortedSet(comparator, rewriteStrategy, handler, persisted); - } - - @Override - public FileByteDocumentSortedSet newInstance(RewritableSortedSet> set, SortedSetFileHandler handler) { - return new FileByteDocumentSortedSet(set, handler); - } - - @Override - public FileByteDocumentSortedSet newInstance(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) - throws IOException { - return new FileByteDocumentSortedSet(set, handler, persist); - } - } -} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeySortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeySortedSet.java index 4f02d74a551..db4e43ec780 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeySortedSet.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeySortedSet.java @@ -2,13 +2,11 @@ import java.io.IOException; import java.util.Comparator; +import java.util.SortedSet; import org.apache.accumulo.core.data.Key; import org.apache.log4j.Logger; -import datawave.query.util.sortedset.rfile.RFileKeyInputStream; -import datawave.query.util.sortedset.rfile.RFileKeyOutputStream; - /** * 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. @@ -65,24 +63,11 @@ public FileKeySortedSet(SortedSetFileHandler handler, boolean persisted) { * @param persisted * a persisted boolean flag */ - public FileKeySortedSet(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { - super(comparator, new KeyFileHandler(handler), new FileKeySortedSet.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 FileKeySortedSet(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { - super(comparator, rewriteStrategy, new KeyFileHandler(handler), new FileKeySortedSet.Factory(), persisted); + public FileKeySortedSet(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + this(handler, persisted); + if (comparator != null) { + throw new UnsupportedOperationException("Cannot supply a comparator for a FileKeySortedSet. Only a Key comparator will work"); + } } /** @@ -93,7 +78,7 @@ public FileKeySortedSet(Comparator comparator, RewriteStrategy rewrite * @param handler * the sorted set file handler */ - public FileKeySortedSet(RewritableSortedSet set, SortedSetFileHandler handler) { + public FileKeySortedSet(SortedSet set, SortedSetFileHandler handler) { super(set, new KeyFileHandler(handler), new FileKeySortedSet.Factory()); } @@ -110,7 +95,7 @@ public FileKeySortedSet(RewritableSortedSet set, SortedSetFileHandler handl * @throws IOException * for issues with read/write */ - public FileKeySortedSet(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { + public FileKeySortedSet(SortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { super(set, new KeyFileHandler(handler), new FileKeySortedSet.Factory(), persist); } @@ -197,22 +182,17 @@ public FileKeySortedSet newInstance(SortedSetFileHandler handler, boolean persis } @Override - public FileKeySortedSet newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + public FileKeySortedSet newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { return new FileKeySortedSet(comparator, handler, persisted); } @Override - public FileKeySortedSet newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted) { - return new FileKeySortedSet(comparator, rewriteStrategy, handler, persisted); - } - - @Override - public FileKeySortedSet newInstance(RewritableSortedSet set, SortedSetFileHandler handler) { + public FileKeySortedSet newInstance(SortedSet set, SortedSetFileHandler handler) { return new FileKeySortedSet(set, handler); } @Override - public FileKeySortedSet newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { + public FileKeySortedSet newInstance(SortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { return new FileKeySortedSet(set, handler, persist); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeyValueSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeyValueSortedSet.java deleted file mode 100644 index 4fd1f6c7e91..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileKeyValueSortedSet.java +++ /dev/null @@ -1,234 +0,0 @@ -package datawave.query.util.sortedset; - -import java.io.IOException; -import java.util.Comparator; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.log4j.Logger; - -import datawave.query.util.sortedset.rfile.RFileKeyValueInputStream; -import datawave.query.util.sortedset.rfile.RFileKeyValueOutputStream; - -/** - * 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 FileKeyValueSortedSet extends FileSortedSet> { - private static Logger log = Logger.getLogger(FileKeyValueSortedSet.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 FileKeyValueSortedSet(FileKeyValueSortedSet 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 FileKeyValueSortedSet(FileKeyValueSortedSet 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 FileKeyValueSortedSet(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 FileKeyValueSortedSet(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultKeyValueComparator() : comparator), new KeyValueFileHandler(handler), new FileKeyValueSortedSet.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 FileKeyValueSortedSet(Comparator> comparator, RewriteStrategy> rewriteStrategy, - SortedSetFileHandler handler, boolean persisted) { - super((comparator == null ? new DefaultKeyValueComparator() : comparator), rewriteStrategy, new KeyValueFileHandler(handler), - new FileKeyValueSortedSet.Factory(), persisted); - } - - /** - * Create an unpersisted sorted set (still in memory) - * - * @param set - * the sorted set - * @param handler - * the sorted set file handler - */ - public FileKeyValueSortedSet(RewritableSortedSet> set, SortedSetFileHandler handler) { - super(set, new KeyValueFileHandler(handler), new FileKeyValueSortedSet.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 FileKeyValueSortedSet(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) throws IOException { - super(set, new KeyValueFileHandler(handler), new FileKeyValueSortedSet.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 FileKeyValueSortedSet clone() { - return (FileKeyValueSortedSet) 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 FileKeyValueSortedSet newInstance(FileSortedSet> other) { - return new FileKeyValueSortedSet((FileKeyValueSortedSet) other); - } - - @Override - public FileKeyValueSortedSet newInstance(FileSortedSet> other, Map.Entry from, Map.Entry to) { - return new FileKeyValueSortedSet((FileKeyValueSortedSet) other, from, to); - } - - @Override - public FileKeyValueSortedSet newInstance(SortedSetFileHandler handler, boolean persisted) { - return new FileKeyValueSortedSet(handler, persisted); - } - - @Override - public FileKeyValueSortedSet newInstance(Comparator> comparator, SortedSetFileHandler handler, boolean persisted) { - return new FileKeyValueSortedSet(comparator, handler, persisted); - } - - @Override - public FileKeyValueSortedSet newInstance(Comparator> comparator, RewriteStrategy> rewriteStategy, - SortedSetFileHandler handler, boolean persisted) { - return new FileKeyValueSortedSet(comparator, rewriteStategy, handler, persisted); - } - - @Override - public FileKeyValueSortedSet newInstance(RewritableSortedSet> set, SortedSetFileHandler handler) { - return new FileKeyValueSortedSet(set, handler); - } - - @Override - public FileKeyValueSortedSet newInstance(RewritableSortedSet> set, SortedSetFileHandler handler, boolean persist) - throws IOException { - return new FileKeyValueSortedSet(set, handler, persist); - } - } -} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSerializableSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSerializableSortedSet.java index 5e5fce939f2..7f70dce9d6f 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSerializableSortedSet.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSerializableSortedSet.java @@ -68,26 +68,10 @@ public FileSerializableSortedSet(TypedSortedSetFileHandler handler, boolean pers * @param persisted * persisted boolean flag */ - public FileSerializableSortedSet(Comparator comparator, TypedSortedSetFileHandler handler, boolean persisted) { + public FileSerializableSortedSet(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 FileSerializableSortedSet(Comparator comparator, RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, boolean persisted) { - super(comparator, rewriteStrategy, handler, new Factory(), persisted); - } - /** * Create an unpersisted sorted set (still in memory) * @@ -96,7 +80,7 @@ public FileSerializableSortedSet(Comparator comparator, RewriteStrategy re * @param handler * a file handler */ - public FileSerializableSortedSet(RewritableSortedSet set, TypedSortedSetFileHandler handler) { + public FileSerializableSortedSet(SortedSet set, TypedSortedSetFileHandler handler) { super(set, handler, new Factory()); } @@ -113,7 +97,7 @@ public FileSerializableSortedSet(RewritableSortedSet set, TypedSortedSetFileH * @throws IOException * for issues with read/write */ - public FileSerializableSortedSet(RewritableSortedSet set, TypedSortedSetFileHandler handler, boolean persist) throws IOException { + public FileSerializableSortedSet(SortedSet set, TypedSortedSetFileHandler handler, boolean persist) throws IOException { super(set, handler, new Factory(), persist); } @@ -276,23 +260,17 @@ public FileSerializableSortedSet newInstance(SortedSetFileHandler handler, bo } @Override - public FileSerializableSortedSet newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { + public FileSerializableSortedSet newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted) { return new FileSerializableSortedSet(comparator, new SerializableFileHandler(handler), persisted); } @Override - public FileSerializableSortedSet newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, - boolean persisted) { - return new FileSerializableSortedSet(comparator, rewriteStrategy, new SerializableFileHandler(handler), persisted); - } - - @Override - public FileSerializableSortedSet newInstance(RewritableSortedSet set, SortedSetFileHandler handler) { + public FileSerializableSortedSet newInstance(SortedSet set, SortedSetFileHandler handler) { return new FileSerializableSortedSet(set, new SerializableFileHandler(handler)); } @Override - public FileSerializableSortedSet newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { + public FileSerializableSortedSet newInstance(SortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException { return new FileSerializableSortedSet(set, new SerializableFileHandler(handler), persist); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSortedSet.java index 96c07b8f532..780c04c9974 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSortedSet.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/FileSortedSet.java @@ -15,10 +15,8 @@ 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; @@ -27,22 +25,31 @@ /** * 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. + * 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. * * @param * type of set */ -public abstract class FileSortedSet extends RewritableSortedSetImpl implements SortedSet, Cloneable { - private static Logger log = Logger.getLogger(FileSortedSet.class); - protected boolean persisted = false; +public abstract class FileSortedSet implements SortedSet, Cloneable { + private static final Logger log = Logger.getLogger(FileSortedSet.class); + protected boolean persisted; protected E[] range; + protected SortedSet set; + // The file handler that handles the underlying io public TypedSortedSetFileHandler handler; // The sort set factory public FileSortedSetFactory factory; + /** + * A class that represents a null object within the set + */ + public static class NullObject implements Serializable { + private static final long serialVersionUID = -5528112099317370355L; + + } + /** * Create a file sorted set from another one * @@ -50,9 +57,9 @@ public abstract class FileSortedSet extends RewritableSortedSetImpl implem * the other sorted set */ public FileSortedSet(FileSortedSet other) { - super(other); this.handler = other.handler; this.factory = other.factory; + this.set = new TreeSet<>(other.set); this.persisted = other.persisted; this.range = other.range; } @@ -73,11 +80,11 @@ public FileSortedSet(FileSortedSet other, E from, E to) { if (persisted) { this.range = (E[]) new Object[] {getStart(from), getEnd(to)}; } else if (to == null) { - this.set = this.set.tailMap(from, true); + this.set = this.set.tailSet(from); } else if (from == null) { - this.set = this.set.headMap(to, false); + this.set = this.set.headSet(to); } else { - this.set = this.set.subMap(from, true, to, false); + this.set = this.set.subSet(from, to); } } } @@ -95,27 +102,7 @@ public FileSortedSet(FileSortedSet other, E from, E to) { public FileSortedSet(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 FileSortedSet(RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { - super(rewriteStrategy); - this.handler = handler; - this.factory = factory; - this.set = new TreeMap<>(); + this.set = new TreeSet<>(); this.persisted = persisted; } @@ -124,8 +111,6 @@ public FileSortedSet(RewriteStrategy rewriteStrategy, TypedSortedSetFileHandl * * @param comparator * the key comparator - * @param rewriteStrategy - * the item rewrite strategy * @param handler * the sorted set file handler * @param persisted @@ -133,32 +118,10 @@ public FileSortedSet(RewriteStrategy rewriteStrategy, TypedSortedSetFileHandl * @param factory * the sorted set factory */ - public FileSortedSet(Comparator comparator, RewriteStrategy rewriteStrategy, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, - boolean persisted) { - super(comparator, rewriteStrategy); + public FileSortedSet(Comparator comparator, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { 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 FileSortedSet(Comparator comparator, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persisted) { - super(comparator); - this.handler = handler; - this.factory = factory; - this.set = new TreeMap<>(comparator); + this.set = new TreeSet<>(comparator); this.persisted = persisted; } @@ -172,11 +135,10 @@ public FileSortedSet(Comparator comparator, TypedSortedSetFileHandler handler * @param factory * the sorted set factory */ - public FileSortedSet(RewritableSortedSet set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory) { + public FileSortedSet(SortedSet 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.set = new TreeSet<>(set); this.persisted = false; } @@ -195,9 +157,14 @@ public FileSortedSet(RewritableSortedSet set, TypedSortedSetFileHandler handl * @throws IOException * for issues with read/write */ - public FileSortedSet(RewritableSortedSet set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persist) throws IOException { - this(set, handler, factory); - if (persist) { + public FileSortedSet(SortedSet set, TypedSortedSetFileHandler handler, FileSortedSetFactory factory, boolean persist) throws IOException { + this.handler = handler; + this.factory = factory; + if (!persist) { + this.set = new TreeSet<>(set); + this.persisted = false; + } else { + this.set = new TreeSet<>(set.comparator()); persist(set, handler); persisted = true; } @@ -238,7 +205,7 @@ public void persist() throws IOException { */ public void persist(TypedSortedSetFileHandler handler) throws IOException { if (!persisted) { - persist(this.set.navigableKeySet(), handler); + persist(this.set, handler); this.set.clear(); persisted = true; } @@ -359,7 +326,7 @@ public void load() throws IOException, ClassNotFoundException { try (SortedSetInputStream stream = getBoundedFileHandler().getInputStream(getStart(), getEnd())) { E obj = stream.readObject(); while (obj != null) { - super.add(obj); + set.add(obj); obj = stream.readObject(); } } @@ -403,7 +370,7 @@ public int size() { throw new IllegalStateException("Unable to get size from file", e); } } else { - return super.size(); + return set.size(); } } @@ -430,7 +397,7 @@ public boolean contains(Object o) { } return false; } else { - return super.contains(o); + return set.contains(o); } } @@ -439,7 +406,7 @@ public Iterator iterator() { if (persisted) { return new FileIterator(); } else { - return super.iterator(); + return set.iterator(); } } @@ -464,7 +431,7 @@ public Object[] toArray() { throw new IllegalStateException("Unable to read file into a complete set", e); } } else { - return super.toArray(); + return set.toArray(); } } @@ -501,7 +468,7 @@ public T[] toArray(T[] a) { throw new IllegalStateException("Unable to read file into a complete set", e); } } else { - return super.toArray(a); + return set.toArray(a); } } @@ -509,8 +476,9 @@ public T[] toArray(T[] a) { public boolean add(E e) { if (persisted) { throw new IllegalStateException("Cannot add an element to a persisted FileSortedSet. Please call load() first."); + } else { + return set.add(e); } - return super.add(e); } @Override @@ -518,7 +486,7 @@ 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 set.remove(o); } } @@ -534,9 +502,6 @@ public boolean containsAll(Collection c) { 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) { @@ -553,7 +518,7 @@ public boolean containsAll(Collection c) { } return false; } else { - return super.containsAll(c); + return set.containsAll(c); } } @@ -562,7 +527,7 @@ 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); + return set.addAll(c); } } @@ -571,7 +536,7 @@ 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); + return set.retainAll(c); } } @@ -580,7 +545,7 @@ 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); + return set.removeAll(c); } } @@ -589,7 +554,7 @@ 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); + return set.removeIf(filter); } } @@ -599,81 +564,72 @@ public void clear() { handler.deleteFile(); persisted = false; } else { - super.clear(); + set.clear(); } } @Override public Comparator comparator() { - return super.comparator(); + return set.comparator(); } @Override - public RewritableSortedSet subSet(E fromElement, E toElement) { + public SortedSet subSet(E fromElement, E toElement) { return factory.newInstance(this, fromElement, toElement); } @Override - public RewritableSortedSet headSet(E toElement) { + public SortedSet headSet(E toElement) { return factory.newInstance(this, null, toElement); } @Override - public RewritableSortedSet tailSet(E fromElement) { + public SortedSet tailSet(E fromElement) { return factory.newInstance(this, fromElement, null); } @Override public E first() { + 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; - } + first = stream.readObject(); + return first; } catch (Exception e) { throw new IllegalStateException(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR, e)); } - } else { - return super.first(); + } else if (!set.isEmpty()) { + first = set.first(); + return first; } + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR)); } @Override public E last() { + 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(); } + return last; } 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 (!set.isEmpty()) { + last = set.last(); + return last; } + throw (NoSuchElementException) new NoSuchElementException().initCause(new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR)); } @Override public String toString() { - if (persisted) { - return handler.toString(); - } else { - return super.toString(); - } + return persisted ? handler.toString() : set.toString(); } /** @@ -865,22 +821,7 @@ public interface FileSortedSetFactory { * a persisted boolean flag * @return a new instance */ - FileSortedSet 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 - */ - FileSortedSet newInstance(Comparator comparator, RewriteStrategy rewriteStrategy, SortedSetFileHandler handler, boolean persisted); + FileSortedSet newInstance(Comparator comparator, SortedSetFileHandler handler, boolean persisted); /** * Create an unpersisted sorted set (still in memory) @@ -891,7 +832,7 @@ public interface FileSortedSetFactory { * the sorted set file handler * @return a new instance */ - FileSortedSet newInstance(RewritableSortedSet set, SortedSetFileHandler handler); + FileSortedSet newInstance(SortedSet set, SortedSetFileHandler handler); /** * factory method @@ -906,7 +847,7 @@ public interface FileSortedSetFactory { * @throws IOException * for problems with read/write */ - FileSortedSet newInstance(RewritableSortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException; + FileSortedSet newInstance(SortedSet set, SortedSetFileHandler handler, boolean persist) throws IOException; } /** diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/HdfsBackedSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/HdfsBackedSortedSet.java index ab0769a0e45..a4209b318f3 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/HdfsBackedSortedSet.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/HdfsBackedSortedSet.java @@ -5,7 +5,9 @@ import java.io.OutputStream; import java.net.MalformedURLException; import java.util.ArrayList; +import java.util.Comparator; import java.util.List; +import java.util.SortedSet; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -16,7 +18,7 @@ import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; -public class HdfsBackedSortedSet extends BufferedFileBackedSortedSet { +public class HdfsBackedSortedSet extends BufferedFileBackedSortedSet implements SortedSet { private static final Logger log = Logger.getLogger(HdfsBackedSortedSet.class); private static final String FILENAME_PREFIX = "SortedSetFile."; @@ -60,13 +62,61 @@ public static HdfsBackedSortedSet.Builder builder() { return new HdfsBackedSortedSet.Builder<>(); } + protected HdfsBackedSortedSet(Builder builder) throws IOException { + super(builder); + this.handlerFactories = createFileHandlerFactories(builder.ivaratorCacheDirs, builder.uniqueSubPath, builder.persistOptions); + setup(builder.persistOptions); + } + protected HdfsBackedSortedSet(HdfsBackedSortedSet other) { super(other); } - protected HdfsBackedSortedSet(Builder builder) throws IOException { - super(builder); - this.handlerFactories = createFileHandlerFactories(builder.ivaratorCacheDirs, builder.uniqueSubPath, builder.persistOptions); + public HdfsBackedSortedSet(List ivaratorCacheDirs, String uniqueSubPath, int maxOpenFiles, int numRetries, + FileSortedSet.PersistOptions persistOptions) throws IOException { + this(ivaratorCacheDirs, uniqueSubPath, maxOpenFiles, numRetries, persistOptions, new FileSerializableSortedSet.Factory()); + } + + public HdfsBackedSortedSet(List ivaratorCacheDirs, String uniqueSubPath, int maxOpenFiles, int numRetries, + FileSortedSet.PersistOptions persistOptions, FileSortedSet.FileSortedSetFactory setFactory) throws IOException { + this(null, ivaratorCacheDirs, uniqueSubPath, maxOpenFiles, numRetries, persistOptions, setFactory); + } + + public HdfsBackedSortedSet(Comparator comparator, List ivaratorCacheDirs, String uniqueSubPath, int maxOpenFiles, + int numRetries, FileSortedSet.PersistOptions persistOptions) throws IOException { + this(comparator, ivaratorCacheDirs, uniqueSubPath, maxOpenFiles, numRetries, persistOptions, new FileSerializableSortedSet.Factory()); + } + + public HdfsBackedSortedSet(Comparator comparator, List ivaratorCacheDirs, String uniqueSubPath, int maxOpenFiles, + int numRetries, FileSortedSet.PersistOptions persistOptions, FileSortedSet.FileSortedSetFactory setFactory) throws IOException { + this(comparator, 10000, ivaratorCacheDirs, uniqueSubPath, maxOpenFiles, numRetries, persistOptions, setFactory); + } + + 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)); + } + return fileHandlerFactories; + } + + public HdfsBackedSortedSet(Comparator comparator, int bufferPersistThreshold, List ivaratorCacheDirs, String uniqueSubPath, + int maxOpenFiles, int numRetries, FileSortedSet.PersistOptions persistOptions) throws IOException { + this(comparator, bufferPersistThreshold, ivaratorCacheDirs, uniqueSubPath, maxOpenFiles, numRetries, persistOptions, + new FileSerializableSortedSet.Factory()); + } + + public HdfsBackedSortedSet(Comparator comparator, int bufferPersistThreshold, List ivaratorCacheDirs, String uniqueSubPath, + int maxOpenFiles, int numRetries, FileSortedSet.PersistOptions persistOptions, FileSortedSet.FileSortedSetFactory setFactory) + throws IOException { + super(comparator, bufferPersistThreshold, maxOpenFiles, numRetries, createFileHandlerFactories(ivaratorCacheDirs, uniqueSubPath, persistOptions), + setFactory); + this.handlerFactories = createFileHandlerFactories(ivaratorCacheDirs, uniqueSubPath, persistOptions); + setup(persistOptions); + } + + private void setup(FileSortedSet.PersistOptions persistOptions) throws IOException { // 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 @@ -82,7 +132,7 @@ protected HdfsBackedSortedSet(Builder builder) throws IOException { 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)); + addSet(setFactory.newInstance(comparator, new SortedSetHdfsFileHandler(fs, file.getPath(), persistOptions), true)); } } } @@ -93,15 +143,6 @@ protected HdfsBackedSortedSet(Builder builder) throws IOException { } } - 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)); - } - return fileHandlerFactories; - } - @Override public void clear() { // This will be a new ArrayList<>() containing the same FileSortedSets diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/MergeSortIterator.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/MergeSortIterator.java new file mode 100644 index 00000000000..f82fe7dd458 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/MergeSortIterator.java @@ -0,0 +1,143 @@ +package datawave.query.util.sortedset; + +import java.util.ArrayList; +import java.util.Collection; +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 datawave.webservice.query.exception.DatawaveErrorCode; +import datawave.webservice.query.exception.QueryException; + +/** + * This is an iterator that will return a sorted set of items (no dups) from an underlying set of sorted sets. This will support null contained in the + * underlying sets iff the underlying sets use a comparator that can handle null values. + * + * + * + * @param + * type for the iterator + */ +public class MergeSortIterator implements Iterator { + + private List> iterators = new ArrayList<>(); + private List lastList = new ArrayList<>(); + private boolean[] finished = null; + private SortedSet set = null; + private boolean populated = false; + private T next = null; + private List> nextIterators = new ArrayList<>(); + + public MergeSortIterator(Collection> sets) { + Comparator comparator = null; + for (SortedSet set : sets) { + comparator = set.comparator(); + Iterator it = set.iterator(); + iterators.add(it); + nextIterators.add(it); + lastList.add(null); + } + this.set = new TreeSet<>(comparator); + 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 T 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(T o1, T 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()) { + T 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/sortedset/MultiSetBackedSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/MultiSetBackedSortedSet.java index 6dbc009b817..880827b3f12 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/MultiSetBackedSortedSet.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/MultiSetBackedSortedSet.java @@ -7,66 +7,27 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.SortedSet; - -import org.apache.commons.lang3.builder.EqualsBuilder; - -import datawave.webservice.query.exception.DatawaveErrorCode; -import datawave.webservice.query.exception.QueryException; +import java.util.TreeSet; /* * 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. + * sorted sets contain the same type of underlying value, and they use the same comparator. + * This will support null contained in the underlying sets iff the underlying sets use a comparator + * that can handle null values. */ -public class MultiSetBackedSortedSet extends AbstractSet implements RewritableSortedSet { +public class MultiSetBackedSortedSet extends AbstractSet implements SortedSet { protected List> sets = new ArrayList<>(); - protected Comparator comparator = null; - protected RewriteStrategy rewriteStrategy = null; - - public MultiSetBackedSortedSet() {} - - public MultiSetBackedSortedSet(List> sets) { - for (SortedSet set : sets) { - addSet(set); - } - } + /** + * Add a set to the underlying sets + * + * @param set + * the 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 * @@ -115,7 +76,7 @@ public boolean contains(Object o) { @Override public Iterator iterator() { - return new MergeSortIterator(); + return new MergeSortIterator<>(sets); } @Override @@ -150,11 +111,14 @@ public void clear() { @Override public Comparator comparator() { - return comparator; + if (!sets.isEmpty()) { + return sets.iterator().next().comparator(); + } + return null; } @Override - public RewritableSortedSet subSet(E fromElement, E toElement) { + public SortedSet subSet(E fromElement, E toElement) { MultiSetBackedSortedSet subSet = new MultiSetBackedSortedSet<>(); for (SortedSet set : sets) { subSet.addSet(set.subSet(fromElement, toElement)); @@ -163,7 +127,7 @@ public RewritableSortedSet subSet(E fromElement, E toElement) { } @Override - public RewritableSortedSet headSet(E toElement) { + public SortedSet headSet(E toElement) { MultiSetBackedSortedSet subSet = new MultiSetBackedSortedSet<>(); for (SortedSet set : sets) { subSet.addSet(set.headSet(toElement)); @@ -172,7 +136,7 @@ public RewritableSortedSet headSet(E toElement) { } @Override - public RewritableSortedSet tailSet(E fromElement) { + public SortedSet tailSet(E fromElement) { MultiSetBackedSortedSet subSet = new MultiSetBackedSortedSet<>(); for (SortedSet set : sets) { subSet.addSet(set.tailSet(fromElement)); @@ -185,7 +149,7 @@ public E first() throws NoSuchElementException { if (sets == null || sets.isEmpty()) { throw new NoSuchElementException("No elements in input sets"); } - SortedSet firstSet = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); + SortedSet firstSet = new TreeSet<>(comparator()); for (SortedSet set : sets) { if (set != null && !set.isEmpty()) { E s = set.first(); @@ -203,7 +167,7 @@ public E last() throws NoSuchElementException { if (sets == null || sets.isEmpty()) { throw new NoSuchElementException("No elements in input sets"); } - SortedSet lastSet = new RewritableSortedSetImpl<>(comparator, rewriteStrategy); + SortedSet lastSet = new TreeSet<>(comparator()); for (SortedSet set : sets) { if (set != null && !set.isEmpty()) { E s = set.last(); @@ -215,134 +179,4 @@ public E last() throws NoSuchElementException { } 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/sortedset/rfile/RFileKeyValueInputStreamBase.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RFileKeyInputStream.java similarity index 66% rename from warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueInputStreamBase.java rename to warehouse/query-core/src/main/java/datawave/query/util/sortedset/RFileKeyInputStream.java index f6373d8ba71..6708b64898a 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueInputStreamBase.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RFileKeyInputStream.java @@ -1,4 +1,4 @@ -package datawave.query.util.sortedset.rfile; +package datawave.query.util.sortedset; import java.io.IOException; import java.io.InputStream; @@ -12,9 +12,7 @@ import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import datawave.query.util.sortedset.FileSortedSet; - -public abstract class RFileKeyValueInputStreamBase implements FileSortedSet.SortedSetInputStream { +public class RFileKeyInputStream implements FileSortedSet.SortedSetInputStream { private final InputStream inputStream; private final long length; private Key start; @@ -24,24 +22,18 @@ public abstract class RFileKeyValueInputStreamBase implements FileSortedSet.S private int size = -1; private static final Range ALL = new Range(); - public RFileKeyValueInputStreamBase(InputStream inputStream, long length) throws IOException { + public RFileKeyInputStream(InputStream inputStream, long length) throws IOException { this.inputStream = inputStream; this.length = length; } - public RFileKeyValueInputStreamBase(InputStream inputStream, long length, Key start, Key end) throws IOException { + public RFileKeyInputStream(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() { + private Iterator> iterator() { if (iterator == null) { Range r = ALL; if (start != null || end != null) { @@ -53,11 +45,12 @@ private Iterator> keyValueIterator() { 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()); + @Override + public Key readObject() throws IOException { + if (iterator().hasNext()) { + Key next = iterator().next().getKey(); + if (RFileKeyOutputStream.SizeKeyUtil.isSizeKey(next)) { + size = RFileKeyOutputStream.SizeKeyUtil.getSize(next); next = null; } return next; diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueOutputStreamBase.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RFileKeyOutputStream.java similarity index 64% rename from warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueOutputStreamBase.java rename to warehouse/query-core/src/main/java/datawave/query/util/sortedset/RFileKeyOutputStream.java index 842338b24ae..74ddf48f763 100644 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueOutputStreamBase.java +++ b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RFileKeyOutputStream.java @@ -1,37 +1,34 @@ -package datawave.query.util.sortedset.rfile; +package datawave.query.util.sortedset; import java.io.IOException; import java.io.OutputStream; -import java.util.Map; 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.commons.collections.keyvalue.UnmodifiableMapEntry; import org.apache.hadoop.io.Text; -public class RFileKeyValueOutputStreamBase { +public class RFileKeyOutputStream implements FileSortedSet.SortedSetOutputStream { 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 { + public RFileKeyOutputStream(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()); + @Override + public void writeObject(Key o) throws IOException { + writer.append(o, EMPTY_VALUE); } + @Override public void writeSize(int i) throws IOException { - writeKeyValue(SizeKeyUtil.getKey(i), EMPTY_VALUE); + writeObject(SizeKeyUtil.getKey(i)); } + @Override public void close() throws IOException { writer.close(); writer = null; diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSet.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSet.java deleted file mode 100644 index 68b09eece2d..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSet.java +++ /dev/null @@ -1,35 +0,0 @@ -package datawave.query.util.sortedset; - -import java.util.SortedSet; - -/** - * A sorted set that can use a replacement strategy to determine when adding an element that is already in the set should result in replacing that element. This - * will support null contained in the underlying sets iff a comparator is supplied that can handle null values. - * - * @param - * type of set - */ -public interface RewritableSortedSet extends SortedSet { - - interface RewriteStrategy { - /** - * Determine if the object should be rewritten - * - * @param original - * @param update - * @return true of the original should be replaced with the update - */ - boolean rewrite(E original, E update); - } - - RewritableSortedSetImpl.RewriteStrategy getRewriteStrategy(); - - E get(E e); - - RewritableSortedSet subSet(E fromElement, E toElement); - - RewritableSortedSet headSet(E toElement); - - RewritableSortedSet tailSet(E fromElement); - -} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSetImpl.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSetImpl.java deleted file mode 100644 index 0b71f9d43ca..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/RewritableSortedSetImpl.java +++ /dev/null @@ -1,324 +0,0 @@ -package datawave.query.util.sortedset; - -import java.util.Collection; -import java.util.Comparator; -import java.util.Iterator; -import java.util.NavigableMap; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.TreeMap; -import java.util.function.Predicate; - -import org.apache.log4j.Logger; - -import datawave.webservice.query.exception.DatawaveErrorCode; -import datawave.webservice.query.exception.QueryException; -import it.unimi.dsi.fastutil.ints.IntIterator; - -/** - * A sorted set that can use a replacement strategy to determine when to replace an element that is already contained in the set. This will support null - * contained in the underlying sets iff a comparator is supplied that can handle null values. - * - * @param - * type of set - */ -public class RewritableSortedSetImpl implements RewritableSortedSet, Cloneable { - private static Logger log = Logger.getLogger(RewritableSortedSetImpl.class); - // using a map to enable replacement of the actual set member (see uses of addResolvingCollisions) - protected NavigableMap set = null; - // When the set contains X and we are adding Y where X == Y, then use this strategy - // to decide which to keep. - protected RewriteStrategy rewriteStrategy = null; - - /** - * Create the rewritable sorted set - * - * @param comparator - * @param rewriteStrategy - */ - public RewritableSortedSetImpl(Comparator comparator, RewriteStrategy rewriteStrategy) { - this.set = new TreeMap<>(comparator); - this.rewriteStrategy = rewriteStrategy; - } - - /** - * Create the rewritable sorted set - * - * @param rewriteStrategy - */ - public RewritableSortedSetImpl(RewriteStrategy rewriteStrategy) { - this(null, rewriteStrategy); - } - - /** - * Create the rewritable sorted set - * - * @param comparator - */ - public RewritableSortedSetImpl(Comparator comparator) { - this(comparator, null); - } - - /** - * Create the rewritable sorted set - */ - public RewritableSortedSetImpl() { - this(null, null); - } - - /** - * Create a file sorted set from another one - * - * @param other - * the other sorted set - */ - public RewritableSortedSetImpl(RewritableSortedSetImpl other) { - this.set = new TreeMap<>(other.set); - this.rewriteStrategy = other.rewriteStrategy; - } - - /** - * 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 RewritableSortedSetImpl(RewritableSortedSetImpl other, E from, E to) { - this(other); - if (from != null || to != null) { - 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); - } - } - } - - @Override - public RewriteStrategy getRewriteStrategy() { - return rewriteStrategy; - } - - /** - * 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() { - return set.size(); - } - - @Override - public boolean isEmpty() { - return set.isEmpty(); - } - - @SuppressWarnings("unchecked") - @Override - public boolean contains(Object o) { - return set.containsKey(o); - } - - @Override - public Iterator iterator() { - return set.values().iterator(); - } - - @Override - public Object[] toArray() { - return set.values().toArray(); - } - - @SuppressWarnings({"unchecked"}) - @Override - public T[] toArray(T[] a) { - return set.values().toArray(a); - } - - @Override - public boolean add(E e) { - return addResolvingCollisions(e); - } - - @Override - public E get(E e) { - return set.get(e); - } - - private boolean addResolvingCollisions(E e) { - // return true if this is a new element to the set - if (set.containsKey(e)) { - if ((rewriteStrategy != null) && rewriteStrategy.rewrite(set.get(e), e)) { - set.put(e, e); - return true; - } - return false; - } - set.put(e, e); - return true; - } - - @Override - public boolean remove(Object o) { - return (set.remove(o) != null); - } - - @SuppressWarnings("unchecked") - @Override - public boolean containsAll(Collection c) { - if (c.isEmpty()) { - return true; - } - return set.keySet().containsAll(c); - } - - @Override - public boolean addAll(Collection c) { - if (!c.isEmpty()) { - return c.stream().map(m -> add(m)).reduce((l, r) -> l || r).get(); - } - return false; - } - - @Override - public boolean retainAll(Collection c) { - return set.keySet().retainAll(c); - } - - @Override - public boolean removeAll(Collection c) { - return set.keySet().removeAll(c); - } - - @Override - public boolean removeIf(Predicate filter) { - return set.keySet().removeIf(filter); - } - - @Override - public void clear() { - set.clear(); - } - - @Override - public Comparator comparator() { - return set.comparator(); - } - - @Override - public RewritableSortedSet subSet(E fromElement, E toElement) { - return new RewritableSortedSetImpl(this, fromElement, toElement); - } - - @Override - public RewritableSortedSet headSet(E toElement) { - return new RewritableSortedSetImpl(this, null, toElement); - } - - @Override - public RewritableSortedSet tailSet(E fromElement) { - return new RewritableSortedSetImpl(this, fromElement, null); - } - - @Override - public E first() { - boolean gotFirst = false; - E first = null; - if (!set.isEmpty()) { - first = set.firstKey(); - gotFirst = true; - } - if (!gotFirst) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_FIRST_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } else { - return set.get(first); - } - } - - @Override - public E last() { - boolean gotLast = false; - E last = null; - if (!set.isEmpty()) { - last = set.lastKey(); - gotLast = true; - } - if (!gotLast) { - QueryException qe = new QueryException(DatawaveErrorCode.FETCH_LAST_ELEMENT_ERROR); - throw (NoSuchElementException) (new NoSuchElementException().initCause(qe)); - } else { - return set.get(last); - } - } - - @Override - public String toString() { - return set.values().toString(); - } - - /** - * Extending classes must implement cloneable - * - * @return A clone - */ - public RewritableSortedSetImpl clone() { - return new RewritableSortedSetImpl(this); - } - - @Override - public boolean equals(Object o) { - if (o == this) { - return true; - } else if (!(o instanceof Set)) { - return false; - } else { - Set s = (Set) o; - return s.size() != this.size() ? false : this.containsAll(s); - } - } - - @Override - public int hashCode() { - int h = 0; - int n = this.size(); - - int k; - for (Iterator i = this.iterator(); n-- != 0; h += k) { - k = i.next().hashCode(); - } - - return h; - } - - /* 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 int compare(E a, E b) { - if (this.set.comparator() != null) { - return this.set.comparator().compare(a, b); - } else { - return ((Comparable) a).compareTo(b); - } - } - -} diff --git a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/KeyValueByteDocumentTransforms.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/KeyValueByteDocumentTransforms.java deleted file mode 100644 index 3a86f14d7ee..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/KeyValueByteDocumentTransforms.java +++ /dev/null @@ -1,64 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -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.Document; -import datawave.query.function.deserializer.DocumentDeserializer; -import datawave.query.function.deserializer.KryoDocumentDeserializer; -import datawave.query.function.serializer.DocumentSerializer; -import datawave.query.function.serializer.KryoDocumentSerializer; - -public class KeyValueByteDocumentTransforms { - - 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/sortedset/rfile/RFileByteDocumentInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileByteDocumentInputStream.java deleted file mode 100644 index 901963b49d3..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileByteDocumentInputStream.java +++ /dev/null @@ -1,34 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; -import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.keyValueToByteDocument; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; - -import datawave.query.attributes.Document; -import datawave.query.util.sortedset.FileSortedSet; - -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/sortedset/rfile/RFileByteDocumentOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileByteDocumentOutputStream.java deleted file mode 100644 index 5fc0c3cbeee..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileByteDocumentOutputStream.java +++ /dev/null @@ -1,21 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import static datawave.query.util.sortedset.rfile.KeyValueByteDocumentTransforms.byteDocumentToKeyValue; - -import java.io.IOException; -import java.io.OutputStream; -import java.util.Map; - -import datawave.query.attributes.Document; -import datawave.query.util.sortedset.FileSortedSet; - -public class RFileByteDocumentOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedSet.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/sortedset/rfile/RFileKeyInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyInputStream.java deleted file mode 100644 index bed9bdb7677..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyInputStream.java +++ /dev/null @@ -1,28 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -import datawave.query.util.sortedset.FileSortedSet; - -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/sortedset/rfile/RFileKeyOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyOutputStream.java deleted file mode 100644 index 14dbf3ce4cb..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyOutputStream.java +++ /dev/null @@ -1,19 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import java.io.IOException; -import java.io.OutputStream; - -import org.apache.accumulo.core.data.Key; - -import datawave.query.util.sortedset.FileSortedSet; - -public class RFileKeyOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedSet.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/sortedset/rfile/RFileKeyValueInputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueInputStream.java deleted file mode 100644 index 07a0a6c0636..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueInputStream.java +++ /dev/null @@ -1,30 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -import datawave.query.util.sortedset.FileSortedSet; - -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/sortedset/rfile/RFileKeyValueOutputStream.java b/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueOutputStream.java deleted file mode 100644 index 5a1bcf7c9d5..00000000000 --- a/warehouse/query-core/src/main/java/datawave/query/util/sortedset/rfile/RFileKeyValueOutputStream.java +++ /dev/null @@ -1,21 +0,0 @@ -package datawave.query.util.sortedset.rfile; - -import java.io.IOException; -import java.io.OutputStream; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -import datawave.query.util.sortedset.FileSortedSet; - -public class RFileKeyValueOutputStream extends RFileKeyValueOutputStreamBase implements FileSortedSet.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/test/java/datawave/query/UniqueTest.java b/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java index 289b281ad74..c9288ded767 100644 --- a/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/UniqueTest.java @@ -1,6 +1,5 @@ package datawave.query; -import java.net.URL; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -153,10 +152,6 @@ 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(); } @@ -187,10 +182,7 @@ protected void runTestQueryWithUniqueness(Set> expected, String quer TransformIterator iter = new DatawaveTransformIterator(logic.iterator(), transformer); List eventList = new ArrayList<>(); while (iter.hasNext()) { - Object o = iter.next(); - if (o != null) { - eventList.add(o); - } + eventList.add(iter.next()); } BaseQueryResponse response = transformer.createResponse(eventList); @@ -203,9 +195,6 @@ 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();) { @@ -217,9 +206,9 @@ protected void runTestQueryWithUniqueness(Set> expected, String quer break; } } - Assert.assertTrue("Failed to find " + event.getMetadata().getInternalId() + " in expected results", found); + Assert.assertTrue(found); } - Assert.assertTrue("Failed to find all expected results. Missing " + expected, expected.isEmpty()); + Assert.assertTrue(expected.isEmpty()); } @Test @@ -237,14 +226,12 @@ 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)); @@ -284,14 +271,12 @@ 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)); @@ -313,14 +298,12 @@ 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)); @@ -372,77 +355,4 @@ 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 00850fba9a6..a4bf1421a30 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,6 +407,8 @@ 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 918b1f6139f..e4a9510fe74 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 @@ -412,8 +412,6 @@ 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 afee2b09606..ff07a788854 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,85 +59,34 @@ 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.MOST_RECENT_UNIQUE, null); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL],field2[ALL],field3[ALL]"); // 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.MOST_RECENT_UNIQUE, null); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY],field3[DAY]"); // 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.MOST_RECENT_UNIQUE, null); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[HOUR],field2[HOUR],field3[HOUR]"); // 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.MOST_RECENT_UNIQUE, null); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[MINUTE],field2[MINUTE],field3[MINUTE]"); // 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.MOST_RECENT_UNIQUE, null); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[ALL],field2[ALL,DAY],field3[DAY],field4[ALL]"); // 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.MOST_RECENT_UNIQUE, null); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY,HOUR],field3[HOUR,MINUTE],field4[ALL,MINUTE],field5[ALL]"); // 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.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"); + assertOption(QueryParameters.UNIQUE_FIELDS, "field1[DAY],field2[DAY,HOUR],field3[HOUR,MINUTE],field4[ALL,MINUTE],field5[ALL]"); } @Test @@ -148,11 +97,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 @@ -163,11 +112,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 @@ -178,11 +127,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 @@ -193,11 +142,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 @@ -208,11 +157,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 @@ -223,11 +172,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 @@ -238,11 +187,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 @@ -253,26 +202,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 deleted file mode 100644 index 13817ab280d..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/transformer/UniqueTransformMostRecentTest.java +++ /dev/null @@ -1,89 +0,0 @@ -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.query.attributes.UniqueGranularity; -import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; -import datawave.query.tables.ShardQueryLogic; -import datawave.query.util.sortedset.FileSortedSet; -import datawave.webservice.query.QueryImpl; - -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 8414dbad2f3..ba0354c12e1 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,7 +2,6 @@ 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; @@ -20,51 +19,45 @@ 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 { - protected static final Random random = new Random(1000); - private static final AtomicLong counter = new AtomicLong(); + private static final Random random = new Random(1000); + private static final List randomValues = new ArrayList<>(); - 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(); + private final List inputDocuments = new ArrayList<>(); + private final List expectedUniqueDocuments = new ArrayList<>(); + private byte[] expectedOrderedFieldValues = null; + private UniqueFields uniqueFields = new UniqueFields(); @BeforeClass public static void setup() { @@ -104,7 +97,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); } @@ -115,7 +108,7 @@ public void testUniquenessWithRandomDocuments() { assertEquals(expectedUniqueDocuments, uniqueDocuments.size()); } - protected int countUniqueness(List input, Set fields) { + private int countUniqueness(List input, Set fields) { Set uniqueValues = new HashSet<>(); for (Document document : input) { Multimap fieldValues = getFieldValues(document, fields); @@ -124,7 +117,7 @@ protected int countUniqueness(List input, Set fields) { return uniqueValues.size(); } - protected Multimap getFieldValues(Document document, Set fields) { + private Multimap getFieldValues(Document document, Set fields) { Multimap values = HashMultimap.create(); for (String docField : document.getDictionary().keySet()) { for (String field : fields) { @@ -141,7 +134,7 @@ protected Multimap getFieldValues(Document document, Set return values; } - protected String getString(Multimap fieldValues) { + private String getString(Multimap fieldValues) { StringBuilder sb = new StringBuilder(); fieldValues.keySet().stream().sorted().forEach((field) -> { if (sb.length() > 0) { @@ -161,12 +154,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"); @@ -178,11 +171,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"); @@ -194,11 +187,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"); @@ -210,11 +203,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"); @@ -227,18 +220,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"); @@ -253,12 +246,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); @@ -271,12 +264,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); @@ -289,12 +282,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); @@ -306,7 +299,7 @@ public void testUniquenessWithTimingMetric() { List input = new ArrayList<>(); List expected = new ArrayList<>(); - String MARKER_STRING = FinalDocumentTrackingIterator.MARKER_TEXT.toString(); + String MARKER_STRING = "\u2735FinalDocument\u2735"; TimingMetadata timingMetadata = new TimingMetadata(); timingMetadata.setNextCount(5l); @@ -333,16 +326,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"); @@ -364,18 +357,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"); @@ -397,18 +390,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"); @@ -431,20 +424,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"); @@ -465,18 +458,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"); @@ -484,56 +477,27 @@ public void testUniquenessWithTwoGroupsAndPartialGroups() { assertOrderedFieldValues(); } - @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() { + private void assertUniqueDocuments() { List actual = getUniqueDocumentsWithUpdateConfigCalls(inputDocuments); Collections.sort(expectedUniqueDocuments); Collections.sort(actual); assertEquals("Unique documents do not match expected", expectedUniqueDocuments, actual); } - protected List getUniqueDocuments(List documents) { + private List getUniqueDocuments(List documents) { Transformer> docToEntry = document -> Maps.immutableEntry(document.getMetadata(), document); TransformIterator> inputIterator = new TransformIterator<>(documents.iterator(), docToEntry); UniqueTransform uniqueTransform = getUniqueTransform(); - Iterator> resultIterator = uniqueTransform.getIterator(inputIterator); + Iterator> resultIterator = Iterators.transform(inputIterator, uniqueTransform); return StreamSupport.stream(Spliterators.spliteratorUnknownSize(resultIterator, Spliterator.ORDERED), false).filter(Objects::nonNull) .map(Map.Entry::getValue).collect(Collectors.toList()); } - protected List getUniqueDocumentsWithUpdateConfigCalls(List documents) { + private List getUniqueDocumentsWithUpdateConfigCalls(List documents) { Transformer> docToEntry = document -> Maps.immutableEntry(document.getMetadata(), document); TransformIterator> inputIterator = new TransformIterator<>(documents.iterator(), docToEntry); UniqueTransform uniqueTransform = getUniqueTransform(); - Iterator> resultIterator = uniqueTransform.getIterator(inputIterator); + Iterator> resultIterator = Iterators.transform(inputIterator, uniqueTransform); ArrayList docs = new ArrayList<>(); while (resultIterator.hasNext()) { Map.Entry next = resultIterator.next(); @@ -545,7 +509,7 @@ protected List getUniqueDocumentsWithUpdateConfigCalls(List return docs; } - protected void assertOrderedFieldValues() { + private void assertOrderedFieldValues() { try { UniqueTransform uniqueTransform = getUniqueTransform(); for (Document d : inputDocuments) { @@ -557,61 +521,53 @@ protected void assertOrderedFieldValues() { } } - protected void givenValueTransformerForFields(UniqueGranularity transformer, String... fields) { + private void givenValueTransformerForFields(UniqueGranularity transformer, String... fields) { Arrays.stream(fields).forEach((field) -> uniqueFields.put(field, transformer)); } - protected void givenValueTransformersForField(String field, UniqueGranularity... transformers) { + private void givenValueTransformersForField(String field, UniqueGranularity... transformers) { Arrays.stream(transformers).forEach((transformer) -> uniqueFields.put(field, transformer)); } - 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 UniqueTransform getUniqueTransform() { + return new UniqueTransform(uniqueFields, Long.MAX_VALUE); } - protected InputDocumentBuilder givenInputDocument() { - return new InputDocumentBuilder("", 0); + private void updateUniqueTransform(UniqueTransform uniqueTransform) { + uniqueTransform.updateConfig(uniqueFields, null); } - protected InputDocumentBuilder givenInputDocument(String cq) { - return new InputDocumentBuilder(cq, 0); + private InputDocumentBuilder givenInputDocument() { + return new InputDocumentBuilder(); } - protected InputDocumentBuilder givenInputDocument(long ts) { - return new InputDocumentBuilder("", ts); + private InputDocumentBuilder givenInputDocument(String docKey) { + return new InputDocumentBuilder(docKey); } - protected InputDocumentBuilder givenInputDocument(String docKey, long ts) { - return new InputDocumentBuilder(docKey, ts); - } - - protected ExpectedOrderedFieldValuesBuilder givenExpectedOrderedFieldValues() { + private ExpectedOrderedFieldValuesBuilder givenExpectedOrderedFieldValues() { return new ExpectedOrderedFieldValuesBuilder(); } - protected class InputDocumentBuilder { + private class InputDocumentBuilder { private final Document document; - InputDocumentBuilder(String cq, long ts) { - Key key = new Key("shardid", "datatype\u0000" + getUid(), cq, ts); - this.document = new Document(key, true); + InputDocumentBuilder() { + this.document = new Document(); 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"}) + 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()); + this.document = new Document(key, true); + inputDocuments.add(document); + this.document.getMetadata().set(key); } @SuppressWarnings({"UnusedReturnValue", "SameParameterValue"}) @@ -634,7 +590,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() { @@ -653,7 +609,7 @@ InputDocumentBuilder isExpectedToBeUnique() { } } - protected class ExpectedOrderedFieldValuesBuilder { + private class ExpectedOrderedFieldValuesBuilder { private Multimap fieldValues = TreeMultimap.create(); @@ -668,16 +624,13 @@ 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 + ":"; - if (fieldValues.isEmpty()) { + String separator = "f-" + field + '/' + (count++) + ":"; + for (String value : fieldValues.get(field)) { output.writeUTF(separator); - } else { - for (String value : fieldValues.get(field)) { - output.writeUTF(separator); - output.writeUTF(value); - 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 d5c5506aba3..898b6481381 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,12 +53,9 @@ 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".getBytes(), (Date) null).toString(); - public static final long sopranoTimeStampDelta = 10; - public static final String caponeUID = UID.builder().newId("Capone".getBytes(), (Date) null).toString(); - public static final long caponeTimeStampDelta = 20; + public static final String sopranoUID = UID.builder().newId("Soprano".toString().getBytes(), (Date) null).toString(); + public static final String caponeUID = UID.builder().newId("Capone".toString().getBytes(), (Date) null).toString(); protected static String normalizeColVal(Map.Entry colVal) { switch (colVal.getKey()) { @@ -97,93 +94,81 @@ 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 + 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); + 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); // CORLEONE date delta is 70 years - 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, + 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, 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); + 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); // 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 + 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); + // 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); // soprano date delta is 50 years - 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, "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, "QUOTE" + "\u0000" + "If you can quote the rules then you can obey them", columnVisibility, - 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); + 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); // 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 + caponeTimeStampDelta, emptyValue); + mutation.put(datatype + "\u0000" + caponeUID, "BIRTH_DATE.0" + "\u0000" + "1910-12-28T00:00:05.000Z", columnVisibility, timeStamp, 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 + 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, "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, "QUOTE" + "\u0000" + "You can get much farther with a kind word and a gun than you can with a kind word alone", columnVisibility, - 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); + 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); bw.addMutation(mutation); @@ -400,10 +385,9 @@ 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, 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); + 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); } finally { if (null != bw) { bw.close(); @@ -624,130 +608,116 @@ 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 + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); // uuid mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("ANDOLINI") + "\u0000" + datatype + "\u0000" + corleoneChildUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); // names mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("SANTINO") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); - mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("FREDO") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); + mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("FREDO") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, + emptyValue); mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("MICHAEL") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("CONSTANZIA") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); - mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("LUCA") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); + mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("LUCA") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, + emptyValue); mutation.put("fi\u0000" + "NOME", lcNoDiacriticsType.normalize("VINCENT") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); // genders mutation.put("fi\u0000" + "GENERE", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); mutation.put("fi\u0000" + "GENERE", lcNoDiacriticsType.normalize("FEMALE") + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + timeStamp, emptyValue); // ages - 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); + 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); // geo for (String normalized : ((OneToManyNormalizerType) geoType).normalizeToMany("POINT(10 10)")) { - mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, - timeStamp + corleoneTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); } // sopranos // uuid mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("SOPRANO") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp + sopranoTimeStampDelta, emptyValue); + timeStamp, emptyValue); // names mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("ANTHONY") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp + sopranoTimeStampDelta, emptyValue); - mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("MEADOW") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp + sopranoTimeStampDelta, emptyValue); + timeStamp, emptyValue); + mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("MEADOW") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, + emptyValue); // genders - mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp + sopranoTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("MALE") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp, + emptyValue); mutation.put("fi\u0000" + "GENDER", lcNoDiacriticsType.normalize("FEMALE") + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, - timeStamp + sopranoTimeStampDelta, emptyValue); + timeStamp, emptyValue); // ages - 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); + 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); // geo for (String normalized : ((OneToManyNormalizerType) geoType).normalizeToMany("POINT(20 20)")) { - mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + sopranoUID, columnVisibility, timeStamp + sopranoTimeStampDelta, - emptyValue); + mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); } // capones // uuid - mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("CAPONE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, - timeStamp + caponeTimeStampDelta, emptyValue); + mutation.put("fi\u0000" + "UUID", lcNoDiacriticsType.normalize("CAPONE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp, + emptyValue); // names mutation.put("fi\u0000" + "NAME", lcNoDiacriticsType.normalize("ALPHONSE") + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, - 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); + 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); // genders - 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); + 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); // ages - 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); + 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); // geo for (String normalized : ((OneToManyNormalizerType) geoType).normalizeToMany("POINT(30 30)")) { - mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + caponeUID, columnVisibility, timeStamp + caponeTimeStampDelta, - emptyValue); + mutation.put("fi\u0000" + "GEO", normalized + "\u0000" + datatype + "\u0000" + corleoneUID, columnVisibility, timeStamp, emptyValue); } // add some index-only fields - 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); + 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); bw.addMutation(mutation); - 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); + 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); } finally { if (null != bw) { bw.close(); @@ -1015,33 +985,31 @@ private static Value getValueForNuthinAndYourHitsForFree() { return new Value(builder.build().toByteArray()); } - private static void addTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid, long timeStampDelta) - throws MutationsRejectedException { + private static void addTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid) throws MutationsRejectedException { Mutation mutation = new Mutation(lcNoDiacriticsType.normalize(phrase)); - mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp + timeStampDelta, + mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp, 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 + timeStampDelta, + mutation.put(field.toUpperCase(), shard + "\u0000" + datatype, columnVisibility, timeStamp, range == WhatKindaRange.SHARD ? getValueForNuthinAndYourHitsForFree() : getValueForBuilderFor(uid)); bw.addMutation(mutation); } } - private static void addFiTfTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid, long timeStampDelta) - throws MutationsRejectedException { + private static void addFiTfTokens(BatchWriter bw, WhatKindaRange range, String field, String phrase, String uid) throws MutationsRejectedException { Mutation fi = new Mutation(shard); - fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(phrase) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, - timeStamp + timeStampDelta, emptyValue); + fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(phrase) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, timeStamp, + 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 + timeStampDelta, emptyValue); + fi.put("fi\u0000" + field.toUpperCase(), lcNoDiacriticsType.normalize(token) + "\u0000" + datatype + "\u0000" + uid, columnVisibility, timeStamp, + emptyValue); tokenOffsetCache.addOffset(new TermAndZone(token, field.toUpperCase()), i); i++; @@ -1054,7 +1022,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 + timeStampDelta, value); + columnVisibility, timeStamp, value); } bw.addMutation(fi); } diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteArraySortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteArraySortedSetTest.java deleted file mode 100644 index 1d3e6d0aa66..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteArraySortedSetTest.java +++ /dev/null @@ -1,21 +0,0 @@ -package datawave.query.util.sortedset; - -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 FileSortedSet.FileSortedSetFactory getFactory() { - return new FileSerializableSortedSet.Factory(); - } -} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteDocumentSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteDocumentSortedSetTest.java deleted file mode 100644 index a1b038237e0..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedByteDocumentSortedSetTest.java +++ /dev/null @@ -1,89 +0,0 @@ -package datawave.query.util.sortedset; - -import static org.junit.Assert.assertEquals; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import javax.annotation.Nullable; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Range; -import org.apache.accumulo.core.data.Value; -import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; - -import datawave.data.type.LcNoDiacriticsType; -import datawave.query.attributes.Content; -import datawave.query.attributes.Document; -import datawave.query.composite.CompositeMetadata; -import datawave.query.predicate.EventDataQueryFieldFilter; -import datawave.query.predicate.EventDataQueryFilter; -import datawave.query.predicate.KeyProjection; -import datawave.query.util.TypeMetadata; - -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 FileSortedSet.FileSortedSetFactory> getFactory() { - return new FileByteDocumentSortedSet.Factory(); - } -} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeySortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeySortedSetTest.java index 1c42597f727..7e31fe0c8c2 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeySortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeySortedSetTest.java @@ -5,7 +5,15 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.util.Comparator; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.SortedSet; +import java.util.TreeSet; import org.apache.accumulo.core.data.Key; import org.junit.After; @@ -13,21 +21,240 @@ import org.junit.Before; import org.junit.Test; -public class BufferedFileBackedKeySortedSetTest extends BufferedFileBackedSortedSetTest { +public class BufferedFileBackedKeySortedSetTest { - @Override - public Key createData(byte[] values) { - return new Key(values); + private final List tempFileHandlers = new ArrayList<>(); + private Key[] data = null; + private int[] sortedOrder = null; + private BufferedFileBackedSortedSet 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 Key[template.length * 2]; + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[i + 11]; + Arrays.fill(buffer, template[i]); + data[i] = new Key(buffer); + } + for (int i = 0; i < template.length; i++) { + byte[] buffer = new byte[10]; + Arrays.fill(buffer, template[i]); + data[i + template.length] = new Key(buffer); + } + 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 BufferedFileBackedSortedSet<>(null, 5, 7, 2, Collections.singletonList(new BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { + @Override + public FileSortedSet.SortedSetFileHandler createHandler() throws IOException { + SortedSetTempFileHandler fileHandler = new SortedSetTempFileHandler(); + tempFileHandlers.add(fileHandler); + return fileHandler; + } + + @Override + public boolean isValid() { + return true; + } + }), new FileKeySortedSet.Factory()); + + // 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++) { + Collections.addAll(set, data); + } + } + + @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 testIterator() { + int index = 0; + for (Iterator it = set.iterator(); it.hasNext();) { + Key value = it.next(); + Key expected = data[sortedOrder[index++]]; + assertEquals(expected, value); + } + set.clear(); + for (Key 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()); + Key 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 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() { + Key expected = data[sortedOrder[data.length - 1]]; + Key value = set.last(); + assertEquals(expected, value); } - @Override - public Comparator getComparator() { - return null; + @Test + public void testFirst() { + Key expected = data[sortedOrder[0]]; + Key value = set.first(); + assertEquals(expected, value); } - @Override - public FileSortedSet.FileSortedSetFactory getFactory() { - return new FileKeySortedSet.Factory(); + @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/sortedset/BufferedFileBackedKeyValueSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeyValueSortedSetTest.java deleted file mode 100644 index 6d1f9cbfb78..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedKeyValueSortedSetTest.java +++ /dev/null @@ -1,62 +0,0 @@ -package datawave.query.util.sortedset; - -import static org.junit.Assert.assertEquals; - -import java.util.Comparator; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -public class 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 FileSortedSet.FileSortedSetFactory> getFactory() { - return new FileKeyValueSortedSet.Factory(); - } - -} diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedRewritableSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedRewritableSortedSetTest.java deleted file mode 100644 index 33035432ed9..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedRewritableSortedSetTest.java +++ /dev/null @@ -1,108 +0,0 @@ -package datawave.query.util.sortedset; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.Map; - -import org.apache.commons.collections.keyvalue.UnmodifiableMapEntry; -import org.junit.Test; - -public abstract class 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 BufferedFileBackedSortedSet.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) - .withMaxOpenFiles(7).withNumRetries(2) - .withHandlerFactories(Collections.singletonList(new BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { - @Override - public FileSortedSet.SortedSetFileHandler createHandler() throws IOException { - 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/sortedset/BufferedFileBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedSortedSetTest.java index 9e1da478072..e700f917169 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/BufferedFileBackedSortedSetTest.java @@ -7,16 +7,12 @@ import java.io.File; import java.io.IOException; -import java.lang.reflect.Array; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; @@ -27,58 +23,33 @@ import org.junit.Before; import org.junit.Test; -public abstract class BufferedFileBackedSortedSetTest { +public class BufferedFileBackedSortedSetTest { - protected final List tempFileHandlers = new ArrayList<>(); - protected E[] data = null; - protected int[] sortedOrder = null; - protected 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 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); - } - } + private final List tempFileHandlers = new ArrayList<>(); + private byte[][] data = null; + private int[] sortedOrder = null; + private BufferedFileBackedSortedSet 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++) { - 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; + data[i] = new byte[i + 11]; + Arrays.fill(data[i], template[i]); } 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; + 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 BufferedFileBackedSortedSet.Builder().withComparator(getComparator()).withRewriteStrategy(getRewriteStrategy()).withBufferPersistThreshold(5) - .withMaxOpenFiles(7).withNumRetries(2) - .withHandlerFactories(Collections.singletonList(new BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { + set = new BufferedFileBackedSortedSet<>(new ByteArrayComparator(), 5, 7, 2, + Collections.singletonList(new BufferedFileBackedSortedSet.SortedSetFileHandlerFactory() { @Override public FileSortedSet.SortedSetFileHandler createHandler() throws IOException { SortedSetTempFileHandler fileHandler = new SortedSetTempFileHandler(); @@ -90,32 +61,12 @@ public FileSortedSet.SortedSetFileHandler createHandler() throws IOException { public boolean isValid() { return true; } - })).withSetFactory(getFactory()).build(); + }), new FileSerializableSortedSet.Factory()); // 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]); - } + Collections.addAll(set, data); } } @@ -234,18 +185,18 @@ public void testRemoveIf() { int expectedSize = data.length; assertFalse(set.isPersisted()); - set.removeIf(new Predicate() { + set.removeIf(new Predicate() { @Override - public boolean test(E bytes) { + public boolean test(byte[] bytes) { return false; } }); assertFalse(set.isPersisted()); assertEquals(expectedSize, set.size()); - set.removeIf(new Predicate() { + set.removeIf(new Predicate() { @Override - public boolean test(E bytes) { + public boolean test(byte[] bytes) { return true; } }); @@ -261,18 +212,18 @@ public void testRemoveIfPersisted() throws IOException { set.persist(); assertTrue(set.isPersisted()); - set.removeIf(new Predicate() { + set.removeIf(new Predicate() { @Override - public boolean test(E bytes) { + public boolean test(byte[] bytes) { return false; } }); assertTrue(set.isPersisted()); assertEquals(expectedSize, set.size()); - set.removeIf(new Predicate() { + set.removeIf(new Predicate() { @Override - public boolean test(E bytes) { + public boolean test(byte[] bytes) { return true; } }); @@ -289,9 +240,9 @@ public void testRemoveAll() { assertFalse(set.isPersisted()); assertEquals(expectedSize, set.size()); - Set dataSet = new TreeSet<>(set.comparator()); - dataSet.addAll(Arrays.asList(data)); - set.removeAll(dataSet); + Set toRemove = new TreeSet(new ByteArrayComparator()); + toRemove.addAll(Arrays.asList(data)); + set.removeAll(toRemove); assertFalse(set.isPersisted()); assertTrue(set.isEmpty()); } @@ -307,9 +258,9 @@ public void testRemoveAllPersisted() throws IOException { assertTrue(set.isPersisted()); assertEquals(expectedSize, set.size()); - Set dataSet = new TreeSet<>(set.comparator()); - dataSet.addAll(Arrays.asList(data)); - set.removeAll(dataSet); + Set toRemove = new TreeSet(new ByteArrayComparator()); + toRemove.addAll(Arrays.asList(data)); + set.removeAll(toRemove); assertTrue(set.isPersisted()); assertTrue(set.isEmpty()); } @@ -317,13 +268,13 @@ public void testRemoveAllPersisted() throws IOException { @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); + for (Iterator it = set.iterator(); it.hasNext();) { + byte[] value = it.next(); + byte[] expected = data[sortedOrder[index++]]; + assertTrue(Arrays.equals(expected, value)); } set.clear(); - for (E value : set) { + for (byte[] value : set) { fail(); } } @@ -334,9 +285,9 @@ public void testIteratorRemove() { int failCount = 0; assertFalse(set.isPersisted()); // calling iterator() will force persistence - for (Iterator it = set.iterator(); it.hasNext();) { + for (Iterator it = set.iterator(); it.hasNext();) { assertTrue(set.isPersisted()); - E value = it.next(); + byte[] value = it.next(); assertTrue(set.contains(value)); try { it.remove(); @@ -354,13 +305,13 @@ public void testIteratorRemove() { @Test public void testComparator() { - Comparator comparator = set.comparator(); - E[] testData = Arrays.copyOf(data, data.length); + Comparator comparator = set.comparator(); + byte[][] 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); + for (byte[] value : set) { + byte[] expected = data[sortedOrder[index++]]; + assertTrue(Arrays.equals(expected, value)); } } @@ -369,8 +320,8 @@ 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()); + 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]]); } @@ -384,8 +335,8 @@ public void testSubSet() { public void testHeadSet() { int end = sortedOrder.length / 3; try { - SortedSet subSet = set.headSet(data[sortedOrder[end]]); - SortedSet expected = new TreeSet<>(set.comparator()); + 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]]); } @@ -399,8 +350,8 @@ public void testHeadSet() { public void testTailSet() { int start = sortedOrder.length / 3; try { - SortedSet subSet = set.tailSet(data[sortedOrder[start]]); - SortedSet expected = new TreeSet<>(set.comparator()); + 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]]); } @@ -412,16 +363,16 @@ public void testTailSet() { @Test public void testLast() { - E expected = data[sortedOrder[data.length - 1]]; - E value = set.last(); - testEquality(expected, value); + byte[] expected = data[sortedOrder[data.length - 1]]; + byte[] value = set.last(); + assertTrue(Arrays.equals(expected, value)); } @Test public void testFirst() { - E expected = data[sortedOrder[0]]; - E value = set.first(); - testEquality(expected, value); + byte[] expected = data[sortedOrder[0]]; + byte[] value = set.first(); + assertTrue(Arrays.equals(expected, value)); } @Test diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/FileSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/FileSortedSetTest.java index c752d88f74d..60479b8fc71 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/FileSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/FileSortedSetTest.java @@ -38,7 +38,7 @@ public int compare(Integer o1, Integer o2) { } }; handler = new SortedSetTempFileHandler(); - set = new FileSerializableSortedSet<>(c, new FileSerializableSortedSet.SerializableFileHandler(handler), false); + set = new FileSerializableSortedSet<>(new TreeSet<>(c), new FileSerializableSortedSet.SerializableFileHandler(handler)); data = new TreeSet<>(c); Random r = new Random(123948710248L); // data.add(null); @@ -330,7 +330,7 @@ public void testNoComparator() throws Exception { assertNotNull(set.comparator()); set.load(); assertNotNull(set.comparator()); - RewritableSortedSet tempData = new RewritableSortedSetImpl<>(); + SortedSet tempData = new TreeSet<>(); for (Integer i : data) { if (i != null) { tempData.add(i); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/HdfsBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/HdfsBackedSortedSetTest.java index 0948b33a278..b65ae2b83ec 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/HdfsBackedSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/HdfsBackedSortedSetTest.java @@ -49,16 +49,7 @@ public void persistReloadTest() throws Exception { String uniquePath = "blah"; - // @formatter:off - @SuppressWarnings("unchecked") - HdfsBackedSortedSet sortedSet = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder() - .withIvaratorCacheDirs(ivaratorCacheDirs) - .withUniqueSubPath(uniquePath) - .withMaxOpenFiles(9999) - .withNumRetries(2) - .withPersistOptions(new FileSortedSet.PersistOptions()) - .build(); - // @formatter:on + HdfsBackedSortedSet sortedSet = new HdfsBackedSortedSet<>(ivaratorCacheDirs, uniquePath, 9999, 2, new FileSortedSet.PersistOptions()); // Add an entry to the sorted set String someTestString = "some test string"; @@ -82,16 +73,7 @@ public void persistReloadTest() throws Exception { Assert.assertTrue(fileStatuses[0].getPath().getName().startsWith("SortedSet")); // Now make sure reloading an ivarator cache dir works - // @formatter:off - @SuppressWarnings("unchecked") - HdfsBackedSortedSet reloadedSortedSet = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder() - .withIvaratorCacheDirs(ivaratorCacheDirs) - .withUniqueSubPath(uniquePath) - .withMaxOpenFiles(9999) - .withNumRetries(2) - .withPersistOptions(new FileSortedSet.PersistOptions()) - .build(); - // @formatter:on + HdfsBackedSortedSet reloadedSortedSet = new HdfsBackedSortedSet<>(ivaratorCacheDirs, uniquePath, 9999, 2, new FileSortedSet.PersistOptions()); Assert.assertEquals(1, reloadedSortedSet.size()); Assert.assertEquals(someTestString, reloadedSortedSet.first()); @@ -126,16 +108,8 @@ public void persistCompactReloadTest() throws Exception { ivaratorCacheDirs.add(new IvaratorCacheDir(new IvaratorCacheDirConfig(dir.toURI().toString(), 1), fs, dir.toURI().toString())); } - // @formatter:off - @SuppressWarnings("unchecked") - HdfsBackedSortedSet firstSortedSet = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder() - .withIvaratorCacheDirs(Collections.singletonList(ivaratorCacheDirs.get(0))) - .withUniqueSubPath(uniquePath) - .withMaxOpenFiles(9999) - .withNumRetries(2) - .withPersistOptions(new FileSortedSet.PersistOptions()) - .build(); - // @formatter:on + HdfsBackedSortedSet firstSortedSet = new HdfsBackedSortedSet<>(Collections.singletonList(ivaratorCacheDirs.get(0)), uniquePath, 9999, 2, + new FileSortedSet.PersistOptions()); // Add an entry to the first sorted set String someTestString = "some test string"; @@ -144,16 +118,8 @@ public void persistCompactReloadTest() throws Exception { // persist the sorted set firstSortedSet.persist(); - // @formatter:off - @SuppressWarnings("unchecked") - HdfsBackedSortedSet thirdSortedSet = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder() - .withIvaratorCacheDirs(Collections.singletonList(ivaratorCacheDirs.get(2))) - .withUniqueSubPath(uniquePath) - .withMaxOpenFiles(9999) - .withNumRetries(2) - .withPersistOptions(new FileSortedSet.PersistOptions()) - .build(); - // @formatter:on + HdfsBackedSortedSet thirdSortedSet = new HdfsBackedSortedSet<>(Collections.singletonList(ivaratorCacheDirs.get(2)), uniquePath, 9999, 2, + new FileSortedSet.PersistOptions()); // Add an entry to the third sorted set String anotherTestString = "another test string"; @@ -181,16 +147,7 @@ public void persistCompactReloadTest() throws Exception { 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") - HdfsBackedSortedSet reloadedSortedSet = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder() - .withIvaratorCacheDirs(ivaratorCacheDirs) - .withUniqueSubPath(uniquePath) - .withMaxOpenFiles(1) - .withNumRetries(2) - .withPersistOptions(new FileSortedSet.PersistOptions()) - .build(); - // @formatter:on + HdfsBackedSortedSet reloadedSortedSet = new HdfsBackedSortedSet<>(ivaratorCacheDirs, uniquePath, 1, 2, new FileSortedSet.PersistOptions()); // Ensure that we have 2 entries total Assert.assertEquals(2, reloadedSortedSet.size()); @@ -225,16 +182,7 @@ public void persistCompactReloadTest() throws Exception { Assert.assertTrue(fileStatuses[0].getPath().getName().startsWith("SortedSet")); // Finally, make sure that the compacted data can be reloaded - // @formatter:off - @SuppressWarnings("unchecked") - HdfsBackedSortedSet compactedSortedSet = (HdfsBackedSortedSet) HdfsBackedSortedSet.builder() - .withIvaratorCacheDirs(ivaratorCacheDirs) - .withUniqueSubPath(uniquePath) - .withMaxOpenFiles(9999) - .withNumRetries(2) - .withPersistOptions(new FileSortedSet.PersistOptions()) - .build(); - // @formatter:on + HdfsBackedSortedSet compactedSortedSet = new HdfsBackedSortedSet<>(ivaratorCacheDirs, uniquePath, 9999, 2, new FileSortedSet.PersistOptions()); // Ensure that we have 3 entries total Assert.assertEquals(3, compactedSortedSet.size()); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MergeSortIteratorTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MergeSortIteratorTest.java index 98da610c69d..472c7f498ca 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MergeSortIteratorTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MergeSortIteratorTest.java @@ -7,7 +7,6 @@ import java.util.ArrayList; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.SortedSet; @@ -56,8 +55,7 @@ public void testIteration() { col.add(set2); col.add(set3); List results = new ArrayList<>(); - - Iterator it = new MultiSetBackedSortedSet(col).iterator(); + MergeSortIterator it = new MergeSortIterator<>(col); try { it.remove(); fail("Expected remove to fail"); @@ -126,7 +124,7 @@ public void testIterationSansHasNext() { col.add(set2); col.add(set3); List results = new ArrayList<>(); - Iterator it = new MultiSetBackedSortedSet(col).iterator(); + MergeSortIterator it = new MergeSortIterator<>(col); while (true) { try { it.remove(); @@ -209,7 +207,7 @@ public int compare(Integer o1, Integer o2) { col.add(set2); col.add(set3); List results = new ArrayList<>(); - Iterator it = new MultiSetBackedSortedSet(col).iterator(); + MergeSortIterator it = new MergeSortIterator<>(col); try { it.remove(); fail("Expected remove to fail"); diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MultiSetBackedSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MultiSetBackedSortedSetTest.java index 3efa34db4d5..091ba311620 100644 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MultiSetBackedSortedSetTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/MultiSetBackedSortedSetTest.java @@ -18,6 +18,9 @@ import org.junit.Before; import org.junit.Test; +import datawave.query.util.sortedset.ByteArrayComparator; +import datawave.query.util.sortedset.MultiSetBackedSortedSet; + public class MultiSetBackedSortedSetTest { private byte[][] data = null; private int[] sortedOrder = null; @@ -590,5 +593,4 @@ public void testFirst() { byte[] value = set.first(); assertArrayEquals(expected, value); } - } diff --git a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/RewritableSortedSetTest.java b/warehouse/query-core/src/test/java/datawave/query/util/sortedset/RewritableSortedSetTest.java deleted file mode 100644 index 6783c48bede..00000000000 --- a/warehouse/query-core/src/test/java/datawave/query/util/sortedset/RewritableSortedSetTest.java +++ /dev/null @@ -1,284 +0,0 @@ -package datawave.query.util.sortedset; - -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.Arrays; -import java.util.Collections; -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 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; - -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/resources/datawave/query/QueryLogicFactory.xml b/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml index fdfced0fba9..1fd86d494e4 100644 --- a/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml +++ b/warehouse/query-core/src/test/resources/datawave/query/QueryLogicFactory.xml @@ -28,23 +28,11 @@ - + - - - - - - - - - - - - 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 1b526db4fcb..c51d6e1818d 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 @@ -41,7 +41,6 @@ import datawave.webservice.query.cachedresults.CacheableQueryRow; import datawave.webservice.query.cachedresults.CacheableQueryRowReader; import datawave.webservice.query.data.ObjectSizeOf; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.logic.QueryLogic; import datawave.webservice.query.logic.QueryLogicFactory; import datawave.webservice.query.logic.QueryLogicTransformer; @@ -302,7 +301,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, QueryException { + QueryMetricFactory metricFactory) throws SQLException { super(metricFactory); this.variableFields.clear(); @@ -570,7 +569,7 @@ private List getViewColumnNames(Connection connection, String view) thro return columns; } - public void activate(Connection connection, QueryLogic queryLogic) throws SQLException, QueryException { + public void activate(Connection connection, QueryLogic queryLogic) throws SQLException { 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 48d9b40df13..8c635216e5e 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 @@ -40,23 +40,15 @@ - + + - - - - - - - - - diff --git a/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsFileOutputMapper.java b/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsFileOutputMapper.java index 08e6d834c21..66651fafe78 100644 --- a/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsFileOutputMapper.java +++ b/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsFileOutputMapper.java @@ -28,7 +28,6 @@ import datawave.webservice.query.Query; import datawave.webservice.query.cache.ResultsPage; import datawave.webservice.query.exception.EmptyObjectException; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.logic.QueryLogic; import datawave.webservice.query.logic.QueryLogicTransformer; import datawave.webservice.result.BaseQueryResponse; @@ -91,7 +90,6 @@ 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/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsTableOutputMapper.java b/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsTableOutputMapper.java index 67904139b61..f840aaae9d7 100644 --- a/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsTableOutputMapper.java +++ b/web-services/map-reduce/src/main/java/datawave/webservice/mr/bulkresults/map/BulkResultsTableOutputMapper.java @@ -17,7 +17,6 @@ import datawave.webservice.query.Query; import datawave.webservice.query.cache.ResultsPage; import datawave.webservice.query.exception.EmptyObjectException; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.logic.QueryLogic; import datawave.webservice.query.logic.QueryLogicTransformer; import datawave.webservice.result.BaseQueryResponse; diff --git a/web-services/query/src/main/java/datawave/webservice/query/logic/BaseQueryLogic.java b/web-services/query/src/main/java/datawave/webservice/query/logic/BaseQueryLogic.java index b2249c2eda2..d096159baff 100644 --- a/web-services/query/src/main/java/datawave/webservice/query/logic/BaseQueryLogic.java +++ b/web-services/query/src/main/java/datawave/webservice/query/logic/BaseQueryLogic.java @@ -21,7 +21,6 @@ import datawave.webservice.common.connection.AccumuloClientConfiguration; import datawave.webservice.query.Query; import datawave.webservice.query.configuration.GenericQueryConfiguration; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.iterator.DatawaveTransformIterator; import datawave.webservice.query.result.event.ResponseObjectFactory; diff --git a/web-services/query/src/main/java/datawave/webservice/query/logic/composite/CompositeQueryLogic.java b/web-services/query/src/main/java/datawave/webservice/query/logic/composite/CompositeQueryLogic.java index 4d69f57785a..9016fbdd859 100644 --- a/web-services/query/src/main/java/datawave/webservice/query/logic/composite/CompositeQueryLogic.java +++ b/web-services/query/src/main/java/datawave/webservice/query/logic/composite/CompositeQueryLogic.java @@ -33,7 +33,6 @@ import datawave.webservice.query.cache.ResultsPage; import datawave.webservice.query.configuration.GenericQueryConfiguration; import datawave.webservice.query.exception.EmptyObjectException; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.logic.BaseQueryLogic; import datawave.webservice.query.logic.QueryLogic; import datawave.webservice.query.logic.QueryLogicTransformer; diff --git a/web-services/query/src/main/java/datawave/webservice/query/logic/filtered/FilteredQueryLogic.java b/web-services/query/src/main/java/datawave/webservice/query/logic/filtered/FilteredQueryLogic.java index a4aa53282b3..25138ec64b0 100644 --- a/web-services/query/src/main/java/datawave/webservice/query/logic/filtered/FilteredQueryLogic.java +++ b/web-services/query/src/main/java/datawave/webservice/query/logic/filtered/FilteredQueryLogic.java @@ -12,7 +12,6 @@ import datawave.security.authorization.UserOperations; import datawave.webservice.query.Query; import datawave.webservice.query.configuration.GenericQueryConfiguration; -import datawave.webservice.query.exception.QueryException; import datawave.webservice.query.iterator.DatawaveTransformIterator; import datawave.webservice.query.logic.DelegatingQueryLogic; import datawave.webservice.query.logic.QueryLogic;