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 bb48229552..979930cfe9 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 @@ -9,9 +9,12 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; -import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -41,11 +44,13 @@ import datawave.query.composite.CompositeMetadata; import datawave.query.composite.CompositeSeeker.FieldIndexCompositeSeeker; import datawave.query.exceptions.DatawaveIvaratorMaxResultsException; +import datawave.query.exceptions.WaitWindowOverrunException; import datawave.query.iterator.CachingIterator; import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.profile.QuerySpan; import datawave.query.iterator.profile.QuerySpanCollector; import datawave.query.iterator.profile.SourceTrackingIterator; +import datawave.query.iterator.waitwindow.WaitWindowObserver; import datawave.query.predicate.TimeFilter; import datawave.query.util.TypeMetadata; import datawave.query.util.sortedset.FileKeySortedSet; @@ -72,6 +77,8 @@ public abstract class DatawaveFieldIndexCachingIteratorJexl extends WrappingIter public abstract static class Builder> { private String queryId; + private String scanId; + private WaitWindowObserver waitWindowObserver; private Text fieldName; protected Text fieldValue; private Predicate datatypeFilter; @@ -108,6 +115,16 @@ public B withQueryId(String queryId) { return self(); } + public B withScanId(String scanId) { + this.scanId = scanId; + return self(); + } + + public B withWaitWindowObserver(WaitWindowObserver waitWindowObserver) { + this.waitWindowObserver = waitWindowObserver; + return self(); + } + public B withFieldName(Text fieldName) { this.fieldName = fieldName; return self(); @@ -253,6 +270,10 @@ public B withIvaratorSourcePool(GenericObjectPool> ivaratorSourcePool = null; + private AtomicBoolean running = new AtomicBoolean(false); // ------------------------------------------------------------------------- // ------------- Constructors @@ -363,6 +382,8 @@ public B withIvaratorSourcePool(GenericObjectPool columnFamilies, boolean inclu // if we are not sorting UIDs, then determine whether we have a cq and capture the lastFiKey Key lastFiKey = null; - if (!sortedUIDs && r.getStartKey().getColumnFamily().getLength() > 0 && r.getStartKey().getColumnQualifier().getLength() > 0) { - Key startKey = r.getStartKey(); - String cq = startKey.getColumnQualifier().toString(); - int fieldnameIndex = cq.indexOf('\0'); - if (fieldnameIndex >= 0) { - String cf = startKey.getColumnFamily().toString(); - lastFiKey = new Key(startKey.getRow().toString(), "fi\0" + cq.substring(0, fieldnameIndex), - cq.substring(fieldnameIndex + 1) + '\0' + cf + '\0'); + Key startKey = r.getStartKey(); + if (!sortedUIDs) { + String cq = WaitWindowObserver.removeMarkers(startKey.getColumnQualifier()).toString(); + if (r.getStartKey().getColumnFamily().getLength() > 0 && cq.length() > 0) { + String cqSuffix = WaitWindowObserver.hasBeginMarker(startKey.getColumnQualifier()) ? "" : "\0"; + int fieldnameIndex = cq.indexOf('\0'); + if (fieldnameIndex >= 0) { + String cf = startKey.getColumnFamily().toString(); + lastFiKey = new Key(startKey.getRow().toString(), "fi\0" + cq.substring(0, fieldnameIndex), + cq.substring(fieldnameIndex + 1) + '\0' + cf + cqSuffix); + } } } @@ -623,7 +651,9 @@ public void seek(Range r, Collection columnFamilies, boolean inclu log.trace("Starting in range: " + boundingFiRanges.get(0)); } Range boundingFiRange = boundingFiRanges.get(0); - boundingFiRange = new Range(lastFiKey, false, boundingFiRange.getEndKey(), boundingFiRange.isEndKeyInclusive()); + // default to startKeyInclusive = false unless we have yielded with begin marker + boolean startKeyInclusive = WaitWindowObserver.hasBeginMarker(startKey.getColumnQualifier()); + boundingFiRange = new Range(lastFiKey, startKeyInclusive, boundingFiRange.getEndKey(), boundingFiRange.isEndKeyInclusive()); boundingFiRanges.set(0, boundingFiRange); if (log.isTraceEnabled()) { log.trace("Reset range to: " + boundingFiRanges.get(0)); @@ -801,14 +831,6 @@ else if (key.compareTo(this.lastRangeSeeked.getStartKey()) < 0) { if (this.setControl.isCancelledQuery()) { this.topKey = null; - } - - if (isTimedOut()) { - log.error("Ivarator query timed out"); - throw new IvaratorException("Ivarator query timed out"); - } - - if (this.setControl.isCancelledQuery()) { log.debug("Ivarator query was cancelled"); throw new IterationInterruptedException("Ivarator query was cancelled"); } @@ -824,15 +846,9 @@ else if (key.compareTo(this.lastRangeSeeked.getStartKey()) < 0) { } if (this.setControl.isCancelledQuery()) { - if (isTimedOut()) { - log.error("Ivarator query timed out"); - throw new IvaratorException("Ivarator query timed out"); - } else { - log.debug("Ivarator query was cancelled"); - throw new IterationInterruptedException("Ivarator query was cancelled"); - } + log.debug("Ivarator query was cancelled"); + throw new IterationInterruptedException("Ivarator query was cancelled"); } - } } @@ -865,16 +881,34 @@ public boolean add(long val) { private void fillSortedSets() throws IOException { String sourceRow = this.fiRow.toString(); - setupRowBasedHdfsBackedSet(sourceRow); - - // if keys is not null, then we already had a completed set which was loaded in setupRowBasedHdfsBackedSet - if (keys != null) { - moveToNextRow(); - return; + // if we are running fillSortedSets as port of a re-seek after yield, then the fillSet threads would not have + // completed when a WaitWindowOverrunException was thrown and therefore the set would not have been marked as + // complete when setupRowBasedHdfsBackedSet was called at the top of this method. We will try to copy the + // RowBasedHdfsBackedSet from the previously used Ivarator since there may be unpersisted results or + // a long-running Ivarator may still be filling the RowBasedHdfsBackedSet + // If this re-seek is from an Ivarator yielding, the startKey will have a yield marker in either the + // colFam (for a shard range) or colQual (for a document range) + boolean usePreviousSortedSet = WaitWindowObserver.hasMarker(lastRangeSeeked.getStartKey()); + if (usePreviousSortedSet) { + if (!copyRowBasedHdfsBackedSetFromPreviousCall()) { + // if copying from the previous call fails, then ensure that the previous IvaratorFutures are removed + // and set usePreviousSortedSet to false to force setup of a new HDFS backed set + usePreviousSortedSet = false; + for (Range r : boundingFiRanges) { + IteratorThreadPoolManager.removeIvaratorFuture(getIvaratorTaskName(r), initEnv); + } + } + } + if (!usePreviousSortedSet) { + setupRowBasedHdfsBackedSet(sourceRow); + // if keys is not null, then we already had a completed set which was loaded in setupRowBasedHdfsBackedSet + if (keys != null) { + moveToNextRow(); + return; + } } - // for each range, fork off a runnable - List> futures = new ArrayList<>(boundingFiRanges.size()); + List futures = new ArrayList<>(boundingFiRanges.size()); if (log.isDebugEnabled()) { log.debug("Processing " + boundingFiRanges + " for " + this); } @@ -885,33 +919,44 @@ private void fillSortedSets() throws IOException { if (log.isTraceEnabled()) { log.trace("range -> " + range); } + // For each range, get either a new or pre-existing IvaratorFuture futures.add(fillSet(range, totalResults)); } boolean failed = false; Exception exception = null; Object result = null; - - // wait for all of the threads to complete - for (Future future : futures) { - checkTiming(); - - if (failed || this.setControl.isCancelledQuery()) { - future.cancel(false); - } else { - try { - result = future.get(); - } catch (Exception e) { - exception = e; - result = e; - } - if (result != null) { - failed = true; - this.setControl.setCancelled(); + Key yieldKey = null; + try { + // wait for all of the threads to complete + for (IvaratorFuture future : futures) { + if (!failed && !this.setControl.isCancelledQuery()) { + try { + result = future.get(this.waitWindowObserver.remainingTimeMs(), TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + // If the remaining time on the WaitWindowObserver has passed, then throw + // a WaitWindowOverrunException to yield at the startKey of the seekRange + yieldKey = this.waitWindowObserver.createYieldKey(lastRangeSeeked.getStartKey(), true); + throw new WaitWindowOverrunException(yieldKey); + } catch (Exception e) { + exception = e; + result = e; + } + if (result != null) { + failed = true; + this.setControl.setCancelled(); + } } } - if (this.setControl.isCancelledQuery()) { - break; + } finally { + if (yieldKey == null) { + // Whether we succeeded or failed, we should remove all Futures associated with this Ivarator + // The only exception is if we interrupted the fillSortedSets with a WaitWindowOverrunException + // in which case we want the IvaratorFutures to remain so that we can reconnect the HDFSBackedSortedSet + // on the next call + for (Range range : boundingFiRanges) { + IteratorThreadPoolManager.removeIvaratorFuture(getIvaratorTaskName(range), this.initEnv); + } } } @@ -920,6 +965,11 @@ private void fillSortedSets() throws IOException { throw new IvaratorException("Failed to complete ivarator cache: " + result, exception); } + if (usePreviousSortedSet) { + forcePersistence(); + setupRowBasedHdfsBackedSet(sourceRow); + } + // now reset the current source to the next viable range moveToNextRow(); } @@ -982,27 +1032,6 @@ protected void startTiming() { startTime = System.currentTimeMillis(); } - /** - * Check if the scan timeout has been reached. Mark as timed out and cancel the query if so. - */ - protected void checkTiming() { - if (System.currentTimeMillis() > (startTime + scanTimeout)) { - // mark as timed out - this.timedOut = true; - // and cancel the query - this.setControl.setCancelled(); - } - } - - /** - * Was the timed out flag set. - * - * @return a boolean if timed out - */ - protected boolean isTimedOut() { - return this.timedOut; - } - /** * Get a source copy. This is only used when retrieving unsorted values. * @@ -1085,6 +1114,32 @@ protected boolean addKey(Key topFiKey, Value value) throws IOException { return false; } + public boolean isRunning() { + return running.get(); + } + + public long getScanTimeout() { + return scanTimeout; + } + + public long getStartTime() { + return startTime; + } + + public void waitUntilComplete() { + if (running.get()) { + synchronized (running) { + while (running.get()) { + try { + running.wait(); + } catch (InterruptedException e) { + + } + } + } + } + } + /** * This method will asynchronously fill the set with matches from within the specified bounding FI range. * @@ -1094,13 +1149,14 @@ protected boolean addKey(Key topFiKey, Value value) throws IOException { * total results * @return the Future */ - protected Future fillSet(final Range boundingFiRange, final TotalResults totalResults) { + protected IvaratorFuture fillSet(final Range boundingFiRange, final TotalResults totalResults) { // this will block until an ivarator source becomes available final SortedKeyValueIterator source = takePoolSource(); // create runnable Runnable runnable = () -> { + running.set(true); if (log.isDebugEnabled()) { log.debug("Starting fillSet(" + boundingFiRange + ')'); } @@ -1125,8 +1181,6 @@ protected Future fillSet(final Range boundingFiRange, final TotalResults tota : null; while (source.hasTop()) { - checkTiming(); - Key top = source.getTopKey(); // if we are setup for composite seeking, seek if we are out of range @@ -1220,11 +1274,23 @@ protected Future fillSet(final Range boundingFiRange, final TotalResults tota if (collectTimingDetails && querySpanCollector != null && querySpan != null) { querySpanCollector.addQuerySpan(querySpan); } + synchronized (running) { + running.set(false); + running.notify(); + } } }; - return IteratorThreadPoolManager.executeIvarator(runnable, DatawaveFieldIndexCachingIteratorJexl.this + " in " + boundingFiRange, this.initEnv); + return IteratorThreadPoolManager.executeIvarator(this, runnable, getIvaratorTaskName(boundingFiRange), this.initEnv); + } + public String getIvaratorTaskName(Range boundingFiRange) { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append(" in "); + sb.append(queryId); + sb.append('_').append(scanId); + sb.append('_').append(boundingFiRange); + return sb.toString(); } /** @@ -1252,6 +1318,43 @@ protected void clearRowBasedHdfsBackedSet() throws IOException { this.set = null; } + protected boolean copyRowBasedHdfsBackedSetFromPreviousCall() { + // Only copy previous Ivarator's RowBasedHdfsBackedSet if all futures are available + // and the Ivarator in the IvaratorFuture is the same object + DatawaveFieldIndexCachingIteratorJexl previousIvarator = null; + for (Range r : boundingFiRanges) { + IvaratorFuture f = IteratorThreadPoolManager.getIvaratorFuture(getIvaratorTaskName(r), initEnv); + if (f == null) { + return false; + } else { + if (previousIvarator == null) { + previousIvarator = f.getIvarator(); + } else { + if (previousIvarator != f.getIvarator()) { + return false; + } + } + } + } + + // ivaratorCacheDirs is declared final, but must be the same + Set currentUriSet = this.ivaratorCacheDirs.stream().map(d -> d.getPathURI()).collect(Collectors.toSet()); + Set previousUriSet = previousIvarator.ivaratorCacheDirs.stream().map(d -> d.getPathURI()).collect(Collectors.toSet()); + if (!currentUriSet.equals(previousUriSet)) { + return false; + } + + // collect timing details from the previous Ivarator and add them to this one + if (this.collectTimingDetails) { + this.querySpanCollector.addQuerySpan(previousIvarator.querySpanCollector.getCombinedQuerySpan(null)); + } + this.currentRow = previousIvarator.currentRow; + this.threadSafeSet = previousIvarator.threadSafeSet; + this.set = previousIvarator.set; + this.setControl = previousIvarator.setControl; + return true; + } + /** * This will setup the set for the specified range. This will attempt to reuse precomputed and persisted sets if we are allowed to. * diff --git a/warehouse/query-core/src/main/java/datawave/core/iterators/IteratorThreadPoolManager.java b/warehouse/query-core/src/main/java/datawave/core/iterators/IteratorThreadPoolManager.java index b0324f86ac..a183c3b795 100644 --- a/warehouse/query-core/src/main/java/datawave/core/iterators/IteratorThreadPoolManager.java +++ b/warehouse/query-core/src/main/java/datawave/core/iterators/IteratorThreadPoolManager.java @@ -1,8 +1,9 @@ package datawave.core.iterators; +import java.util.HashSet; import java.util.Map; import java.util.Objects; -import java.util.OptionalInt; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -16,6 +17,9 @@ import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.log4j.Logger; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; + /** * */ @@ -26,31 +30,88 @@ public class IteratorThreadPoolManager { private static final String EVALUATOR_THREAD_PROP = "tserver.datawave.evaluation.threads"; private static final String EVALUATOR_THREAD_NAME = "DATAWAVE Evaluation"; private static final int DEFAULT_THREAD_POOL_SIZE = 100; + private static final String IVARATOR_MAX_TIME_AFTER_ACCESS_PROP = "tserver.datawave.ivarator.maxTimeAfterAccess"; + private static final String IVARATOR_MAX_SCAN_TIMEOUT_PROP = "tserver.datawave.ivarator.maxScanTimeout"; + private static final long DEFAULT_IVARATOR_MAX_TIME_AFTER_ACCESS = TimeUnit.HOURS.toMillis(1); + private static final long DEFAULT_IVARATOR_MAX_SCAN_TIMEOUT = TimeUnit.HOURS.toMillis(1); private Map threadPools = new TreeMap<>(); - + private Cache ivaratorFutures; + // If an IvaratorFuture is not accessed within this time, then it will be removed from the cache + private long ivaratorMaxTimeAfterAccess; + // Each Ivarator has a scanTimeout. This is a system-wide limit which could be + // useful in causing all Ivarators to terminate if necessary + private long ivaratorMaxScanTimeout; private static final Object instanceSemaphore = new Object(); private static final String instanceId = Integer.toHexString(instanceSemaphore.hashCode()); private static volatile IteratorThreadPoolManager instance; private IteratorThreadPoolManager(IteratorEnvironment env) { - // create the thread pools - createExecutorService(IVARATOR_THREAD_PROP, IVARATOR_THREAD_NAME, env); - createExecutorService(EVALUATOR_THREAD_PROP, EVALUATOR_THREAD_NAME, env); - } - - private ThreadPoolExecutor createExecutorService(final String prop, final String name, IteratorEnvironment env) { - final AccumuloConfiguration accumuloConfiguration; + final AccumuloConfiguration config; if (env != null) { - accumuloConfiguration = env.getConfig(); + config = env.getConfig(); } else { - accumuloConfiguration = DefaultConfiguration.getInstance(); + config = DefaultConfiguration.getInstance(); } - final ThreadPoolExecutor service = createExecutorService(getMaxThreads(prop, accumuloConfiguration), name + " (" + instanceId + ')'); + // create the thread pools + createExecutorService(IVARATOR_THREAD_PROP, IVARATOR_THREAD_NAME, config); + createExecutorService(EVALUATOR_THREAD_PROP, EVALUATOR_THREAD_NAME, config); + ivaratorMaxTimeAfterAccess = getLongPropertyValue(IVARATOR_MAX_TIME_AFTER_ACCESS_PROP, DEFAULT_IVARATOR_MAX_TIME_AFTER_ACCESS, config); + log.info("Using " + ivaratorMaxTimeAfterAccess + " ms for " + IVARATOR_MAX_TIME_AFTER_ACCESS_PROP); + ivaratorMaxScanTimeout = getLongPropertyValue(IVARATOR_MAX_SCAN_TIMEOUT_PROP, DEFAULT_IVARATOR_MAX_SCAN_TIMEOUT, config); + log.info("Using " + ivaratorMaxScanTimeout + " ms for " + IVARATOR_MAX_SCAN_TIMEOUT_PROP); + // This thread will check for changes to ivaratorMaxTimeAfterAccess and ivaratorMaxScanTimeout + ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(config).scheduleWithFixedDelay(() -> { + try { + long max = getLongPropertyValue(IVARATOR_MAX_TIME_AFTER_ACCESS_PROP, DEFAULT_IVARATOR_MAX_TIME_AFTER_ACCESS, config); + if (ivaratorMaxTimeAfterAccess != max) { + log.info("Changing " + IVARATOR_MAX_TIME_AFTER_ACCESS_PROP + " to " + max + " ms"); + ivaratorMaxTimeAfterAccess = max; + } + } catch (Throwable t) { + log.error(t, t); + } + try { + long max = getLongPropertyValue(IVARATOR_MAX_SCAN_TIMEOUT_PROP, DEFAULT_IVARATOR_MAX_SCAN_TIMEOUT, config); + if (ivaratorMaxScanTimeout != max) { + log.info("Changing " + IVARATOR_MAX_SCAN_TIMEOUT_PROP + " to " + max + " ms"); + ivaratorMaxScanTimeout = max; + } + } catch (Throwable t) { + log.error(t, t); + } + }, 1, 10, TimeUnit.SECONDS); + + // If the IvaratorFuture has been not been accessed in ivaratorMaxTimeAfterAccess, then cancel and remove from the cache + ivaratorFutures = Caffeine.newBuilder().expireAfterAccess(ivaratorMaxTimeAfterAccess, TimeUnit.MILLISECONDS).evictionListener((t, f, removalCause) -> { + ((IvaratorFuture) f).cancel(true); + }).build(); + + // If Ivarator has been been running for a time greater than either its scanTimeout or the maxScanTimeout, + // then stop the Ivarator and remove the future from the cache + ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(config).scheduleWithFixedDelay(() -> { + Set tasks = new HashSet<>(ivaratorFutures.asMap().keySet()); + long now = System.currentTimeMillis(); + for (String taskName : tasks) { + IvaratorFuture future = getIvaratorFuture(taskName, env); + if (future != null) { + DatawaveFieldIndexCachingIteratorJexl ivarator = future.getIvarator(); + long elapsed = now - ivarator.getStartTime(); + long ivaratorScanTimeout = ivarator.getScanTimeout(); + if (!ivarator.isRunning() && ((elapsed > ivaratorScanTimeout) || (elapsed > ivaratorMaxScanTimeout))) { + removeIvaratorFuture(taskName, env); + } + } + } + }, 1, 60, TimeUnit.SECONDS); + } + + private ThreadPoolExecutor createExecutorService(final String prop, final String name, final AccumuloConfiguration config) { + final ThreadPoolExecutor service = createExecutorService(getIntPropertyValue(prop, DEFAULT_THREAD_POOL_SIZE, config), name + " (" + instanceId + ')'); threadPools.put(name, service); - ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(accumuloConfiguration).scheduleWithFixedDelay(() -> { + ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(config).scheduleWithFixedDelay(() -> { try { - int max = getMaxThreads(prop, accumuloConfiguration); + int max = getIntPropertyValue(prop, DEFAULT_THREAD_POOL_SIZE, config); if (service.getMaximumPoolSize() != max) { log.info("Changing " + prop + " to " + max); service.setMaximumPoolSize(max); @@ -70,15 +131,23 @@ private ThreadPoolExecutor createExecutorService(int maxThreads, String name) { return pool; } - private int getMaxThreads(final String prop, AccumuloConfiguration conf) { - if (conf != null) { + private int getIntPropertyValue(final String prop, int defaultValue, AccumuloConfiguration config) { + if (config != null) { Map properties = new TreeMap<>(); - conf.getProperties(properties, k -> Objects.equals(k, prop)); + config.getProperties(properties, k -> Objects.equals(k, prop)); if (properties.containsKey(prop)) { return Integer.parseInt(properties.get(prop)); } } - return DEFAULT_THREAD_POOL_SIZE; + return defaultValue; + } + + private long getLongPropertyValue(final String prop, long defaultValue, AccumuloConfiguration config) { + String valueStr = config.get(prop); + if (valueStr != null) { + return Long.parseLong(valueStr); + } + return defaultValue; } private static IteratorThreadPoolManager instance(IteratorEnvironment env) { @@ -104,12 +173,31 @@ private Future execute(String name, final Runnable task, final String taskNam }); } - public static Future executeIvarator(Runnable task, String taskName, IteratorEnvironment env) { - return instance(env).execute(IVARATOR_THREAD_NAME, task, taskName); + public static IvaratorFuture getIvaratorFuture(String taskName, IteratorEnvironment env) { + return instance(env).ivaratorFutures.getIfPresent(taskName); + } + + public static void removeIvaratorFuture(String taskName, IteratorEnvironment env) { + IvaratorFuture future = instance(env).ivaratorFutures.getIfPresent(taskName); + if (future != null) { + if (future.getIvarator().isRunning()) { + future.cancel(true); + future.getIvarator().waitUntilComplete(); + } + instance(env).ivaratorFutures.invalidate(taskName); + } + } + + public static IvaratorFuture executeIvarator(DatawaveFieldIndexCachingIteratorJexl ivarator, Runnable task, String taskName, IteratorEnvironment env) { + IvaratorFuture future = instance(env).ivaratorFutures.getIfPresent(taskName); + if (future == null) { + future = new IvaratorFuture(instance(env).execute(IVARATOR_THREAD_NAME, task, taskName), ivarator); + instance(env).ivaratorFutures.put(taskName, future); + } + return future; } public static Future executeEvaluation(Runnable task, String taskName, IteratorEnvironment env) { return instance(env).execute(EVALUATOR_THREAD_NAME, task, taskName); } - } diff --git a/warehouse/query-core/src/main/java/datawave/core/iterators/IvaratorFuture.java b/warehouse/query-core/src/main/java/datawave/core/iterators/IvaratorFuture.java new file mode 100644 index 0000000000..ea3c35401a --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/core/iterators/IvaratorFuture.java @@ -0,0 +1,46 @@ +package datawave.core.iterators; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class IvaratorFuture implements Future { + + final private Future future; + final private DatawaveFieldIndexCachingIteratorJexl ivarator; + + public IvaratorFuture(Future future, DatawaveFieldIndexCachingIteratorJexl ivarator) { + this.future = future; + this.ivarator = ivarator; + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return this.future.cancel(mayInterruptIfRunning); + } + + @Override + public boolean isCancelled() { + return this.future.isCancelled(); + } + + @Override + public boolean isDone() { + return this.future.isDone(); + } + + @Override + public Object get() throws InterruptedException, ExecutionException { + return this.future.get(); + } + + @Override + public Object get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + return this.future.get(timeout, unit); + } + + public DatawaveFieldIndexCachingIteratorJexl getIvarator() { + return ivarator; + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/attributes/WaitWindowExceededMetadata.java b/warehouse/query-core/src/main/java/datawave/query/attributes/WaitWindowExceededMetadata.java new file mode 100644 index 0000000000..dd79daed13 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/attributes/WaitWindowExceededMetadata.java @@ -0,0 +1,12 @@ +package datawave.query.attributes; + +/** + * Indicates that the scan session was ended after exceeding the wait window + */ +public class WaitWindowExceededMetadata extends Metadata { + + @Override + public int size() { + return 1; + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/exceptions/WaitWindowOverrunException.java b/warehouse/query-core/src/main/java/datawave/query/exceptions/WaitWindowOverrunException.java new file mode 100644 index 0000000000..f9487eaf1d --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/exceptions/WaitWindowOverrunException.java @@ -0,0 +1,36 @@ +package datawave.query.exceptions; + +import org.apache.accumulo.core.data.Key; + +public class WaitWindowOverrunException extends RuntimeException { + + private Key yieldKey; + + public WaitWindowOverrunException(Key yieldKey) { + this.yieldKey = yieldKey; + } + + public Key getYieldKey() { + return yieldKey; + } + + // disallow other constructors that don't include a yieldKey + private WaitWindowOverrunException() { + super(); + } + + // disallow constructors that don't include a yieldKey + private WaitWindowOverrunException(String message) { + super(message); + } + + // disallow constructors that don't include a yieldKey + private WaitWindowOverrunException(String message, Throwable cause) { + super(message, cause); + } + + // disallow constructors that don't include a yieldKey + private WaitWindowOverrunException(Throwable cause) { + super(cause); + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/function/KeyToDocumentData.java b/warehouse/query-core/src/main/java/datawave/query/function/KeyToDocumentData.java index 5e1e147b5f..ebde94e295 100644 --- a/warehouse/query-core/src/main/java/datawave/query/function/KeyToDocumentData.java +++ b/warehouse/query-core/src/main/java/datawave/query/function/KeyToDocumentData.java @@ -180,7 +180,7 @@ public Entry apply(Entry from) { logStop(keyRange.getStartKey()); return Maps.immutableEntry(new DocumentData(from.getKey(), docKeys, attrs, false), from.getValue()); } catch (IOException e) { - log.error("Unable to collection document attributes for evaluation: " + keyRange, e); + log.error("Unable to collect document attributes for evaluation: " + keyRange, e); QueryException qe = new QueryException(DatawaveErrorCode.DOCUMENT_EVALUATION_ERROR, e); throw new DatawaveFatalQueryException(qe); } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/EventDataScanNestedIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/EventDataScanNestedIterator.java index 3517e6b06b..7e007717f3 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/EventDataScanNestedIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/EventDataScanNestedIterator.java @@ -16,6 +16,7 @@ import com.google.common.base.Predicate; import datawave.query.attributes.Document; +import datawave.query.iterator.waitwindow.WaitWindowObserver; /** * @@ -51,7 +52,7 @@ protected Key nextStartKey(Key key) { @Override public Key move(Key minimum) { if (totalRange != null) { - Range newRange = totalRange; + Range newRange; if (totalRange.contains(minimum)) { newRange = new Range(minimum, true, totalRange.getEndKey(), totalRange.isEndKeyInclusive()); } else { @@ -112,7 +113,7 @@ public void seek(Range range, Collection columnFamilies, boolean i this.inclusive = inclusive; // determine if we have been torn down and rebuilt - if (!range.isInfiniteStartKey() && !range.isStartKeyInclusive()) { + if (!range.isInfiniteStartKey() && !range.isStartKeyInclusive() && !WaitWindowObserver.hasMarker(range.getStartKey())) { move(nextStartKey(range.getStartKey())); } else { source.seek(range, columnFamilies, inclusive); 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 7b39f0b719..c2b426e49a 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 @@ -96,6 +96,8 @@ import datawave.query.iterator.profile.QuerySpan; import datawave.query.iterator.profile.QuerySpanCollector; import datawave.query.iterator.profile.SourceTrackingIterator; +import datawave.query.iterator.waitwindow.WaitWindowObserver; +import datawave.query.iterator.waitwindow.WaitWindowOverseerIterator; import datawave.query.jexl.DatawaveJexlContext; import datawave.query.jexl.JexlASTHelper; import datawave.query.jexl.StatefulArithmetic; @@ -177,6 +179,7 @@ public class QueryIterator extends QueryOptions implements YieldingKeyValueItera protected Key key; protected Value value; protected YieldCallback yield; + protected WaitWindowObserver waitWindowObserver = new WaitWindowObserver(); protected IteratorEnvironment myEnvironment; @@ -222,6 +225,7 @@ public QueryIterator(QueryIterator other, IteratorEnvironment env) { this.typeMetadata = other.typeMetadata; this.exceededOrEvaluationCache = other.exceededOrEvaluationCache; this.trackingSpan = other.trackingSpan; + this.waitWindowObserver = other.waitWindowObserver; // Defer to QueryOptions to re-set all of the query options super.deepCopy(other); } @@ -240,6 +244,9 @@ public void init(SortedKeyValueIterator source, Map op throw new IllegalArgumentException("Could not initialize QueryIterator with " + options); } + // affects the way waitWindowObserver creates some yieldKeys + this.waitWindowObserver.setSortedUIDs(sortedUIDs); + // We want to add in spoofed dataTypes for Aggregation/Evaluation to // ensure proper numeric evaluation. this.typeMetadata = new TypeMetadata(this.getTypeMetadata()); @@ -262,7 +269,8 @@ public void init(SortedKeyValueIterator source, Map op if (gatherTimingDetails()) { this.trackingSpan = new MultiThreadedQuerySpan(getStatsdClient()); - this.source = new SourceTrackingIterator(trackingSpan, source); + this.waitWindowObserver.setTrackingSpan(this.trackingSpan); + this.source = new SourceTrackingIterator(this.trackingSpan, source); } else { this.source = source; } @@ -317,21 +325,32 @@ private boolean hasValidBasePath(IvaratorCacheDirConfig config) throws Interrupt @Override public boolean hasTop() { + this.waitWindowObserver.yieldOnOverrun(); boolean yielded = (this.yield != null) && this.yield.hasYielded(); boolean result = (!yielded) && (this.key != null) && (this.value != null); if (log.isTraceEnabled()) { log.trace("hasTop() " + result); } + if (result == false) { + // ensure that the timerTask is cancelled + this.waitWindowObserver.stop(); + } return result; } @Override public void enableYielding(YieldCallback yieldCallback) { this.yield = yieldCallback; + this.waitWindowObserver.setYieldCallback(yieldCallback); } @Override public void next() throws IOException { + // If collectTimingDetails, then we return an WAIT_WINDOW_OVERRUN first, then yield. + // In this case, we are ready to yield and the iterator stack is not in a state to iterate + if (this.waitWindowObserver.isReadyToYield()) { + return; + } getActiveQueryLog().get(getQueryId()).beginCall(this.originalRange, ActiveQuery.CallType.NEXT); try { if (log.isTraceEnabled()) { @@ -357,7 +376,8 @@ public void next() throws IOException { public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { // preserve the original range for use with the Final Document tracking iterator because it is placed after the ResultCountingIterator // so the FinalDocumentTracking iterator needs the start key with the count already appended - originalRange = range; + this.originalRange = range; + this.waitWindowObserver.start(range, yieldThresholdMs); getActiveQueryLog().get(getQueryId()).beginCall(this.originalRange, ActiveQuery.CallType.SEEK); ActiveQueryLog.getInstance().get(getQueryId()).beginCall(this.originalRange, ActiveQuery.CallType.SEEK); @@ -400,11 +420,9 @@ public void seek(Range range, Collection columnFamilies, boolean i // determine whether this is a document specific range Range documentRange = isDocumentSpecificRange(range) ? range : null; - - // if we have a document specific range, but the key is not - // inclusive then we have already returned the document; this scan - // is done - if (documentRange != null && !documentRange.isStartKeyInclusive()) { + // if we have a non-inclusive document specific range and this isn't a re-seeked document + // range after a yield then we have already returned the document and this scan is done + if (documentRange != null && !documentRange.isStartKeyInclusive() && !WaitWindowObserver.hasBeginMarker(documentRange.getStartKey())) { if (log.isTraceEnabled()) { log.trace("Received non-inclusive event specific range: " + documentRange); } @@ -442,7 +460,7 @@ else if (documentRange != null && (!this.isContainsIndexOnlyTerms() && this.getT // evaluation within a thread pool PipelineIterator pipelineIter = PipelineFactory.createIterator(this.seekKeySource, getMaxEvaluationPipelines(), getMaxPipelineCachedResults(), getSerialPipelineRequest(), querySpanCollector, trackingSpan, this, sourceForDeepCopies.deepCopy(myEnvironment), myEnvironment, - yield, yieldThresholdMs, columnFamilies, inclusive); + yield, yieldThresholdMs, waitWindowObserver, columnFamilies, inclusive); pipelineIter.setCollectTimingDetails(collectTimingDetails); // TODO pipelineIter.setStatsdHostAndPort(statsdHostAndPort); @@ -513,18 +531,22 @@ else if (documentRange != null && (!this.isContainsIndexOnlyTerms() && this.getT // now add the result count to the keys (required when not sorting UIDs) // Cannot do this on document specific ranges as the count would place the keys outside the initial range if (!sortedUIDs && documentRange == null) { - this.serializedDocuments = new ResultCountingIterator(serializedDocuments, resultCount, yield); + ResultCountingIterator resultCountingIterator = new ResultCountingIterator(serializedDocuments, resultCount, yield); + // this is necessary to allow the waitWindowObserver to use the same logic and count + // when modifying the yieldKey + this.waitWindowObserver.setResultCountingIterator(resultCountingIterator); + this.serializedDocuments = resultCountingIterator; } else if (this.sortedUIDs) { // we have sorted UIDs, so we can mask out the cq - this.serializedDocuments = new KeyAdjudicator<>(serializedDocuments, yield); + this.serializedDocuments = new KeyAdjudicator<>(serializedDocuments); } // only add the final document tracking iterator which sends stats back to the client if collectTimingDetails is true if (collectTimingDetails) { // if there is no document to return, then add an empty document // to store the timing metadata - this.serializedDocuments = new FinalDocumentTrackingIterator(querySpanCollector, trackingSpan, originalRange, this.serializedDocuments, - this.getReturnType(), this.isReducedResponse(), this.isCompressResults(), this.yield); + this.serializedDocuments = new FinalDocumentTrackingIterator(querySpanCollector, trackingSpan, waitWindowObserver, originalRange, + this.serializedDocuments, this.getReturnType(), this.isReducedResponse(), this.isCompressResults(), this.yield); } if (log.isTraceEnabled()) { KryoDocumentDeserializer dser = new KryoDocumentDeserializer(); @@ -1153,14 +1175,19 @@ private void prepareKeyValue() { this.key = entry.getKey(); this.value = entry.getValue(); - } else { - if (log.isTraceEnabled()) { + if (log.isTraceEnabled() && (this.yield == null || !this.yield.hasYielded())) { log.trace("Exhausted all keys"); } this.key = null; this.value = null; } + if (this.yield != null && this.yield.hasYielded()) { + // ensure that the timerTask is cancelled + this.waitWindowObserver.stop(); + this.key = null; + this.value = null; + } } @Override @@ -1386,7 +1413,7 @@ protected NestedIterator getOrSetKeySource(final Range documentRange, ASTJe sourceIter = getEventDataNestedIterator(source); } - return sourceIter; + return new WaitWindowOverseerIterator(sourceIter, this.myEnvironment); } /** @@ -1433,6 +1460,7 @@ protected IteratorBuildingVisitor createIteratorBuildingVisitor(Class next() { return next; } - private Key addKeyCount(Key key) { + public Key addKeyCount(Key key) { resultCount.getAndIncrement(); return new Key(key.getRow(), new Text(NumericalEncoder.encode(Long.toString(resultCount.get())) + '\0' + key.getColumnFamily()), key.getColumnQualifier(), key.getColumnVisibility(), key.getTimestamp()); diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AbstractIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AbstractIteratorBuilder.java index 5899b14516..c9242045e3 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AbstractIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AbstractIteratorBuilder.java @@ -9,6 +9,7 @@ import com.google.common.collect.HashMultimap; import datawave.query.iterator.NestedIterator; +import datawave.query.iterator.waitwindow.WaitWindowObserver; /** * Provides semantics for adding sources to a nested iterator but deferring the creation of iterator. This is meant to be used in a visitor. @@ -29,6 +30,10 @@ public abstract class AbstractIteratorBuilder implements IteratorBuilder { protected String queryId; + protected String scanId; + + protected WaitWindowObserver waitWindowObserver; + public boolean isSortedUIDs() { return sortedUIDs; } @@ -115,6 +120,14 @@ public void setQueryId(String queryId) { this.queryId = queryId; } + public void setScanId(String scanId) { + this.scanId = scanId; + } + + public void setWaitWindowObserver(WaitWindowObserver waitWindowObserver) { + this.waitWindowObserver = waitWindowObserver; + } + /** * Checks to see if a given {@code } mapping has been observed before by this builder. This method is intended to only be called when a * visitor is building an IndexIterator. diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AndIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AndIteratorBuilder.java index 1396f70d9c..e04f9c2dc1 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AndIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/AndIteratorBuilder.java @@ -29,6 +29,6 @@ public NestedIterator build() { if (includes.isEmpty()) { throw new IllegalStateException("AndIterator has no inclusive sources!"); } - return new AndIterator(includes, excludes); + return new AndIterator(includes, excludes, waitWindowObserver); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexFilterIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexFilterIteratorBuilder.java index f66b0fde3a..e578422a2b 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexFilterIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexFilterIteratorBuilder.java @@ -91,6 +91,9 @@ public NestedIterator build() { .withTypeMetadata(typeMetadata) .withIteratorEnv(env) .withIvaratorSourcePool(ivaratorSourcePool) + .withQueryId(queryId) + .withScanId(scanId) + .withWaitWindowObserver(waitWindowObserver) .build(); // @formatter:on diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexListIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexListIteratorBuilder.java index a4a08139d8..3db1bcfdbb 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexListIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexListIteratorBuilder.java @@ -97,7 +97,10 @@ public NestedIterator build() { .withCompositeSeekThreshold(compositeSeekThreshold) .withTypeMetadata(typeMetadata) .withIvaratorSourcePool(ivaratorSourcePool) - .withIteratorEnv(env); + .withIteratorEnv(env) + .withQueryId(queryId) + .withScanId(scanId) + .withWaitWindowObserver(waitWindowObserver); // @formatter:on if (values != null) { builder = builder.withValues(values); diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRangeIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRangeIteratorBuilder.java index b9ec35d23a..853d81bb36 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRangeIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRangeIteratorBuilder.java @@ -94,6 +94,9 @@ public NestedIterator build() { .withSubRanges(subRanges) .withIteratorEnv(env) .withIvaratorSourcePool(ivaratorSourcePool) + .withQueryId(queryId) + .withScanId(scanId) + .withWaitWindowObserver(waitWindowObserver) .build(); // @formatter:on diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRegexIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRegexIteratorBuilder.java index fcf21ab21f..5d4ca09c2c 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRegexIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/IndexRegexIteratorBuilder.java @@ -74,6 +74,9 @@ public NestedIterator build() { .withTypeMetadata(typeMetadata) .withIteratorEnv(env) .withIvaratorSourcePool(ivaratorSourcePool) + .withQueryId(queryId) + .withScanId(scanId) + .withWaitWindowObserver(waitWindowObserver) .build(); // @formatter:on if (collectTimingDetails) { diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/OrIteratorBuilder.java b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/OrIteratorBuilder.java index 3c291ef950..05988a5916 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/builder/OrIteratorBuilder.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/builder/OrIteratorBuilder.java @@ -8,7 +8,7 @@ public class OrIteratorBuilder extends AbstractIteratorBuilder { @SuppressWarnings({"rawtypes", "unchecked"}) @Override public NestedIterator build() { - return new OrIterator(includes, excludes); + return new OrIterator(includes, excludes, waitWindowObserver); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/logic/AndIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/logic/AndIterator.java index 144e10bded..271ba24d9c 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/logic/AndIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/logic/AndIterator.java @@ -13,16 +13,19 @@ import java.util.SortedSet; import org.apache.accumulo.core.data.ByteSequence; +import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.log4j.Logger; import com.google.common.collect.TreeMultimap; import datawave.query.attributes.Document; +import datawave.query.exceptions.WaitWindowOverrunException; import datawave.query.iterator.NestedIterator; import datawave.query.iterator.SeekableIterator; import datawave.query.iterator.Util; import datawave.query.iterator.Util.Transformer; +import datawave.query.iterator.waitwindow.WaitWindowObserver; /** * Performs a merge join of the child iterators. It is expected that all child iterators return values in sorted order. @@ -31,6 +34,7 @@ public class AndIterator> implements NestedIterator, // temporary stores of uninitialized streams of iterators private List> includes, excludes, contextIncludes, contextExcludes; + private WaitWindowObserver waitWindowObserver; private Map transforms; private Transformer transformer; @@ -45,10 +49,15 @@ public class AndIterator> implements NestedIterator, private static final Logger log = Logger.getLogger(AndIterator.class); public AndIterator(Iterable> sources) { - this(sources, null); + this(sources, null, null); } public AndIterator(Iterable> sources, Iterable> filters) { + this(sources, filters, null); + } + + public AndIterator(Iterable> sources, Iterable> filters, WaitWindowObserver waitWindowObserver) { + this.waitWindowObserver = waitWindowObserver; includes = new LinkedList<>(); contextIncludes = new LinkedList<>(); for (NestedIterator src : sources) { @@ -80,31 +89,44 @@ public void initialize() { // nestedIteratorComparator will keep a deterministic ordering, unlike hashCodeComparator Comparator> itrComp = Util.nestedIteratorComparator(); - transformer = Util.keyTransformer(); - transforms = new HashMap<>(); + try { - includeHeads = TreeMultimap.create(keyComp, itrComp); - includeHeads = initSubtree(includeHeads, includes, transformer, transforms, true); + transformer = Util.keyTransformer(); + transforms = new HashMap<>(); - if (excludes.isEmpty()) { - excludeHeads = Util.getEmpty(); - } else { - excludeHeads = TreeMultimap.create(keyComp, itrComp); - // pass null in for transforms as excludes are not returned - excludeHeads = initSubtree(excludeHeads, excludes, transformer, null, false); - } + includeHeads = TreeMultimap.create(keyComp, itrComp); + includeHeads = initSubtree(includeHeads, includes, transformer, transforms, true); - if (!contextIncludes.isEmpty()) { - contextIncludeHeads = TreeMultimap.create(keyComp, itrComp); - contextIncludeNullHeads = TreeMultimap.create(keyComp, itrComp); - } + if (excludes.isEmpty()) { + excludeHeads = Util.getEmpty(); + } else { + excludeHeads = TreeMultimap.create(keyComp, itrComp); + // pass null in for transforms as excludes are not returned + excludeHeads = initSubtree(excludeHeads, excludes, transformer, null, false); + } - if (contextExcludes != null && !contextExcludes.isEmpty()) { - contextExcludeHeads = TreeMultimap.create(keyComp, itrComp); - contextExcludeNullHeads = TreeMultimap.create(keyComp, itrComp); - } + if (!contextIncludes.isEmpty()) { + contextIncludeHeads = TreeMultimap.create(keyComp, itrComp); + contextIncludeNullHeads = TreeMultimap.create(keyComp, itrComp); + } - next(); + if (contextExcludes != null && !contextExcludes.isEmpty()) { + contextExcludeHeads = TreeMultimap.create(keyComp, itrComp); + contextExcludeNullHeads = TreeMultimap.create(keyComp, itrComp); + } + + next(); + } catch (WaitWindowOverrunException e) { + // if prev != null then it's a match that has not been returned + T highest = null; + if (!includeHeads.isEmpty()) { + highest = includeHeads.keySet().last(); + } + Key possibleYieldKey = (prev != null) ? (Key) prev : (Key) highest; + // When comparing possible yield keys in the AndIterator, we choose the highest + // key because the uids of the sources need to be equal to return a match + this.waitWindowObserver.propagateException(possibleYieldKey, true, false, e); + } } public boolean isInitialized() { @@ -168,63 +190,75 @@ public T next() { prev = next; prevDocument = document; + T lowest = null; + T highest = null; - // look through includes for candidates if there are any - while (!includeHeads.isEmpty()) { - SortedSet topKeys = includeHeads.keySet(); - T lowest = topKeys.first(); - T highest = topKeys.last(); - - // short circuit if possible from a supplied evaluation context - if (evaluationContext != null) { - int lowestCompare = lowest.compareTo(evaluationContext); - int highestCompare = highest.compareTo(evaluationContext); - - if (lowestCompare > 0 || highestCompare > 0) { - // if any value is beyond the evaluationContext it's not possible to intersect - break; - } + try { + // look through includes for candidates if there are any + while (!includeHeads.isEmpty()) { + SortedSet topKeys = includeHeads.keySet(); + lowest = topKeys.first(); + highest = topKeys.last(); + + // short circuit if possible from a supplied evaluation context + if (evaluationContext != null) { + checkWaitWindow(evaluationContext); + int lowestCompare = lowest.compareTo(evaluationContext); + int highestCompare = highest.compareTo(evaluationContext); + + if (lowestCompare > 0 || highestCompare > 0) { + // if any value is beyond the evaluationContext it's not possible to intersect + break; + } - // advance anything less than the evaluation context to the evaluation context - SortedSet toMove = topKeys.headSet(evaluationContext); - if (!toMove.isEmpty()) { - includeHeads = moveIterators(toMove, evaluationContext); - continue; + // advance anything less than the evaluation context to the evaluation context + SortedSet toMove = topKeys.headSet(evaluationContext); + if (!toMove.isEmpty()) { + includeHeads = moveIterators(toMove, evaluationContext); + continue; + } } - } - - // if the highest and lowest are the same we are currently intersecting - if (lowest.equals(highest)) { - // make sure this value isn't filtered - if (!NegationFilter.isFiltered(lowest, excludeHeads, transformer)) { - // use this value as a candidate against any includes/excludes that require context - if (applyContextRequired(lowest)) { - // found a match, set next/document and advance - next = transforms.get(lowest); - document = Util.buildNewDocument(includeHeads.values()); + checkWaitWindow(highest); + // if the highest and lowest are the same we are currently intersecting + if (lowest.equals(highest)) { + // make sure this value isn't filtered + if (!NegationFilter.isFiltered(lowest, excludeHeads, transformer)) { + // use this value as a candidate against any includes/excludes that require context + if (applyContextRequired(lowest)) { + // found a match, set next/document and advance + next = transforms.get(lowest); + document = Util.buildNewDocument(includeHeads.values()); + includeHeads = advanceIterators(lowest); + break; + } + } else { + // filtered, advance the iterators (which are all currently pointing at the same point) includeHeads = advanceIterators(lowest); - break; } } else { - // filtered, advance the iterators (which are all currently pointing at the same point) - includeHeads = advanceIterators(lowest); + // haven't converged yet, take the next highest and move it + T nextHighest = topKeys.headSet(highest).last(); + includeHeads = moveIterators(nextHighest, highest); } - } else { - // haven't converged yet, take the next highest and move it - T nextHighest = topKeys.headSet(highest).last(); - includeHeads = moveIterators(nextHighest, highest); } - } - // for cases where there are no sources the only source for a candidate is the evaluationContext. - if (isContextRequired()) { - // test exclude for the candidate in case there are excludes - if (!NegationFilter.isFiltered(evaluationContext, excludeHeads, transformer)) { - if (applyContextRequired(evaluationContext)) { - next = evaluationContext; - document = Util.buildNewDocument(Collections.emptyList()); + // for cases where there are no sources the only source for a candidate is the evaluationContext. + if (isContextRequired()) { + checkWaitWindow(evaluationContext); + // test exclude for the candidate in case there are excludes + if (!NegationFilter.isFiltered(evaluationContext, excludeHeads, transformer)) { + if (applyContextRequired(evaluationContext)) { + next = evaluationContext; + document = Util.buildNewDocument(Collections.emptyList()); + } } } + } catch (WaitWindowOverrunException e) { + // if prev != null then it's a match that has not been returned + Key possibleYieldKey = (prev != null) ? (Key) prev : (Key) highest; + // When comparing possible yield keys in the AndIterator, we choose the highest + // key because the uids of the sources need to be equal to return a match + this.waitWindowObserver.propagateException(possibleYieldKey, true, false, e); } // if we didn't move after the loop, then we don't have a next after this @@ -235,6 +269,16 @@ public T next() { return prev; } + private void checkWaitWindow(T key) { + if (this.waitWindowObserver != null) { + Key yieldKey = (prev != null) ? (Key) prev : (Key) key; + if (yieldKey != null) { + // YIELD_AT_BEGIN because neither prev nor key have been returned yet + this.waitWindowObserver.checkWaitWindow(yieldKey, true); + } + } + } + public void remove() { throw new UnsupportedOperationException("This iterator does not support remove."); } @@ -259,6 +303,10 @@ public void seek(Range range, Collection columnFamilies, boolean i ((SeekableIterator) itr).seek(range, columnFamilies, inclusive); } } + } catch (WaitWindowOverrunException e) { + // When comparing possible yield keys in the AndIterator, we choose the highest + // key because the uids of the sources need to be equal to return a match + this.waitWindowObserver.propagateException(null, true, false, e); } catch (Exception e) { include.remove(); if (includes.isEmpty()) { diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/logic/IndexIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/logic/IndexIterator.java index e7ea5e8470..9406f31134 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/logic/IndexIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/logic/IndexIterator.java @@ -25,6 +25,7 @@ import datawave.query.iterator.DocumentIterator; import datawave.query.iterator.LimitedSortedKeyValueIterator; import datawave.query.iterator.Util; +import datawave.query.iterator.waitwindow.WaitWindowObserver; import datawave.query.jexl.functions.FieldIndexAggregator; import datawave.query.jexl.functions.IdentityAggregator; import datawave.query.predicate.SeekingFilter; @@ -424,10 +425,12 @@ protected void seek(SortedKeyValueIterator source, Range r) throws IO * @return the field index range */ protected Range buildIndexRange(Range r) { - Key startKey = permuteRangeKey(r.getStartKey(), r.isStartKeyInclusive()); + // include startKey if we yielded to the beginning of a document range + boolean includeStartKey = WaitWindowObserver.hasBeginMarker(r.getStartKey().getColumnQualifier()) ? true : r.isStartKeyInclusive(); + Key startKey = permuteRangeKey(r.getStartKey(), includeStartKey); Key endKey = permuteRangeKey(r.getEndKey(), r.isEndKeyInclusive()); - return new Range(startKey, r.isStartKeyInclusive(), endKey, r.isEndKeyInclusive()); + return new Range(startKey, includeStartKey, endKey, r.isEndKeyInclusive()); } /** diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/logic/OrIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/logic/OrIterator.java index 3a5e780f47..ae682e15af 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/logic/OrIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/logic/OrIterator.java @@ -13,11 +13,15 @@ import java.util.SortedSet; import java.util.TreeSet; +import org.apache.accumulo.core.data.Key; + import com.google.common.collect.TreeMultimap; import datawave.query.attributes.Document; +import datawave.query.exceptions.WaitWindowOverrunException; import datawave.query.iterator.NestedIterator; import datawave.query.iterator.Util; +import datawave.query.iterator.waitwindow.WaitWindowObserver; /** * Performs a deduping merge of iterators. @@ -38,15 +42,21 @@ public class OrIterator> implements NestedIterator { private T prev; private T next; + private WaitWindowObserver waitWindowObserver; private Document prevDocument, document; private T evaluationContext; public OrIterator(Iterable> sources) { - this(sources, null); + this(sources, null, null); } public OrIterator(Iterable> sources, Iterable> filters) { + this(sources, filters, null); + } + + public OrIterator(Iterable> sources, Iterable> filters, WaitWindowObserver waitWindowObserver) { + this.waitWindowObserver = waitWindowObserver; includes = new LinkedList<>(); contextIncludes = new LinkedList<>(); for (NestedIterator src : sources) { @@ -79,20 +89,32 @@ public void initialize() { transformer = Util.keyTransformer(); transforms = new HashMap<>(); - includeHeads = TreeMultimap.create(keyComp, itrComp); - initSubtree(includeHeads, includes, transformer, transforms, false); + try { + includeHeads = TreeMultimap.create(keyComp, itrComp); + initSubtree(includeHeads, includes, transformer, transforms, false); - if (contextIncludes.size() > 0) { - contextIncludeHeads = TreeMultimap.create(keyComp, itrComp); - contextIncludeNullHeads = TreeMultimap.create(keyComp, itrComp); - } + if (contextIncludes.size() > 0) { + contextIncludeHeads = TreeMultimap.create(keyComp, itrComp); + contextIncludeNullHeads = TreeMultimap.create(keyComp, itrComp); + } - if (contextExcludes.size() > 0) { - contextExcludeHeads = TreeMultimap.create(keyComp, itrComp); - contextExcludeNullHeads = TreeMultimap.create(keyComp, itrComp); - } + if (contextExcludes.size() > 0) { + contextExcludeHeads = TreeMultimap.create(keyComp, itrComp); + contextExcludeNullHeads = TreeMultimap.create(keyComp, itrComp); + } - next(); + next(); + } catch (WaitWindowOverrunException e) { + // if prev != null then it's a match that has not been returned + T lowest = includeHeads.keySet().first(); + Key possibleYieldKey = null; + if (prev != null || lowest != null) { + possibleYieldKey = (prev != null) ? (Key) prev : (Key) lowest; + } + // When comparing possible yield keys in the OrIterator, we choose the lowest + // key because a match in any source is a match + waitWindowObserver.propagateException(possibleYieldKey, true, true, e); + } } public boolean hasNext() { @@ -119,56 +141,68 @@ public T next() { SortedSet candidateSet = new TreeSet<>(Util.keyComparator()); T lowest; - if (includeHeads.keySet().size() > 0) { - lowest = includeHeads.keySet().first(); - candidateSet.add(lowest); - } + try { + if (includeHeads.keySet().size() > 0) { + lowest = includeHeads.keySet().first(); + candidateSet.add(lowest); + } - T lowestContextInclude = null; - if (evaluationContext != null) { - if (contextIncludes.size() > 0) { - // get the lowest union and add it for contextRequiredIncludes - lowestContextInclude = NestedIteratorContextUtil.union(evaluationContext, contextIncludes, contextIncludeHeads, contextIncludeNullHeads, - transformer); - if (lowestContextInclude != null) { - candidateSet.add(lowestContextInclude); + T lowestContextInclude = null; + if (evaluationContext != null) { + if (contextIncludes.size() > 0) { + // get the lowest union and add it for contextRequiredIncludes + lowestContextInclude = NestedIteratorContextUtil.union(evaluationContext, contextIncludes, contextIncludeHeads, contextIncludeNullHeads, + transformer); + if (lowestContextInclude != null) { + candidateSet.add(lowestContextInclude); + } } - } - if (contextExcludes.size() > 0) { - // DeMorgan's Law: (~A) OR (~B) == ~(A AND B) - // for an exclude intersect the evaluation context with the set and then as long as the result doesn't match it is a candidate - T intersectExclude = NestedIteratorContextUtil.intersect(evaluationContext, contextExcludes, contextExcludeHeads, contextExcludeNullHeads, - transformer); - if (!evaluationContext.equals(intersectExclude)) { - candidateSet.add(evaluationContext); + if (contextExcludes.size() > 0) { + // DeMorgan's Law: (~A) OR (~B) == ~(A AND B) + // for an exclude intersect the evaluation context with the set and then as long as the result doesn't match it is a candidate + T intersectExclude = NestedIteratorContextUtil.intersect(evaluationContext, contextExcludes, contextExcludeHeads, contextExcludeNullHeads, + transformer); + if (!evaluationContext.equals(intersectExclude)) { + candidateSet.add(evaluationContext); + } } } - } - // take the lowest of the candidates - if (candidateSet.size() > 0) { - lowest = candidateSet.first(); - - // decide how to construct the document - if (lowest.equals(lowestContextInclude)) { - // build it from the contextIncludeHeads - next = lowestContextInclude; - document = Util.buildNewDocument(contextIncludeHeads.get(next)); - } else if (includeHeads.keySet().size() > 0 && lowest.equals(includeHeads.keySet().first())) { - // build it from the includeHeads - next = transforms.get(lowest); - document = Util.buildNewDocument(includeHeads.get(lowest)); - } else { - // nothing to build it from all we know is that it wasn't in the exclude set - next = evaluationContext; - document = Util.buildNewDocument(Collections.emptyList()); - } + // take the lowest of the candidates + if (candidateSet.size() > 0) { + lowest = candidateSet.first(); + checkWaitWindow(lowest); + + // decide how to construct the document + if (lowest.equals(lowestContextInclude)) { + // build it from the contextIncludeHeads + next = lowestContextInclude; + document = Util.buildNewDocument(contextIncludeHeads.get(next)); + } else if (includeHeads.keySet().size() > 0 && lowest.equals(includeHeads.keySet().first())) { + // build it from the includeHeads + next = transforms.get(lowest); + document = Util.buildNewDocument(includeHeads.get(lowest)); + } else { + // nothing to build it from all we know is that it wasn't in the exclude set + next = evaluationContext; + document = Util.buildNewDocument(Collections.emptyList()); + } - // regardless of where we hit make sure to advance includeHeads if it matches there - if (includeHeads != null && includeHeads.containsKey(lowest)) { - includeHeads = advanceIterators(lowest); + // regardless of where we hit make sure to advance includeHeads if it matches there + if (includeHeads != null && includeHeads.containsKey(lowest)) { + includeHeads = advanceIterators(lowest); + } + } + } catch (WaitWindowOverrunException e) { + // if prev != null then it's a match that has not been returned + Key possibleYieldKey = null; + if (prev != null || !candidateSet.isEmpty()) { + possibleYieldKey = (prev != null) ? (Key) prev : (Key) candidateSet.first(); } + // When comparing possible yield keys in the OrIterator, we choose the lowest + // key because a match in any sources can return a match + waitWindowObserver.propagateException(possibleYieldKey, true, true, e); } // the loop couldn't find a new next, so set next to null because we're done after this @@ -179,6 +213,16 @@ public T next() { return prev; } + private void checkWaitWindow(T key) { + if (this.waitWindowObserver != null) { + Key yieldKey = (prev != null) ? (Key) prev : (Key) key; + if (yieldKey != null) { + // YIELD_AT_BEGIN because neither prev nor key have been returned yet + this.waitWindowObserver.checkWaitWindow(yieldKey, true); + } + } + } + /** * Test all layers of cache for the minimum, then if necessary advance heads * diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/Pipeline.java b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/Pipeline.java index 955514c309..9f637e5bfc 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/Pipeline.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/Pipeline.java @@ -3,16 +3,14 @@ import java.util.Iterator; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.log4j.Logger; import datawave.query.attributes.Document; import datawave.query.iterator.DocumentSpecificNestedIterator; import datawave.query.iterator.NestedQueryIterator; -import datawave.query.iterator.profile.QuerySpanCollector; /** * A pipeline that can be executed as a runnable @@ -28,15 +26,13 @@ public class Pipeline implements Runnable { // the result private Entry result = null; + // exception + private RuntimeException exception = null; // the pipeline private Iterator> iterator = null; + private AtomicBoolean running = new AtomicBoolean(false); - private QuerySpanCollector querySpanCollector = null; - - public Pipeline(QuerySpanCollector querySpanCollector, SortedKeyValueIterator sourceForDeepCopy) { - this.querySpanCollector = querySpanCollector; - this.iterator = null; - } + public Pipeline() {} public void setSourceIterator(Iterator> sourceIter) { this.iterator = sourceIter; @@ -55,25 +51,57 @@ public Map.Entry getSource() { } public void clear() { + this.exception = null; this.result = null; this.documentSpecificSource.setDocumentKey(null); } public Entry getResult() { - return result; - } - - @Override - public void run() { - if (iterator.hasNext()) { - result = iterator.next(); + if (exception == null) { + return result; } else { - result = null; + throw exception; } + } - if (log.isTraceEnabled()) { - log.trace("next() returned " + result); + public void waitUntilComplete() { + if (running.get()) { + synchronized (running) { + while (running.get()) { + try { + running.wait(); + } catch (InterruptedException e) { + + } + } + } } } + @Override + public void run() { + try { + running.set(true); + if (iterator.hasNext()) { + result = iterator.next(); + } else { + result = null; + } + + if (log.isTraceEnabled()) { + log.trace("next() returned " + result); + } + } catch (Exception e) { + if (e instanceof RuntimeException) { + exception = (RuntimeException) e; + } else { + exception = new RuntimeException(e); + } + } finally { + synchronized (running) { + running.set(false); + running.notify(); + } + } + } } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineFactory.java b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineFactory.java index cbd7751ce2..2921d4c1f2 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineFactory.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineFactory.java @@ -13,6 +13,7 @@ import datawave.query.iterator.QueryIterator; import datawave.query.iterator.profile.QuerySpan; import datawave.query.iterator.profile.QuerySpanCollector; +import datawave.query.iterator.waitwindow.WaitWindowObserver; public class PipelineFactory { @@ -50,13 +51,13 @@ public class PipelineFactory { public static PipelineIterator createIterator(NestedIterator documents, int maxPipelines, int maxCachedResults, boolean requestSerialPipeline, QuerySpanCollector querySpanCollector, QuerySpan querySpan, QueryIterator sourceIterator, SortedKeyValueIterator sourceForDeepCopy, IteratorEnvironment env, YieldCallback yield, long yieldThresholdMs, - Collection columnFamilies, boolean inclusive) { + WaitWindowObserver waitWindowObserver, Collection columnFamilies, boolean inclusive) { if (maxPipelines > 1 && !requestSerialPipeline) { return new PipelineIterator(documents, maxPipelines, maxCachedResults, querySpanCollector, querySpan, sourceIterator, sourceForDeepCopy, env, yield, - yieldThresholdMs, columnFamilies, inclusive); + yieldThresholdMs, waitWindowObserver, columnFamilies, inclusive); } else { return new SerialIterator(documents, maxPipelines, maxCachedResults, querySpanCollector, querySpan, sourceIterator, sourceForDeepCopy, env, yield, - yieldThresholdMs, columnFamilies, inclusive); + yieldThresholdMs, waitWindowObserver, columnFamilies, inclusive); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineIterator.java index 01214d848e..a7c3fb83f6 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelineIterator.java @@ -1,14 +1,20 @@ package datawave.query.iterator.pipeline; +import java.util.AbstractMap; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.LinkedList; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.Queue; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; @@ -16,17 +22,18 @@ import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iterators.YieldCallback; -import org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException; import org.apache.log4j.Logger; import datawave.core.iterators.IteratorThreadPoolManager; import datawave.query.attributes.Document; +import datawave.query.exceptions.WaitWindowOverrunException; import datawave.query.iterator.NestedIterator; import datawave.query.iterator.NestedQuery; import datawave.query.iterator.NestedQueryIterator; import datawave.query.iterator.QueryIterator; import datawave.query.iterator.profile.QuerySpan; import datawave.query.iterator.profile.QuerySpanCollector; +import datawave.query.iterator.waitwindow.WaitWindowObserver; import datawave.query.util.Tuple2; /** @@ -50,10 +57,14 @@ public class PipelineIterator implements Iterator> { protected IteratorEnvironment env; protected Collection columnFamilies; protected boolean inclusive; + protected final WaitWindowObserver waitWindowObserver; + protected Key keyFromWaitWindowOverrun = null; + protected Entry result = null; public PipelineIterator(NestedIterator documents, int maxPipelines, int maxCachedResults, QuerySpanCollector querySpanCollector, QuerySpan querySpan, QueryIterator sourceIterator, SortedKeyValueIterator sourceForDeepCopy, IteratorEnvironment env, - YieldCallback yieldCallback, long yieldThresholdMs, Collection columnFamilies, boolean inclusive) { + YieldCallback yieldCallback, long yieldThresholdMs, WaitWindowObserver waitWindowObserver, Collection columnFamilies, + boolean inclusive) { this.docSource = documents; this.pipelines = new PipelinePool(maxPipelines, querySpanCollector, sourceIterator, sourceForDeepCopy, env); this.evaluationQueue = new LinkedList<>(); @@ -63,6 +74,7 @@ public PipelineIterator(NestedIterator documents, int maxPipelines, int max this.querySpan = querySpan; this.env = env; this.yield = yieldCallback; + this.waitWindowObserver = waitWindowObserver; this.yieldThresholdMs = yieldThresholdMs; this.columnFamilies = columnFamilies; this.inclusive = inclusive; @@ -79,16 +91,24 @@ public void setCollectTimingDetails(boolean collectTimingDetails) { */ @Override public boolean hasNext() { - // if we had already yielded, then leave gracefully - if (yield != null && yield.hasYielded()) { - return false; + // if we have exceeded the wait window, then return true if + // collectTimingDetails, otherwise yield and return false + if (this.keyFromWaitWindowOverrun != null) { + boolean yieldToBeginning = WaitWindowObserver.hasBeginMarker(this.keyFromWaitWindowOverrun); + result = handleWaitWindowOverrun(this.keyFromWaitWindowOverrun, yieldToBeginning); + } else { + result = getNext(); + if (result != null && WaitWindowObserver.hasMarker(result.getKey())) { + boolean yieldToBeginning = WaitWindowObserver.hasBeginMarker(result.getKey()); + result = handleWaitWindowOverrun(result.getKey(), yieldToBeginning); + } } - Entry next = getNext(false); - if (log.isTraceEnabled()) { - log.trace("QueryIterator.hasNext() -> " + (next == null ? null : next.getKey())); + boolean yielded = (this.yield != null) && this.yield.hasYielded(); + if (!yielded && log.isTraceEnabled()) { + log.trace("PipelineIterator.hasNext() -> " + (result == null ? null : result.getKey())); } - return (next != null); + return (!yielded) && (result != null); } /* @@ -103,24 +123,23 @@ public Entry next() { return null; } - Entry next = getNext(true); if (log.isTraceEnabled()) { - log.trace("QueryIterator.next() -> " + (next == null ? null : next.getKey())); + log.trace("PipelineIterator.next() -> " + (result == null ? null : result.getKey())); } - return next; + Entry returnResult = result; + result = null; + return returnResult; } /** - * Get the next non-null result from the queue. Pop/remove that pipeline as specified. + * Get the next non-null result from the queue. * - * @param remove - * flag to remove * @return the next non-null entry. null if there are no more entries to get. */ - private Entry getNext(boolean remove) { + private Entry getNext() { try { if (log.isTraceEnabled()) { - log.trace("getNext(" + remove + ") start: " + evaluationQueue.size() + " cached: " + results.size()); + log.trace("getNext start: " + evaluationQueue.size() + " cached: " + results.size()); } // cache the next non-null result if we do not already have one @@ -129,36 +148,40 @@ private Entry getNext(boolean remove) { } if (log.isTraceEnabled()) { - log.trace("getNext(" + remove + ") cache: " + evaluationQueue.size() + " cached: " + results.size()); + log.trace("getNext cache: " + evaluationQueue.size() + " cached: " + results.size()); } // flush any completed results to the results queue flushCompletedResults(); if (log.isTraceEnabled()) { - log.trace("getNext(" + remove + ") flush: " + evaluationQueue.size() + " cached: " + results.size()); + log.trace("getNext flush: " + evaluationQueue.size() + " cached: " + results.size()); + } + + // ensure that the evaluation queue is filled if there is anything to evaluate + fillEvaluationQueue(); + + if (log.isTraceEnabled()) { + log.trace("getNext fill: " + evaluationQueue.size() + " cached: " + results.size()); } // get/remove and return the next result, null if we are done Entry next = null; if (!results.isEmpty()) { - if (remove) { - next = results.poll(); - } else { - next = results.peek(); - } + next = results.poll(); } return next; + } catch (WaitWindowOverrunException e) { + Key yieldKey = e.getYieldKey(); + return handleWaitWindowOverrun(yieldKey, WaitWindowObserver.hasBeginMarker(yieldKey)); } catch (Exception e) { - // cancel out existing executions - cancel(); - - // if we yielded, then leave gracefully if (yield != null && yield.hasYielded()) { + // if we yielded, then leave gracefully return null; } - + // if we have not yielded, then cancel existing evaluations + cancel(); log.error("Failed to retrieve evaluation pipeline result", e); throw new RuntimeException("Failed to retrieve evaluation pipeline result", e); } @@ -175,28 +198,36 @@ private Entry getNext(boolean remove) { private void cacheNextResult() throws InterruptedException, ExecutionException { Entry result = null; - long startMs = System.currentTimeMillis(); while (!evaluationQueue.isEmpty() && result == null) { // we must have at least evaluated one thing in order to yield, otherwise we will have not progressed at all if (yield != null && lastKeyEvaluated != null) { - long delta = System.currentTimeMillis() - startMs; - if (delta > yieldThresholdMs) { - yield.yield(lastKeyEvaluated); - if (log.isDebugEnabled()) - log.debug("Yielding at " + lastKeyEvaluated); - throw new IterationInterruptedException("Yielding at " + lastKeyEvaluated); + long remainingTimeMs = waitWindowObserver.remainingTimeMs(); + if (remainingTimeMs <= 0) { + List yieldKeys = Collections.singletonList(waitWindowObserver.createYieldKey(lastKeyEvaluated, false)); + throwExceptionOnWaitWindowOverrun(yieldKeys); } try { - result = poll(yieldThresholdMs - delta); + result = poll(remainingTimeMs); + // put the result into the queue if non-null + if (result != null) { + results.add(result); + } + // ensure that the evaluation queue is filled if there is anything to evaluate + fillEvaluationQueue(); } catch (TimeoutException e) { - yield.yield(lastKeyEvaluated); - if (log.isDebugEnabled()) - log.debug("Yielding at " + lastKeyEvaluated); - throw new IterationInterruptedException("Yielding at " + lastKeyEvaluated); + // lastKeyEvaluated either succeeded and is in results or failed so we can yield past it + List yieldKeys = Collections.singletonList(waitWindowObserver.createYieldKey(lastKeyEvaluated, false)); + throwExceptionOnWaitWindowOverrun(yieldKeys); } } else { try { result = poll(Long.MAX_VALUE); + // put the result into the queue if non-null + if (result != null) { + results.add(result); + } + // ensure that the evaluation queue is filled if there is anything to evaluate + fillEvaluationQueue(); } catch (TimeoutException e) { // should be impossible with a Long.MAX_VALUE, but we can wait another 292 million years log.error("We have been waiting for 292 million years, trying again"); @@ -205,6 +236,74 @@ private void cacheNextResult() throws InterruptedException, ExecutionException { } } + private List getPossibleYieldKeys(List additionalYieldKeys) { + List possibleYieldKeys = new ArrayList<>(); + // Add additional passed-in yieldKeys + possibleYieldKeys.addAll(additionalYieldKeys); + // Create and add a yieldKey for each unevaluated key in the evaluationQueue + for (Tuple2,Pipeline> t : evaluationQueue) { + possibleYieldKeys.add(waitWindowObserver.createYieldKey(t.second().getSource().getKey(), true)); + } + // Create and add a yieldKey for each not-yet returned result in the results list + for (Entry r : results) { + possibleYieldKeys.add(waitWindowObserver.createYieldKey(r.getKey(), true)); + } + // dedupe and sort possibleYieldKeys + return possibleYieldKeys.stream().distinct().collect(Collectors.toList()); + } + + private Key findLowestYieldKey(List possibleYieldKeys) { + possibleYieldKeys.sort(WaitWindowObserver.keyComparator); + // if there is more than one key and the lowest is YIELD_TO_END, then we can remove the first + while (possibleYieldKeys.size() > 1 && WaitWindowObserver.hasEndMarker(possibleYieldKeys.get(0))) { + possibleYieldKeys.remove(0); + } + Key yieldKey; + if (possibleYieldKeys.size() > 1) { + yieldKey = waitWindowObserver.lowestYieldKey(possibleYieldKeys); + if (log.isTraceEnabled()) { + log.trace("Yielding at:" + yieldKey + " after evaluating keys:" + possibleYieldKeys); + } + } else { + yieldKey = possibleYieldKeys.get(0); + } + return yieldKey; + } + + // If collectTimingDetails == true, then we wil set the (future) yieldKey in waitWindowObserver and return + // an entry with the yield key and a WAIT_WINDOW_OVERRUN document to which the timing details can be added + // If collectTimingDetails == false, then we yield and return a null + private Map.Entry handleWaitWindowOverrun(Key key, boolean yieldToBeginning) { + Map.Entry result = null; + List additionalYieldKeys = Collections.singletonList(waitWindowObserver.createYieldKey(key, yieldToBeginning)); + List possibleYieldKeys = getPossibleYieldKeys(additionalYieldKeys); + Key yieldKey = findLowestYieldKey(possibleYieldKeys); + if (collectTimingDetails) { + if (log.isDebugEnabled()) { + log.debug("WaitWindowOverrun at " + yieldKey); + } + waitWindowObserver.setYieldKey(yieldKey); + result = new AbstractMap.SimpleEntry<>(yieldKey, WaitWindowObserver.getWaitWindowOverrunDocument()); + } else { + if (log.isDebugEnabled()) { + log.debug("Yielding at " + yieldKey); + } + yield.yield(yieldKey); + } + // Either we're yielding or returning a WAIT_WINDOW_OVERRUN document. + // We're done using the evaluationQueue and results and can cancel and clear + log.debug("Cancelling remaining evaluations and removing results due to yield"); + cancel(); + return result; + } + + // This exception should be caught in getNext which will call handleWaitWindowOverrun + private void throwExceptionOnWaitWindowOverrun(List yieldKeys) { + List possibleYieldKeys = getPossibleYieldKeys(yieldKeys); + Key yieldKey = findLowestYieldKey(possibleYieldKeys); + throw new WaitWindowOverrunException(yieldKey); + } + /** * flush the results from the evaluation queue that are complete up to the max number of cached results * @@ -216,7 +315,10 @@ private void cacheNextResult() throws InterruptedException, ExecutionException { private void flushCompletedResults() throws InterruptedException, ExecutionException { while (!evaluationQueue.isEmpty() && evaluationQueue.peek().first().isDone() && results.size() < this.maxResults) { try { - poll(Long.MAX_VALUE); + Entry result = poll(Long.MAX_VALUE); + if (result != null) { + results.add(result); + } } catch (TimeoutException e) { // should be impossible with a Long.MAX_VALUE, but we can wait another 292 million years log.error("We have been waiting for 292 million years, trying again"); @@ -238,10 +340,10 @@ private void flushCompletedResults() throws InterruptedException, ExecutionExcep * for timeout exceptions */ private Entry poll(long waitMs) throws InterruptedException, ExecutionException, TimeoutException { - // get the next evaluated result - Tuple2,Pipeline> nextFuture = evaluationQueue.poll(); + // get the next entry on the evaluationQueue but do not remove + Tuple2,Pipeline> nextFuture = evaluationQueue.peek(); - Entry result = null; + Entry nextEntry; try { if (log.isTraceEnabled()) { Key docKey = nextFuture.second().getSource().getKey(); @@ -264,55 +366,67 @@ private Entry poll(long waitMs) throws InterruptedException, Execu nextFuture.first().get(); // pull the result - result = nextFuture.second().getResult(); + nextEntry = nextFuture.second().getResult(); if (log.isTraceEnabled()) { Key docKey = nextFuture.second().getSource().getKey(); - log.trace("Polling for result from " + docKey + " was " + (result == null ? "empty" : "successful")); + log.trace("Polling for result from " + docKey + " was " + (nextEntry == null ? "empty" : "successful")); } // record the last evaluated key lastKeyEvaluated = nextFuture.second().getSource().getKey(); + // remove completed Entry,Pipeline> from the evaluation queue + evaluationQueue.remove(); + // return the pipeline for reuse + pipelines.checkIn(nextFuture.second()); + } catch (TimeoutException e) { + // timeout means that we are yielding + throw e; } catch (Exception e) { Key docKey = nextFuture.second().getSource().getKey(); - log.error("Failed polling for result from " + docKey + "; cancelling remaining evaluations and flushing results", e); - cancel(); + log.error("Failed polling for result from " + docKey, e); throw e; - } finally { - // return the pipeline for reuse - pipelines.checkIn(nextFuture.second()); } + return nextEntry; + } - // start a new evaluation if we can - if (docSource.hasNext()) { + private void fillEvaluationQueue() { + // start a new evaluation for any available sources if there is room in the evaluationQueue + while (docSource.hasNext() && evaluationQueue.size() < pipelines.maxPipelines) { Key keySource = docSource.next(); - NestedQuery nestedQuery = null; - if (docSource instanceof NestedQueryIterator) { - nestedQuery = ((NestedQueryIterator) this.docSource).getNestedQuery(); - } + if (WaitWindowObserver.hasMarker(keySource)) { + List yieldKeys = new ArrayList<>(); + yieldKeys.add(waitWindowObserver.createYieldKey(keySource, WaitWindowObserver.hasBeginMarker(keySource))); + if (lastKeyEvaluated != null) { + // lastKeyEvaluated was updated in poll() and may be a valid result + yieldKeys.add(lastKeyEvaluated); + } + throwExceptionOnWaitWindowOverrun(yieldKeys); + } else { + NestedQuery nestedQuery = null; + if (docSource instanceof NestedQueryIterator) { + nestedQuery = ((NestedQueryIterator) this.docSource).getNestedQuery(); + } - evaluate(keySource, docSource.document(), nestedQuery, columnFamilies, inclusive); - if (collectTimingDetails) { - querySpanCollector.addQuerySpan(querySpan); + evaluate(keySource, docSource.document(), nestedQuery, columnFamilies, inclusive); + if (collectTimingDetails) { + querySpanCollector.addQuerySpan(querySpan); + } } } - - // put the result into the queue if non-null - if (result != null) { - results.add(result); - } - - return result; } /** - * Cancel all of the queued evaluations + * Cancel queued evaluations and clear results */ private void cancel() { while (!evaluationQueue.isEmpty()) { - Tuple2,Pipeline> nextFuture = evaluationQueue.poll(); - nextFuture.first().cancel(true); - pipelines.checkIn(nextFuture.second()); + Tuple2,Pipeline> pair = evaluationQueue.poll(); + Future future = pair.first(); + Pipeline pipeline = pair.second(); + future.cancel(false); + pipeline.waitUntilComplete(); + pipelines.checkIn(pipeline); } results.clear(); } @@ -332,11 +446,15 @@ public void startPipeline() { for (int i = 0; i < maxPipelines && this.docSource.hasNext(); i++) { Key keySource = this.docSource.next(); + if (keySource != null && WaitWindowObserver.hasMarker(keySource)) { + this.keyFromWaitWindowOverrun = keySource; + break; + } if (isNested) { nestedQuery = ((NestedQueryIterator) this.docSource).getNestedQuery(); - } - if (log.isTraceEnabled()) { - log.trace("evaluating nested " + nestedQuery); + if (log.isTraceEnabled()) { + log.trace("evaluating nested " + nestedQuery); + } } evaluate(keySource, this.docSource.document(), nestedQuery, columnFamilies, inclusive); } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelinePool.java b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelinePool.java index edbfc16b5a..023f3726ed 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelinePool.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/PipelinePool.java @@ -76,7 +76,7 @@ public Pipeline checkOut(Key key, Document doc, NestedQuery nestedQuery, Co } } } else if (checkedIn.size() + checkedOut.size() < maxPipelines) { - pipeline = new Pipeline(this.querySpanCollector, sourceForDeepCopy.deepCopy(env)); + pipeline = new Pipeline(); NestedQueryIterator nq = pipeline.getDocumentSpecificSource(); if (null != nestedQuery) { nq.setCurrentQuery(nestedQuery); diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/SerialIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/SerialIterator.java index c0ba487210..021bb3560d 100644 --- a/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/SerialIterator.java +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/pipeline/SerialIterator.java @@ -1,6 +1,8 @@ package datawave.query.iterator.pipeline; +import java.util.AbstractMap; import java.util.Collection; +import java.util.Map; import java.util.Map.Entry; import org.apache.accumulo.core.data.ByteSequence; @@ -18,6 +20,7 @@ import datawave.query.iterator.QueryIterator; import datawave.query.iterator.profile.QuerySpan; import datawave.query.iterator.profile.QuerySpanCollector; +import datawave.query.iterator.waitwindow.WaitWindowObserver; public class SerialIterator extends PipelineIterator { @@ -25,13 +28,12 @@ public class SerialIterator extends PipelineIterator { protected Pipeline currentPipeline; - protected Entry result = null; - public SerialIterator(NestedIterator documents, int maxPipelines, int maxCachedResults, QuerySpanCollector querySpanCollector, QuerySpan querySpan, QueryIterator sourceIterator, SortedKeyValueIterator sourceForDeepCopy, IteratorEnvironment env, - YieldCallback yieldCallback, long yieldThresholdMs, Collection columnFamilies, boolean include) { + YieldCallback yieldCallback, long yieldThresholdMs, WaitWindowObserver waitWindowObserver, Collection columnFamilies, + boolean include) { super(documents, maxPipelines, maxCachedResults, querySpanCollector, querySpan, sourceIterator, sourceForDeepCopy, env, yieldCallback, yieldThresholdMs, - columnFamilies, include); + waitWindowObserver, columnFamilies, include); } @Override @@ -41,23 +43,16 @@ public boolean hasNext() { return false; } - if (null == result) { - long start = System.currentTimeMillis(); - while (this.docSource.hasNext()) { - Key docKey = this.docSource.next(); - Document doc = this.docSource.document(); + while (result == null && this.docSource.hasNext()) { + Key docKey = this.docSource.next(); + Document doc = this.docSource.document(); + if (WaitWindowObserver.hasMarker(docKey) || waitWindowObserver.waitWindowOverrun()) { + result = handleWaitWindowOverrun(docKey, true); + break; + } else { currentPipeline.setSource(Maps.immutableEntry(docKey, doc)); currentPipeline.run(); result = currentPipeline.getResult(); - if (null != result) - break; - if (yield != null && ((System.currentTimeMillis() - start) > yieldThresholdMs)) { - yield.yield(docKey); - if (log.isDebugEnabled()) { - log.debug("Yielding at " + docKey); - } - break; - } } } return result != null; @@ -83,14 +78,41 @@ public void remove() { public void startPipeline() { if (this.docSource.hasNext()) { - currentPipeline = pipelines.checkOut(this.docSource.next(), this.docSource.document(), null, columnFamilies, inclusive); - currentPipeline.run(); - result = currentPipeline.getResult(); - if (null == result) { - hasNext(); + Key docKey = this.docSource.next(); + Document doc = this.docSource.document(); + if (WaitWindowObserver.hasMarker(docKey) || waitWindowObserver.waitWindowOverrun()) { + result = handleWaitWindowOverrun(docKey, true); + } else { + currentPipeline = pipelines.checkOut(docKey, doc, null, columnFamilies, inclusive); + currentPipeline.run(); + result = currentPipeline.getResult(); + if (null == result) { + hasNext(); + } } } else { result = null; } } + + // If collectTimingDetails == true, then we wil set the (future) yieldKey in waitWindowObserver and return + // an entry with the yield key and a WAIT_WINDOW_OVERRUN document to which the timing details can be added + // If collectTimingDetails == false, then we yield and return a null + private Map.Entry handleWaitWindowOverrun(Key docKey, boolean yieldToBeginning) { + Map.Entry result = null; + Key yieldKey = waitWindowObserver.createYieldKey(docKey, yieldToBeginning); + if (collectTimingDetails) { + waitWindowObserver.setYieldKey(yieldKey); + result = new AbstractMap.SimpleEntry<>(yieldKey, WaitWindowObserver.getWaitWindowOverrunDocument()); + if (log.isDebugEnabled()) { + log.debug("WaitWindowOverrun at " + yieldKey); + } + } else { + yield.yield(yieldKey); + if (log.isDebugEnabled()) { + log.debug("Yielding at " + yieldKey); + } + } + return result; + } } 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 8c636fc0ea..611dc70f47 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 @@ -18,11 +18,13 @@ import datawave.query.DocumentSerialization; import datawave.query.attributes.Document; +import datawave.query.attributes.WaitWindowExceededMetadata; import datawave.query.function.LogTiming; import datawave.query.function.serializer.KryoDocumentSerializer; import datawave.query.function.serializer.ToStringDocumentSerializer; import datawave.query.function.serializer.WritableDocumentSerializer; import datawave.query.iterator.Util; +import datawave.query.iterator.waitwindow.WaitWindowObserver; public class FinalDocumentTrackingIterator implements Iterator> { @@ -41,10 +43,12 @@ public class FinalDocumentTrackingIterator implements Iterator> itr, - DocumentSerialization.ReturnType returnType, boolean isReducedResponse, boolean isCompressResults, YieldCallback yield) { + public FinalDocumentTrackingIterator(QuerySpanCollector querySpanCollector, QuerySpan querySpan, WaitWindowObserver waitWindowObserver, Range seekRange, + Iterator> itr, DocumentSerialization.ReturnType returnType, boolean isReducedResponse, boolean isCompressResults, + YieldCallback yield) { this.itr = itr; this.seekRange = seekRange; this.returnType = returnType; @@ -52,6 +56,7 @@ public FinalDocumentTrackingIterator(QuerySpanCollector querySpanCollector, Quer this.isCompressResults = isCompressResults; this.querySpanCollector = querySpanCollector; this.querySpan = querySpan; + this.waitWindowObserver = waitWindowObserver; this.yield = yield; // check for the special case where we were torn down just after returning the final document @@ -77,20 +82,25 @@ public static boolean isFinalDocumentKey(Key k) { return false; } - private Map.Entry getStatsEntry(Key statsKey) { + private Map.Entry getStatsEntry(Key statsKey, boolean waitWindowOverrun) { - // now add our marker - statsKey = new Key(statsKey.getRow(), statsKey.getColumnFamily(), Util.appendText(statsKey.getColumnQualifier(), MARKER_TEXT), - statsKey.getColumnVisibility(), statsKey.getTimestamp()); + if (!waitWindowOverrun) { + // now add our marker + statsKey = new Key(statsKey.getRow(), statsKey.getColumnFamily(), Util.appendText(statsKey.getColumnQualifier(), MARKER_TEXT), + statsKey.getColumnVisibility(), statsKey.getTimestamp()); + } HashMap documentMap = new HashMap(); QuerySpan combinedQuerySpan = querySpanCollector.getCombinedQuerySpan(this.querySpan); + Document document = new Document(); if (combinedQuerySpan != null) { - Document document = new Document(); LogTiming.addTimingMetadata(document, combinedQuerySpan); - documentMap.put(statsKey, document); } + if (waitWindowOverrun) { + document.put(WaitWindowObserver.WAIT_WINDOW_OVERRUN, new WaitWindowExceededMetadata()); + } + documentMap.put(statsKey, document); Iterator> emptyDocumentIterator = documentMap.entrySet().iterator(); Iterator> serializedDocuments = null; @@ -156,13 +166,17 @@ public Map.Entry next() { } } - nextEntry = getStatsEntry(statsKey); + nextEntry = getStatsEntry(statsKey, false); statsEntryReturned = true; } } else { nextEntry = this.itr.next(); if (nextEntry != null) { + if (WaitWindowObserver.hasMarker(nextEntry.getKey())) { + nextEntry = getStatsEntry(nextEntry.getKey(), true); + statsEntryReturned = true; + } this.lastKey = nextEntry.getKey(); } } diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/waitwindow/WaitWindowObserver.java b/warehouse/query-core/src/main/java/datawave/query/iterator/waitwindow/WaitWindowObserver.java new file mode 100644 index 0000000000..8c028f19b2 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/waitwindow/WaitWindowObserver.java @@ -0,0 +1,417 @@ +package datawave.query.iterator.waitwindow; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.PartialKey; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.iterators.YieldCallback; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.io.Text; +import org.apache.log4j.Logger; + +import datawave.query.attributes.Document; +import datawave.query.attributes.DocumentKey; +import datawave.query.attributes.WaitWindowExceededMetadata; +import datawave.query.exceptions.WaitWindowOverrunException; +import datawave.query.iterator.ResultCountingIterator; +import datawave.query.iterator.profile.QuerySpan; + +// This class maintains common state and logic to determine if the QueryIterator's stack of boolean logic +// should yield to ensure that resources are shared appropriately, to return collected metrics if configured, +// and to ensure that there is still a client waiting for a response. +public class WaitWindowObserver { + + private static final Logger log = Logger.getLogger(WaitWindowObserver.class); + public static final String WAIT_WINDOW_OVERRUN = "WAIT_WINDOW_OVERRUN"; + public static final String YIELD_AT_BEGIN_STR = "!YIELD_AT_BEGIN"; + public static final Text YIELD_AT_BEGIN = new Text(YIELD_AT_BEGIN_STR); + public static final String YIELD_AT_END_STR = "\uffffYIELD_AT_END"; + public static final Text YIELD_AT_END = new Text(YIELD_AT_END_STR); + private static Timer timer = null; + private boolean readyToYield = false; + + public static final Comparator keyComparator = (o1, o2) -> { + if (o1 instanceof Key) { + return ((Key) o1).compareTo((Key) o2, PartialKey.ROW_COLFAM_COLQUAL); + } + return o1.compareTo(o2); + }; + + public static final Comparator keyComparatorReversed = (o1, o2) -> { + if (o1 instanceof Key) { + return ((Key) o2).compareTo((Key) o1, PartialKey.ROW_COLFAM_COLQUAL); + } + return o2.compareTo(o1); + }; + + protected YieldCallback yieldCallback = null; + // When the wait window is over. Set during the initial seek + protected long endOfWaitWindow; + // Remaining time in the wait window. Updated by the timerTask + protected AtomicLong remainingTimeMs = new AtomicLong(); + protected TimerTask timerTask = new WaitWindowTimerTask(); + // How often the timerTask gets run + protected long checkPeriod = 50; + // Seek range of the QueryIterator. Used to ensure that yieldKey is in the range. + protected Range seekRange = null; + // When collectTimingDetails==true, we set the yieldKey, return a WAIT_WINDOW_OVERRUN + // document (that will also have the timingDetails), and then yield to this key on + // a subsequent call to QueryIterator.hasTop + protected Key yieldKey = null; + + // Used for the unsortedUID case to prepend the colFam with an encoded count + protected ResultCountingIterator resultCountingIterator = null; + protected boolean sortedUIDs = true; + + // Used to record the yield that happens + protected QuerySpan trackingSpan = null; + + public WaitWindowObserver() { + + } + + // Using the WaitWindowTask in a Timer will limit the number of times that System.currentTimeMillis() + // is called while still decrementing remainingTimeMs as appropriate to enable yields. + private class WaitWindowTimerTask extends TimerTask { + @Override + public void run() { + long remaining = WaitWindowObserver.this.endOfWaitWindow - System.currentTimeMillis(); + WaitWindowObserver.this.remainingTimeMs.set(remaining); + // self-cancel this task when the wait window is exhausted + if (remaining <= 0) { + this.cancel(); + } + } + } + + // Ensure that we are only creating one Timer object per JVM (tablet server) for + // scheduling WaitWindowTimerTasks. Use the double null check to limit synchronization + // and prevent a race condition that overwrites WaitWindowObserver.timer. + private static Timer getTimer() { + if (WaitWindowObserver.timer == null) { + synchronized (WaitWindowObserver.class) { + if (WaitWindowObserver.timer == null) { + WaitWindowObserver.timer = new Timer(); + } + } + } + return WaitWindowObserver.timer; + } + + public void start(Range seekRange, long yieldThresholdMs) { + this.seekRange = seekRange; + this.remainingTimeMs.set(yieldThresholdMs); + this.endOfWaitWindow = yieldThresholdMs + System.currentTimeMillis(); + WaitWindowObserver.getTimer().schedule(this.timerTask, this.checkPeriod, this.checkPeriod); + } + + // Ensure that the WaitWindowTimerTask is cancelled. Called from QueryIterator.hasTop. + public void stop() { + this.timerTask.cancel(); + } + + // Called from waitWindowOverrun() and from places that use a timeout for polling or retrieving Future results + public long remainingTimeMs() { + if (this.yieldCallback == null) { + return Long.MAX_VALUE; + } else { + return this.remainingTimeMs.get(); + } + } + + public boolean waitWindowOverrun() { + return remainingTimeMs.get() <= 0; + } + + public void checkWaitWindow(Key currentKey, boolean yieldToBeginning) { + if (this.yieldCallback != null && waitWindowOverrun()) { + Key currentYieldKey = createYieldKey(currentKey, yieldToBeginning); + if (this.yieldKey == null) { + throw new WaitWindowOverrunException(currentYieldKey); + } else { + throw new WaitWindowOverrunException(lowestYieldKey(Arrays.asList(this.yieldKey, currentYieldKey))); + } + } + } + + // There can be many embedded AndIterators, OrIterators, and Ivarators where a WaitWindowOverrunException + // can be thrown from. As the exception makes its way to the top of the call chain, we need to evaluate + // the yieldKey at each level. + public void propagateException(Key key, boolean yieldToBeginning, boolean keepLowest, WaitWindowOverrunException e) { + Key yieldKey; + if (key == null) { + yieldKey = e.getYieldKey(); + } else { + Collection keys = Arrays.asList(e.getYieldKey(), createYieldKey(key, yieldToBeginning)); + if (keepLowest) { + yieldKey = lowestYieldKey(keys); + } else { + yieldKey = highestYieldKey(keys); + } + } + throw new WaitWindowOverrunException(yieldKey); + } + + // When yieldKey is set, yield on the second call from QueryIterator.hasTop so that the first call can return the + // document that contains the WAIT_WINDOW_OVERRUN and TIMING_METADATA attributes + public void yieldOnOverrun() { + if (this.yieldCallback != null && this.yieldKey != null && !this.yieldCallback.hasYielded()) { + if (readyToYield) { + if (this.resultCountingIterator != null) { + this.yieldKey = this.resultCountingIterator.addKeyCount(this.yieldKey); + } + yieldCallback.yield(this.yieldKey); + if (log.isDebugEnabled()) { + log.debug("Yielding at " + this.yieldKey); + } + } else { + readyToYield = true; + } + } + } + + // Create a yield key with YIELD_AT_BEGIN or YIELD_AT_END marker + public Key createYieldKey(Key yieldKey, boolean yieldToBeginning) { + if (isShardKey(yieldKey)) { + return createShardYieldKey(yieldKey, yieldToBeginning); + } else { + return createDocumentYieldKey(yieldKey, yieldToBeginning); + } + } + + // Create a key that sorts either before or after all field keys for this document key. + // A colQual starting with ! sorts before all keys whose colFam starts with an alphanumeric character. + // A colQual starting with \uffff sorts after all keys whose colFam starts with an alphanumeric character. + // We are adding sort-irrelevant marker text after that symbol to easily identify the key + public Key createShardYieldKey(Key key, boolean yieldToBeginning) { + // if key already contains YIELD_AT_END then we must yield to the end + Text marker = yieldToBeginning && !hasEndMarker(key) ? YIELD_AT_BEGIN : YIELD_AT_END; + return ensureYieldKeyAfterRangeStart(new Key(key.getRow(), marker)); + } + + // Create a key that sorts either before or after all field keys for this document key. + // A colQual starting with ! sorts before all keys with the same row/colFam and an alphanumeric colQual + // A colQual starting with \uffff sorts after all keys with the same row/colFam and an alphanumeric colQual + // Also adding a sort-irrelevant marker text after that symbol to easily identify the key + public Key createDocumentYieldKey(Key key, boolean yieldToBeginning) { + // if key already contains YIELD_AT_END then we must yield to the end + Text marker = yieldToBeginning && !hasEndMarker(key) ? YIELD_AT_BEGIN : YIELD_AT_END; + Key newKey; + if (sortedUIDs) { + Key documentKey = new DocumentKey(key, true).getDocKey(); + newKey = new Key(documentKey.getRow(), documentKey.getColumnFamily(), marker); + } else { + Text colQual; + if (hasMarker(key.getColumnQualifier())) { + colQual = key.getColumnQualifier(); + } else { + String origColQual = key.getColumnQualifier().toString(); + if (origColQual.isEmpty()) { + colQual = marker; + } else if (yieldToBeginning) { + colQual = new Text(marker.toString() + "\0" + origColQual); + } else { + colQual = new Text(origColQual + marker.toString()); + } + } + newKey = new Key(key.getRow(), key.getColumnFamily(), colQual); + } + return ensureYieldKeyAfterRangeStart(newKey); + } + + // When the current seekRange is non-inclusive, we can not return the startKey of the range + // as a yield key. Instead, we have to return the following key. + private Key ensureYieldKeyAfterRangeStart(Key key) { + if (!this.seekRange.isStartKeyInclusive()) { + Key seekStartKey = this.seekRange.getStartKey(); + boolean isShardKey = isShardKey(seekStartKey); + if (isShardKey && key.compareTo(seekStartKey, PartialKey.ROW_COLFAM) <= 0) { + // shard range + return seekStartKey.followingKey(PartialKey.ROW_COLFAM); + } + if (!isShardKey && key.compareTo(seekStartKey, PartialKey.ROW_COLFAM_COLQUAL) <= 0) { + // document range + return seekStartKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL); + } + } + return key; + } + + private Key yieldKeyAfterOverrun(Key key) { + Text row = key.getRow(); + Text colFam = key.getColumnFamily(); + Text colQual = key.getColumnQualifier(); + if (hasBeginMarker(key)) { + if (hasBeginMarker(colFam)) { + colFam = new Text(colFam.toString().replace(YIELD_AT_BEGIN_STR, YIELD_AT_BEGIN_STR + "\0")); + } else { + colQual = new Text(colQual.toString().replace(YIELD_AT_BEGIN_STR, YIELD_AT_BEGIN_STR + "\0")); + } + } else if (hasEndMarker(key)) { + if (hasEndMarker(colFam)) { + colFam = new Text(colFam.toString().replace(YIELD_AT_END_STR, YIELD_AT_END_STR + "\0")); + } else { + colQual = new Text(colQual.toString().replace(YIELD_AT_END_STR, YIELD_AT_END_STR + "\0")); + } + } + return new Key(row, colFam, colQual); + } + + public void setYieldKey(Key yieldKey) { + // only do this once + if (this.trackingSpan != null && this.yieldKey == null) { + this.trackingSpan.yield(); + } + // we can't return a key and then yield at the same key + this.yieldKey = yieldKeyAfterOverrun(yieldKey); + } + + public boolean isReadyToYield() { + return readyToYield && yieldKey != null; + } + + public Key getYieldKey() { + return yieldKey; + } + + static public boolean isShardKey(Key key) { + Text colFam = key.getColumnFamily(); + return colFam.equals(new Text()) || hasBeginMarker(colFam); + } + + static public boolean hasMarker(Key key) { + return hasBeginMarker(key) || hasEndMarker(key); + } + + static public boolean hasMarker(Text text) { + return hasBeginMarker(text) || hasEndMarker(text); + } + + static public boolean hasBeginMarker(Key key) { + return hasBeginMarker(key.getColumnFamily()) || hasBeginMarker(key.getColumnQualifier()); + } + + static public boolean hasBeginMarker(Text text) { + return text.toString().contains(YIELD_AT_BEGIN_STR); + } + + static public boolean hasEndMarker(Key key) { + return hasEndMarker(key.getColumnFamily()) || hasEndMarker(key.getColumnQualifier()); + } + + static public boolean hasEndMarker(Text text) { + return text.toString().contains(YIELD_AT_END_STR); + } + + static public Text removeMarkers(Text text) { + String str = text.toString(); + if (hasBeginMarker(text)) { + int yieldAtBeginStrLength = YIELD_AT_BEGIN_STR.length(); + if (str.length() > yieldAtBeginStrLength) { + str = str.substring(yieldAtBeginStrLength + 1); + // strip null characters after YIELD_AT_BEGIN (now at start of str) + str = StringUtils.stripStart(str, "\0"); + } else { + str = ""; + } + return new Text(str); + } else if (hasEndMarker(text)) { + int marker = str.indexOf(WaitWindowObserver.YIELD_AT_END_STR); + if (marker > 0) { + str = str.substring(0, marker); + } else { + str = ""; + } + return new Text(str); + } else { + return text; + } + } + + static public Document getWaitWindowOverrunDocument() { + Document document = new Document(); + document.put(WAIT_WINDOW_OVERRUN, new WaitWindowExceededMetadata()); + return document; + } + + public Key lowestYieldKey(Collection keys) { + Collection keySet = new HashSet<>(keys); + Text lowestRow = keySet.stream().sorted(keyComparator).findFirst().get().getRow(); + List keysInRowSortedIncreasing = keySet.stream().filter(k -> k.getRow().equals(lowestRow)).sorted(keyComparator).collect(Collectors.toList()); + Key lowestKey = keysInRowSortedIncreasing.stream().findFirst().get(); + if (keySet.size() > 1 && !sortedUIDs) { + Optional lowestColQual = keysInRowSortedIncreasing.stream().map(k -> WaitWindowObserver.removeMarkers(k.getColumnQualifier()).toString()) + .filter(Predicate.not(String::isEmpty)).sorted().findFirst(); + if (removeMarkers(lowestKey.getColumnQualifier()).getLength() > 0 && lowestColQual.isPresent()) { + Key lowestKeyWithLowestColQual = keysInRowSortedIncreasing.stream() + .filter(k -> WaitWindowObserver.removeMarkers(k.getColumnQualifier()).toString().equals(lowestColQual.get())).findFirst().get(); + int compare = lowestKeyWithLowestColQual.compareTo(lowestKey, PartialKey.ROW_COLFAM_COLQUAL); + if (compare <= 0) { + lowestKey = lowestKeyWithLowestColQual; + } else { + Text colQual = lowestKeyWithLowestColQual.getColumnQualifier(); + lowestKey = createYieldKey(new Key(lowestKey.getRow(), lowestKey.getColumnFamily(), colQual), hasBeginMarker(lowestKey)); + } + } + } + return lowestKey; + } + + public Key highestYieldKey(Collection keys) { + Collection keySet = new HashSet<>(keys); + Text higestRow = keySet.stream().sorted(keyComparatorReversed).findFirst().get().getRow(); + List keysInRowSortedDecreasing = keySet.stream().filter(k -> k.getRow().equals(higestRow)).sorted(keyComparatorReversed) + .collect(Collectors.toList()); + Key highestKey = keysInRowSortedDecreasing.stream().findFirst().get(); + if (keysInRowSortedDecreasing.size() > 1 && !sortedUIDs) { + Optional highestColQual = keysInRowSortedDecreasing.stream().map(k -> WaitWindowObserver.removeMarkers(k.getColumnQualifier()).toString()) + .filter(Predicate.not(String::isEmpty)).sorted(Comparator.reverseOrder()).findFirst(); + if (highestColQual.isPresent()) { + Optional highestKeyWithHighestColQualOpt = keysInRowSortedDecreasing.stream() + .filter(k -> WaitWindowObserver.removeMarkers(k.getColumnQualifier()).toString().equals(highestColQual.get())).findFirst(); + if (removeMarkers(highestKey.getColumnQualifier()).getLength() > 0 && highestKeyWithHighestColQualOpt.isPresent()) { + Key highestKeyWithHighestColQual = highestKeyWithHighestColQualOpt.get(); + int compare = highestKeyWithHighestColQual.compareTo(highestKey, PartialKey.ROW_COLFAM_COLQUAL); + if (compare <= 0) { + highestKey = highestKeyWithHighestColQual; + } else { + Text colQual = highestKeyWithHighestColQual.getColumnQualifier(); + highestKey = createYieldKey(new Key(highestKey.getRow(), highestKey.getColumnFamily(), colQual), hasBeginMarker(highestKey)); + } + } + } + } + return highestKey; + } + + public void setResultCountingIterator(ResultCountingIterator resultCountingIterator) { + this.resultCountingIterator = resultCountingIterator; + } + + public void setYieldCallback(YieldCallback yieldCallback) { + this.yieldCallback = yieldCallback; + } + + public void setSortedUIDs(boolean sortedUIDs) { + this.sortedUIDs = sortedUIDs; + } + + public boolean getSortedUIDs() { + return sortedUIDs; + } + + public void setTrackingSpan(QuerySpan trackingSpan) { + this.trackingSpan = trackingSpan; + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/iterator/waitwindow/WaitWindowOverseerIterator.java b/warehouse/query-core/src/main/java/datawave/query/iterator/waitwindow/WaitWindowOverseerIterator.java new file mode 100644 index 0000000000..9cf7242652 --- /dev/null +++ b/warehouse/query-core/src/main/java/datawave/query/iterator/waitwindow/WaitWindowOverseerIterator.java @@ -0,0 +1,87 @@ +package datawave.query.iterator.waitwindow; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Map; + +import org.apache.accumulo.core.data.ByteSequence; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.iterators.IteratorEnvironment; + +import datawave.query.attributes.Document; +import datawave.query.exceptions.WaitWindowOverrunException; +import datawave.query.iterator.NestedIterator; +import datawave.query.iterator.SeekableNestedIterator; + +public class WaitWindowOverseerIterator extends SeekableNestedIterator { + + private Map.Entry nextFromWaitWindowOverrunException = null; + + public WaitWindowOverseerIterator(NestedIterator source, IteratorEnvironment env) { + super(source, env); + } + + @Override + public void initialize() { + // skip initialize if we've already had a WaitWindowOverrunException + if (this.nextFromWaitWindowOverrunException == null) { + try { + super.initialize(); + } catch (WaitWindowOverrunException e) { + this.nextFromWaitWindowOverrunException = new AbstractMap.SimpleEntry<>(e.getYieldKey(), WaitWindowObserver.getWaitWindowOverrunDocument()); + } + } + } + + @Override + public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { + try { + super.seek(range, columnFamilies, inclusive); + } catch (WaitWindowOverrunException e) { + this.nextFromWaitWindowOverrunException = new AbstractMap.SimpleEntry<>(e.getYieldKey(), WaitWindowObserver.getWaitWindowOverrunDocument()); + } + } + + @Override + public boolean hasNext() { + if (this.nextFromWaitWindowOverrunException == null) { + try { + boolean hasNext = super.hasNext(); + return hasNext; + } catch (WaitWindowOverrunException e) { + this.nextFromWaitWindowOverrunException = new AbstractMap.SimpleEntry<>(e.getYieldKey(), WaitWindowObserver.getWaitWindowOverrunDocument()); + return true; + } + } else { + return true; + } + } + + @Override + public Key next() { + Key next; + if (this.nextFromWaitWindowOverrunException == null) { + try { + next = super.next(); + } catch (WaitWindowOverrunException e) { + next = e.getYieldKey(); + } + } else { + next = this.nextFromWaitWindowOverrunException.getKey(); + } + return next; + } + + @Override + public Document document() { + if (this.nextFromWaitWindowOverrunException == null) { + return super.document(); + } else { + Document document = this.nextFromWaitWindowOverrunException.getValue(); + this.nextFromWaitWindowOverrunException = null; + return document; + } + } +} diff --git a/warehouse/query-core/src/main/java/datawave/query/jexl/functions/KeyAdjudicator.java b/warehouse/query-core/src/main/java/datawave/query/jexl/functions/KeyAdjudicator.java index eafafab680..7671a7d4b8 100644 --- a/warehouse/query-core/src/main/java/datawave/query/jexl/functions/KeyAdjudicator.java +++ b/warehouse/query-core/src/main/java/datawave/query/jexl/functions/KeyAdjudicator.java @@ -4,12 +4,13 @@ import java.util.Map.Entry; import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.iterators.YieldCallback; import org.apache.hadoop.io.Text; import com.google.common.base.Function; import com.google.common.collect.Maps; +import datawave.query.iterator.waitwindow.WaitWindowObserver; + /** * Key adjudicator, will take an accumulo key based entry whose value is specified by T. * @@ -19,24 +20,21 @@ public class KeyAdjudicator implements Iterator>, Function,Entry> { public static final Text COLUMN_QUALIFIER_SUFFIX = new Text("\uffff"); - public static final Text EMPTY_COLUMN_QUALIFIER = new Text(); private final Text colQualRef; private final Iterator> source; - private final YieldCallback yield; - public KeyAdjudicator(Iterator> source, Text colQualRef, YieldCallback yield) { + public KeyAdjudicator(Iterator> source, Text colQualRef) { this.colQualRef = colQualRef; this.source = source; - this.yield = yield; } - public KeyAdjudicator(Iterator> source, YieldCallback yield) { - this(source, COLUMN_QUALIFIER_SUFFIX, yield); + public KeyAdjudicator(Iterator> source) { + this(source, COLUMN_QUALIFIER_SUFFIX); } public KeyAdjudicator(Text colQualRef) { - this(null, colQualRef, null); + this(null, colQualRef); } public KeyAdjudicator() { @@ -46,17 +44,18 @@ public KeyAdjudicator() { @Override public Entry apply(Entry entry) { final Key entryKey = entry.getKey(); - return Maps.immutableEntry(new Key(entryKey.getRow(), entryKey.getColumnFamily(), colQualRef, entryKey.getColumnVisibility(), entryKey.getTimestamp()), - entry.getValue()); + if (WaitWindowObserver.hasMarker(entryKey)) { + return entry; + } else { + return Maps.immutableEntry( + new Key(entryKey.getRow(), entryKey.getColumnFamily(), colQualRef, entryKey.getColumnVisibility(), entryKey.getTimestamp()), + entry.getValue()); + } } @Override public boolean hasNext() { - boolean hasNext = source.hasNext(); - if (yield != null && yield.hasYielded()) { - yield.yield(apply((Entry) Maps.immutableEntry(yield.getPositionAndReset(), null)).getKey()); - } - return hasNext; + return source.hasNext(); } @Override diff --git a/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/IteratorBuildingVisitor.java b/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/IteratorBuildingVisitor.java index 34f9bf569b..93361d5940 100644 --- a/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/IteratorBuildingVisitor.java +++ b/warehouse/query-core/src/main/java/datawave/query/jexl/visitors/IteratorBuildingVisitor.java @@ -92,6 +92,7 @@ import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; import datawave.query.iterator.logic.OrIterator; import datawave.query.iterator.profile.QuerySpanCollector; +import datawave.query.iterator.waitwindow.WaitWindowObserver; import datawave.query.jexl.ArithmeticJexlEngines; import datawave.query.jexl.DatawaveJexlContext; import datawave.query.jexl.DatawaveJexlEngine; @@ -149,6 +150,7 @@ public class IteratorBuildingVisitor extends BaseVisitor { protected List ivaratorCacheDirConfigs; protected String queryId; protected String scanId; + protected WaitWindowObserver waitWindowObserver; protected String ivaratorCacheSubDirPrefix = ""; protected long ivaratorCacheScanPersistThreshold = 100000L; protected long ivaratorCacheScanTimeout = 1000L * 60 * 60; @@ -363,6 +365,7 @@ public Object visit(ASTAndNode and, Object data) { // Create an AndIterator and recursively add the children AbstractIteratorBuilder andItr = new AndIteratorBuilder(); andItr.negateAsNeeded(data); + andItr.setWaitWindowObserver(this.waitWindowObserver); and.childrenAccept(this, andItr); // If there is no parent @@ -459,7 +462,7 @@ private NestedIterator buildExceededFromTermFrequency(String identifier, Je builder.setField(identifier); NestedIterator tfIterator = builder.build(); - OrIterator tfMerge = new OrIterator(Arrays.asList(tfIterator, eventFieldIterator)); + OrIterator tfMerge = new OrIterator(Arrays.asList(tfIterator, eventFieldIterator), null, waitWindowObserver); return tfMerge; } else { QueryException qe = new QueryException(DatawaveErrorCode.UNEXPECTED_SOURCE_NODE, MessageFormat.format("{0}", "buildExceededFromTermFrequency")); @@ -511,6 +514,7 @@ public Object visit(ASTOrNode or, Object data) { } else { // Create an OrIterator and recursively add the children AbstractIteratorBuilder orItr = new OrIteratorBuilder(); + orItr.setWaitWindowObserver(waitWindowObserver); orItr.setSortedUIDs(sortedUIDs); orItr.negateAsNeeded(data); or.childrenAccept(this, orItr); @@ -1411,6 +1415,8 @@ public boolean keep(Key k) { */ public void ivarate(IvaratorBuilder builder, JexlNode rootNode, JexlNode sourceNode, Object data) throws IOException { builder.setQueryId(queryId); + builder.setScanId(scanId); + builder.setWaitWindowObserver(waitWindowObserver); builder.setSource(unsortedIvaratorSource); builder.setTimeFilter(timeFilter); builder.setTypeMetadata(typeMetadata); @@ -1684,6 +1690,11 @@ public IteratorBuildingVisitor setScanId(String scanId) { return this; } + public IteratorBuildingVisitor setWaitWindowObserver(WaitWindowObserver waitWindowObserver) { + this.waitWindowObserver = waitWindowObserver; + return this; + } + public IteratorBuildingVisitor setIvaratorCacheSubDirPrefix(String ivaratorCacheSubDirPrefix) { this.ivaratorCacheSubDirPrefix = (ivaratorCacheSubDirPrefix == null ? "" : ivaratorCacheSubDirPrefix); return this; diff --git a/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransformerSupport.java b/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransformerSupport.java index 07da72a261..2fbb5e98cd 100644 --- a/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransformerSupport.java +++ b/warehouse/query-core/src/main/java/datawave/query/transformer/DocumentTransformerSupport.java @@ -39,6 +39,7 @@ import datawave.query.function.deserializer.DocumentDeserializer; import datawave.query.iterator.QueryOptions; import datawave.query.iterator.profile.QuerySpan; +import datawave.query.iterator.waitwindow.WaitWindowObserver; import datawave.query.jexl.JexlASTHelper; import datawave.util.StringUtils; import datawave.util.time.DateHelper; @@ -258,8 +259,10 @@ protected void extractMetrics(Document document, Key documentKey) { log.info(sb.toString()); } } - if (dictionary.size() == 1) { - // this document contained only timing metadata + boolean metadata = dictionary.containsKey(LogTiming.TIMING_METADATA); + boolean waitWindowOverrun = dictionary.containsKey(WaitWindowObserver.WAIT_WINDOW_OVERRUN); + if ((dictionary.size() == 1 && metadata) || waitWindowOverrun) { + // this document contained only timing metadata or contains the WAIT_WINDOW_OVERRUN marker throw new EmptyObjectException(); } } diff --git a/warehouse/query-core/src/test/java/datawave/query/IvaratorReloadTest.java b/warehouse/query-core/src/test/java/datawave/query/IvaratorReloadTest.java index bcfc8765f7..c7f4377019 100644 --- a/warehouse/query-core/src/test/java/datawave/query/IvaratorReloadTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/IvaratorReloadTest.java @@ -34,6 +34,7 @@ import datawave.query.iterator.SortedListKeyValueIterator; import datawave.query.iterator.ivarator.IvaratorCacheDir; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; +import datawave.query.iterator.waitwindow.WaitWindowObserver; /** * Verify that a rebuild of an ivarator will reuse the files from after a tear-down/rebuild @@ -115,6 +116,7 @@ public static DatawaveFieldIndexRangeIteratorJexl createRangeIvarator(FileSystem .withTypeMetadata(null) .withSubRanges(null) .withIvaratorSourcePool(createIvaratorSourcePool(10)) + .withWaitWindowObserver(new WaitWindowObserver()) .build(); // @formatter:on } diff --git a/warehouse/query-core/src/test/java/datawave/query/IvaratorYieldingTest.java b/warehouse/query-core/src/test/java/datawave/query/IvaratorYieldingTest.java index 7e7a672d97..82e180af36 100644 --- a/warehouse/query-core/src/test/java/datawave/query/IvaratorYieldingTest.java +++ b/warehouse/query-core/src/test/java/datawave/query/IvaratorYieldingTest.java @@ -1,8 +1,6 @@ package datawave.query; import static datawave.query.iterator.QueryOptions.SORTED_UIDS; -import static datawave.query.testframework.RawDataManager.JEXL_AND_OP; -import static datawave.query.testframework.RawDataManager.RE_OP; import java.io.IOException; import java.net.URL; @@ -20,7 +18,6 @@ import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iterators.YieldCallback; -import org.apache.accumulo.core.iterators.YieldingKeyValueIterator; import org.apache.log4j.Logger; import org.junit.Before; import org.junit.BeforeClass; @@ -71,8 +68,6 @@ public static void setUp() throws Exception { public void setup() throws IOException { TimeZone.setDefault(TimeZone.getTimeZone("GMT")); - logic.setCollectTimingDetails(true); - logic.setFullTableScanEnabled(true); // this should force regex expansion into ivarators logic.setMaxValueExpansionThreshold(1); @@ -87,28 +82,62 @@ public void setup() throws IOException { logic.setYieldThresholdMs(1); logic.getQueryPlanner().setQueryIteratorClass(YieldingQueryIterator.class); + } - logic.setMaxEvaluationPipelines(1); + @Test + public void testSerial_SortedUIDs_TimingDetails() throws Exception { + runTest(1, true, true); } @Test - public void testIvaratorInterruptedAndYieldSorted() throws Exception { - Map params = new HashMap<>(); - // both required in order to force ivarator to call fillSets - params.put(SORTED_UIDS, "true"); - logic.getConfig().setUnsortedUIDsEnabled(false); + public void testSerial_SortedUIDs_NoTimingDetails() throws Exception { + runTest(1, true, false); + } - String query = CitiesDataType.CityField.STATE.name() + "=~'.*[a-z].*' && filter:includeRegex(" + CitiesDataType.CityField.STATE.name() + ",'ma.*')"; - String expected = CitiesDataType.CityField.STATE.name() + "=~'ma.*'"; - runTest(query, expected, params); + @Test + public void testSerial_UnSortedUIDs_TimingDetails() throws Exception { + runTest(1, false, true); + } + + @Test + public void testSerial_UnSortedUIDs_NoTimingDetails() throws Exception { + runTest(1, false, false); + } + + @Test + public void testPipeline_SortedUIDs_TimingDetails() throws Exception { + runTest(4, true, true); + } + + @Test + public void testPipeline_SortedUIDs_NoTimingDetails() throws Exception { + runTest(4, true, false); } @Test - public void testIvaratorInterruptedAndYieldUnsorted() throws Exception { - String query = CitiesDataType.CityField.STATE.name() + RE_OP + "'.*[a-z].*'" + JEXL_AND_OP + "filter:includeRegex(" - + CitiesDataType.CityField.STATE.name() + ",'ma.*')"; - String expected = CitiesDataType.CityField.STATE.name() + "=~'ma.*'"; - runTest(query, expected); + public void testPipeline_UnSortedUIDs_TimingDetails() throws Exception { + runTest(4, false, true); + } + + @Test + public void testPipeline_UnSortedUIDs_NoTimingDetails() throws Exception { + runTest(4, false, false); + } + + public void runTest(int pipelines, boolean sortedUIDs, boolean timingDetails) throws Exception { + Map params = new HashMap<>(); + if (sortedUIDs) { + // both required in order to force ivarator to call fillSets + params.put(SORTED_UIDS, "true"); + } + logic.setUnsortedUIDsEnabled(!sortedUIDs); + logic.setCollectTimingDetails(timingDetails); + logic.setLogTimingDetails(timingDetails); + logic.setMaxEvaluationPipelines(pipelines); + + String query = CitiesDataType.CityField.STATE.name() + "=~'.*[a-z].*' && filter:includeRegex(" + CitiesDataType.CityField.STATE.name() + ",'m.*')"; + String expected = CitiesDataType.CityField.STATE.name() + "=~'m.*'"; + runTest(query, expected, params); } public static class YieldingQueryIterator implements SortedKeyValueIterator { @@ -121,6 +150,7 @@ public static class YieldingQueryIterator implements SortedKeyValueIterator __columnFamilies; private boolean __inclusive; + private Key lastResultKey = null; public YieldingQueryIterator() { __delegate = new QueryIterator(); @@ -142,7 +172,18 @@ public void init(SortedKeyValueIterator source, Map op @Override public boolean hasTop() { - return __delegate.hasTop(); + boolean hasTop = __delegate.hasTop(); + while (__yield.hasYielded()) { + try { + Key yieldKey = __yield.getPositionAndReset(); + checkYieldKey(yieldKey); + createAndSeekNewQueryIterator(yieldKey); + hasTop = __delegate.hasTop(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return hasTop; } @Override @@ -150,21 +191,28 @@ public void enableYielding(YieldCallback yieldCallback) { throw new UnsupportedOperationException("Yielding being handled internally"); } + private void checkYieldKey(Key yieldKey) throws IOException { + if (!__range.contains(yieldKey)) { + throw new IllegalStateException("Yielded to key outside of range " + yieldKey + " not in " + __range); + } + if (lastResultKey != null && yieldKey.compareTo(lastResultKey) <= 0) { + throw new IOException( + "Underlying iterator yielded to a position that does not follow the last key returned: " + yieldKey + " <= " + lastResultKey); + } + } + + private void createAndSeekNewQueryIterator(Key yieldKey) throws IOException { + log.debug("Yielded at " + yieldKey + " after seeking range " + __range); + __delegate = new QueryIterator(); + __delegate.init(__source, __options, __env); + __delegate.enableYielding(__yield); + __range = new Range(yieldKey, false, __range.getEndKey(), __range.isEndKeyInclusive()); + __delegate.seek(__range, __columnFamilies, __inclusive); + } + @Override public void next() throws IOException { __delegate.next(); - while (__yield.hasYielded()) { - Key key = __yield.getPositionAndReset(); - if (!__range.contains(key)) { - throw new IllegalStateException("Yielded to key outside of range"); - } - __delegate = new QueryIterator(); - __delegate.init(__source, __options, __env); - __delegate.enableYielding(__yield); - __range = new Range(key, false, __range.getEndKey(), __range.isEndKeyInclusive()); - log.info("Yielded at " + __range.getStartKey()); - __delegate.seek(__range, __columnFamilies, __inclusive); - } } @Override @@ -173,23 +221,17 @@ public void seek(Range range, Collection columnFamilies, boolean i __columnFamilies = columnFamilies; __inclusive = inclusive; __delegate.seek(range, columnFamilies, inclusive); - while (__yield.hasYielded()) { - Key key = __yield.getPositionAndReset(); - if (!__range.contains(key)) { - throw new IllegalStateException("Yielded to key outside of range"); - } - __delegate = new QueryIterator(); - __delegate.init(__source, __options, __env); - __delegate.enableYielding(__yield); - __range = new Range(key, false, __range.getEndKey(), __range.isEndKeyInclusive()); - log.info("Yielded at " + __range.getStartKey()); - __delegate.seek(__range, __columnFamilies, __inclusive); - } } @Override public Key getTopKey() { - return __delegate.getTopKey(); + Key resultKey = __delegate.getTopKey(); + if (lastResultKey != null && resultKey != null && resultKey.compareTo(lastResultKey) < 0) { + throw new IllegalStateException( + "Result key does not follow the last key returned -- results should be sorted: " + resultKey + " <= " + lastResultKey); + } + lastResultKey = resultKey; + return resultKey; } @Override diff --git a/warehouse/query-core/src/test/java/datawave/query/ancestor/AncestorQueryIteratorIT.java b/warehouse/query-core/src/test/java/datawave/query/ancestor/AncestorQueryIteratorIT.java index 7cf3734244..852613cd08 100644 --- a/warehouse/query-core/src/test/java/datawave/query/ancestor/AncestorQueryIteratorIT.java +++ b/warehouse/query-core/src/test/java/datawave/query/ancestor/AncestorQueryIteratorIT.java @@ -1,19 +1,14 @@ package datawave.query.ancestor; -import java.io.IOException; - -import org.junit.Before; - import datawave.query.iterator.QueryIteratorIT; /** * AncestorQueryIterator integration tests. Ancestor Query should find any hits event query finds plus its own unique cases */ public class AncestorQueryIteratorIT extends QueryIteratorIT { - @Before - public void setup() throws IOException { - super.setup(); - iterator = new AncestorQueryIterator(); + + protected Class getIteratorClass() { + return AncestorQueryIterator.class; } /** diff --git a/warehouse/query-core/src/test/java/datawave/query/iterator/QueryIteratorIT.java b/warehouse/query-core/src/test/java/datawave/query/iterator/QueryIteratorIT.java index 7ed1ee8d3b..8525e251e4 100644 --- a/warehouse/query-core/src/test/java/datawave/query/iterator/QueryIteratorIT.java +++ b/warehouse/query-core/src/test/java/datawave/query/iterator/QueryIteratorIT.java @@ -18,7 +18,6 @@ import static datawave.query.iterator.QueryOptions.TERM_FREQUENCIES_REQUIRED; import static datawave.query.iterator.QueryOptions.TERM_FREQUENCY_FIELDS; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -34,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.stream.Collectors; import org.apache.accumulo.core.conf.DefaultConfiguration; @@ -62,9 +62,7 @@ import datawave.query.attributes.Attributes; import datawave.query.attributes.Document; import datawave.query.function.JexlEvaluation; -import datawave.query.function.deserializer.KryoDocumentDeserializer; import datawave.query.iterator.ivarator.IvaratorCacheDirConfig; -import datawave.query.predicate.EventDataQueryFilter; import datawave.query.util.TypeMetadata; /** @@ -76,11 +74,10 @@ public class QueryIteratorIT extends EasyMockSupport { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - protected QueryIterator iterator; + protected TestLookupTask lookupTask; protected SortedListKeyValueIterator baseIterator; protected Map options; protected IteratorEnvironment environment; - protected EventDataQueryFilter filter; protected TypeMetadata typeMetadata; // Default row is for day 20190314, shard 0 @@ -89,11 +86,18 @@ public class QueryIteratorIT extends EasyMockSupport { // Default if test does not specify a datatype protected static final String DEFAULT_DATATYPE = "dataType1"; + // Event time + protected long eventTimestamp = 11; + public Path tempPath; + protected Class getIteratorClass() { + return QueryIterator.class; + } + @Before public void setup() throws IOException { - iterator = new QueryIterator(); + lookupTask = new TestLookupTask(getIteratorClass()); options = new HashMap<>(); tempPath = temporaryFolder.newFolder().toPath(); @@ -117,7 +121,10 @@ public void setup() throws IOException { options.put(NON_INDEXED_DATATYPES, nonIndexedTypes); // set a query id - options.put(QUERY_ID, "000001"); + options.put(QUERY_ID, UUID.randomUUID().toString()); + + // set a scanId + options.put(SCAN_ID, UUID.randomUUID().toString()); // set a scanId options.put(SCAN_ID, "000002"); @@ -146,7 +153,6 @@ public void setup() throws IOException { environment = createMock(IteratorEnvironment.class); EasyMock.expect(environment.getConfig()).andReturn(DefaultConfiguration.getInstance()).anyTimes(); - filter = createMock(EventDataQueryFilter.class); } @After @@ -154,58 +160,58 @@ public void cleanUp() { tempPath.toFile().deleteOnExit(); } - private List> addEvent(long eventTime, String uid) { - return addEvent(DEFAULT_ROW, DEFAULT_DATATYPE, eventTime, uid); + protected List> addEvent(String uid) { + return addEvent(DEFAULT_ROW, DEFAULT_DATATYPE, uid); } - private List> addEvent(String row, String dataType, long eventTime, String uid) { + protected List> addEvent(String row, String dataType, String uid) { List> listSource = new ArrayList<>(); // indexed - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD1", "a", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "EVENT_FIELD1", "a", dataType, uid, eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD1", "a", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "EVENT_FIELD1", "a", dataType, uid), new Value())); // unindexed - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD2", "b", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD3", "c", dataType, uid, eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD2", "b", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD3", "c", dataType, uid), new Value())); // indexed - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD4", "d", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "EVENT_FIELD4", "d", dataType, uid, eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD4", "d", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "EVENT_FIELD4", "d", dataType, uid), new Value())); // unindexed - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD5", "e", dataType, uid, eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD5", "e", dataType, uid), new Value())); // indexed - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD6", "f", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "EVENT_FIELD6", "f", dataType, uid, eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "EVENT_FIELD6", "f", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "EVENT_FIELD6", "f", dataType, uid), new Value())); // add some indexed TF fields - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(DEFAULT_ROW, "TF_FIELD1", "a,, b,,, c,,", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "a b c", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "a", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "b", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "c", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD1", "a", dataType, uid, eventTime), getTFValue(0))); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD1", "b", dataType, uid, eventTime), getTFValue(1))); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD1", "c", dataType, uid, eventTime), getTFValue(2))); - - listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "TF_FIELD2", ",x, ,y, ,z,", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "x y z", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "x", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "y", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "z", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD2", "x", dataType, uid, eventTime), getTFValue(23))); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD2", "y", dataType, uid, eventTime), getTFValue(24))); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD2", "z", dataType, uid, eventTime), getTFValue(25))); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(DEFAULT_ROW, "TF_FIELD1", "a,, b,,, c,,", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "a b c", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "a", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "b", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD1", "c", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD1", "a", dataType, uid), getTFValue(0))); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD1", "b", dataType, uid), getTFValue(1))); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD1", "c", dataType, uid), getTFValue(2))); + + listSource.add(new AbstractMap.SimpleEntry<>(getEvent(row, "TF_FIELD2", ",x, ,y, ,z,", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "x y z", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "x", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "y", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD2", "z", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD2", "x", dataType, uid), getTFValue(23))); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD2", "y", dataType, uid), getTFValue(24))); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD2", "z", dataType, uid), getTFValue(25))); // add an index only TF - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD4", "d", dataType, uid, eventTime), getTFValue(3))); - listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD4", "d", dataType, uid, eventTime), getTFValue(3))); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "TF_FIELD4", "d", dataType, uid), getTFValue(3))); + listSource.add(new AbstractMap.SimpleEntry<>(getTF(row, "TF_FIELD4", "d", dataType, uid), getTFValue(3))); // add some index only field data - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD1", "apple", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD1", "pear", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD1", "orange", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD2", "beef", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD2", "chicken", dataType, uid, eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD2", "pork", dataType, uid, eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD1", "apple", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD1", "pear", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD1", "orange", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD2", "beef", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD2", "chicken", dataType, uid), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI(row, "INDEX_ONLY_FIELD2", "pork", dataType, uid), new Value())); return listSource; } @@ -213,7 +219,7 @@ private List> addEvent(String row, String dataType, long ev protected List> configureTestData(long eventTime) { List> listSource = new ArrayList<>(); - listSource.addAll(addEvent(eventTime, "123.345.456")); + listSource.addAll(addEvent("123.345.456")); return listSource; } @@ -296,7 +302,7 @@ public void indexOnly_documentSpecific_secondEvent_test() throws IOException { // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "INDEX_ONLY_FIELD1 == 'apple'"; - indexOnly_test(seekRange, query, false, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + indexOnly_test(seekRange, query, false, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -306,7 +312,7 @@ public void indexOnly_shardRange_secondEvent_test() throws IOException { String query = "INDEX_ONLY_FIELD1 == 'apple'"; Map.Entry>> secondEvent = getBaseExpectedEvent("123.345.457"); secondEvent.getValue().put("INDEX_ONLY_FIELD1", Arrays.asList(new String[] {"apple"})); - indexOnly_test(seekRange, query, false, addEvent(11, "123.345.457"), Arrays.asList(secondEvent)); + indexOnly_test(seekRange, query, false, addEvent("123.345.457"), Arrays.asList(secondEvent)); } @Test @@ -346,7 +352,7 @@ public void indexOnly_trailingRegex_documentSpecific_secondEvent_test() throws I // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "((_Value_ = true) && (INDEX_ONLY_FIELD1 =~ 'ap.*'))"; - indexOnly_test(seekRange, query, false, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + indexOnly_test(seekRange, query, false, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -356,7 +362,7 @@ public void indexOnly_trailingRegex_shardRange_secondEvent_test() throws IOExcep String query = "((_Value_ = true) && (INDEX_ONLY_FIELD1 =~ 'ap.*'))"; Map.Entry>> secondEvent = getBaseExpectedEvent("123.345.457"); secondEvent.getValue().put("INDEX_ONLY_FIELD1", Arrays.asList(new String[] {"apple"})); - indexOnly_test(seekRange, query, false, addEvent(11, "123.345.457"), Arrays.asList(secondEvent)); + indexOnly_test(seekRange, query, false, addEvent("123.345.457"), Arrays.asList(secondEvent)); } @Test @@ -396,7 +402,7 @@ public void indexOnly_leadingRegex_documentSpecific_secondEvent_test() throws IO // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "((_Value_ = true) && (INDEX_ONLY_FIELD1 =~ '.*le'))"; - indexOnly_test(seekRange, query, false, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + indexOnly_test(seekRange, query, false, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -406,7 +412,7 @@ public void indexOnly_leadingRegex_shardRange_secondEvent_test() throws IOExcept String query = "((_Value_ = true) && (INDEX_ONLY_FIELD1 =~ '.*le'))"; Map.Entry>> secondEvent = getBaseExpectedEvent("123.345.457"); secondEvent.getValue().put("INDEX_ONLY_FIELD1", Arrays.asList(new String[] {"apple"})); - indexOnly_test(seekRange, query, false, addEvent(11, "123.345.457"), Arrays.asList(secondEvent)); + indexOnly_test(seekRange, query, false, addEvent("123.345.457"), Arrays.asList(secondEvent)); } @Test @@ -464,7 +470,7 @@ public void event_documentSpecific_secondEvent_test() throws IOException { // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "EVENT_FIELD2 == 'b'"; - event_test(seekRange, query, false, null, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + event_test(seekRange, query, false, null, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -472,7 +478,7 @@ public void event_shardRange_secondEvent_test() throws IOException { // build the seek range for a document specific pull Range seekRange = getShardRange(); String query = "EVENT_FIELD2 == 'b'"; - event_test(seekRange, query, false, null, addEvent(11, "123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); + event_test(seekRange, query, false, null, addEvent("123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); } @Test @@ -512,7 +518,7 @@ public void event_trailingRegex_documentSpecific_secondEvent_test() throws IOExc // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "EVENT_FIELD2 =~ 'b.*'"; - event_test(seekRange, query, false, null, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + event_test(seekRange, query, false, null, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -520,7 +526,7 @@ public void event_trailingRegex_shardRange_secondEvent_test() throws IOException // build the seek range for a document specific pull Range seekRange = getShardRange(); String query = "EVENT_FIELD2 =~ 'b.*'"; - event_test(seekRange, query, false, null, addEvent(11, "123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); + event_test(seekRange, query, false, null, addEvent("123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); } @Test @@ -560,7 +566,7 @@ public void event_leadingRegex_documentSpecific_secondEvent_test() throws IOExce // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "EVENT_FIELD2 =~ '.*b'"; - event_test(seekRange, query, false, null, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + event_test(seekRange, query, false, null, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -568,7 +574,7 @@ public void event_leadingRegex_shardRange_secondEvent_test() throws IOException // build the seek range for a document specific pull Range seekRange = getShardRange(); String query = "EVENT_FIELD2 =~ '.*b'"; - event_test(seekRange, query, false, null, addEvent(11, "123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); + event_test(seekRange, query, false, null, addEvent("123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); } @Test(expected = IOException.class) @@ -643,7 +649,7 @@ public void index_documentSpecific_secondEvent_test() throws IOException { // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "EVENT_FIELD4 == 'd'"; - index_test(seekRange, query, false, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + index_test(seekRange, query, false, addEvent("123.345.457"), Collections.EMPTY_LIST); } /** @@ -657,7 +663,7 @@ public void index_shardRange_secondEvent_test() throws IOException { // build the seek range for a document specific pull Range seekRange = getShardRange(); String query = "EVENT_FIELD4 == 'd'"; - index_test(seekRange, query, false, addEvent(11, "123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); + index_test(seekRange, query, false, addEvent("123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); } @Test @@ -697,7 +703,7 @@ public void index_trailingRegex_documentSpecific_secondEvent_test() throws IOExc // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "((_Value_ = true) && (EVENT_FIELD4 =~ 'd.*'))"; - index_test(seekRange, query, false, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + index_test(seekRange, query, false, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -705,7 +711,7 @@ public void index_trailingRegex_shardRange_secondEvent_test() throws IOException // build the seek range for a document specific pull Range seekRange = getShardRange(); String query = "((_Value_ = true) && (EVENT_FIELD4 =~ 'd.*'))"; - index_test(seekRange, query, false, addEvent(11, "123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); + index_test(seekRange, query, false, addEvent("123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); } @Test @@ -745,7 +751,7 @@ public void index_leadingRegex_documentSpecific_secondEvent_test() throws IOExce // build the seek range for a document specific pull Range seekRange = getDocumentRange("123.345.456"); String query = "((_Value_ = true) && (EVENT_FIELD4 =~ '.*d'))"; - index_test(seekRange, query, false, addEvent(11, "123.345.457"), Collections.EMPTY_LIST); + index_test(seekRange, query, false, addEvent("123.345.457"), Collections.EMPTY_LIST); } @Test @@ -753,7 +759,7 @@ public void index_leadingRegex_shardRange_secondEvent_test() throws IOException // build the seek range for a document specific pull Range seekRange = getShardRange(); String query = "((_Value_ = true) && (EVENT_FIELD4 =~ '.*d'))"; - index_test(seekRange, query, false, addEvent(11, "123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); + index_test(seekRange, query, false, addEvent("123.345.457"), Arrays.asList(getBaseExpectedEvent("123.345.457"))); } @Test @@ -1028,9 +1034,7 @@ public void tf_event_exceededValue_negated_leadingWildcardMissIndexOnly_shardRan } protected void configureIterator() { - // configure iterator - iterator.setEvaluationFilter(filter); - iterator.setTypeMetadata(typeMetadata); + lookupTask.setTypeMetadata(typeMetadata); } /** @@ -1058,8 +1062,7 @@ protected void event_test(Range seekRange, String query, boolean miss, Map.Entry replayAll(); - iterator.init(baseIterator, options, environment); - iterator.seek(seekRange, Collections.EMPTY_LIST, true); + List> results = lookupTask.lookup(baseIterator, options, environment, Collections.singletonList(seekRange)); verifyAll(); @@ -1074,7 +1077,7 @@ protected void event_test(Range seekRange, String query, boolean miss, Map.Entry } } hits.addAll(otherHits); - eval(hits); + eval(results, hits); } /** @@ -1102,8 +1105,7 @@ protected void index_test(Range seekRange, String query, boolean miss, List> results = lookupTask.lookup(baseIterator, options, environment, Collections.singletonList(seekRange)); verifyAll(); @@ -1114,7 +1116,7 @@ protected void index_test(Range seekRange, String query, boolean miss, List> results = lookupTask.lookup(baseIterator, options, environment, Collections.singletonList(seekRange)); verifyAll(); @@ -1162,7 +1163,7 @@ protected void indexOnly_test(Range seekRange, String query, boolean miss, List< } hits.addAll(otherHits); - eval(hits); + eval(results, hits); } /** @@ -1192,8 +1193,7 @@ protected void tf_test(Range seekRange, String query, Map.Entry> results = lookupTask.lookup(baseIterator, options, environment, Collections.singletonList(seekRange)); verifyAll(); @@ -1203,7 +1203,7 @@ protected void tf_test(Range seekRange, String query, Map.Entry>> getBaseExpectedEvent(String ui } protected Map.Entry>> getBaseExpectedEvent(String row, String dataType, String uid) { - Key hitKey = new Key(row, dataType + Constants.NULL + uid); + Key hitKey = new Key(row, dataType + Constants.NULL + uid, "", "", eventTimestamp); Map> expectedDocument = new HashMap<>(); expectedDocument.put("EVENT_FIELD1", Arrays.asList(new String[] {"a"})); expectedDocument.put("EVENT_FIELD2", Arrays.asList(new String[] {"b"})); @@ -1284,13 +1284,14 @@ protected boolean isExpectHitTerm() { return options.get(HIT_LIST) != null && Boolean.parseBoolean(options.get(HIT_LIST)); } - protected void eval(List>>> toEval) throws IOException { + protected void eval(List> results, List>>> toEval) throws IOException { Iterator>>> evalIterator = toEval.iterator(); while (evalIterator.hasNext()) { Map.Entry>> evalPair = evalIterator.next(); - eval(evalPair.getKey(), evalPair.getValue()); + eval(results, evalPair.getKey(), evalPair.getValue()); } - assertFalse(iterator.hasTop()); + // there should be no other hits + assertTrue("Should be no more results", results.isEmpty()); } /** @@ -1303,23 +1304,20 @@ protected void eval(List>>> toEval) throws * @throws IOException * IOException for issues with read/write */ - protected void eval(Key docKeyHit, Map> docKeys) throws IOException { + protected void eval(List> results, Key docKeyHit, Map> docKeys) throws IOException { // asserts for a hit or miss if (docKeyHit == null) { - assertFalse(iterator.hasTop()); + assertTrue(results.isEmpty()); } else { - assertTrue("Expected hit, but got none", iterator.hasTop()); - Key next = iterator.getTopKey(); + assertTrue("Expected hit, but got none", results.size() > 0); + Map.Entry entry = results.remove(0); + Key next = entry.getKey(); assertNotNull(next); - assertEquals(next.getRow().toString(), docKeyHit.getRow().toString()); - assertEquals(next.getColumnFamily().toString(), docKeyHit.getColumnFamily().toString()); + assertEquals(docKeyHit.getRow().toString(), next.getRow().toString()); + assertEquals(docKeyHit.getColumnFamily().toString(), next.getColumnFamily().toString()); // asserts for document build - Value topValue = iterator.getTopValue(); - assertNotNull(topValue); - Map.Entry deserializedValue = deserialize(topValue); - assertNotNull(deserializedValue.getValue()); - Document d = deserializedValue.getValue(); + Document d = entry.getValue(); assertNotNull(d); // -1 is for RECORD_ID field and -1 for HIT_LIST if configured @@ -1368,26 +1366,18 @@ protected void eval(Key docKeyHit, Map> docKeys) throws IOEx assertEquals(0, expected.size()); } } - - // there should be no other hits - iterator.next(); } } - private Map.Entry deserialize(Value value) { - KryoDocumentDeserializer dser = new KryoDocumentDeserializer(); - return dser.apply(new AbstractMap.SimpleEntry(null, value)); - } - // support methods - protected Key getTF(String field, String value, String uid, long timestamp) { - return getTF(DEFAULT_ROW, field, value, DEFAULT_DATATYPE, uid, timestamp); + protected Key getTF(String field, String value, String uid) { + return getTF(DEFAULT_ROW, field, value, DEFAULT_DATATYPE, uid); } - protected Key getTF(String row, String field, String value, String dataType, String uid, long timestamp) { + protected Key getTF(String row, String field, String value, String dataType, String uid) { // CQ = dataType\0UID\0Normalized field value\0Field name return new Key(row, "tf", dataType + Constants.NULL_BYTE_STRING + uid + Constants.NULL_BYTE_STRING + value + Constants.NULL_BYTE_STRING + field, - timestamp); + eventTimestamp); } // Generate TermFrequencyOffsets from a key @@ -1397,20 +1387,20 @@ protected Value getTFValue(int position) { return new Value(info.toByteArray()); } - protected Key getFI(String field, String value, String uid, long timestamp) { - return getFI(DEFAULT_ROW, field, value, DEFAULT_DATATYPE, uid, timestamp); + protected Key getFI(String field, String value, String uid) { + return getFI(DEFAULT_ROW, field, value, DEFAULT_DATATYPE, uid); } - protected Key getFI(String row, String field, String value, String dataType, String uid, long timestamp) { + protected Key getFI(String row, String field, String value, String dataType, String uid) { return new Key(row, "fi" + Constants.NULL_BYTE_STRING + field.toUpperCase(), - value + Constants.NULL_BYTE_STRING + dataType + Constants.NULL_BYTE_STRING + uid, timestamp); + value + Constants.NULL_BYTE_STRING + dataType + Constants.NULL_BYTE_STRING + uid, eventTimestamp); } - protected Key getEvent(String field, String value, String uid, long timestamp) { - return getEvent(DEFAULT_ROW, field, value, DEFAULT_DATATYPE, uid, timestamp); + protected Key getEvent(String field, String value, String uid) { + return getEvent(DEFAULT_ROW, field, value, DEFAULT_DATATYPE, uid); } - protected Key getEvent(String row, String field, String value, String dataType, String uid, long timestamp) { - return new Key(row, dataType + Constants.NULL + uid, field + Constants.NULL + value, timestamp); + protected Key getEvent(String row, String field, String value, String dataType, String uid) { + return new Key(row, dataType + Constants.NULL + uid, field + Constants.NULL + value, eventTimestamp); } } diff --git a/warehouse/query-core/src/test/java/datawave/query/iterator/TestLookupTask.java b/warehouse/query-core/src/test/java/datawave/query/iterator/TestLookupTask.java new file mode 100644 index 0000000000..98f3e453ed --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/iterator/TestLookupTask.java @@ -0,0 +1,114 @@ +package datawave.query.iterator; + +import static datawave.query.function.LogTiming.TIMING_METADATA; +import static datawave.query.iterator.waitwindow.WaitWindowObserver.WAIT_WINDOW_OVERRUN; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.IteratorEnvironment; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.YieldCallback; +import org.apache.log4j.Logger; + +import datawave.query.attributes.Document; +import datawave.query.function.deserializer.KryoDocumentDeserializer; +import datawave.query.iterator.waitwindow.WaitWindowObserver; +import datawave.query.util.TypeMetadata; + +public class TestLookupTask { + + private static final Logger log = Logger.getLogger(TestLookupTask.class); + private TypeMetadata typeMetadata = null; + private Class iteratorClass; + private QueryIterator iterator; + + public TestLookupTask(Class iteratorClass) { + this.iteratorClass = iteratorClass; + } + + public void setTypeMetadata(TypeMetadata typeMetadata) { + this.typeMetadata = typeMetadata; + } + + private QueryIterator init(SortedKeyValueIterator source, Map options, IteratorEnvironment env, YieldCallback yield) + throws IOException { + try { + QueryIterator iter = this.iteratorClass.newInstance(); + iter.setTypeMetadata(this.typeMetadata); + iter.init(source, options, env); + iter.enableYielding(yield); + return iter; + } catch (Exception e) { + throw new IOException(e); + } + } + + public List> lookup(SortedKeyValueIterator source, Map options, IteratorEnvironment env, + List ranges) throws IOException { + List> results = new ArrayList<>(); + YieldCallback yield = new YieldCallback(); + this.iterator = init(source, options, env, yield); + for (Range range : ranges) { + boolean rangeCompleted = false; + Range r = range; + while (!rangeCompleted) { + log.trace("Seeking to range:" + r); + this.iterator.seek(r, Collections.EMPTY_LIST, false); + while (this.iterator.hasTop()) { + Document document = deserializeAndFilterDocument(this.iterator.getTopValue()); + if (document.getDictionary().size() > 0) { + results.add(new AbstractMap.SimpleEntry(this.iterator.getTopKey(), document)); + } else { + log.trace("Filtering out invalid document"); + } + this.iterator.next(); + } + if (yield.hasYielded()) { + Key yieldKey = yield.getPositionAndReset(); + if (!r.contains(yieldKey)) { + throw new IllegalStateException("Yielded to key outside of range " + yieldKey + " not in " + r); + } + if (!results.isEmpty()) { + Key lastResultKey = results.get(results.size() - 1).getKey(); + if (lastResultKey != null && yieldKey.compareTo(lastResultKey) <= 0) { + throw new IllegalStateException("Underlying iterator yielded to a position that does not follow the last key returned: " + yieldKey + + " <= " + lastResultKey); + } + } + log.debug("Yielded at " + yieldKey + " after seeking range " + r); + yield = new YieldCallback<>(); + this.iterator = init(source, options, env, yield); + r = new Range(yieldKey, false, r.getEndKey(), r.isEndKeyInclusive()); + } else { + rangeCompleted = true; + } + } + } + return results; + } + + private Document deserializeAndFilterDocument(Value value) { + Map.Entry deserializedValue = deserialize(value); + Document d = deserializedValue.getValue(); + Document filteredDocument = new Document(); + d.getDictionary().entrySet().stream().forEach(e -> { + if (!e.getKey().equals(TIMING_METADATA) && !e.getKey().equals(WAIT_WINDOW_OVERRUN)) { + filteredDocument.put(e.getKey(), e.getValue()); + } + }); + return filteredDocument; + } + + private Map.Entry deserialize(Value value) { + KryoDocumentDeserializer dser = new KryoDocumentDeserializer(); + return dser.apply(new AbstractMap.SimpleEntry(null, value)); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/iterator/WaitWindowQueryIteratorPipelineIT.java b/warehouse/query-core/src/test/java/datawave/query/iterator/WaitWindowQueryIteratorPipelineIT.java new file mode 100644 index 0000000000..9a66cf6c8a --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/iterator/WaitWindowQueryIteratorPipelineIT.java @@ -0,0 +1,18 @@ +package datawave.query.iterator; + +import static datawave.query.iterator.QueryOptions.MAX_EVALUATION_PIPELINES; +import static datawave.query.iterator.QueryOptions.SERIAL_EVALUATION_PIPELINE; + +import org.junit.Before; + +/** + * Integration test for WaitWindowObserver using the QueryIteratorIT and the PipelineIterator + */ +public class WaitWindowQueryIteratorPipelineIT extends WaitWindowQueryIteratorSerialIT { + + @Before + public void setupPipelineIterator() { + options.put(SERIAL_EVALUATION_PIPELINE, "false"); + options.put(MAX_EVALUATION_PIPELINES, "4"); + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/iterator/WaitWindowQueryIteratorSerialIT.java b/warehouse/query-core/src/test/java/datawave/query/iterator/WaitWindowQueryIteratorSerialIT.java new file mode 100644 index 0000000000..55095447a7 --- /dev/null +++ b/warehouse/query-core/src/test/java/datawave/query/iterator/WaitWindowQueryIteratorSerialIT.java @@ -0,0 +1,133 @@ +package datawave.query.iterator; + +import static datawave.query.iterator.QueryOptions.COLLECT_TIMING_DETAILS; +import static datawave.query.iterator.QueryOptions.MAX_EVALUATION_PIPELINES; +import static datawave.query.iterator.QueryOptions.SERIAL_EVALUATION_PIPELINE; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.junit.Before; +import org.junit.Test; + +import datawave.query.iterator.waitwindow.WaitWindowObserver; + +/** + * Integration test for WaitWindowObserver using the QueryIteratorIT and the SerialIterator + */ +public class WaitWindowQueryIteratorSerialIT extends QueryIteratorIT { + + @Override + protected Class getIteratorClass() { + return WaitWindowQueryIterator.class; + } + + @Before + public void setCollectDetails() { + options.put(COLLECT_TIMING_DETAILS, "true"); + } + + private void createEvents(int numEvents, List>>> otherHits, List> otherData) { + for (int x = 0; x < numEvents; x++) { + List> l = new ArrayList<>(); + String uid = "123.345." + String.format("%04d", x); + l.addAll(addEvent(uid)); + int mod = x % 3; + if (mod == 1 || mod == 2) { + l = l.stream().filter(e -> !e.getKey().getColumnQualifier().toString().startsWith("EVENT_FIELD1")).collect(Collectors.toList()); + l = l.stream().filter(e -> !e.getKey().getColumnFamily().toString().endsWith("EVENT_FIELD1")).collect(Collectors.toList()); + } + if (mod == 0 || mod == 2) { + l = l.stream().filter(e -> !e.getKey().getColumnQualifier().toString().startsWith("EVENT_FIELD4")).collect(Collectors.toList()); + l = l.stream().filter(e -> !e.getKey().getColumnFamily().toString().endsWith("EVENT_FIELD4")).collect(Collectors.toList()); + } + if (mod == 0 || mod == 1) { + l = l.stream().filter(e -> !e.getKey().getColumnQualifier().toString().startsWith("EVENT_FIELD6")).collect(Collectors.toList()); + l = l.stream().filter(e -> !e.getKey().getColumnFamily().toString().endsWith("EVENT_FIELD6")).collect(Collectors.toList()); + } + if (mod == 3) { + otherHits.add(getBaseExpectedEvent(uid)); + } + otherData.addAll(l); + } + } + + @Test + public void many_events() throws IOException { + List>>> otherHits = new ArrayList<>(); + List> otherData = new ArrayList<>(); + createEvents(2000, otherHits, otherData); + Range seekRange = getShardRange(); + String query = "EVENT_FIELD1 == 'a' && EVENT_FIELD4 == 'd' && EVENT_FIELD6 == 'f'"; + tf_test(seekRange, query, getBaseExpectedEvent("123.345.456"), otherData, otherHits); + } + + static public class WaitWindowQueryIterator extends QueryIterator { + + public WaitWindowQueryIterator() { + super(); + this.waitWindowObserver = new TestWaitWindowObserver(); + } + } + + // Custom WaitWindowObserver that allows checksBeforeYield checks + // before either yielding or reyurning a WAIT_WINDOW_OERRUN + // It will also limit the number of reseeks on the same startKey using maxNulls + static public class TestWaitWindowObserver extends WaitWindowObserver { + + private AtomicLong checksBeforeYield = new AtomicLong(2); + private long maxNulls = 5; + + public TestWaitWindowObserver() { + + } + + @Override + public void start(Range seekRange, long yieldThresholdMs) { + super.start(seekRange, Long.MAX_VALUE); + // limit the number of times that the test yields in the + // same place by counting the final null characters + Key startKey = seekRange.getStartKey(); + String s = null; + if (WaitWindowObserver.hasMarker(startKey.getColumnFamily())) { + s = startKey.getColumnFamily().toString(); + } else if (WaitWindowObserver.hasMarker(startKey.getColumnQualifier())) { + s = startKey.getColumnQualifier().toString(); + } + if (s != null && s.endsWith("\0")) { + long numNulls = 0; + for (int x = s.length() - 1; x > 0 && s.charAt(x) == '\0'; x--) { + numNulls++; + } + if (numNulls >= maxNulls) { + checksBeforeYield.set(Long.MAX_VALUE); + } + } + } + + @Override + public boolean waitWindowOverrun() { + if (checksBeforeYield.decrementAndGet() <= 0) { + return true; + } else { + return false; + } + } + + @Override + public long remainingTimeMs() { + if (checksBeforeYield.decrementAndGet() <= 0) { + return 0; + } else { + return Long.MAX_VALUE; + } + } + } +} diff --git a/warehouse/query-core/src/test/java/datawave/query/testframework/QueryLogicTestHarness.java b/warehouse/query-core/src/test/java/datawave/query/testframework/QueryLogicTestHarness.java index 3cdf08174b..9f2cc2fcac 100644 --- a/warehouse/query-core/src/test/java/datawave/query/testframework/QueryLogicTestHarness.java +++ b/warehouse/query-core/src/test/java/datawave/query/testframework/QueryLogicTestHarness.java @@ -21,6 +21,7 @@ import datawave.query.attributes.TypeAttribute; import datawave.query.function.deserializer.KryoDocumentDeserializer; import datawave.query.iterator.profile.FinalDocumentTrackingIterator; +import datawave.query.iterator.waitwindow.WaitWindowObserver; import datawave.webservice.query.logic.BaseQueryLogic; public class QueryLogicTestHarness { @@ -79,12 +80,16 @@ public void assertLogicResults(BaseQueryLogic> logic, Colle } for (Map.Entry entry : logic) { - if (FinalDocumentTrackingIterator.isFinalDocumentKey(entry.getKey())) { + if (FinalDocumentTrackingIterator.isFinalDocumentKey(entry.getKey()) || WaitWindowObserver.hasMarker(entry.getKey())) { continue; } final Document document = this.deserializer.apply(entry).getValue(); + if (document.containsKey(WaitWindowObserver.WAIT_WINDOW_OVERRUN)) { + continue; + } + // check all of the types to ensure that all are keepers as defined in the // AttributeFactory class int count = 0; diff --git a/warehouse/query-core/src/test/java/datawave/query/tld/TLDQueryIteratorIT.java b/warehouse/query-core/src/test/java/datawave/query/tld/TLDQueryIteratorIT.java index cc6753427a..fe4d7c0195 100644 --- a/warehouse/query-core/src/test/java/datawave/query/tld/TLDQueryIteratorIT.java +++ b/warehouse/query-core/src/test/java/datawave/query/tld/TLDQueryIteratorIT.java @@ -18,6 +18,7 @@ import org.junit.Before; import org.junit.Test; +import datawave.data.type.LcNoDiacriticsType; import datawave.query.Constants; import datawave.query.iterator.QueryIteratorIT; import datawave.query.iterator.QueryOptions; @@ -27,10 +28,14 @@ */ public class TLDQueryIteratorIT extends QueryIteratorIT { + @Override + protected Class getIteratorClass() { + return TLDQueryIterator.class; + } + @Before public void setup() throws IOException { super.setup(); - iterator = new TLDQueryIterator(); // update indexed options.put(INDEXED_FIELDS, options.get(INDEXED_FIELDS) + ",TF_FIELD3"); @@ -50,8 +55,7 @@ public void setup() throws IOException { @Override protected void configureIterator() { // configure iterator - iterator.setEvaluationFilter(null); - iterator.setTypeMetadata(typeMetadata); + lookupTask.setTypeMetadata(typeMetadata); } @Test @@ -206,30 +210,30 @@ private List> configureTLDTestData(long eventTime) { List> listSource = super.configureTestData(eventTime); // add some indexed TF fields in a child - listSource.add(new AbstractMap.SimpleEntry<>(getEvent("TF_FIELD1", ",,q ,r, ,s,", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "q r s", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "q", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "r", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "s", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD1", "q", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD1", "r", "123.345.456.1", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD1", "s", "123.345.456.1", eventTime), new Value())); - - listSource.add(new AbstractMap.SimpleEntry<>(getEvent("TF_FIELD2", ",d, ,e, ,f,", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "d e f", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "d", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "e", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "f", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD2", "d", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD2", "e", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD2", "f", "123.345.456.2", eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent("TF_FIELD1", ",,q ,r, ,s,", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "q r s", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "q", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "r", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD1", "s", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD1", "q", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD1", "r", "123.345.456.1"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD1", "s", "123.345.456.1"), new Value())); + + listSource.add(new AbstractMap.SimpleEntry<>(getEvent("TF_FIELD2", ",d, ,e, ,f,", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "d e f", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "d", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "e", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD2", "f", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD2", "d", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD2", "e", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getTF("TF_FIELD2", "f", "123.345.456.2"), new Value())); // add some event data for children - listSource.add(new AbstractMap.SimpleEntry<>(getEvent("EVENT_FIELD7", "1", "123.345.456.1", eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent("EVENT_FIELD7", "1", "123.345.456.1"), new Value())); // add some non-event data that is unique for children - listSource.add(new AbstractMap.SimpleEntry<>(getEvent("TF_FIELD3", "z", "123.345.456.2", eventTime), new Value())); - listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD3", "z", "123.345.456.2", eventTime), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getEvent("TF_FIELD3", "z", "123.345.456.2"), new Value())); + listSource.add(new AbstractMap.SimpleEntry<>(getFI("TF_FIELD3", "z", "123.345.456.2"), new Value())); return listSource; } diff --git a/warehouse/query-core/src/test/resources/log4j.properties b/warehouse/query-core/src/test/resources/log4j.properties index 6646cecab8..b30f367354 100644 --- a/warehouse/query-core/src/test/resources/log4j.properties +++ b/warehouse/query-core/src/test/resources/log4j.properties @@ -1,7 +1,7 @@ log4j.rootLogger=DEBUG, R log4j.appender.R=org.apache.log4j.ConsoleAppender log4j.appender.R.layout=org.apache.log4j.PatternLayout -log4j.appender.R.layout.ConversionPattern=%d %p %C:%L %t %m%n +log4j.appender.R.layout.ConversionPattern=%d %p %t %C:%L %t %m%n log4j.appender.R.encoding=UTF-8 #log4j.logger.org.apache.commons.jci=OFF