You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/06/02 19:55:55 UTC
svn commit: r1599282 - in
/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene:
index/IndexWriter.java search/LiveFieldValues.java
Author: mikemccand
Date: Mon Jun 2 17:55:55 2014
New Revision: 1599282
URL: http://svn.apache.org/r1599282
Log:
if infoStream is enabled after IW init still print the dir/config
Modified:
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1599282&r1=1599281&r2=1599282&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Mon Jun 2 17:55:55 2014
@@ -261,6 +261,7 @@ public class IndexWriter implements Clos
private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<>();
private long mergeGen;
private boolean stopMerges;
+ private boolean didMessageState;
final AtomicInteger flushCount = new AtomicInteger();
final AtomicInteger flushDeletesCount = new AtomicInteger();
@@ -842,7 +843,8 @@ public class IndexWriter implements Clos
}
private void messageState() {
- if (infoStream.isEnabled("IW")) {
+ if (infoStream.isEnabled("IW") && didMessageState == false) {
+ didMessageState = true;
infoStream.message("IW", "\ndir=" + directory + "\n" +
"index=" + segString() + "\n" +
"version=" + Constants.LUCENE_VERSION + "\n" +
@@ -1885,6 +1887,11 @@ public class IndexWriter implements Clos
private synchronized boolean updatePendingMerges(MergeTrigger trigger, int maxNumSegments)
throws IOException {
+
+ // In case infoStream was disabled on init, but then enabled at some
+ // point, try again to log the config here:
+ messageState();
+
assert maxNumSegments == -1 || maxNumSegments > 0;
assert trigger != null;
if (stopMerges) {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java?rev=1599282&r1=1599281&r2=1599282&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java Mon Jun 2 17:55:55 2014
@@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentHa
* the same time by two threads, because in this case you
* cannot in general know which thread "won". */
+// TODO: should this class handle deletions better...?
public abstract class LiveFieldValues<S,T> implements ReferenceManager.RefreshListener, Closeable {
private volatile Map<String,T> current = new ConcurrentHashMap<>();
@@ -40,6 +41,7 @@ public abstract class LiveFieldValues<S,
private final ReferenceManager<S> mgr;
private final T missingValue;
+ /** The missingValue must be non-null. */
public LiveFieldValues(ReferenceManager<S> mgr, T missingValue) {
this.missingValue = missingValue;
this.mgr = mgr;