/** * Discards a segment file when the log no longer requires it. The file may be left on disk if the archive script * requires it. (Potentially blocking operation) */ void discard(boolean deleteFile) { close(); if (deleteFile) FileUtils.deleteWithConfirm(logFile); manager.addSize(-onDiskSize()); }
/** * Close the segment file. */ synchronized void close() { discardUnusedTail(); waitForModifications(); lastSyncedOffset = buffer.capacity(); internalClose(); }
public String dirtyString() { StringBuilder sb = new StringBuilder(); for (UUID cfId : getDirtyCFIDs()) { CFMetaData m = Schema.instance.getCFMetaData(cfId); sb.append(m == null ? "<deleted>" : m.cfName).append(" (").append(cfId).append("), "); } return sb.toString(); }
/** * Stop writing to this file, sync and close it. Does nothing if the file is already closed. */ synchronized void close() { discardUnusedTail(); sync(true); assert buffer == null; }
protected void runMayThrow() throws IOException { segment.waitForFinalSync(); String command = archiveCommand.replace("%name", segment.getName()); command = command.replace("%path", segment.getPath()); exec(command); } }));
nextMarker = allocate(SYNC_MARKER_SIZE); if (nextMarker < 0) discardUnusedTail(); close = true; waitForModifications(); sectionEnd = close ? endOfBuffer : nextMarker; write(startMarker, sectionEnd); lastMarkerOffset = sectionEnd; flush(startMarker, sectionEnd); lastSyncedOffset = lastMarkerOffset = nextMarker; internalClose(); syncComplete.signalAll();
segment.markClean(cfId, lowerBound, upperBound); if (segment.isUnused()) (iter.hasNext() ? "" : " active"), segment, segment.dirtyString()); if (segment.contains(upperBound)) break;
nextMarker = allocate(SYNC_MARKER_SIZE); boolean close = false; if (nextMarker < 0) discardUnusedTail(); close = true; waitForModifications(); waitForModifications(); internalClose(); throw new FSWriteError(e, getPath());
last.waitForModifications(); for (UUID cfId : last.getDirtyCFIDs()) segment.markClean(cfId, segment.getContext()); if (segment.isUnused()) recycleSegment(segment);
final CommitLogPosition maxCommitLogPosition = segments.get(segments.size() - 1).getCurrentCommitLogPosition(); for (UUID dirtyCFId : segment.getDirtyCFIDs()) segment.markClean(dirtyCFId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
@Override public Map<String, Double> getActiveSegmentCompressionRatios() { Map<String, Double> segmentRatios = new TreeMap<>(); for (CommitLogSegment seg : segmentManager.getActiveSegments()) segmentRatios.put(seg.getName(), 1.0 * seg.onDiskSize() / seg.contentSize()); return segmentRatios; }
public void discard(CommitLogSegment segment, boolean delete) { segment.close(); addSize(-segment.onDiskSize()); cdcSizeTracker.processDiscardedSegment(segment); if (segment.getCDCState() == CDCState.CONTAINS) FileUtils.renameWithConfirm(segment.logFile.getAbsolutePath(), DatabaseDescriptor.getCDCLogLocation() + File.separator + segment.logFile.getName()); else { if (delete) FileUtils.deleteWithConfirm(segment.logFile); } }
/** * Recycle processes an unneeded segment file for reuse. * * @return a new CommitLogSegment representing the newly reusable segment. */ CommitLogSegment recycle() { try { sync(); } catch (FSWriteError e) { logger.error("I/O error flushing {} {}", this, e.getMessage()); throw e; } close(); return new CommitLogSegment(getPath()); }
/** * Allocate space in this buffer for the provided mutation, and return the allocated Allocation object. * Returns null if there is not enough space in this segment, and a new segment is needed. */ @SuppressWarnings("resource") //we pass the op order around Allocation allocate(Mutation mutation, int size) { final OpOrder.Group opGroup = appendOrder.start(); try { int position = allocate(size); if (position < 0) { opGroup.close(); return null; } markDirty(mutation, position); return new Allocation(this, opGroup, position, (ByteBuffer) buffer.duplicate().position(position).limit(position + size)); } catch (Throwable t) { opGroup.close(); throw t; } }
public void maybeArchive(final CommitLogSegment segment) { if (Strings.isNullOrEmpty(archiveCommand)) return; archivePending.put(segment.getName(), executor.submit(new WrappedRunnable() { protected void runMayThrow() throws IOException { segment.waitForFinalSync(); String command = NAME.matcher(archiveCommand).replaceAll(Matcher.quoteReplacement(segment.getName())); command = PATH.matcher(command).replaceAll(Matcher.quoteReplacement(segment.getPath())); exec(command); } })); }
@Override public String toString() { return "CommitLogSegment(" + getPath() + ')'; }
/** * Constructs a new segment file. */ CommitLogSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) { this.manager = manager; id = getNextId(); descriptor = new CommitLogDescriptor(id, commitLog.configuration.getCompressorClass(), commitLog.configuration.getEncryptionContext()); logFile = new File(manager.storageDirectory, descriptor.fileName()); try { channel = FileChannel.open(logFile.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE); fd = NativeLibrary.getfd(channel); } catch (IOException e) { throw new FSWriteError(e, logFile); } buffer = createBuffer(commitLog); }
/** * Reserve space in the current segment for the provided mutation or, if there isn't space available, * create a new segment. For CDC mutations, allocation is expected to throw WTE if the segment disallows CDC mutations. * * @param mutation Mutation to allocate in segment manager * @param size total size (overhead + serialized) of mutation * @return the created Allocation object * @throws WriteTimeoutException If segment disallows CDC mutations, we throw WTE */ @Override public CommitLogSegment.Allocation allocate(Mutation mutation, int size) throws WriteTimeoutException { CommitLogSegment segment = allocatingFrom(); CommitLogSegment.Allocation alloc; throwIfForbidden(mutation, segment); while ( null == (alloc = segment.allocate(mutation, size)) ) { // Failed to allocate, so move to a new segment with enough room if possible. advanceAllocatingFrom(segment); segment = allocatingFrom(); throwIfForbidden(mutation, segment); } if (mutation.trackedByCDC()) segment.setCDCState(CDCState.CONTAINS); return alloc; }
final ReplayPosition maxReplayPosition = segments.get(segments.size() - 1).getContext(); for (UUID dirtyCFId : segment.getDirtyCFIDs()) segment.markClean(dirtyCFId, segment.getContext());