github.com/balzaczyy/golucene@v0.0.0-20151210033525-d0be9ee89713/core/store/compoundFileWriter.go (about) 1 package store 2 3 import ( 4 "container/list" 5 "errors" 6 "fmt" 7 "github.com/balzaczyy/golucene/core/codec" 8 "github.com/balzaczyy/golucene/core/util" 9 "sort" 10 "sync" 11 ) 12 13 // store/CompoundFileWriter.java 14 15 type AtomicBool struct { 16 *sync.RWMutex 17 v bool 18 } 19 20 func NewAtomicBool() *AtomicBool { 21 return &AtomicBool{&sync.RWMutex{}, false} 22 } 23 24 func (b *AtomicBool) Get() bool { 25 b.RLock() 26 defer b.RUnlock() 27 return b.v 28 } 29 30 func (b *AtomicBool) CompareAndSet(from, to bool) bool { 31 b.Lock() 32 defer b.Unlock() 33 if b.v == from { 34 b.v = to 35 return true 36 } 37 return false 38 } 39 40 type FileEntry struct { 41 file string // source file 42 length, offset int64 // temporary holder for the start of this file's data section 43 dir Directory // which contains the file. 44 } 45 46 // Combines multiple files into a single compound file 47 type CompoundFileWriter struct { 48 sync.Locker 49 directory Directory 50 entries map[string]*FileEntry 51 seenIDs map[string]bool 52 // all entries that are written to a sep. file but not yet moved into CFS 53 pendingEntries *list.List 54 closed bool 55 dataOut IndexOutput 56 outputTaken *AtomicBool 57 entryTableName string 58 dataFileName string 59 } 60 61 /* 62 Create the compound stream in the specified file. The filename is the 63 entire name (no extensions are added). 64 */ 65 func newCompoundFileWriter(dir Directory, name string) *CompoundFileWriter { 66 assert2(dir != nil, "directory cannot be nil") 67 assert2(name != "", "name cannot be empty") 68 return &CompoundFileWriter{ 69 Locker: &sync.Mutex{}, 70 directory: dir, 71 entries: make(map[string]*FileEntry), 72 seenIDs: make(map[string]bool), 73 pendingEntries: list.New(), 74 outputTaken: NewAtomicBool(), 75 entryTableName: util.SegmentFileName( 76 util.StripExtension(name), 77 "", 78 COMPOUND_FILE_ENTRIES_EXTENSION, 79 ), 80 dataFileName: name, 81 } 82 } 83 84 func (w *CompoundFileWriter) output(ctx IOContext) (IndexOutput, error) { 85 w.Lock() 86 defer w.Unlock() 87 if w.dataOut == nil { 88 var success = false 89 defer func() { 90 if !success { 91 util.CloseWhileSuppressingError(w.dataOut) 92 } 93 }() 94 95 var err error 96 w.dataOut, err = w.directory.CreateOutput(w.dataFileName, ctx) 97 if err != nil { 98 return nil, err 99 } 100 err = codec.WriteHeader(w.dataOut, CFD_DATA_CODEC, CFD_VERSION_CURRENT) 101 if err != nil { 102 return nil, err 103 } 104 success = true 105 } 106 return w.dataOut, nil 107 } 108 109 /* Closes all resouces and writes the entry table */ 110 func (w *CompoundFileWriter) Close() (err error) { 111 if w.closed { 112 fmt.Println("CompoundFileWriter is already closed.") 113 return nil 114 } 115 116 // TODO this code should clean up after itself (remove partial .cfs/.cfe) 117 if err = func() (err error) { 118 var success = false 119 defer func() { 120 if success { 121 util.Close(w.dataOut) 122 } else { 123 util.CloseWhileSuppressingError(w.dataOut) 124 } 125 }() 126 127 assert2(w.pendingEntries.Len() == 0 && !w.outputTaken.Get(), 128 "CFS has pending open files") 129 w.closed = true 130 // open the compound stream; we can safely use IO_CONTEXT_DEFAULT 131 // here because this will only open the output if no file was 132 // added to the CFS 133 _, err = w.output(IO_CONTEXT_DEFAULT) 134 if err != nil { 135 return 136 } 137 assert(w.dataOut != nil) 138 err = codec.WriteFooter(w.dataOut) 139 if err != nil { 140 return 141 } 142 success = true 143 return nil 144 }(); err != nil { 145 return 146 } 147 148 var entryTableOut IndexOutput 149 var success = false 150 defer func() { 151 if success { 152 util.Close(entryTableOut) 153 } else { 154 util.CloseWhileSuppressingError(entryTableOut) 155 } 156 }() 157 entryTableOut, err = w.directory.CreateOutput(w.entryTableName, IO_CONTEXT_DEFAULT) 158 if err != nil { 159 return 160 } 161 err = w.writeEntryTable(w.entries, entryTableOut) 162 if err != nil { 163 return 164 } 165 success = true 166 return 167 } 168 169 func (w *CompoundFileWriter) ensureOpen() { 170 assert2(!w.closed, "CFS Directory is already closed") 171 } 172 173 /* Copy the contents of the file with specified extension into the provided output stream. */ 174 func (w *CompoundFileWriter) copyFileEntry(dataOut IndexOutput, fileEntry *FileEntry) (n int64, err error) { 175 var is IndexInput 176 is, err = fileEntry.dir.OpenInput(fileEntry.file, IO_CONTEXT_READONCE) 177 if err != nil { 178 return 0, err 179 } 180 var success = false 181 defer func() { 182 if success { 183 err = util.Close(is) 184 // copy successful - delete file 185 if err == nil { 186 fileEntry.dir.DeleteFile(fileEntry.file) // ignore error 187 } 188 } else { 189 util.CloseWhileSuppressingError(is) 190 } 191 }() 192 193 startPtr := dataOut.FilePointer() 194 length := fileEntry.length 195 err = dataOut.CopyBytes(is, length) 196 if err != nil { 197 return 0, err 198 } 199 // verify that the output length diff is equal to original file 200 endPtr := dataOut.FilePointer() 201 diff := endPtr - startPtr 202 if diff != length { 203 return 0, errors.New(fmt.Sprintf( 204 "Difference in the output file offsets %v does not match the original file length %v", 205 diff, length)) 206 } 207 fileEntry.offset = startPtr 208 success = true 209 return length, nil 210 } 211 212 func (w *CompoundFileWriter) writeEntryTable(entries map[string]*FileEntry, 213 entryOut IndexOutput) (err error) { 214 if err = codec.WriteHeader(entryOut, CFD_ENTRY_CODEC, CFD_VERSION_CURRENT); err == nil { 215 if err = entryOut.WriteVInt(int32(len(entries))); err == nil { 216 var names []string 217 for name, _ := range entries { 218 names = append(names, name) 219 } 220 sort.Strings(names) 221 for _, name := range names { 222 // for _, fe := range entries { 223 fe := entries[name] 224 if err = Stream(entryOut). 225 WriteString(util.StripSegmentName(fe.file)). 226 WriteLong(fe.offset). 227 WriteLong(fe.length). 228 Close(); err != nil { 229 break 230 } 231 } 232 } 233 } 234 if err == nil { 235 err = codec.WriteFooter(entryOut) 236 } 237 return err 238 } 239 240 func (w *CompoundFileWriter) createOutput(name string, context IOContext) (IndexOutput, error) { 241 w.ensureOpen() 242 var success = false 243 var outputLocked = false 244 defer func() { 245 if !success { 246 delete(w.entries, name) 247 if outputLocked { // release the output lock if not successful 248 assert(w.outputTaken.Get()) 249 w.releaseOutputLock() 250 } 251 } 252 }() 253 254 assert2(name != "", "name must not be empty") 255 _, ok := w.entries[name] 256 assert2(!ok, "File %v already exists", name) 257 entry := &FileEntry{} 258 entry.file = name 259 w.entries[name] = entry 260 id := util.StripSegmentName(name) 261 _, ok = w.seenIDs[id] 262 assert2(!ok, "file='%v' maps to id='%v', which was already written", name, id) 263 w.seenIDs[id] = true 264 265 var out *DirectCFSIndexOutput 266 if outputLocked := w.outputTaken.CompareAndSet(false, true); outputLocked { 267 o, err := w.output(context) 268 if err != nil { 269 return nil, err 270 } 271 out = newDirectCFSIndexOutput(w, o, entry, false) 272 } else { 273 entry.dir = w.directory 274 o, err := w.directory.CreateOutput(name, context) 275 if err != nil { 276 return nil, err 277 } 278 out = newDirectCFSIndexOutput(w, o, entry, true) 279 } 280 success = true 281 return out, nil 282 } 283 284 func (w *CompoundFileWriter) releaseOutputLock() { 285 w.outputTaken.CompareAndSet(true, false) 286 } 287 288 func (w *CompoundFileWriter) prunePendingEntries() error { 289 // claim the output and copy all pending files in 290 if w.outputTaken.CompareAndSet(false, true) { 291 defer func() { 292 cas := w.outputTaken.CompareAndSet(true, false) 293 assert(cas) 294 }() 295 for w.pendingEntries.Len() > 0 { 296 head := w.pendingEntries.Front() 297 w.pendingEntries.Remove(head) 298 entry := head.Value.(*FileEntry) 299 out, err := w.output(NewIOContextForFlush(&FlushInfo{0, entry.length})) 300 if err == nil { 301 _, err = w.copyFileEntry(out, entry) 302 } 303 if err != nil { 304 return err 305 } 306 w.entries[entry.file] = entry 307 } 308 } 309 return nil 310 } 311 312 type DirectCFSIndexOutput struct { 313 *IndexOutputImpl 314 owner *CompoundFileWriter 315 delegate IndexOutput 316 offset int64 317 closed bool 318 entry *FileEntry 319 writtenBytes int64 320 isSeparate bool 321 } 322 323 func newDirectCFSIndexOutput(owner *CompoundFileWriter, 324 delegate IndexOutput, entry *FileEntry, isSeparate bool) *DirectCFSIndexOutput { 325 ans := &DirectCFSIndexOutput{ 326 owner: owner, 327 delegate: delegate, 328 entry: entry, 329 offset: delegate.FilePointer(), 330 isSeparate: isSeparate, 331 } 332 ans.entry.offset = ans.offset 333 ans.IndexOutputImpl = NewIndexOutput(ans) 334 return ans 335 } 336 337 func (out *DirectCFSIndexOutput) Flush() error { 338 panic("not implemented yet") 339 } 340 341 func (out *DirectCFSIndexOutput) Close() error { 342 if out.closed { 343 return nil 344 } 345 out.closed = true 346 out.entry.length = out.writtenBytes 347 if out.isSeparate { 348 err := out.delegate.Close() 349 if err != nil { 350 return err 351 } 352 // we are a separate file - push into the pending entries 353 out.owner.pendingEntries.PushBack(out.entry) 354 } else { 355 // we have been written into the CFS directly - release the lock 356 out.owner.releaseOutputLock() 357 } 358 // now prune all pending entries and push them into the CFS 359 return out.owner.prunePendingEntries() 360 } 361 362 func (out *DirectCFSIndexOutput) FilePointer() int64 { 363 panic("not implemented yet") 364 } 365 366 func (out *DirectCFSIndexOutput) WriteByte(b byte) error { 367 panic("not implemented yet") 368 } 369 370 func (out *DirectCFSIndexOutput) WriteBytes(b []byte) error { 371 assert(!out.closed) 372 out.writtenBytes += int64(len(b)) 373 return out.delegate.WriteBytes(b) 374 } 375 376 func (out *DirectCFSIndexOutput) Checksum() int64 { 377 return out.delegate.Checksum() 378 }