github.com/balzaczyy/golucene@v0.0.0-20151210033525-d0be9ee89713/core/index/dwpt.go (about) 1 package index 2 3 import ( 4 "fmt" 5 "github.com/balzaczyy/golucene/core/analysis" 6 . "github.com/balzaczyy/golucene/core/codec/spi" 7 . "github.com/balzaczyy/golucene/core/index/model" 8 "github.com/balzaczyy/golucene/core/store" 9 "github.com/balzaczyy/golucene/core/util" 10 "log" 11 "sync/atomic" 12 ) 13 14 // index/DocumentsWriterPerThread.java 15 16 const DWPT_VERBOSE = false 17 18 // Returns the DocConsumer that the DocumentsWriter calls to 19 // process the documents. 20 type IndexingChain func(documentsWriterPerThread *DocumentsWriterPerThread) DocConsumer 21 22 var defaultIndexingChain = func(dwpt *DocumentsWriterPerThread) DocConsumer { 23 return newDefaultIndexingChain(dwpt) 24 } 25 26 type docState struct { 27 docWriter *DocumentsWriterPerThread 28 analyzer analysis.Analyzer 29 infoStream util.InfoStream 30 similarity Similarity 31 docID int 32 doc []IndexableField 33 } 34 35 func newDocState(docWriter *DocumentsWriterPerThread, infoStream util.InfoStream) *docState { 36 return &docState{ 37 docWriter: docWriter, 38 infoStream: infoStream, 39 } 40 } 41 42 func (ds *docState) testPoint(name string) { 43 ds.docWriter.testPoint(name) 44 } 45 46 func (ds *docState) clear() { 47 // don't hold onto doc nor analyzer, in case it is largish: 48 ds.doc = nil 49 ds.analyzer = nil 50 } 51 52 type FlushedSegment struct { 53 segmentInfo *SegmentCommitInfo 54 fieldInfos FieldInfos 55 segmentUpdates *FrozenBufferedUpdates 56 liveDocs util.MutableBits 57 delCount int 58 } 59 60 func newFlushedSegment(segmentInfo *SegmentCommitInfo, 61 fieldInfos FieldInfos, segmentUpdates *BufferedUpdates, 62 liveDocs util.MutableBits, delCount int) *FlushedSegment { 63 64 var sd *FrozenBufferedUpdates 65 if segmentUpdates != nil && segmentUpdates.any() { 66 sd = freezeBufferedUpdates(segmentUpdates, true) 67 } 68 return &FlushedSegment{segmentInfo, fieldInfos, sd, liveDocs, delCount} 69 } 70 71 type DocumentsWriterPerThread struct { 72 codec Codec 73 directory *store.TrackingDirectoryWrapper 74 directoryOrig store.Directory 75 docState *docState 76 consumer DocConsumer 77 _bytesUsed util.Counter 78 79 // Deletes for our still-in-RAM (to be flushed next) segment 80 pendingUpdates *BufferedUpdates 81 segmentInfo *SegmentInfo // Current segment we are working on 82 aborting bool // True if an abort is pending 83 hasAborted bool // True if the last exception throws by #updateDocument was aborting 84 85 fieldInfos *FieldInfosBuilder 86 infoStream util.InfoStream 87 numDocsInRAM int // the number of RAM resident documents 88 deleteQueue *DocumentsWriterDeleteQueue 89 deleteSlice *DeleteSlice 90 byteBlockAllocator util.ByteAllocator 91 intBlockAllocator util.IntAllocator 92 pendingNumDocs *int64 93 indexWriterConfig LiveIndexWriterConfig 94 95 filesToDelete map[string]bool 96 } 97 98 func newDocumentsWriterPerThread(segmentName string, 99 directory store.Directory, indexWriterConfig LiveIndexWriterConfig, 100 infoStream util.InfoStream, deleteQueue *DocumentsWriterDeleteQueue, 101 fieldInfos *FieldInfosBuilder, pendingNumDocs *int64) *DocumentsWriterPerThread { 102 103 counter := util.NewCounter() 104 ans := &DocumentsWriterPerThread{ 105 directoryOrig: directory, 106 directory: store.NewTrackingDirectoryWrapper(directory), 107 fieldInfos: fieldInfos, 108 pendingNumDocs: pendingNumDocs, 109 indexWriterConfig: indexWriterConfig, 110 infoStream: infoStream, 111 codec: indexWriterConfig.Codec(), 112 _bytesUsed: counter, 113 byteBlockAllocator: util.NewDirectTrackingAllocator(counter), 114 pendingUpdates: newBufferedUpdates(), 115 intBlockAllocator: newIntBlockAllocator(counter), 116 deleteQueue: deleteQueue, 117 deleteSlice: deleteQueue.newSlice(), 118 segmentInfo: NewSegmentInfo(directory, util.VERSION_LATEST, segmentName, -1, false, indexWriterConfig.Codec(), nil), 119 filesToDelete: make(map[string]bool), 120 } 121 ans.docState = newDocState(ans, infoStream) 122 ans.docState.similarity = indexWriterConfig.Similarity() 123 assert2(ans.numDocsInRAM == 0, "num docs %v", ans.numDocsInRAM) 124 if DWPT_VERBOSE && infoStream.IsEnabled("DWPT") { 125 infoStream.Message("DWPT", "init seg=%v delQueue=%v", segmentName, deleteQueue) 126 } 127 // this should be the last call in the ctor 128 // it really sucks that we need to pull this within the ctor and pass this ref to the chain! 129 ans.consumer = indexWriterConfig.indexingChain()(ans) 130 return ans 131 } 132 133 /* 134 Called if we hit an error at a bad time (when updating the index 135 files) and must discard all currently buffered docs. This resets our 136 state, discarding any docs added since last flush. 137 */ 138 func (dwpt *DocumentsWriterPerThread) abort(createdFiles map[string]bool) { 139 assert(createdFiles != nil) 140 log.Printf("now abort seg=%v", dwpt.segmentInfo.Name) 141 dwpt.hasAborted, dwpt.aborting = true, true 142 defer func() { 143 dwpt.aborting = false 144 if dwpt.infoStream.IsEnabled("DWPT") { 145 dwpt.infoStream.Message("DWPT", "done abort") 146 } 147 }() 148 149 if dwpt.infoStream.IsEnabled("DWPT") { 150 dwpt.infoStream.Message("DWPT", "now abort") 151 } 152 dwpt.consumer.abort() 153 154 dwpt.pendingUpdates.clear() 155 dwpt.directory.EachCreatedFiles(func(file string) { 156 createdFiles[file] = true 157 }) 158 } 159 160 func (dwpt *DocumentsWriterPerThread) setAborting() { 161 dwpt.aborting = true 162 } 163 164 func (dwpt *DocumentsWriterPerThread) checkAndResetHasAborted() (res bool) { 165 res, dwpt.hasAborted = dwpt.hasAborted, false 166 return 167 } 168 169 func (dwpt *DocumentsWriterPerThread) testPoint(msg string) { 170 if dwpt.infoStream.IsEnabled("TP") { 171 dwpt.infoStream.Message("TP", msg) 172 } 173 } 174 175 /* 176 Anything that will add N docs to the index should reserve first to 177 make sure it's allowed. 178 */ 179 func (dwpt *DocumentsWriterPerThread) reserveDoc() { 180 if atomic.AddInt64(dwpt.pendingNumDocs, 1) > int64(actualMaxDocs) { 181 // reserve failed 182 atomic.AddInt64(dwpt.pendingNumDocs, -1) 183 panic(fmt.Sprintf("number of documents in the index cannot exceed %v", actualMaxDocs)) 184 } 185 } 186 187 func (dwpt *DocumentsWriterPerThread) updateDocument(doc []IndexableField, 188 analyzer analysis.Analyzer, delTerm *Term) error { 189 190 dwpt.testPoint("DocumentsWriterPerThread addDocument start") 191 assert(dwpt.deleteQueue != nil) 192 dwpt.docState.doc = doc 193 dwpt.docState.analyzer = analyzer 194 dwpt.docState.docID = dwpt.numDocsInRAM 195 if DWPT_VERBOSE && dwpt.infoStream.IsEnabled("DWPT") { 196 dwpt.infoStream.Message("DWPT", "update delTerm=%v docID=%v seg=%v ", 197 delTerm, dwpt.docState.docID, dwpt.segmentInfo.Name) 198 } 199 // Even on error, the document is still added (but marked deleted), 200 // so we don't need to un-reserve at that point. Aborting errors 201 // will actually "lose" more than one document, so the counter will 202 // be "wrong" in that case, but it's very hard to fix (we can't 203 // easily distinguish aborting vs non-aborting errors): 204 dwpt.reserveDoc() 205 if err := func() error { 206 var success = false 207 defer func() { 208 if !success { 209 if !dwpt.aborting { 210 // mark document as deleted 211 dwpt.deleteDocID(dwpt.docState.docID) 212 dwpt.numDocsInRAM++ 213 } else { 214 dwpt.abort(dwpt.filesToDelete) 215 } 216 } 217 }() 218 defer dwpt.docState.clear() 219 if err := dwpt.consumer.processDocument(); err != nil { 220 return err 221 } 222 success = true 223 return nil 224 }(); err != nil { 225 return err 226 } 227 dwpt.finishDocument(delTerm) 228 return nil 229 } 230 231 func (dwpt *DocumentsWriterPerThread) updateDocuments(doc []IndexableField, 232 analyzer analysis.Analyzer, delTerm *Term) error { 233 234 dwpt.testPoint("DocumentsWriterPerThread addDocument start") 235 panic("not implemented yet") 236 } 237 238 func (w *DocumentsWriterPerThread) finishDocument(delTerm *Term) { 239 // here we actually finish the document in two steps: 240 // 1. push the delete into the queue and update out slice. 241 // 2. increment the DWPT private document id. 242 // 243 // the updated slice we get from 1. holds all the deletes that have 244 // occurred since we updated the slice the last time. 245 applySlice := w.numDocsInRAM != 0 246 if delTerm != nil { 247 w.deleteQueue.add(delTerm, w.deleteSlice) 248 assertn(w.deleteSlice.isTailItem(delTerm), "expected the delete term as the tail item") 249 } else { 250 if !w.deleteQueue.updateSlice(w.deleteSlice) { 251 applySlice = false 252 } 253 } 254 255 if applySlice { 256 w.deleteSlice.apply(w.pendingUpdates, w.numDocsInRAM) 257 } else { // if we don't need to apply we must reset! 258 w.deleteSlice.reset() 259 } 260 w.numDocsInRAM++ 261 } 262 263 /* 264 Buffer a specific docID for deletion. Currenlty only used when we hit 265 an error when adding a document 266 */ 267 func (dwpt *DocumentsWriterPerThread) deleteDocID(docIDUpto int) { 268 dwpt.pendingUpdates.addDocID(docIDUpto) 269 // NOTE: we do not trigger flush here. This is 270 // potentially a RAM leak, if you have an app that tries 271 // to add docs but every single doc always hits a 272 // non-aborting exception. Allowing a flush here gets 273 // very messy because we are only invoked when handling 274 // exceptions so to do this properly, while handling an 275 // exception we'd have to go off and flush new deletes 276 // which is risky (likely would hit some other 277 // confounding exception). 278 } 279 280 /* 281 Prepares this DWPT fo flushing. This method will freeze and return 282 the DWDQs global buffer and apply all pending deletes to this DWPT. 283 */ 284 func (dwpt *DocumentsWriterPerThread) prepareFlush() *FrozenBufferedUpdates { 285 assert(dwpt.numDocsInRAM > 0) 286 globalDeletes := dwpt.deleteQueue.freezeGlobalBuffer(dwpt.deleteSlice) 287 // deleteSlice can possibly be nil if we have hit non-aborting 288 // errors during adding a document. 289 if dwpt.deleteSlice != nil { 290 // apply all deletes before we flush and release the delete slice 291 dwpt.deleteSlice.apply(dwpt.pendingUpdates, dwpt.numDocsInRAM) 292 assert(dwpt.deleteSlice.isEmpty()) 293 dwpt.deleteSlice.reset() 294 } 295 return globalDeletes 296 } 297 298 /* Flush all pending docs to a new segment */ 299 func (dwpt *DocumentsWriterPerThread) flush() (fs *FlushedSegment, err error) { 300 assert(dwpt.numDocsInRAM > 0) 301 assert2(dwpt.deleteSlice.isEmpty(), "all deletes must be applied in prepareFlush") 302 dwpt.segmentInfo.SetDocCount(dwpt.numDocsInRAM) 303 numBytesUsed := dwpt.bytesUsed() 304 flushState := NewSegmentWriteState(dwpt.infoStream, dwpt.directory, 305 dwpt.segmentInfo, dwpt.fieldInfos.Finish(), 306 dwpt.indexWriterConfig.TermIndexInterval(), dwpt.pendingUpdates, 307 store.NewIOContextForFlush(&store.FlushInfo{dwpt.numDocsInRAM, numBytesUsed})) 308 startMBUsed := float64(numBytesUsed) / 1024 / 1024 309 310 // Apply delete-by-docID now (delete-byDocID only happens when an 311 // error is hit processing that doc, e.g., if analyzer has some 312 // problem with the text): 313 if delCount := len(dwpt.pendingUpdates.docIDs); delCount > 0 { 314 flushState.LiveDocs = dwpt.codec.LiveDocsFormat().NewLiveDocs(dwpt.numDocsInRAM) 315 for _, delDocID := range dwpt.pendingUpdates.docIDs { 316 flushState.LiveDocs.Clear(delDocID) 317 } 318 flushState.DelCountOnFlush = delCount 319 atomic.AddInt64(&dwpt.pendingUpdates.bytesUsed, -int64(delCount)*BYTES_PER_DEL_DOCID) 320 dwpt.pendingUpdates.docIDs = nil 321 } 322 323 if dwpt.aborting { 324 if dwpt.infoStream.IsEnabled("DWPT") { 325 dwpt.infoStream.Message("DWPT", "flush: skip because aborting is set") 326 } 327 return nil, nil 328 } 329 330 if dwpt.infoStream.IsEnabled("DWPT") { 331 dwpt.infoStream.Message("DWPT", "flush postings as segment %v numDocs=%v", 332 flushState.SegmentInfo.Name, dwpt.numDocsInRAM) 333 } 334 335 var success = false 336 defer func() { 337 if !success { 338 dwpt.abort(dwpt.filesToDelete) 339 } 340 }() 341 342 err = dwpt.consumer.flush(flushState) 343 if err != nil { 344 return nil, err 345 } 346 dwpt.pendingUpdates.terms = make(map[*Term]int) 347 files := make(map[string]bool) 348 dwpt.directory.EachCreatedFiles(func(name string) { 349 files[name] = true 350 }) 351 dwpt.segmentInfo.SetFiles(files) 352 353 info := NewSegmentCommitInfo(dwpt.segmentInfo, 0, -1, -1, -1) 354 if dwpt.infoStream.IsEnabled("DWPT") { 355 dwpt.infoStream.Message("DWPT", "new segment has %v deleted docs", 356 check(flushState.LiveDocs == nil, 0, 357 flushState.SegmentInfo.DocCount()-flushState.DelCountOnFlush)) 358 dwpt.infoStream.Message("DWPT", "new segment has %v; %v; %v; %v; %v", 359 check(flushState.FieldInfos.HasVectors, "vectors", "no vectors"), 360 check(flushState.FieldInfos.HasNorms, "norms", "no norms"), 361 check(flushState.FieldInfos.HasDocValues, "docValues", "no docValues"), 362 check(flushState.FieldInfos.HasProx, "prox", "no prox"), 363 check(flushState.FieldInfos.HasFreq, "freqs", "no freqs")) 364 dwpt.infoStream.Message("DWPT", "flushedFiles=%v", info.Files()) 365 dwpt.infoStream.Message("DWPT", "flushed codec=%v", dwpt.codec) 366 } 367 368 var segmentUpdates *BufferedUpdates 369 if len(dwpt.pendingUpdates.queries) > 0 { 370 segmentUpdates = dwpt.pendingUpdates 371 } 372 373 if dwpt.infoStream.IsEnabled("DWPT") { 374 numBytes, err := info.SizeInBytes() 375 if err != nil { 376 return nil, err 377 } 378 newSegmentSize := float64(numBytes) / 1024 / 1024 379 dwpt.infoStream.Message("DWPT", 380 "flushed: segment=%v ramUsed=%v MB newFlushedSize(includes docstores)=%v MB docs/MB=%v", 381 dwpt.segmentInfo.Name, startMBUsed, newSegmentSize, 382 float64(flushState.SegmentInfo.DocCount())/newSegmentSize) 383 } 384 385 assert(dwpt.segmentInfo != nil) 386 387 fs = newFlushedSegment(info, flushState.FieldInfos, segmentUpdates, 388 flushState.LiveDocs, flushState.DelCountOnFlush) 389 err = dwpt.sealFlushedSegment(fs) 390 if err != nil { 391 return nil, err 392 } 393 success = true 394 395 return fs, nil 396 } 397 398 func check(ok bool, v1, v2 interface{}) interface{} { 399 if ok { 400 return v1 401 } 402 return v2 403 } 404 405 /* 406 Seals the SegmentInfo for the new flushed segment and persists the 407 deleted documents MutableBits 408 */ 409 func (dwpt *DocumentsWriterPerThread) sealFlushedSegment(flushedSegment *FlushedSegment) error { 410 assert(flushedSegment != nil) 411 412 newSegment := flushedSegment.segmentInfo 413 414 setDiagnostics(newSegment.Info, SOURCE_FLUSH) 415 416 segSize, err := newSegment.SizeInBytes() 417 if err != nil { 418 return err 419 } 420 context := store.NewIOContextForFlush(&store.FlushInfo{ 421 newSegment.Info.DocCount(), 422 segSize, 423 }) 424 425 var success = false 426 defer func() { 427 if !success { 428 if dwpt.infoStream.IsEnabled("DWPT") { 429 dwpt.infoStream.Message( 430 "DWPT", "hit error relating compound file for newly flushed segment %v", 431 newSegment.Info.Name) 432 } 433 } 434 }() 435 436 if dwpt.indexWriterConfig.UseCompoundFile() { 437 files, err := createCompoundFile( 438 dwpt.infoStream, dwpt.directory, 439 CheckAbortNone(0), newSegment.Info, context) 440 if err != nil { 441 return err 442 } 443 for _, file := range files { 444 dwpt.filesToDelete[file] = true 445 } 446 newSegment.Info.SetUseCompoundFile(true) 447 } 448 449 // Have codec write SegmentInfo. Must do this after creating CFS so 450 // that 1) .si isn't slurped into CFS, and 2) .si reflects 451 // useCompoundFile=true change above: 452 err = dwpt.codec.SegmentInfoFormat().SegmentInfoWriter().Write( 453 dwpt.directory, 454 newSegment.Info, 455 flushedSegment.fieldInfos, 456 context) 457 if err != nil { 458 return err 459 } 460 461 // TODO: ideally we would freeze newSegment here!! 462 // because any changes after writing the .si will be lost... 463 464 // Must write deleted docs after the CFS so we don't slurp the del 465 // file into CFS: 466 if flushedSegment.liveDocs != nil { 467 delCount := flushedSegment.delCount 468 assert(delCount > 0) 469 if dwpt.infoStream.IsEnabled("DWPT") { 470 dwpt.infoStream.Message("DWPT", "flush: write %v deletes gen=%v", 471 delCount, flushedSegment.segmentInfo.DelGen()) 472 } 473 474 // TODO: we should prune the segment if it's 100% deleted... but 475 // merge will also catch it. 476 477 // TODO: in the NRT case id'd be better to hand this del vector 478 // over to the shortly-to-be-opened SegmentReader and let it 479 // carry the changes; there's no reason to use filesystem as 480 // intermediary here. 481 482 info := flushedSegment.segmentInfo 483 codec := info.Info.Codec().(Codec) 484 err = codec.LiveDocsFormat().WriteLiveDocs(flushedSegment.liveDocs, 485 dwpt.directory, info, delCount, context) 486 if err != nil { 487 return err 488 } 489 newSegment.SetDelCount(delCount) 490 newSegment.AdvanceDelGen() 491 } 492 493 success = true 494 return nil 495 } 496 497 func (dwpt *DocumentsWriterPerThread) bytesUsed() int64 { 498 return dwpt._bytesUsed.Get() + atomic.LoadInt64(&dwpt.pendingUpdates.bytesUsed) 499 } 500 501 // L600 502 // if you increase this, you must fix field cache impl for 503 // Terms/TermsIndex requires <= 32768 504 const MAX_TERM_LENGTH_UTF8 = util.BYTE_BLOCK_SIZE - 2 505 506 type IntBlockAllocator struct { 507 *util.IntAllocatorImpl 508 blockSize int 509 bytesUsed util.Counter 510 } 511 512 func newIntBlockAllocator(bytesUsed util.Counter) *IntBlockAllocator { 513 return &IntBlockAllocator{ 514 IntAllocatorImpl: util.NewIntAllocator(util.INT_BLOCK_SIZE), 515 blockSize: util.INT_BLOCK_SIZE, 516 bytesUsed: bytesUsed, 517 } 518 } 519 520 func (alloc *IntBlockAllocator) Recycle(blocks [][]int) { 521 alloc.bytesUsed.AddAndGet(int64(-len(blocks) * alloc.blockSize * util.NUM_BYTES_INT)) 522 for i, _ := range blocks { 523 blocks[i] = nil 524 } 525 } 526 527 func (w *DocumentsWriterPerThread) String() string { 528 return fmt.Sprintf( 529 "DocumentsWriterPerThread [pendingUpdates=%v, segment=%v, aborting=%v, numDocsInRAM=%v, deleteQueue=%v]", 530 w.pendingUpdates, func() string { 531 if w.segmentInfo != nil { 532 return w.segmentInfo.Name 533 } 534 return "null" 535 }(), w.aborting, w.numDocsInRAM, w.deleteQueue) 536 }