You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2015/11/23 21:46:19 UTC

[08/50] [abbrv] nifi git commit: NIFI-748 addressed PR comments - made DocReader package private - polished logic in read(..) method to avoid escaping the loop - added call to sorting logic in LuceneUtil.groupDocsByStorageFileName(..) to ensure that prev

NIFI-748 addressed PR comments
- made DocReader package private
- polished logic in read(..) method to avoid escaping the loop
- added call to sorting logic in LuceneUtil.groupDocsByStorageFileName(..) to ensure that previous behavior and assumptions in read(..) methodd are preserved
- other minor polishing


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/15880f9f
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/15880f9f
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/15880f9f

Branch: refs/heads/NIFI-655
Commit: 15880f9fcc78e7a040e0d64d2a517390122e3fe3
Parents: a4d93c6
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Mon Nov 16 08:39:23 2015 -0500
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Nov 16 08:39:23 2015 -0500

----------------------------------------------------------------------
 .../nifi/provenance/lucene/DocsReader.java      | 21 ++++++++++----------
 .../nifi/provenance/lucene/LuceneUtil.java      | 15 ++++++++++++--
 2 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/15880f9f/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
index 7bd800b..703a5b8 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
@@ -22,6 +22,7 @@ import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -43,7 +44,7 @@ import org.apache.lucene.search.TopDocs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class DocsReader {
+class DocsReader {
     private final Logger logger = LoggerFactory.getLogger(DocsReader.class);
 
     public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles,
@@ -106,12 +107,13 @@ public class DocsReader {
     public Set<ProvenanceEventRecord> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles,
             final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
 
+        if (retrievalCount.get() >= maxResults) {
+            return Collections.emptySet();
+        }
+
         final long start = System.nanoTime();
 
         Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>();
-        if (retrievalCount.get() >= maxResults) {
-            return matchingRecords;
-        }
 
         Map<String, List<Document>> byStorageNameDocGroups = LuceneUtil.groupDocsByStorageFileName(docs);
 
@@ -123,17 +125,16 @@ public class DocsReader {
             if (provenanceEventFile != null) {
                 try (RecordReader reader = RecordReaders.newRecordReader(provenanceEventFile, allProvenanceLogFiles,
                         maxAttributeChars)) {
-                    for (Document document : byStorageNameDocGroups.get(storageFileName)) {
-                        ProvenanceEventRecord eRec = this.getRecord(document, reader);
+
+                    Iterator<Document> docIter = byStorageNameDocGroups.get(storageFileName).iterator();
+                    while (docIter.hasNext() && retrievalCount.incrementAndGet() < maxResults){
+                        ProvenanceEventRecord eRec = this.getRecord(docIter.next(), reader);
                         if (eRec != null) {
                             matchingRecords.add(eRec);
                             eventsReadThisFile++;
-
-                            if (retrievalCount.incrementAndGet() >= maxResults) {
-                                break;
-                            }
                         }
                     }
+
                 } catch (Exception e) {
                     logger.warn("Failed while trying to read Provenance Events. The event file '"
                             + provenanceEventFile.getAbsolutePath() +

http://git-wip-us.apache.org/repos/asf/nifi/blob/15880f9f/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneUtil.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneUtil.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneUtil.java
index aa50b94..08a99d6 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneUtil.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneUtil.java
@@ -130,8 +130,14 @@ public class LuceneUtil {
         return luceneQuery;
     }
 
+    /**
+     * Will sort documents by filename and then file offset so that we can
+     * retrieve the records efficiently
+     *
+     * @param documents
+     *            list of {@link Document}s
+     */
     public static void sortDocsForRetrieval(final List<Document> documents) {
-        // sort by filename and then file offset so that we can retrieve the records efficiently
         Collections.sort(documents, new Comparator<Document>() {
             @Override
             public int compare(final Document o1, final Document o2) {
@@ -167,7 +173,9 @@ public class LuceneUtil {
      * Will group documents based on the {@link FieldNames#STORAGE_FILENAME}.
      *
      * @param documents
-     *            list of {@link Document}s
+     *            list of {@link Document}s which will be sorted via
+     *            {@link #sortDocsForRetrieval(List)} for more efficient record
+     *            retrieval.
      * @return a {@link Map} of document groups with
      *         {@link FieldNames#STORAGE_FILENAME} as key and {@link List} of
      *         {@link Document}s as value.
@@ -181,6 +189,9 @@ public class LuceneUtil {
             }
             documentGroups.get(fileName).add(document);
         }
+        for (List<Document> groupedDocuments : documentGroups.values()) {
+            sortDocsForRetrieval(groupedDocuments);
+        }
         return documentGroups;
     }
 }