mirror of https://github.com/apache/lucene.git
LUCENE-5680: add aotmic DocValues updates
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1598272 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a627b55bcd
commit
f99a9d7ab0
|
@ -119,6 +119,9 @@ New Features
|
||||||
optimized for primary-key (ID) fields that also record a version
|
optimized for primary-key (ID) fields that also record a version
|
||||||
(long) for each ID. (Robert Muir, Mike McCandless)
|
(long) for each ID. (Robert Muir, Mike McCandless)
|
||||||
|
|
||||||
|
* LUCENE-5680: Add ability to atomically update a set of DocValues
|
||||||
|
fields. (Shai Erera)
|
||||||
|
|
||||||
Changes in Backwards Compatibility Policy
|
Changes in Backwards Compatibility Policy
|
||||||
|
|
||||||
* LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This
|
* LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This
|
||||||
|
|
|
@ -101,7 +101,7 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||||
private final int bitsPerValue;
|
private final int bitsPerValue;
|
||||||
|
|
||||||
public BinaryDocValuesFieldUpdates(String field, int maxDoc) {
|
public BinaryDocValuesFieldUpdates(String field, int maxDoc) {
|
||||||
super(field, Type.BINARY);
|
super(field, FieldInfo.DocValuesType.BINARY);
|
||||||
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
||||||
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
||||||
offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||||
|
|
|
@ -33,8 +33,6 @@ abstract class DocValuesFieldUpdates {
|
||||||
|
|
||||||
protected static final int PAGE_SIZE = 1024;
|
protected static final int PAGE_SIZE = 1024;
|
||||||
|
|
||||||
static enum Type { NUMERIC, BINARY }
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An iterator over documents and their updated values. Only documents with
|
* An iterator over documents and their updated values. Only documents with
|
||||||
* updates are returned by this iterator, and the documents are returned in
|
* updates are returned by this iterator, and the documents are returned in
|
||||||
|
@ -100,7 +98,7 @@ abstract class DocValuesFieldUpdates {
|
||||||
return ramBytesPerDoc;
|
return ramBytesPerDoc;
|
||||||
}
|
}
|
||||||
|
|
||||||
DocValuesFieldUpdates getUpdates(String field, Type type) {
|
DocValuesFieldUpdates getUpdates(String field, FieldInfo.DocValuesType type) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case NUMERIC:
|
case NUMERIC:
|
||||||
return numericDVUpdates.get(field);
|
return numericDVUpdates.get(field);
|
||||||
|
@ -111,7 +109,7 @@ abstract class DocValuesFieldUpdates {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
DocValuesFieldUpdates newUpdates(String field, Type type, int maxDoc) {
|
DocValuesFieldUpdates newUpdates(String field, FieldInfo.DocValuesType type, int maxDoc) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case NUMERIC:
|
case NUMERIC:
|
||||||
assert numericDVUpdates.get(field) == null;
|
assert numericDVUpdates.get(field) == null;
|
||||||
|
@ -135,9 +133,9 @@ abstract class DocValuesFieldUpdates {
|
||||||
}
|
}
|
||||||
|
|
||||||
final String field;
|
final String field;
|
||||||
final Type type;
|
final FieldInfo.DocValuesType type;
|
||||||
|
|
||||||
protected DocValuesFieldUpdates(String field, Type type) {
|
protected DocValuesFieldUpdates(String field, FieldInfo.DocValuesType type) {
|
||||||
this.field = field;
|
this.field = field;
|
||||||
this.type = type;
|
this.type = type;
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,7 +39,7 @@ abstract class DocValuesUpdate {
|
||||||
*/
|
*/
|
||||||
private static final int RAW_SIZE_IN_BYTES = 8*NUM_BYTES_OBJECT_HEADER + 8*NUM_BYTES_OBJECT_REF + 8*NUM_BYTES_INT;
|
private static final int RAW_SIZE_IN_BYTES = 8*NUM_BYTES_OBJECT_HEADER + 8*NUM_BYTES_OBJECT_REF + 8*NUM_BYTES_INT;
|
||||||
|
|
||||||
final DocValuesFieldUpdates.Type type;
|
final FieldInfo.DocValuesType type;
|
||||||
final Term term;
|
final Term term;
|
||||||
final String field;
|
final String field;
|
||||||
final Object value;
|
final Object value;
|
||||||
|
@ -52,7 +52,7 @@ abstract class DocValuesUpdate {
|
||||||
* @param field the {@link NumericDocValuesField} to update
|
* @param field the {@link NumericDocValuesField} to update
|
||||||
* @param value the updated value
|
* @param value the updated value
|
||||||
*/
|
*/
|
||||||
protected DocValuesUpdate(DocValuesFieldUpdates.Type type, Term term, String field, Object value) {
|
protected DocValuesUpdate(FieldInfo.DocValuesType type, Term term, String field, Object value) {
|
||||||
this.type = type;
|
this.type = type;
|
||||||
this.term = term;
|
this.term = term;
|
||||||
this.field = field;
|
this.field = field;
|
||||||
|
@ -82,7 +82,7 @@ abstract class DocValuesUpdate {
|
||||||
private static final long RAW_VALUE_SIZE_IN_BYTES = NUM_BYTES_ARRAY_HEADER + 2*NUM_BYTES_INT + NUM_BYTES_OBJECT_REF;
|
private static final long RAW_VALUE_SIZE_IN_BYTES = NUM_BYTES_ARRAY_HEADER + 2*NUM_BYTES_INT + NUM_BYTES_OBJECT_REF;
|
||||||
|
|
||||||
BinaryDocValuesUpdate(Term term, String field, BytesRef value) {
|
BinaryDocValuesUpdate(Term term, String field, BytesRef value) {
|
||||||
super(DocValuesFieldUpdates.Type.BINARY, term, field, value);
|
super(FieldInfo.DocValuesType.BINARY, term, field, value);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -96,7 +96,7 @@ abstract class DocValuesUpdate {
|
||||||
static final class NumericDocValuesUpdate extends DocValuesUpdate {
|
static final class NumericDocValuesUpdate extends DocValuesUpdate {
|
||||||
|
|
||||||
NumericDocValuesUpdate(Term term, String field, Long value) {
|
NumericDocValuesUpdate(Term term, String field, Long value) {
|
||||||
super(DocValuesFieldUpdates.Type.NUMERIC, term, field, value);
|
super(FieldInfo.DocValuesType.NUMERIC, term, field, value);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -30,13 +30,10 @@ import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.index.DocumentsWriterFlushQueue.SegmentFlushTicket;
|
import org.apache.lucene.index.DocumentsWriterFlushQueue.SegmentFlushTicket;
|
||||||
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
|
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
|
||||||
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
||||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
|
||||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
|
||||||
import org.apache.lucene.index.IndexWriter.Event;
|
import org.apache.lucene.index.IndexWriter.Event;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.util.BytesRef;
|
|
||||||
import org.apache.lucene.util.InfoStream;
|
import org.apache.lucene.util.InfoStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -156,16 +153,9 @@ final class DocumentsWriter implements Closeable {
|
||||||
return applyAllDeletes( deleteQueue);
|
return applyAllDeletes( deleteQueue);
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized boolean updateNumericDocValue(Term term, String field, long value) throws IOException {
|
synchronized boolean updateDocValues(DocValuesUpdate... updates) throws IOException {
|
||||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||||
deleteQueue.addNumericUpdate(new NumericDocValuesUpdate(term, field, Long.valueOf(value)));
|
deleteQueue.addDocValuesUpdates(updates);
|
||||||
flushControl.doOnDelete();
|
|
||||||
return applyAllDeletes(deleteQueue);
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized boolean updateBinaryDocValue(Term term, String field, BytesRef value) throws IOException {
|
|
||||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
|
||||||
deleteQueue.addBinaryUpdate(new BinaryDocValuesUpdate(term, field, value));
|
|
||||||
flushControl.doOnDelete();
|
flushControl.doOnDelete();
|
||||||
return applyAllDeletes(deleteQueue);
|
return applyAllDeletes(deleteQueue);
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import java.util.concurrent.locks.ReentrantLock;
|
||||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes
|
* {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes
|
||||||
|
@ -109,13 +110,8 @@ final class DocumentsWriterDeleteQueue {
|
||||||
tryApplyGlobalSlice();
|
tryApplyGlobalSlice();
|
||||||
}
|
}
|
||||||
|
|
||||||
void addNumericUpdate(NumericDocValuesUpdate update) {
|
void addDocValuesUpdates(DocValuesUpdate... updates) {
|
||||||
add(new NumericUpdateNode(update));
|
add(new DocValuesUpdatesNode(updates));
|
||||||
tryApplyGlobalSlice();
|
|
||||||
}
|
|
||||||
|
|
||||||
void addBinaryUpdate(BinaryDocValuesUpdate update) {
|
|
||||||
add(new BinaryUpdateNode(update));
|
|
||||||
tryApplyGlobalSlice();
|
tryApplyGlobalSlice();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -392,37 +388,40 @@ final class DocumentsWriterDeleteQueue {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final class NumericUpdateNode extends Node<NumericDocValuesUpdate> {
|
private static final class DocValuesUpdatesNode extends Node<DocValuesUpdate[]> {
|
||||||
|
|
||||||
NumericUpdateNode(NumericDocValuesUpdate update) {
|
DocValuesUpdatesNode(DocValuesUpdate... updates) {
|
||||||
super(update);
|
super(updates);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void apply(BufferedUpdates bufferedUpdates, int docIDUpto) {
|
void apply(BufferedUpdates bufferedUpdates, int docIDUpto) {
|
||||||
bufferedUpdates.addNumericUpdate(item, docIDUpto);
|
for (DocValuesUpdate update : item) {
|
||||||
|
switch (update.type) {
|
||||||
|
case NUMERIC:
|
||||||
|
bufferedUpdates.addNumericUpdate(new NumericDocValuesUpdate(update.term, update.field, (Long) update.value), docIDUpto);
|
||||||
|
break;
|
||||||
|
case BINARY:
|
||||||
|
bufferedUpdates.addBinaryUpdate(new BinaryDocValuesUpdate(update.term, update.field, (BytesRef) update.value), docIDUpto);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException(update.type + " DocValues updates not supported yet!");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "update=" + item;
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("docValuesUpdates: ");
|
||||||
|
if (item.length > 0) {
|
||||||
|
sb.append("term=").append(item[0].term).append("; updates: [");
|
||||||
|
for (DocValuesUpdate update : item) {
|
||||||
|
sb.append(update.field).append(':').append(update.value).append(',');
|
||||||
}
|
}
|
||||||
|
sb.setCharAt(sb.length()-1, ']');
|
||||||
}
|
}
|
||||||
|
return sb.toString();
|
||||||
private static final class BinaryUpdateNode extends Node<BinaryDocValuesUpdate> {
|
|
||||||
|
|
||||||
BinaryUpdateNode(BinaryDocValuesUpdate update) {
|
|
||||||
super(update);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
void apply(BufferedUpdates bufferedUpdates, int docIDUpto) {
|
|
||||||
bufferedUpdates.addBinaryUpdate(item, docIDUpto);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "update=" + item;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -32,14 +32,17 @@ import java.util.Iterator;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
import java.util.Queue;
|
import java.util.Queue;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.codecs.Codec;
|
import org.apache.lucene.codecs.Codec;
|
||||||
|
import org.apache.lucene.document.Field;
|
||||||
|
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||||
|
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||||
import org.apache.lucene.index.FieldInfos.FieldNumbers;
|
import org.apache.lucene.index.FieldInfos.FieldNumbers;
|
||||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||||
|
@ -1429,7 +1432,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
throw new IllegalArgumentException("can only update existing numeric-docvalues fields!");
|
throw new IllegalArgumentException("can only update existing numeric-docvalues fields!");
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (docWriter.updateNumericDocValue(term, field, value)) {
|
if (docWriter.updateDocValues(new NumericDocValuesUpdate(term, field, value))) {
|
||||||
processEvents(true, false);
|
processEvents(true, false);
|
||||||
}
|
}
|
||||||
} catch (OutOfMemoryError oom) {
|
} catch (OutOfMemoryError oom) {
|
||||||
|
@ -1471,7 +1474,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
throw new IllegalArgumentException("can only update existing binary-docvalues fields!");
|
throw new IllegalArgumentException("can only update existing binary-docvalues fields!");
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (docWriter.updateBinaryDocValue(term, field, value)) {
|
if (docWriter.updateDocValues(new BinaryDocValuesUpdate(term, field, value))) {
|
||||||
processEvents(true, false);
|
processEvents(true, false);
|
||||||
}
|
}
|
||||||
} catch (OutOfMemoryError oom) {
|
} catch (OutOfMemoryError oom) {
|
||||||
|
@ -1479,6 +1482,56 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates documents' DocValues fields to the given values. Each field update
|
||||||
|
* is applied to the set of documents that are associated with the
|
||||||
|
* {@link Term} to the same value. All updates are atomically applied and
|
||||||
|
* flushed together.
|
||||||
|
*
|
||||||
|
* <p>
|
||||||
|
* <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
|
||||||
|
* close the writer. See <a href="#OOME">above</a> for details.
|
||||||
|
* </p>
|
||||||
|
*
|
||||||
|
* @param updates
|
||||||
|
* the updates to apply
|
||||||
|
* @throws CorruptIndexException
|
||||||
|
* if the index is corrupt
|
||||||
|
* @throws IOException
|
||||||
|
* if there is a low-level IO error
|
||||||
|
*/
|
||||||
|
public void updateDocValues(Term term, Field... updates) throws IOException {
|
||||||
|
ensureOpen();
|
||||||
|
DocValuesUpdate[] dvUpdates = new DocValuesUpdate[updates.length];
|
||||||
|
for (int i = 0; i < updates.length; i++) {
|
||||||
|
final Field f = updates[i];
|
||||||
|
final DocValuesType dvType = f.fieldType().docValueType();
|
||||||
|
if (dvType == null) {
|
||||||
|
throw new IllegalArgumentException("can only update NUMERIC or BINARY fields! field=" + f.name());
|
||||||
|
}
|
||||||
|
if (!globalFieldNumberMap.contains(f.name(), dvType)) {
|
||||||
|
throw new IllegalArgumentException("can only update existing docvalues fields! field=" + f.name() + ", type=" + dvType);
|
||||||
|
}
|
||||||
|
switch (dvType) {
|
||||||
|
case NUMERIC:
|
||||||
|
dvUpdates[i] = new NumericDocValuesUpdate(term, f.name(), (Long) f.numericValue());
|
||||||
|
break;
|
||||||
|
case BINARY:
|
||||||
|
dvUpdates[i] = new BinaryDocValuesUpdate(term, f.name(), f.binaryValue());
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("can only update NUMERIC or BINARY fields: field=" + f.name() + ", type=" + dvType);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
if (docWriter.updateDocValues(dvUpdates)) {
|
||||||
|
processEvents(true, false);
|
||||||
|
}
|
||||||
|
} catch (OutOfMemoryError oom) {
|
||||||
|
handleOOM(oom, "updateDocValues");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// for test purpose
|
// for test purpose
|
||||||
final synchronized int getSegmentCount(){
|
final synchronized int getSegmentCount(){
|
||||||
return segmentInfos.size();
|
return segmentInfos.size();
|
||||||
|
|
|
@ -86,7 +86,7 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||||
private int size;
|
private int size;
|
||||||
|
|
||||||
public NumericDocValuesFieldUpdates(String field, int maxDoc) {
|
public NumericDocValuesFieldUpdates(String field, int maxDoc) {
|
||||||
super(field, Type.NUMERIC);
|
super(field, FieldInfo.DocValuesType.NUMERIC);
|
||||||
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
||||||
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
||||||
values = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
values = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||||
|
|
|
@ -911,7 +911,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
final IndexWriter writer = new IndexWriter(dir, conf);
|
final IndexWriter writer = new IndexWriter(dir, conf);
|
||||||
|
|
||||||
// create index
|
// create index
|
||||||
final int numThreads = TestUtil.nextInt(random(), 3, 6);
|
final int numFields = TestUtil.nextInt(random(), 1, 4);
|
||||||
final int numDocs = atLeast(2000);
|
final int numDocs = atLeast(2000);
|
||||||
for (int i = 0; i < numDocs; i++) {
|
for (int i = 0; i < numDocs; i++) {
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
|
@ -923,7 +923,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
else if (group < 0.8) g = "g2";
|
else if (group < 0.8) g = "g2";
|
||||||
else g = "g3";
|
else g = "g3";
|
||||||
doc.add(new StringField("updKey", g, Store.NO));
|
doc.add(new StringField("updKey", g, Store.NO));
|
||||||
for (int j = 0; j < numThreads; j++) {
|
for (int j = 0; j < numFields; j++) {
|
||||||
long value = random().nextInt();
|
long value = random().nextInt();
|
||||||
doc.add(new BinaryDocValuesField("f" + j, toBytes(value)));
|
doc.add(new BinaryDocValuesField("f" + j, toBytes(value)));
|
||||||
doc.add(new BinaryDocValuesField("cf" + j, toBytes(value * 2))); // control, always updated to f * 2
|
doc.add(new BinaryDocValuesField("cf" + j, toBytes(value * 2))); // control, always updated to f * 2
|
||||||
|
@ -931,14 +931,13 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final int numThreads = TestUtil.nextInt(random(), 3, 6);
|
||||||
final CountDownLatch done = new CountDownLatch(numThreads);
|
final CountDownLatch done = new CountDownLatch(numThreads);
|
||||||
final AtomicInteger numUpdates = new AtomicInteger(atLeast(100));
|
final AtomicInteger numUpdates = new AtomicInteger(atLeast(100));
|
||||||
|
|
||||||
// same thread updates a field as well as reopens
|
// same thread updates a field as well as reopens
|
||||||
Thread[] threads = new Thread[numThreads];
|
Thread[] threads = new Thread[numThreads];
|
||||||
for (int i = 0; i < threads.length; i++) {
|
for (int i = 0; i < threads.length; i++) {
|
||||||
final String f = "f" + i;
|
|
||||||
final String cf = "cf" + i;
|
|
||||||
threads[i] = new Thread("UpdateThread-" + i) {
|
threads[i] = new Thread("UpdateThread-" + i) {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
@ -953,10 +952,13 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
else if (group < 0.5) t = new Term("updKey", "g1");
|
else if (group < 0.5) t = new Term("updKey", "g1");
|
||||||
else if (group < 0.8) t = new Term("updKey", "g2");
|
else if (group < 0.8) t = new Term("updKey", "g2");
|
||||||
else t = new Term("updKey", "g3");
|
else t = new Term("updKey", "g3");
|
||||||
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t);
|
|
||||||
|
final int field = random().nextInt(numFields);
|
||||||
|
final String f = "f" + field;
|
||||||
|
final String cf = "cf" + field;
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t + " field=" + field);
|
||||||
long updValue = random.nextInt();
|
long updValue = random.nextInt();
|
||||||
writer.updateBinaryDocValue(t, f, toBytes(updValue));
|
writer.updateDocValues(t, new BinaryDocValuesField(f, toBytes(updValue)), new BinaryDocValuesField(cf, toBytes(updValue*2)));
|
||||||
writer.updateBinaryDocValue(t, cf, toBytes(updValue * 2));
|
|
||||||
|
|
||||||
if (random.nextDouble() < 0.2) {
|
if (random.nextDouble() < 0.2) {
|
||||||
// delete a random document
|
// delete a random document
|
||||||
|
@ -1012,7 +1014,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
BytesRef scratch = new BytesRef();
|
BytesRef scratch = new BytesRef();
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
for (int i = 0; i < numThreads; i++) {
|
for (int i = 0; i < numFields; i++) {
|
||||||
BinaryDocValues bdv = r.getBinaryDocValues("f" + i);
|
BinaryDocValues bdv = r.getBinaryDocValues("f" + i);
|
||||||
BinaryDocValues control = r.getBinaryDocValues("cf" + i);
|
BinaryDocValues control = r.getBinaryDocValues("cf" + i);
|
||||||
Bits docsWithBdv = r.getDocsWithField("f" + i);
|
Bits docsWithBdv = r.getDocsWithField("f" + i);
|
||||||
|
@ -1054,8 +1056,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
int doc = random().nextInt(numDocs);
|
int doc = random().nextInt(numDocs);
|
||||||
Term t = new Term("id", "doc" + doc);
|
Term t = new Term("id", "doc" + doc);
|
||||||
long value = random().nextLong();
|
long value = random().nextLong();
|
||||||
writer.updateBinaryDocValue(t, "f", toBytes(value));
|
writer.updateDocValues(t, new BinaryDocValuesField("f", toBytes(value)), new BinaryDocValuesField("cf", toBytes(value*2)));
|
||||||
writer.updateBinaryDocValue(t, "cf", toBytes(value * 2));
|
|
||||||
DirectoryReader reader = DirectoryReader.open(writer, true);
|
DirectoryReader reader = DirectoryReader.open(writer, true);
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
|
@ -1146,8 +1147,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
// update some docs to a random value
|
// update some docs to a random value
|
||||||
long value = random().nextInt();
|
long value = random().nextInt();
|
||||||
Term term = new Term("id", RandomPicks.randomFrom(random(), randomTerms));
|
Term term = new Term("id", RandomPicks.randomFrom(random(), randomTerms));
|
||||||
writer.updateBinaryDocValue(term, "bdv", toBytes(value));
|
writer.updateDocValues(term, new BinaryDocValuesField("bdv", toBytes(value)), new BinaryDocValuesField("control", toBytes(value * 2)));
|
||||||
writer.updateBinaryDocValue(term, "control", toBytes(value * 2));
|
|
||||||
writer.shutdown();
|
writer.shutdown();
|
||||||
|
|
||||||
Directory dir2 = newDirectory();
|
Directory dir2 = newDirectory();
|
||||||
|
@ -1252,8 +1252,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
|
||||||
int field = random.nextInt(numBinaryFields);
|
int field = random.nextInt(numBinaryFields);
|
||||||
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
||||||
long value = random.nextInt();
|
long value = random.nextInt();
|
||||||
writer.updateBinaryDocValue(updateTerm, "f" + field, toBytes(value));
|
writer.updateDocValues(updateTerm, new BinaryDocValuesField("f" + field, toBytes(value)),
|
||||||
writer.updateBinaryDocValue(updateTerm, "cf" + field, toBytes(value * 2));
|
new BinaryDocValuesField("cf" + field, toBytes(value * 2)));
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.shutdown();
|
writer.shutdown();
|
||||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.lucene.analysis.MockTokenizer;
|
||||||
import org.apache.lucene.analysis.Token;
|
import org.apache.lucene.analysis.Token;
|
||||||
import org.apache.lucene.analysis.TokenFilter;
|
import org.apache.lucene.analysis.TokenFilter;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.Analyzer.TokenStreamComponents;
|
|
||||||
import org.apache.lucene.document.BinaryDocValuesField;
|
import org.apache.lucene.document.BinaryDocValuesField;
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
|
@ -1979,8 +1978,6 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
|
|
||||||
shouldFail.set(true);
|
shouldFail.set(true);
|
||||||
boolean doClose = false;
|
boolean doClose = false;
|
||||||
int updatingDocID = -1;
|
|
||||||
long updatingValue = -1;
|
|
||||||
try {
|
try {
|
||||||
boolean defaultCodecSupportsFieldUpdates = defaultCodecSupportsFieldUpdates();
|
boolean defaultCodecSupportsFieldUpdates = defaultCodecSupportsFieldUpdates();
|
||||||
for(int i=0;i<numDocs;i++) {
|
for(int i=0;i<numDocs;i++) {
|
||||||
|
@ -1993,28 +1990,18 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
System.out.println(" update id=" + docid + " to value " + value);
|
System.out.println(" update id=" + docid + " to value " + value);
|
||||||
}
|
}
|
||||||
Term idTerm = new Term("id", Integer.toString(docid));
|
Term idTerm = new Term("id", Integer.toString(docid));
|
||||||
updatingDocID = docid; // record that we're updating that document
|
|
||||||
updatingValue = value; // and its updating value
|
|
||||||
if (random().nextBoolean()) { // update only numeric field
|
if (random().nextBoolean()) { // update only numeric field
|
||||||
w.updateNumericDocValue(idTerm, "f", value);
|
w.updateDocValues(idTerm, new NumericDocValuesField("f", value), new NumericDocValuesField("cf", value*2));
|
||||||
w.updateNumericDocValue(idTerm, "cf", value * 2);
|
|
||||||
} else if (random().nextBoolean()) {
|
} else if (random().nextBoolean()) {
|
||||||
w.updateBinaryDocValue(idTerm, "bf", TestBinaryDocValuesUpdates.toBytes(value));
|
w.updateDocValues(idTerm, new BinaryDocValuesField("bf", TestBinaryDocValuesUpdates.toBytes(value)),
|
||||||
w.updateBinaryDocValue(idTerm, "bcf", TestBinaryDocValuesUpdates.toBytes(value * 2));
|
new BinaryDocValuesField("bcf", TestBinaryDocValuesUpdates.toBytes(value*2)));
|
||||||
} else {
|
} else {
|
||||||
w.updateNumericDocValue(idTerm, "f", value);
|
w.updateDocValues(idTerm,
|
||||||
w.updateNumericDocValue(idTerm, "cf", value * 2);
|
new NumericDocValuesField("f", value),
|
||||||
w.updateBinaryDocValue(idTerm, "bf", TestBinaryDocValuesUpdates.toBytes(value));
|
new NumericDocValuesField("cf", value*2),
|
||||||
w.updateBinaryDocValue(idTerm, "bcf", TestBinaryDocValuesUpdates.toBytes(value * 2));
|
new BinaryDocValuesField("bf", TestBinaryDocValuesUpdates.toBytes(value)),
|
||||||
|
new BinaryDocValuesField("bcf", TestBinaryDocValuesUpdates.toBytes(value*2)));
|
||||||
}
|
}
|
||||||
// record that we successfully updated the document. this is
|
|
||||||
// important when we later assert the value of the DV fields of
|
|
||||||
// that document - since we update two fields that depend on each
|
|
||||||
// other, could be that one of the fields successfully updates,
|
|
||||||
// while the other fails (since we turn on random exceptions).
|
|
||||||
// while this is supported, it makes the test raise false alarms.
|
|
||||||
updatingDocID = -1;
|
|
||||||
updatingValue = -1;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// sometimes do both deletes and updates
|
// sometimes do both deletes and updates
|
||||||
|
@ -2062,18 +2049,6 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
shouldFail.set(false);
|
shouldFail.set(false);
|
||||||
|
|
||||||
if (updatingDocID != -1) {
|
|
||||||
// Updating this document did not succeed. Since the fields we assert on
|
|
||||||
// depend on each other, and the update may have gone through halfway,
|
|
||||||
// replay the update on both numeric and binary DV fields, so later
|
|
||||||
// asserts succeed.
|
|
||||||
Term idTerm = new Term("id", ""+updatingDocID);
|
|
||||||
w.updateNumericDocValue(idTerm, "f", updatingValue);
|
|
||||||
w.updateNumericDocValue(idTerm, "cf", updatingValue * 2);
|
|
||||||
w.updateBinaryDocValue(idTerm, "bf", TestBinaryDocValuesUpdates.toBytes(updatingValue));
|
|
||||||
w.updateBinaryDocValue(idTerm, "bcf", TestBinaryDocValuesUpdates.toBytes(updatingValue * 2));
|
|
||||||
}
|
|
||||||
|
|
||||||
IndexReader r;
|
IndexReader r;
|
||||||
|
|
||||||
if (doClose && w != null) {
|
if (doClose && w != null) {
|
||||||
|
|
|
@ -160,6 +160,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
final IndexWriter writer = new IndexWriter(dir, conf);
|
final IndexWriter writer = new IndexWriter(dir, conf);
|
||||||
|
|
||||||
// create index
|
// create index
|
||||||
|
final int numFields = TestUtil.nextInt(random(), 2, 4);
|
||||||
final int numThreads = TestUtil.nextInt(random(), 3, 6);
|
final int numThreads = TestUtil.nextInt(random(), 3, 6);
|
||||||
final int numDocs = atLeast(2000);
|
final int numDocs = atLeast(2000);
|
||||||
for (int i = 0; i < numDocs; i++) {
|
for (int i = 0; i < numDocs; i++) {
|
||||||
|
@ -172,7 +173,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
else if (group < 0.8) g = "g2";
|
else if (group < 0.8) g = "g2";
|
||||||
else g = "g3";
|
else g = "g3";
|
||||||
doc.add(new StringField("updKey", g, Store.NO));
|
doc.add(new StringField("updKey", g, Store.NO));
|
||||||
for (int j = 0; j < numThreads; j++) {
|
for (int j = 0; j < numFields; j++) {
|
||||||
long value = random().nextInt();
|
long value = random().nextInt();
|
||||||
doc.add(new BinaryDocValuesField("f" + j, TestBinaryDocValuesUpdates.toBytes(value)));
|
doc.add(new BinaryDocValuesField("f" + j, TestBinaryDocValuesUpdates.toBytes(value)));
|
||||||
doc.add(new NumericDocValuesField("cf" + j, value * 2)); // control, always updated to f * 2
|
doc.add(new NumericDocValuesField("cf" + j, value * 2)); // control, always updated to f * 2
|
||||||
|
@ -186,8 +187,6 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
// same thread updates a field as well as reopens
|
// same thread updates a field as well as reopens
|
||||||
Thread[] threads = new Thread[numThreads];
|
Thread[] threads = new Thread[numThreads];
|
||||||
for (int i = 0; i < threads.length; i++) {
|
for (int i = 0; i < threads.length; i++) {
|
||||||
final String f = "f" + i;
|
|
||||||
final String cf = "cf" + i;
|
|
||||||
threads[i] = new Thread("UpdateThread-" + i) {
|
threads[i] = new Thread("UpdateThread-" + i) {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
@ -203,10 +202,13 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
else if (group < 0.8) t = new Term("updKey", "g2");
|
else if (group < 0.8) t = new Term("updKey", "g2");
|
||||||
else t = new Term("updKey", "g3");
|
else t = new Term("updKey", "g3");
|
||||||
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t);
|
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t);
|
||||||
|
int field = random().nextInt(numFields);
|
||||||
|
final String f = "f" + field;
|
||||||
|
final String cf = "cf" + field;
|
||||||
long updValue = random.nextInt();
|
long updValue = random.nextInt();
|
||||||
// System.err.println("[" + Thread.currentThread().getName() + "] t=" + t + ", f=" + f + ", updValue=" + updValue);
|
// System.err.println("[" + Thread.currentThread().getName() + "] t=" + t + ", f=" + f + ", updValue=" + updValue);
|
||||||
writer.updateBinaryDocValue(t, f, TestBinaryDocValuesUpdates.toBytes(updValue));
|
writer.updateDocValues(t, new BinaryDocValuesField(f, TestBinaryDocValuesUpdates.toBytes(updValue)),
|
||||||
writer.updateNumericDocValue(t, cf, updValue * 2);
|
new NumericDocValuesField(cf, updValue*2));
|
||||||
|
|
||||||
if (random.nextDouble() < 0.2) {
|
if (random.nextDouble() < 0.2) {
|
||||||
// delete a random document
|
// delete a random document
|
||||||
|
@ -262,7 +264,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
BytesRef scratch = new BytesRef();
|
BytesRef scratch = new BytesRef();
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
for (int i = 0; i < numThreads; i++) {
|
for (int i = 0; i < numFields; i++) {
|
||||||
BinaryDocValues bdv = r.getBinaryDocValues("f" + i);
|
BinaryDocValues bdv = r.getBinaryDocValues("f" + i);
|
||||||
NumericDocValues control = r.getNumericDocValues("cf" + i);
|
NumericDocValues control = r.getNumericDocValues("cf" + i);
|
||||||
Bits docsWithBdv = r.getDocsWithField("f" + i);
|
Bits docsWithBdv = r.getDocsWithField("f" + i);
|
||||||
|
@ -270,8 +272,8 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
Bits liveDocs = r.getLiveDocs();
|
Bits liveDocs = r.getLiveDocs();
|
||||||
for (int j = 0; j < r.maxDoc(); j++) {
|
for (int j = 0; j < r.maxDoc(); j++) {
|
||||||
if (liveDocs == null || liveDocs.get(j)) {
|
if (liveDocs == null || liveDocs.get(j)) {
|
||||||
assertEquals(docsWithBdv.get(j), docsWithControl.get(j));
|
assertTrue(docsWithBdv.get(j));
|
||||||
if (docsWithBdv.get(j)) {
|
assertTrue(docsWithControl.get(j));
|
||||||
long ctrlValue = control.get(j);
|
long ctrlValue = control.get(j);
|
||||||
long bdvValue = TestBinaryDocValuesUpdates.getValue(bdv, j, scratch) * 2;
|
long bdvValue = TestBinaryDocValuesUpdates.getValue(bdv, j, scratch) * 2;
|
||||||
// if (ctrlValue != bdvValue) {
|
// if (ctrlValue != bdvValue) {
|
||||||
|
@ -282,7 +284,6 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
reader.close();
|
reader.close();
|
||||||
|
|
||||||
dir.close();
|
dir.close();
|
||||||
|
@ -310,8 +311,8 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
int doc = random().nextInt(numDocs);
|
int doc = random().nextInt(numDocs);
|
||||||
Term t = new Term("id", "doc" + doc);
|
Term t = new Term("id", "doc" + doc);
|
||||||
long value = random().nextLong();
|
long value = random().nextLong();
|
||||||
writer.updateBinaryDocValue(t, "f", TestBinaryDocValuesUpdates.toBytes(value));
|
writer.updateDocValues(t, new BinaryDocValuesField("f", TestBinaryDocValuesUpdates.toBytes(value)),
|
||||||
writer.updateNumericDocValue(t, "cf", value * 2);
|
new NumericDocValuesField("cf", value*2));
|
||||||
DirectoryReader reader = DirectoryReader.open(writer, true);
|
DirectoryReader reader = DirectoryReader.open(writer, true);
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
|
@ -373,8 +374,8 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
|
||||||
int field = random.nextInt(numBinaryFields);
|
int field = random.nextInt(numBinaryFields);
|
||||||
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
||||||
long value = random.nextInt();
|
long value = random.nextInt();
|
||||||
writer.updateBinaryDocValue(updateTerm, "f" + field, TestBinaryDocValuesUpdates.toBytes(value));
|
writer.updateDocValues(updateTerm, new BinaryDocValuesField("f"+field, TestBinaryDocValuesUpdates.toBytes(value)),
|
||||||
writer.updateNumericDocValue(updateTerm, "cf" + field, value * 2);
|
new NumericDocValuesField("cf"+field, value*2));
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.shutdown();
|
writer.shutdown();
|
||||||
|
|
|
@ -893,7 +893,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
final IndexWriter writer = new IndexWriter(dir, conf);
|
final IndexWriter writer = new IndexWriter(dir, conf);
|
||||||
|
|
||||||
// create index
|
// create index
|
||||||
final int numThreads = TestUtil.nextInt(random(), 3, 6);
|
final int numFields = TestUtil.nextInt(random(), 1, 4);
|
||||||
final int numDocs = atLeast(2000);
|
final int numDocs = atLeast(2000);
|
||||||
for (int i = 0; i < numDocs; i++) {
|
for (int i = 0; i < numDocs; i++) {
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
|
@ -905,7 +905,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
else if (group < 0.8) g = "g2";
|
else if (group < 0.8) g = "g2";
|
||||||
else g = "g3";
|
else g = "g3";
|
||||||
doc.add(new StringField("updKey", g, Store.NO));
|
doc.add(new StringField("updKey", g, Store.NO));
|
||||||
for (int j = 0; j < numThreads; j++) {
|
for (int j = 0; j < numFields; j++) {
|
||||||
long value = random().nextInt();
|
long value = random().nextInt();
|
||||||
doc.add(new NumericDocValuesField("f" + j, value));
|
doc.add(new NumericDocValuesField("f" + j, value));
|
||||||
doc.add(new NumericDocValuesField("cf" + j, value * 2)); // control, always updated to f * 2
|
doc.add(new NumericDocValuesField("cf" + j, value * 2)); // control, always updated to f * 2
|
||||||
|
@ -913,14 +913,13 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final int numThreads = TestUtil.nextInt(random(), 3, 6);
|
||||||
final CountDownLatch done = new CountDownLatch(numThreads);
|
final CountDownLatch done = new CountDownLatch(numThreads);
|
||||||
final AtomicInteger numUpdates = new AtomicInteger(atLeast(100));
|
final AtomicInteger numUpdates = new AtomicInteger(atLeast(100));
|
||||||
|
|
||||||
// same thread updates a field as well as reopens
|
// same thread updates a field as well as reopens
|
||||||
Thread[] threads = new Thread[numThreads];
|
Thread[] threads = new Thread[numThreads];
|
||||||
for (int i = 0; i < threads.length; i++) {
|
for (int i = 0; i < threads.length; i++) {
|
||||||
final String f = "f" + i;
|
|
||||||
final String cf = "cf" + i;
|
|
||||||
threads[i] = new Thread("UpdateThread-" + i) {
|
threads[i] = new Thread("UpdateThread-" + i) {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
@ -935,10 +934,13 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
else if (group < 0.5) t = new Term("updKey", "g1");
|
else if (group < 0.5) t = new Term("updKey", "g1");
|
||||||
else if (group < 0.8) t = new Term("updKey", "g2");
|
else if (group < 0.8) t = new Term("updKey", "g2");
|
||||||
else t = new Term("updKey", "g3");
|
else t = new Term("updKey", "g3");
|
||||||
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t);
|
|
||||||
|
final int field = random().nextInt(numFields);
|
||||||
|
final String f = "f" + field;
|
||||||
|
final String cf = "cf" + field;
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t + " field=" + field);
|
||||||
long updValue = random.nextInt();
|
long updValue = random.nextInt();
|
||||||
writer.updateNumericDocValue(t, f, updValue);
|
writer.updateDocValues(t, new NumericDocValuesField(f, updValue), new NumericDocValuesField(cf, updValue*2));
|
||||||
writer.updateNumericDocValue(t, cf, updValue * 2);
|
|
||||||
|
|
||||||
if (random.nextDouble() < 0.2) {
|
if (random.nextDouble() < 0.2) {
|
||||||
// delete a random document
|
// delete a random document
|
||||||
|
@ -993,7 +995,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
DirectoryReader reader = DirectoryReader.open(dir);
|
DirectoryReader reader = DirectoryReader.open(dir);
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
for (int i = 0; i < numThreads; i++) {
|
for (int i = 0; i < numFields; i++) {
|
||||||
NumericDocValues ndv = r.getNumericDocValues("f" + i);
|
NumericDocValues ndv = r.getNumericDocValues("f" + i);
|
||||||
NumericDocValues control = r.getNumericDocValues("cf" + i);
|
NumericDocValues control = r.getNumericDocValues("cf" + i);
|
||||||
Bits docsWithNdv = r.getDocsWithField("f" + i);
|
Bits docsWithNdv = r.getDocsWithField("f" + i);
|
||||||
|
@ -1035,8 +1037,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
int doc = random().nextInt(numDocs);
|
int doc = random().nextInt(numDocs);
|
||||||
Term t = new Term("id", "doc" + doc);
|
Term t = new Term("id", "doc" + doc);
|
||||||
long value = random().nextLong();
|
long value = random().nextLong();
|
||||||
writer.updateNumericDocValue(t, "f", value);
|
writer.updateDocValues(t, new NumericDocValuesField("f", value), new NumericDocValuesField("cf", value*2));
|
||||||
writer.updateNumericDocValue(t, "cf", value * 2);
|
|
||||||
DirectoryReader reader = DirectoryReader.open(writer, true);
|
DirectoryReader reader = DirectoryReader.open(writer, true);
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
|
@ -1128,8 +1129,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
// update some docs to a random value
|
// update some docs to a random value
|
||||||
long value = random().nextInt();
|
long value = random().nextInt();
|
||||||
Term term = new Term("id", RandomPicks.randomFrom(random(), randomTerms));
|
Term term = new Term("id", RandomPicks.randomFrom(random(), randomTerms));
|
||||||
writer.updateNumericDocValue(term, "ndv", value);
|
writer.updateDocValues(term, new NumericDocValuesField("ndv", value), new NumericDocValuesField("control", value*2));
|
||||||
writer.updateNumericDocValue(term, "control", value * 2);
|
|
||||||
writer.shutdown();
|
writer.shutdown();
|
||||||
|
|
||||||
Directory dir2 = newDirectory();
|
Directory dir2 = newDirectory();
|
||||||
|
@ -1235,8 +1235,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
int field = random.nextInt(numNumericFields);
|
int field = random.nextInt(numNumericFields);
|
||||||
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms));
|
||||||
long value = random.nextInt();
|
long value = random.nextInt();
|
||||||
writer.updateNumericDocValue(updateTerm, "f" + field, value);
|
writer.updateDocValues(updateTerm, new NumericDocValuesField("f"+field, value), new NumericDocValuesField("cf"+field, value*2));
|
||||||
writer.updateNumericDocValue(updateTerm, "cf" + field, value * 2);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.shutdown();
|
writer.shutdown();
|
||||||
|
|
|
@ -25,6 +25,7 @@ import java.util.Random;
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.codecs.Codec;
|
import org.apache.lucene.codecs.Codec;
|
||||||
|
import org.apache.lucene.document.Field;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -239,6 +240,11 @@ public class RandomIndexWriter implements Closeable {
|
||||||
w.updateBinaryDocValue(term, field, value);
|
w.updateBinaryDocValue(term, field, value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void updateDocValues(Term term, Field... updates) throws IOException {
|
||||||
|
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
|
||||||
|
w.updateDocValues(term, updates);
|
||||||
|
}
|
||||||
|
|
||||||
public void deleteDocuments(Term term) throws IOException {
|
public void deleteDocuments(Term term) throws IOException {
|
||||||
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
|
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
|
||||||
w.deleteDocuments(term);
|
w.deleteDocuments(term);
|
||||||
|
|
Loading…
Reference in New Issue