diff --git a/.gitignore b/.gitignore index db6c0826ca..c060b2d8b2 100644 --- a/.gitignore +++ b/.gitignore @@ -30,3 +30,14 @@ resources/*.xml *.o .vscode cpp/pixels-retina/third_party/ + +# AI tools +.codex +.claude/ +.cursor/ +.continue/ +.aider* +.ai/ +.notes/ +CLAUDE.local.md +AGENTS.md.local diff --git a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/CompactExecutor.java b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/CompactExecutor.java index ec8c0501c0..b2a6d20281 100644 --- a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/CompactExecutor.java +++ b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/CompactExecutor.java @@ -21,6 +21,7 @@ import com.google.common.base.Joiner; import io.pixelsdb.pixels.cli.Main; +import io.pixelsdb.pixels.common.exception.MetadataException; import io.pixelsdb.pixels.common.exception.RetinaException; import io.pixelsdb.pixels.common.metadata.MetadataService; import io.pixelsdb.pixels.common.metadata.domain.Compact; @@ -261,7 +262,10 @@ public void execute(Namespace ns, String command) throws Exception // Issue #192: wait for the compaction to complete. compactExecutor.shutdown(); while (!compactExecutor.awaitTermination(100, TimeUnit.SECONDS)); - metadataService.addFiles(compactFiles); + if (!metadataService.addFiles(compactFiles)) + { + throw new MetadataException("failed to add compact files to metadata"); + } if (retinaService.isEnabled()) { diff --git a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/ImportExecutor.java b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/ImportExecutor.java index 140ded28c6..c2c7b8c3b7 100644 --- a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/ImportExecutor.java +++ b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/ImportExecutor.java @@ -20,6 +20,7 @@ package io.pixelsdb.pixels.cli.executor; import com.google.common.collect.ImmutableList; +import io.pixelsdb.pixels.common.exception.MetadataException; import io.pixelsdb.pixels.common.metadata.MetadataService; import io.pixelsdb.pixels.common.metadata.domain.File; import io.pixelsdb.pixels.common.metadata.domain.Layout; @@ -67,7 +68,10 @@ public void execute(Namespace ns, String command) throws Exception try { List importFiles = getImportFiles(ordered, writableLayout); - metadataService.addFiles(importFiles); + if (!metadataService.addFiles(importFiles)) + { + throw new MetadataException("failed to import pixels files into metadata"); + } System.out.println(command + " is successful"); } catch (Exception e) diff --git a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/LoadExecutor.java b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/LoadExecutor.java index 765f031a39..fde71d3da1 100644 --- a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/LoadExecutor.java +++ b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/executor/LoadExecutor.java @@ -93,7 +93,10 @@ public void execute(Namespace ns, String command) throws Exception { File file = loadedInfo.loadedFile; Path path = loadedInfo.loadedPath; - metadataService.updateFile(file); + if (!metadataService.updateFile(file)) + { + throw new MetadataException("failed to publish loaded file " + file.getName()); + } try { diff --git a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/load/AbstractPixelsConsumer.java b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/load/AbstractPixelsConsumer.java index cb1d3c32f5..373ca3b83c 100644 --- a/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/load/AbstractPixelsConsumer.java +++ b/pixels-cli/src/main/java/io/pixelsdb/pixels/cli/load/AbstractPixelsConsumer.java @@ -163,11 +163,14 @@ private void cleanupTemporaryFiles() { for (File tmpFile : tmpFiles) { - if (tmpFile.getType() == File.Type.TEMPORARY) + if (tmpFile.getType() == File.Type.TEMPORARY_INGEST) { try { - metadataService.deleteFiles(Collections.singletonList((tmpFile.getId()))); + if (!metadataService.deleteFiles(Collections.singletonList((tmpFile.getId())))) + { + throw new MetadataException("failed to delete temporary load file " + tmpFile.getId()); + } } catch (MetadataException e) { e.printStackTrace(); @@ -207,11 +210,14 @@ protected File openTmpFile(String fileName, Path filePath) throws MetadataExcept { File file = new File(); file.setName(fileName); - file.setType(File.Type.TEMPORARY); + file.setType(File.Type.TEMPORARY_INGEST); file.setNumRowGroup(1); file.setPathId(filePath.getId()); String tmpFilePath = filePath.getUri() + "/" + fileName; - this.metadataService.addFiles(Collections.singletonList(file)); + if (!this.metadataService.addFiles(Collections.singletonList(file))) + { + throw new MetadataException("failed to add temporary load file " + tmpFilePath); + } file.setId(metadataService.getFileId(tmpFilePath)); return file; } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/index/MainIndexBuffer.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/index/MainIndexBuffer.java index 5ee71ba582..e8efb46fc5 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/index/MainIndexBuffer.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/index/MainIndexBuffer.java @@ -53,6 +53,40 @@ public class MainIndexBuffer implements Closeable private final MainIndexCache indexCache; private boolean populateCache = false; + public static final class FlushSnapshot + { + private final long fileId; + private final int entryCount; + private final List rowIdRanges; + + private FlushSnapshot(long fileId, int entryCount, List rowIdRanges) + { + this.fileId = fileId; + this.entryCount = entryCount; + this.rowIdRanges = Collections.unmodifiableList(new ArrayList<>(rowIdRanges)); + } + + public long getFileId() + { + return fileId; + } + + public int getEntryCount() + { + return entryCount; + } + + public List getRowIdRanges() + { + return rowIdRanges; + } + + public boolean isEmpty() + { + return entryCount == 0; + } + } + /** * Create a main index buffer and bind the main index cache to it. * Entries put into this buffer will also be put into the cache. @@ -143,20 +177,19 @@ public IndexProto.RowLocation lookup(long rowId) throws MainIndexException } /** - * Flush the (row id -> row location) mappings of the given file id into ranges and remove them from the buffer. - * This method does not evict the main index cache bind to this buffer as the cached entries are not out of date. - * However, this method may disable synchronous cache population and clear the cache if remaining file ids in the - * buffer is below or equals to the {@link #CACHE_POP_ENABLE_THRESHOLD}. + * Build a stable snapshot of the (row id -> row location) mappings of the given file id. + * This method must not mutate the buffer or cache; callers should only discard the buffered + * entries after the snapshot has been durably committed. * @param fileId the given file id to flush - * @return the flushed row id ranges to be persisited into the storage + * @return the row id range snapshot to be persisted into the storage * @throws MainIndexException */ - public List flush(long fileId) throws MainIndexException + public FlushSnapshot snapshotForFlush(long fileId) throws MainIndexException { Map fileBuffer = this.indexBuffer.get(fileId); if (fileBuffer == null) { - return null; + return new FlushSnapshot(fileId, 0, Collections.emptyList()); } ImmutableList.Builder ranges = ImmutableList.builder(); RowIdRange.Builder currRangeBuilder = new RowIdRange.Builder(); @@ -210,16 +243,34 @@ public List flush(long fileId) throws MainIndexException // release the flushed file index buffer if(fileBuffer.size() != rowIds.length) { - throw new MainIndexException("FileBuffer Changed while flush"); + throw new MainIndexException("FileBuffer changed while building flush snapshot"); + } + return new FlushSnapshot(fileId, rowIds.length, ranges.build()); + } + + /** + * Discard a flush snapshot after the backing store has durably committed it. + * @param snapshot the committed snapshot + * @throws MainIndexException if the buffer no longer matches the committed snapshot + */ + public void discardFlushed(FlushSnapshot snapshot) throws MainIndexException + { + if (snapshot.isEmpty()) + { + return; + } + Map fileBuffer = this.indexBuffer.get(snapshot.getFileId()); + if (fileBuffer == null || fileBuffer.size() != snapshot.getEntryCount()) + { + throw new MainIndexException("FileBuffer changed before committed flush discard"); } fileBuffer.clear(); - this.indexBuffer.remove(fileId); + this.indexBuffer.remove(snapshot.getFileId()); if (this.indexBuffer.size() <= CACHE_POP_ENABLE_THRESHOLD) { this.populateCache = false; this.indexCache.evictAllEntries(); } - return ranges.build(); } public List cachedFileIds() diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/MetadataService.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/MetadataService.java index 8835f63ac7..f840c21ce7 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/MetadataService.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/MetadataService.java @@ -1361,7 +1361,7 @@ public boolean addFiles(Collection files) throws MetadataException { throw new MetadataException("failed to add file", e); } - return false; + return true; } /** @@ -1420,7 +1420,7 @@ public File.Type getFileType(String filePathUri) throws MetadataException { throw new MetadataException("response token does not match."); } - return File.Type.valueOf(response.getFileType().getNumber()); + return File.Type.valueOf(response.getFileTypeValue()); } catch (Exception e) { @@ -1428,6 +1428,9 @@ public File.Type getFileType(String filePathUri) throws MetadataException } } + /** + * Return query-visible REGULAR files under the path. + */ public List getFiles(long pathId) throws MetadataException { String token = UUID.randomUUID().toString(); @@ -1476,7 +1479,7 @@ public boolean updateFile(File file) throws MetadataException { throw new MetadataException("failed to update file", e); } - return false; + return true; } public boolean deleteFiles(List fileIds) throws MetadataException @@ -1502,7 +1505,7 @@ public boolean deleteFiles(List fileIds) throws MetadataException { throw new MetadataException("failed to delete files", e); } - return false; + return true; } /** @@ -1537,8 +1540,8 @@ public File getFileById(long fileId) throws MetadataException } /** - * Atomically promote a TEMPORARY file to REGULAR and delete the old files. - * @param newFileId the id of the new TEMPORARY file to promote + * Atomically promote a temporary file to REGULAR and delete the old files. + * @param newFileId the id of the new temporary file to promote * @param oldFileIds the ids of old files to delete * @throws MetadataException if the request fails */ diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/domain/File.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/domain/File.java index 7dd46ecdc3..a567b82939 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/domain/File.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/metadata/domain/File.java @@ -33,22 +33,37 @@ */ public class File extends Base { - /** - * Files such as loaded and compacted are marked as REGULAR, while file - * created by pixelsWriterImpl during build are marked as TEMPORARY. - */ public enum Type { - TEMPORARY, REGULAR; + TEMPORARY_INGEST(0), + REGULAR(1), + TEMPORARY_GC(2), + RETIRED(3); + + private final int number; + + Type(int number) + { + this.number = number; + } + + public int getNumber() + { + return number; + } public static Type valueOf(int number) { switch (number) { case 0: - return TEMPORARY; + return TEMPORARY_INGEST; case 1: return REGULAR; + case 2: + return TEMPORARY_GC; + case 3: + return RETIRED; default: throw new InvalidArgumentException("invalid number for File.Type"); } @@ -61,6 +76,7 @@ public static Type valueOf(int number) private long minRowId; private long maxRowId; private long pathId; + private Long cleanupAt; public File() { @@ -70,11 +86,12 @@ public File(MetadataProto.File file) { this.setId(file.getId()); this.name = file.getName(); - this.type = Type.valueOf(file.getType().getNumber()); + this.type = Type.valueOf(file.getTypeValue()); this.numRowGroup = file.getNumRowGroup(); this.minRowId = file.getMinRowId(); this.maxRowId = file.getMaxRowId(); this.pathId = file.getPathId(); + this.cleanupAt = file.hasCleanupAt() ? file.getCleanupAt() : null; } public String getName() @@ -137,6 +154,16 @@ public void setPathId(long pathId) this.pathId = pathId; } + public Long getCleanupAt() + { + return cleanupAt; + } + + public void setCleanupAt(Long cleanupAt) + { + this.cleanupAt = cleanupAt; + } + public static List convertFiles(List protoFiles) { requireNonNull(protoFiles, "protoFiles is null"); @@ -182,8 +209,14 @@ public static String getFilePath(Path path, File file) @Override public MetadataProto.File toProto() { - return MetadataProto.File.newBuilder().setId(this.getId()).setName(this.name) - .setTypeValue(this.type.ordinal()).setNumRowGroup(this.numRowGroup) - .setMinRowId(this.minRowId).setMaxRowId(this.maxRowId).setPathId(this.pathId).build(); + MetadataProto.File.Builder builder = MetadataProto.File.newBuilder() + .setId(this.getId()).setName(this.name) + .setTypeValue(this.type.getNumber()).setNumRowGroup(this.numRowGroup) + .setMinRowId(this.minRowId).setMaxRowId(this.maxRowId).setPathId(this.pathId); + if (this.cleanupAt != null) + { + builder.setCleanupAt(this.cleanupAt); + } + return builder.build(); } } diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/metadata/domain/TestFileDomain.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/metadata/domain/TestFileDomain.java new file mode 100644 index 0000000000..5e66d66e29 --- /dev/null +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/metadata/domain/TestFileDomain.java @@ -0,0 +1,336 @@ +/* + * Copyright 2026 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.metadata.domain; + +import io.pixelsdb.pixels.common.exception.InvalidArgumentException; +import io.pixelsdb.pixels.daemon.MetadataProto; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Unit tests for {@link File} that exercise the c01.1 contract: + *
    + *
  • {@link File.Type} now carries an explicit numeric tag (no longer relies on {@code ordinal()}).
  • + *
  • The four enum constants — {@code TEMPORARY_INGEST(0)}, {@code REGULAR(1)}, + * {@code TEMPORARY_GC(2)}, {@code RETIRED(3)} — must round-trip cleanly through both + * {@link MetadataProto.File} and the domain object.
  • + *
  • {@link File#getCleanupAt()} is an optional field: it must be preserved across + * {@link File#toProto()} / {@code new File(MetadataProto.File)} when present and absent.
  • + *
+ * + * @author tdd-guide + * @create 2026-05-13 + */ +public class TestFileDomain +{ + // ------------------------------------------------------------------------- + // File.Type — numeric tags + // ------------------------------------------------------------------------- + + /** + * The domain {@link File.Type#getNumber()} must agree with the proto-generated + * {@link MetadataProto.File.Type#getNumber()} for every constant we publish. + * This guards against the previous implementation that relied on + * {@code ordinal()} and would silently re-number constants when the enum order changed. + */ + @Test + public void typeNumber_isConsistentWithProtoEnum() + { + assertEquals(MetadataProto.File.Type.TEMPORARY_INGEST.getNumber(), + File.Type.TEMPORARY_INGEST.getNumber()); + assertEquals(MetadataProto.File.Type.REGULAR.getNumber(), + File.Type.REGULAR.getNumber()); + assertEquals(MetadataProto.File.Type.TEMPORARY_GC.getNumber(), + File.Type.TEMPORARY_GC.getNumber()); + assertEquals(MetadataProto.File.Type.RETIRED.getNumber(), + File.Type.RETIRED.getNumber()); + } + + // ------------------------------------------------------------------------- + // File.Type.valueOf(int) — happy path + boundaries + // ------------------------------------------------------------------------- + + @Test + public void typeValueOf_resolvesAllKnownNumbers() + { + assertSame(File.Type.TEMPORARY_INGEST, File.Type.valueOf(0)); + assertSame(File.Type.REGULAR, File.Type.valueOf(1)); + assertSame(File.Type.TEMPORARY_GC, File.Type.valueOf(2)); + assertSame(File.Type.RETIRED, File.Type.valueOf(3)); + } + + @Test + public void typeValueOf_rejectsInvalidNumbers() + { + // Test various boundary cases for invalid type numbers + int[] invalidNumbers = {-1, 4, Integer.MAX_VALUE, Integer.MIN_VALUE}; + + for (int invalidNumber : invalidNumbers) + { + try + { + File.Type.valueOf(invalidNumber); + fail("expected InvalidArgumentException for number: " + invalidNumber); + } + catch (InvalidArgumentException expected) + { + assertNotNull("Exception message should not be null for number: " + invalidNumber, + expected.getMessage()); + } + } + } + + /** + * Round-trip: every constant survives {@code num -> valueOf -> getNumber}. + */ + @Test + public void typeValueOf_roundTripForAllConstants() + { + for (File.Type t : File.Type.values()) + { + assertSame("round-trip failed for " + t, + t, File.Type.valueOf(t.getNumber())); + } + } + + // ------------------------------------------------------------------------- + // cleanupAt — getter / setter + // ------------------------------------------------------------------------- + + @Test + public void cleanupAt_defaultsToNullOnNoArgConstructor() + { + File f = new File(); + assertNull("a freshly constructed File must have a null cleanupAt", f.getCleanupAt()); + } + + @Test + public void cleanupAt_setterAcceptsValueAndNull() + { + File f = new File(); + f.setCleanupAt(123_456_789L); + assertEquals(Long.valueOf(123_456_789L), f.getCleanupAt()); + + // explicit clear must be supported (used after promote-to-REGULAR) + f.setCleanupAt(null); + assertNull(f.getCleanupAt()); + } + + // ------------------------------------------------------------------------- + // toProto / fromProto round-trip + // ------------------------------------------------------------------------- + + /** + * When {@code cleanupAt == null}, {@link File#toProto()} must NOT set the optional + * field on the wire. Otherwise downstream consumers calling {@code hasCleanupAt()} + * would see a spurious zero deadline. + */ + @Test + public void toProto_omitsCleanupAt_whenDomainValueIsNull() + { + File f = makeFile(1L, "n.pxl", File.Type.TEMPORARY_INGEST, 1, 0L, 0L, 1L, null); + + MetadataProto.File proto = f.toProto(); + + assertFalse("cleanupAt must be absent on the wire when domain value is null", + proto.hasCleanupAt()); + } + + /** + * cleanupAt = 0L is a legitimate value (epoch start); it must NOT be confused with "absent". + * Without this guard, a naïve {@code if (cleanupAt != 0)} check would silently drop the field. + */ + @Test + public void toProto_includesCleanupAt_whenValueIsZero() + { + File f = makeFile(1L, "z.pxl", File.Type.RETIRED, 1, 0L, 0L, 1L, 0L); + + MetadataProto.File proto = f.toProto(); + + assertTrue("cleanupAt = 0L must be carried on the wire (zero != absent)", + proto.hasCleanupAt()); + assertEquals(0L, proto.getCleanupAt()); + } + + @Test + public void fromProto_preservesCleanupAt_whenSet() + { + long deadline = 1_700_000_123_456L; + MetadataProto.File proto = MetadataProto.File.newBuilder() + .setId(42L) + .setName("retired.pxl") + .setTypeValue(File.Type.RETIRED.getNumber()) + .setNumRowGroup(2) + .setMinRowId(0L) + .setMaxRowId(127L) + .setPathId(9L) + .setCleanupAt(deadline) + .build(); + + File f = new File(proto); + + assertEquals(42L, f.getId()); + assertEquals("retired.pxl", f.getName()); + assertSame(File.Type.RETIRED, f.getType()); + assertEquals(2, f.getNumRowGroup()); + assertEquals(0L, f.getMinRowId()); + assertEquals(127L, f.getMaxRowId()); + assertEquals(9L, f.getPathId()); + assertNotNull("cleanupAt must be retained from the proto", f.getCleanupAt()); + assertEquals(Long.valueOf(deadline), f.getCleanupAt()); + } + + /** + * If the proto omits the optional cleanupAt, the domain object MUST observe {@code null} + * (not 0L). This is the reciprocal of {@link #toProto_omitsCleanupAt_whenDomainValueIsNull()}. + */ + @Test + public void fromProto_returnsNullCleanupAt_whenAbsent() + { + MetadataProto.File proto = MetadataProto.File.newBuilder() + .setId(1L) + .setName("tmp.pxl") + .setTypeValue(File.Type.TEMPORARY_GC.getNumber()) + .setNumRowGroup(1) + .setMinRowId(0L) + .setMaxRowId(0L) + .setPathId(1L) + .build(); + + File f = new File(proto); + + assertNull("absent cleanupAt on the wire must materialise as null in the domain", + f.getCleanupAt()); + } + + /** + * End-to-end round-trip — domain → proto → domain — must be lossless for every {@link File.Type}. + */ + @Test + public void roundTrip_domainProtoDomain_isLossless_forEveryType() + { + for (File.Type t : File.Type.values()) + { + // RETIRED carries cleanupAt; the others should not. We deliberately set cleanupAt + // independently of type to verify the domain object preserves whatever it is given. + Long cleanup = (t == File.Type.RETIRED) ? 1_700_000_000_999L : null; + File original = makeFile(7L, "x_" + t + ".pxl", t, 1, 0L, 63L, 3L, cleanup); + + File restored = new File(original.toProto()); + + assertEquals("id mismatch for " + t, original.getId(), restored.getId()); + assertEquals("name mismatch for " + t, original.getName(), restored.getName()); + assertSame("type mismatch for " + t, original.getType(), restored.getType()); + assertEquals("numRowGroup mismatch for " + t, + original.getNumRowGroup(), restored.getNumRowGroup()); + assertEquals("minRowId mismatch for " + t, + original.getMinRowId(), restored.getMinRowId()); + assertEquals("maxRowId mismatch for " + t, + original.getMaxRowId(), restored.getMaxRowId()); + assertEquals("pathId mismatch for " + t, + original.getPathId(), restored.getPathId()); + assertEquals("cleanupAt mismatch for " + t, + original.getCleanupAt(), restored.getCleanupAt()); + } + } + + // ------------------------------------------------------------------------- + // convertFiles / revertFiles + // ------------------------------------------------------------------------- + + @Test + public void convertFiles_handlesEmptyList() + { + List result = File.convertFiles(Collections.emptyList()); + assertNotNull(result); + assertTrue(result.isEmpty()); + } + + @Test(expected = NullPointerException.class) + public void convertFiles_rejectsNullInput() + { + File.convertFiles(null); + } + + @Test + public void convertFiles_thenRevertFiles_isLossless() + { + MetadataProto.File p1 = MetadataProto.File.newBuilder() + .setId(10L).setName("a.pxl") + .setTypeValue(File.Type.REGULAR.getNumber()) + .setNumRowGroup(1).setMinRowId(0L).setMaxRowId(63L).setPathId(1L) + .build(); + MetadataProto.File p2 = MetadataProto.File.newBuilder() + .setId(11L).setName("b.pxl") + .setTypeValue(File.Type.RETIRED.getNumber()) + .setNumRowGroup(2).setMinRowId(64L).setMaxRowId(127L).setPathId(1L) + .setCleanupAt(1_700_000_000_000L) + .build(); + + List domain = File.convertFiles(Arrays.asList(p1, p2)); + assertEquals(2, domain.size()); + assertSame(File.Type.REGULAR, domain.get(0).getType()); + assertNull(domain.get(0).getCleanupAt()); + assertSame(File.Type.RETIRED, domain.get(1).getType()); + assertEquals(Long.valueOf(1_700_000_000_000L), domain.get(1).getCleanupAt()); + + List back = File.revertFiles(domain); + assertEquals(2, back.size()); + assertEquals(p1, back.get(0)); + assertEquals(p2, back.get(1)); + } + + @Test(expected = NullPointerException.class) + public void revertFiles_rejectsNullInput() + { + File.revertFiles(null); + } + + // ------------------------------------------------------------------------- + // helpers + // ------------------------------------------------------------------------- + + private static File makeFile(long id, String name, File.Type type, + int numRowGroup, long minRowId, long maxRowId, + long pathId, Long cleanupAt) + { + File f = new File(); + f.setId(id); + f.setName(name); + f.setType(type); + f.setNumRowGroup(numRowGroup); + f.setMinRowId(minRowId); + f.setMaxRowId(maxRowId); + f.setPathId(pathId); + f.setCleanupAt(cleanupAt); + return f; + } +} diff --git a/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/FileDao.java b/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/FileDao.java index 73b921008b..d400256e7a 100644 --- a/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/FileDao.java +++ b/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/FileDao.java @@ -38,6 +38,9 @@ public List getAll() throw new UnsupportedOperationException("getAll is not supported."); } + /** + * Return query-visible REGULAR files under a path. + */ public abstract List getAllByPathId(long pathId); public abstract MetadataProto.File getByPathIdAndFileName(long pathId, String fileName); @@ -75,8 +78,8 @@ public boolean save (MetadataProto.File file) abstract public boolean deleteByIds (List ids); /** - * Atomically promote a TEMPORARY file to REGULAR and delete the old files in a single transaction. - * @param newFileId the id of the new TEMPORARY file to promote + * Atomically promote a temporary file to REGULAR and delete the old files in a single transaction. + * @param newFileId the id of the new temporary file to promote * @param oldFileIds the ids of old files to delete * @return true if the transaction committed successfully */ diff --git a/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/RdbFileDao.java b/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/RdbFileDao.java index 1af30d564b..8db4d04783 100644 --- a/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/RdbFileDao.java +++ b/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/RdbFileDao.java @@ -42,6 +42,36 @@ public RdbFileDao() { } private static final MetaDBUtil db = MetaDBUtil.Instance(); + private static MetadataProto.File buildFile(ResultSet rs) throws SQLException + { + MetadataProto.File.Builder builder = MetadataProto.File.newBuilder() + .setId(rs.getLong("FILE_ID")) + .setName(rs.getString("FILE_NAME")) + .setTypeValue(rs.getInt("FILE_TYPE")) + .setNumRowGroup(rs.getInt("FILE_NUM_RG")) + .setMinRowId(rs.getLong("FILE_MIN_ROW_ID")) + .setMaxRowId(rs.getLong("FILE_MAX_ROW_ID")) + .setPathId(rs.getLong("PATHS_PATH_ID")); + long cleanupAt = rs.getLong("FILE_CLEANUP_AT"); + if (!rs.wasNull()) + { + builder.setCleanupAt(cleanupAt); + } + return builder.build(); + } + + private static void setCleanupAt(PreparedStatement pst, int index, MetadataProto.File file) throws SQLException + { + if (file.getTypeValue() == MetadataProto.File.Type.RETIRED.getNumber() && file.hasCleanupAt()) + { + pst.setLong(index, file.getCleanupAt()); + } + else + { + pst.setNull(index, Types.BIGINT); + } + } + @Override public MetadataProto.File getById(long id) { @@ -51,13 +81,7 @@ public MetadataProto.File getById(long id) ResultSet rs = st.executeQuery("SELECT * FROM FILES WHERE FILE_ID=" + id); if (rs.next()) { - return MetadataProto.File.newBuilder().setId(id) - .setName(rs.getString("FILE_NAME")) - .setTypeValue(rs.getInt("FILE_TYPE")) - .setNumRowGroup(rs.getInt("FILE_NUM_RG")) - .setMinRowId(rs.getLong("FILE_MIN_ROW_ID")) - .setMaxRowId(rs.getLong("FILE_MAX_ROW_ID")) - .setPathId(rs.getLong("PATHS_PATH_ID")).build(); + return buildFile(rs); } } catch (SQLException e) { @@ -71,22 +95,17 @@ public MetadataProto.File getById(long id) public List getAllByPathId(long pathId) { Connection conn = db.getConnection(); - try (Statement st = conn.createStatement()) + String sql = "SELECT * FROM FILES WHERE FILE_TYPE = ? AND PATHS_PATH_ID = ?"; + try (PreparedStatement st = conn.prepareStatement(sql)) { - // Issue #932: Add empty file markers and ignore empty files when retrieving file lists. - ResultSet rs = st.executeQuery("SELECT * FROM FILES WHERE FILE_TYPE <> 0 AND PATHS_PATH_ID=" + pathId); + // Query-visible file enumeration is REGULAR-only. + st.setInt(1, MetadataProto.File.Type.REGULAR.getNumber()); + st.setLong(2, pathId); + ResultSet rs = st.executeQuery(); List files = new ArrayList<>(); while (rs.next()) { - MetadataProto.File.Builder builder = MetadataProto.File.newBuilder() - .setId(rs.getLong("FILE_ID")) - .setTypeValue(rs.getInt("FILE_TYPE")) - .setName(rs.getString("FILE_NAME")) - .setNumRowGroup(rs.getInt("FILE_NUM_RG")) - .setMinRowId(rs.getLong("FILE_MIN_ROW_ID")) - .setMaxRowId(rs.getLong("FILE_MAX_ROW_ID")) - .setPathId(rs.getLong("PATHS_PATH_ID")); - files.add(builder.build()); + files.add(buildFile(rs)); } return files; } catch (SQLException e) @@ -101,7 +120,7 @@ public List getAllByPathId(long pathId) public MetadataProto.File getByPathIdAndFileName(long pathId, String fileName) { Connection conn = db.getConnection(); - String sql = "SELECT FILE_ID, FILE_TYPE, FILE_NUM_RG, FILE_MIN_ROW_ID, FILE_MAX_ROW_ID FROM FILES WHERE PATHS_PATH_ID=? AND FILE_NAME=?"; + String sql = "SELECT * FROM FILES WHERE PATHS_PATH_ID=? AND FILE_NAME=?"; try (PreparedStatement st = conn.prepareStatement(sql)) { st.setLong(1, pathId); @@ -109,14 +128,7 @@ public MetadataProto.File getByPathIdAndFileName(long pathId, String fileName) ResultSet rs = st.executeQuery(); if (rs.next()) { - return MetadataProto.File.newBuilder() - .setId(rs.getLong("FILE_ID")) - .setName(fileName) - .setTypeValue(rs.getInt("FILE_TYPE")) - .setNumRowGroup(rs.getInt("FILE_NUM_RG")) - .setMinRowId(rs.getLong("FILE_MIN_ROW_ID")) - .setMaxRowId(rs.getLong("FILE_MAX_ROW_ID")) - .setPathId(pathId).build(); + return buildFile(rs); } } catch (SQLException e) { @@ -156,7 +168,8 @@ public long insert(MetadataProto.File file) "`FILE_NUM_RG`," + "`FILE_MIN_ROW_ID`," + "`FILE_MAX_ROW_ID`," + - "`PATHS_PATH_ID`) VALUES (?,?,?,?,?,?)"; + "`PATHS_PATH_ID`," + + "`FILE_CLEANUP_AT`) VALUES (?,?,?,?,?,?,?)"; try (PreparedStatement pst = conn.prepareStatement(sql)) { pst.setString(1, file.getName()); @@ -165,6 +178,7 @@ public long insert(MetadataProto.File file) pst.setLong(4, file.getMinRowId()); pst.setLong(5, file.getMaxRowId()); pst.setLong(6, file.getPathId()); + setCleanupAt(pst, 7, file); if (pst.executeUpdate() == 1) { ResultSet rs = pst.executeQuery("SELECT LAST_INSERT_ID()"); @@ -199,7 +213,8 @@ public boolean insertBatch(List files) "`FILE_NUM_RG`," + "`FILE_MIN_ROW_ID`," + "`FILE_MAX_ROW_ID`," + - "`PATHS_PATH_ID`) VALUES (?,?,?,?,?,?)"; + "`PATHS_PATH_ID`," + + "`FILE_CLEANUP_AT`) VALUES (?,?,?,?,?,?,?)"; try (PreparedStatement pst = conn.prepareStatement(sql)) { for (MetadataProto.File file : files) @@ -210,6 +225,7 @@ public boolean insertBatch(List files) pst.setLong(4, file.getMinRowId()); pst.setLong(5, file.getMaxRowId()); pst.setLong(6, file.getPathId()); + setCleanupAt(pst, 7, file); pst.addBatch(); } pst.executeBatch(); @@ -230,7 +246,8 @@ public boolean update(MetadataProto.File file) "`FILE_TYPE` = ?," + "`FILE_NUM_RG` = ?," + "`FILE_MIN_ROW_ID` = ?," + - "`FILE_MAX_ROW_ID` = ?\n" + + "`FILE_MAX_ROW_ID` = ?," + + "`FILE_CLEANUP_AT` = ?\n" + "WHERE `FILE_ID` = ?"; try (PreparedStatement pst = conn.prepareStatement(sql)) { @@ -239,7 +256,8 @@ public boolean update(MetadataProto.File file) pst.setInt(3, file.getNumRowGroup()); pst.setLong(4, file.getMinRowId()); pst.setLong(5, file.getMaxRowId()); - pst.setLong(6, file.getId()); + setCleanupAt(pst, 6, file); + pst.setLong(7, file.getId()); return pst.executeUpdate() == 1; } catch (SQLException e) { @@ -279,7 +297,7 @@ public boolean atomicSwapFiles(long newFileId, List oldFileIds) { conn.setAutoCommit(false); try (PreparedStatement pst = conn.prepareStatement( - "UPDATE FILES SET FILE_TYPE=? WHERE FILE_ID=?")) + "UPDATE FILES SET FILE_TYPE=?, FILE_CLEANUP_AT=NULL WHERE FILE_ID=?")) { pst.setInt(1, MetadataProto.File.Type.REGULAR.getNumber()); pst.setLong(2, newFileId); diff --git a/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/retina/RetinaServerImpl.java b/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/retina/RetinaServerImpl.java index 09218beef5..d2763698b9 100644 --- a/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/retina/RetinaServerImpl.java +++ b/pixels-daemon/src/main/java/io/pixelsdb/pixels/daemon/retina/RetinaServerImpl.java @@ -75,9 +75,18 @@ public class RetinaServerImpl extends RetinaWorkerServiceGrpc.RetinaWorkerServic */ public RetinaServerImpl() { - this.metadataService = MetadataService.Instance(); - this.indexService = IndexServiceProvider.getService(IndexServiceProvider.ServiceMode.local); - this.retinaResourceManager = RetinaResourceManager.Instance(); + this(MetadataService.Instance(), + IndexServiceProvider.getService(IndexServiceProvider.ServiceMode.local), + RetinaResourceManager.Instance()); + } + + RetinaServerImpl(MetadataService metadataService, IndexService indexService, + RetinaResourceManager retinaResourceManager) + { + this.metadataService = requireNonNull(metadataService, "metadataService is null"); + this.indexService = requireNonNull(indexService, "indexService is null"); + this.retinaResourceManager = requireNonNull(retinaResourceManager, "retinaResourceManager is null"); + int totalBuckets = Integer.parseInt(ConfigFactory.Instance().getProperty("index.bucket.num")); this.indexOptionPool = new IndexOption[totalBuckets]; for (int i = 0; i < totalBuckets; i++) @@ -86,91 +95,95 @@ public RetinaServerImpl() this.indexOptionPool[i].setVNodeId(i); } - startRetinaMetricsLogThread(); try { - logger.info("Pre-loading checkpoints..."); - this.retinaResourceManager.recoverCheckpoints(); + initializeRetinaResources(); + this.retinaResourceManager.startBackgroundGc(); + startRetinaMetricsLogThread(); + logger.info("Retina service is ready"); + } + catch (Exception e) + { + logger.error("Error while initializing RetinaServerImpl", e); + throw new IllegalStateException("Failed to initialize RetinaServerImpl", e); + } + } + + private void initializeRetinaResources() throws Exception + { + logger.info("Pre-loading checkpoints..."); + this.retinaResourceManager.recoverCheckpoints(); - List schemas = this.metadataService.getSchemas(); - for (Schema schema : schemas) + List schemas = this.metadataService.getSchemas(); + for (Schema schema : schemas) + { + List tables = this.metadataService.getTables(schema.getName()); + for (Table table : tables) { - List
tables = this.metadataService.getTables(schema.getName()); - for (Table table : tables) + List layouts = this.metadataService.getLayouts(schema.getName(), table.getName()); + List files = new LinkedList<>(); + for (Layout layout : layouts) { - List layouts = this.metadataService.getLayouts(schema.getName(), table.getName()); - List files = new LinkedList<>(); - for (Layout layout : layouts) + if (layout.isReadable()) { - if (layout.isReadable()) - { - /* - * Issue #946: always add visibility to all files - */ - // add visibility for ordered files - List orderedPaths = layout.getOrderedPaths(); - validateOrderedOrCompactPaths(orderedPaths); - List orderedFiles = this.metadataService.getFiles(orderedPaths.get(0).getId()); - files.addAll(orderedFiles.stream() - .map(file -> orderedPaths.get(0).getUri() + "/" + file.getName()) - .collect(Collectors.toList())); - - // add visibility for compact files - List compactPaths = layout.getCompactPaths(); - validateOrderedOrCompactPaths(compactPaths); - List compactFiles = this.metadataService.getFiles(compactPaths.get(0).getId()); - files.addAll(compactFiles.stream() - .map(file -> compactPaths.get(0).getUri() + "/" + file.getName()) - .collect(Collectors.toList())); - } + /* + * Issue #946: always add visibility to all files + */ + // add visibility for ordered files + List orderedPaths = layout.getOrderedPaths(); + validateOrderedOrCompactPaths(orderedPaths); + List orderedFiles = this.metadataService.getFiles(orderedPaths.get(0).getId()); + files.addAll(orderedFiles.stream() + .map(file -> orderedPaths.get(0).getUri() + "/" + file.getName()) + .collect(Collectors.toList())); + + // add visibility for compact files + List compactPaths = layout.getCompactPaths(); + validateOrderedOrCompactPaths(compactPaths); + List compactFiles = this.metadataService.getFiles(compactPaths.get(0).getId()); + files.addAll(compactFiles.stream() + .map(file -> compactPaths.get(0).getUri() + "/" + file.getName()) + .collect(Collectors.toList())); } + } - int threadNum = Integer.parseInt - (ConfigFactory.Instance().getProperty("retina.service.init.threads")); - ExecutorService executorService = Executors.newFixedThreadPool(threadNum); - AtomicBoolean success = new AtomicBoolean(true); - AtomicReference e = new AtomicReference<>(); - try + int threadNum = Integer.parseInt + (ConfigFactory.Instance().getProperty("retina.service.init.threads")); + ExecutorService executorService = Executors.newFixedThreadPool(threadNum); + AtomicBoolean success = new AtomicBoolean(true); + AtomicReference e = new AtomicReference<>(); + try + { + for (String filePath : files) { - for (String filePath : files) + executorService.submit(() -> { - executorService.submit(() -> + try { - try - { - this.retinaResourceManager.addVisibility(filePath); - } - catch (Exception ex) - { - success.set(false); - e.set(ex); - } - }); - } - } - finally - { - executorService.shutdown(); - } - - if (success.get()) - { - executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); + this.retinaResourceManager.addVisibility(filePath); + } + catch (Exception ex) + { + success.set(false); + e.set(ex); + } + }); } + } + finally + { + executorService.shutdown(); + } - if (!success.get()) - { - throw new RetinaException("Can't add visibility", e.get()); - } + executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); - this.retinaResourceManager.addWriteBuffer(schema.getName(), table.getName()); + if (!success.get()) + { + throw new RetinaException("Can't add visibility", e.get()); } + + this.retinaResourceManager.addWriteBuffer(schema.getName(), table.getName()); } - logger.info("Retina service is ready"); - } - catch (Exception e) - { - logger.error("Error while initializing RetinaServerImpl", e); } } diff --git a/pixels-daemon/src/main/resources/pixels_metadata.mwb b/pixels-daemon/src/main/resources/pixels_metadata.mwb index 3a9176fa93..0874f98600 100644 Binary files a/pixels-daemon/src/main/resources/pixels_metadata.mwb and b/pixels-daemon/src/main/resources/pixels_metadata.mwb differ diff --git a/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/TestRdbFileDaoCleanupAt.java b/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/TestRdbFileDaoCleanupAt.java new file mode 100644 index 0000000000..47879c9afb --- /dev/null +++ b/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/metadata/dao/impl/TestRdbFileDaoCleanupAt.java @@ -0,0 +1,406 @@ +/* + * Copyright 2026 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.daemon.metadata.dao.impl; + +import io.pixelsdb.pixels.common.utils.MetaDBUtil; +import io.pixelsdb.pixels.daemon.MetadataProto; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.lang.reflect.Field; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Types; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Mockito-based unit tests for the c01.1 changes in {@link RdbFileDao} that govern how the + * optional {@code FILE_CLEANUP_AT} column is persisted and restored. + * + *

Contract under test: + *

    + *
  • On INSERT / UPDATE: {@code FILE_CLEANUP_AT} is bound to a real {@code long} only when + * {@code type == RETIRED && hasCleanupAt()}. Every other combination must bind {@code NULL}.
  • + *
  • On SELECT: a {@code wasNull()} column on the result set must materialise as + * {@code !proto.hasCleanupAt()} on the wire.
  • + *
  • {@code atomicSwapFiles} must clear {@code FILE_CLEANUP_AT} (set to NULL) when it + * promotes the new file to {@code REGULAR}, otherwise stale deadlines would leak across + * the swap boundary.
  • + *
+ * + *

The DAO calls {@code MetaDBUtil.Instance().getConnection()} on every method, so we + * inject a mock {@link Connection} into the singleton via reflection. This keeps the test + * a true unit test (no JDBC driver, no schema, no network). + * + * @author tdd-guide + * @create 2026-05-13 + */ +public class TestRdbFileDaoCleanupAt +{ + private static final int RETIRED_VALUE = MetadataProto.File.Type.RETIRED.getNumber(); + private static final int REGULAR_VALUE = MetadataProto.File.Type.REGULAR.getNumber(); + private static final int TEMPORARY_INGEST_VALUE = + MetadataProto.File.Type.TEMPORARY_INGEST.getNumber(); + + private Connection mockConn; + private Connection originalConn; + + private RdbFileDao dao; + + @Before + public void setUp() throws Exception + { + mockConn = mock(Connection.class); + // The DAO does conn.isValid(30) on lazy reconnect; force it to report healthy so the + // production code path stays on our mock rather than re-acquiring a real connection. + when(mockConn.isValid(anyInt())).thenReturn(true); + + originalConn = swapConnection(mockConn); + dao = new RdbFileDao(); + } + + @After + public void tearDown() throws Exception + { + // Always restore the real connection so subsequent tests in the same JVM are unaffected. + swapConnection(originalConn); + } + + // ------------------------------------------------------------------------- + // INSERT — single row + // ------------------------------------------------------------------------- + + /** + * For non-RETIRED file types (REGULAR, TEMPORARY_INGEST, TEMPORARY_GC), the DAO must NOT persist a cleanup deadline, + * even if a stray {@code cleanupAt} happens to be present on the proto. + */ + @Test + public void insert_nonRetiredFileTypes_bindCleanupAtAsNull() throws Exception + { + // Test REGULAR file with stray cleanupAt value + PreparedStatement pst1 = stubPreparedStatementForInsert(); + MetadataProto.File regularFile = baseFile("a.pxl", REGULAR_VALUE) + .setCleanupAt(123_456_789L) // deliberately stray; type != RETIRED so MUST be ignored + .build(); + dao.insert(regularFile); + verify(pst1).setNull(7, Types.BIGINT); + verify(pst1, never()).setLong(eq(7), anyLong()); + + // Test TEMPORARY_INGEST file (no cleanupAt) + PreparedStatement pst2 = stubPreparedStatementForInsert(); + MetadataProto.File ingestFile = baseFile("ingest.pxl", TEMPORARY_INGEST_VALUE).build(); + dao.insert(ingestFile); + verify(pst2).setNull(7, Types.BIGINT); + verify(pst2, never()).setLong(eq(7), anyLong()); + } + + /** + * RETIRED file binding tests covering various cleanupAt scenarios + */ + @Test + public void insert_retiredFile_bindingScenarios() throws Exception + { + // Test RETIRED file with cleanup deadline + PreparedStatement pst1 = stubPreparedStatementForInsert(); + long deadline = 1_700_000_000_000L; + MetadataProto.File retiredWithDeadline = baseFile("retired.pxl", RETIRED_VALUE) + .setCleanupAt(deadline) + .build(); + dao.insert(retiredWithDeadline); + verify(pst1).setLong(7, deadline); + verify(pst1, never()).setNull(eq(7), anyInt()); + + // Test RETIRED file without cleanupAt (should bind NULL) + PreparedStatement pst2 = stubPreparedStatementForInsert(); + MetadataProto.File retiredNoDeadline = baseFile("retired_unset.pxl", RETIRED_VALUE).build(); + dao.insert(retiredNoDeadline); + verify(pst2).setNull(7, Types.BIGINT); + verify(pst2, never()).setLong(eq(7), anyLong()); + + // Test RETIRED file with cleanupAt = 0L (should bind as long zero, not NULL) + PreparedStatement pst3 = stubPreparedStatementForInsert(); + MetadataProto.File retiredZero = baseFile("retired_zero.pxl", RETIRED_VALUE) + .setCleanupAt(0L) + .build(); + dao.insert(retiredZero); + verify(pst3).setLong(7, 0L); + verify(pst3, never()).setNull(eq(7), anyInt()); + } + + // ------------------------------------------------------------------------- + // INSERT BATCH — verifies per-row binding semantics + // ------------------------------------------------------------------------- + + @Test + public void insertBatch_mixedTypes_bindsCleanupAtPerRow() throws Exception + { + PreparedStatement pst = stubPreparedStatementForInsert(); + + MetadataProto.File regular = baseFile("r.pxl", REGULAR_VALUE).build(); + MetadataProto.File retiredWithDeadline = baseFile("d.pxl", RETIRED_VALUE) + .setCleanupAt(42L).build(); + MetadataProto.File retiredNoDeadline = baseFile("nd.pxl", RETIRED_VALUE).build(); + + dao.insertBatch(Arrays.asList(regular, retiredWithDeadline, retiredNoDeadline)); + + // Two rows must bind NULL (regular + retired-without-deadline), one row binds a long. + verify(pst, times(2)).setNull(7, Types.BIGINT); + verify(pst, times(1)).setLong(7, 42L); + verify(pst).executeBatch(); + } + + // ------------------------------------------------------------------------- + // UPDATE — index 6 carries cleanupAt (id is bound at index 7) + // ------------------------------------------------------------------------- + + /** + * UPDATE operation binding tests for different file types and cleanupAt scenarios + */ + @Test + public void update_bindingScenarios() throws Exception + { + // Test REGULAR file - should bind cleanupAt as NULL + PreparedStatement pst1 = mock(PreparedStatement.class); + when(mockConn.prepareStatement(anyString())).thenReturn(pst1); + when(pst1.executeUpdate()).thenReturn(1); + + MetadataProto.File regularFile = baseFile("u.pxl", REGULAR_VALUE).setId(7L).build(); + boolean ok1 = dao.update(regularFile); + + assertTrue(ok1); + verify(pst1).setNull(6, Types.BIGINT); + verify(pst1).setLong(7, 7L); // WHERE FILE_ID = ? + + // Test RETIRED file with cleanup deadline - should bind as long + PreparedStatement pst2 = mock(PreparedStatement.class); + when(mockConn.prepareStatement(anyString())).thenReturn(pst2); + when(pst2.executeUpdate()).thenReturn(1); + + long deadline = 1_700_000_000_999L; + MetadataProto.File retiredFile = baseFile("u.pxl", RETIRED_VALUE) + .setId(8L) + .setCleanupAt(deadline) + .build(); + boolean ok2 = dao.update(retiredFile); + + assertTrue(ok2); + verify(pst2).setLong(6, deadline); + verify(pst2).setLong(7, 8L); + } + + // ------------------------------------------------------------------------- + // atomicSwapFiles — cleanupAt must be reset to NULL on promote + // ------------------------------------------------------------------------- + + /** + * The promote step must use the SQL fragment {@code FILE_CLEANUP_AT=NULL}. Without it, + * a file that was previously RETIRED and is being recycled into a fresh REGULAR slot + * would silently retain its deadline, eventually getting GC'd while live. + */ + @Test + public void atomicSwapFiles_promoteSqlClearsCleanupAt() throws Exception + { + PreparedStatement updatePst = mock(PreparedStatement.class); + PreparedStatement deletePst = mock(PreparedStatement.class); + + when(mockConn.prepareStatement(anyString())).thenAnswer(inv -> { + String sql = inv.getArgument(0); + if (sql.startsWith("UPDATE")) + { + return updatePst; + } + if (sql.startsWith("DELETE")) + { + return deletePst; + } + return mock(PreparedStatement.class); + }); + + boolean ok = dao.atomicSwapFiles(101L, Arrays.asList(11L, 12L)); + assertTrue(ok); + + // Capture the actual SQL string the production code sent to the JDBC driver. + org.mockito.ArgumentCaptor sqlCaptor = org.mockito.ArgumentCaptor.forClass(String.class); + verify(mockConn, atLeastOnce()).prepareStatement(sqlCaptor.capture()); + boolean clearsCleanupAt = false; + for (String sql : sqlCaptor.getAllValues()) + { + if (sql.contains("FILE_TYPE=?") && sql.contains("FILE_CLEANUP_AT=NULL")) + { + clearsCleanupAt = true; + break; + } + } + assertTrue("promote SQL must clear FILE_CLEANUP_AT to NULL together with the type update", + clearsCleanupAt); + + // The promote binds REGULAR + the new id, then commits. These behaviours are tied + // to the same transaction as the DELETE, so we check both ran on the same connection. + verify(updatePst).setInt(1, REGULAR_VALUE); + verify(updatePst).setLong(2, 101L); + verify(updatePst).executeUpdate(); + verify(deletePst).setLong(1, 11L); + verify(deletePst).setLong(2, 12L); + verify(deletePst).executeUpdate(); + verify(mockConn).setAutoCommit(false); + verify(mockConn).commit(); + } + + @Test + public void atomicSwapFiles_rollsBackOnSqlException() throws Exception + { + when(mockConn.prepareStatement(anyString())) + .thenThrow(new SQLException("boom")); + + boolean ok = dao.atomicSwapFiles(1L, Collections.singletonList(2L)); + + assertFalse("atomicSwapFiles must report failure when the JDBC layer throws", ok); + verify(mockConn).setAutoCommit(false); + verify(mockConn).rollback(); + verify(mockConn).setAutoCommit(true); // finally block must restore auto-commit + verify(mockConn, never()).commit(); + } + + // ------------------------------------------------------------------------- + // SELECT (buildFile) — cleanupAt round-trip from ResultSet to proto + // ------------------------------------------------------------------------- + + /** + * SELECT operation tests covering different cleanupAt scenarios from ResultSet to proto + */ + @Test + public void getById_cleanupAtRoundTripScenarios() throws Exception + { + // Test scenario 1: ResultSet with cleanupAt value (non-NULL) + Statement st1 = mock(Statement.class); + ResultSet rs1 = mock(ResultSet.class); + when(mockConn.createStatement()).thenReturn(st1); + when(st1.executeQuery(anyString())).thenReturn(rs1); + when(rs1.next()).thenReturn(true).thenReturn(false); + + when(rs1.getLong("FILE_ID")).thenReturn(99L); + when(rs1.getString("FILE_NAME")).thenReturn("x.pxl"); + when(rs1.getInt("FILE_TYPE")).thenReturn(RETIRED_VALUE); + when(rs1.getInt("FILE_NUM_RG")).thenReturn(2); + when(rs1.getLong("FILE_MIN_ROW_ID")).thenReturn(0L); + when(rs1.getLong("FILE_MAX_ROW_ID")).thenReturn(127L); + when(rs1.getLong("PATHS_PATH_ID")).thenReturn(5L); + when(rs1.getLong("FILE_CLEANUP_AT")).thenReturn(1_700_000_000_000L); + when(rs1.wasNull()).thenReturn(false); + + MetadataProto.File proto1 = dao.getById(99L); + + assertNotNull(proto1); + assertEquals(99L, proto1.getId()); + assertEquals(MetadataProto.File.Type.RETIRED, proto1.getType()); + assertTrue("non-NULL FILE_CLEANUP_AT column must surface as hasCleanupAt()", + proto1.hasCleanupAt()); + assertEquals(1_700_000_000_000L, proto1.getCleanupAt()); + + // Test scenario 2: ResultSet with NULL cleanupAt + Statement st2 = mock(Statement.class); + ResultSet rs2 = mock(ResultSet.class); + when(mockConn.createStatement()).thenReturn(st2); + when(st2.executeQuery(anyString())).thenReturn(rs2); + when(rs2.next()).thenReturn(true).thenReturn(false); + + when(rs2.getLong("FILE_ID")).thenReturn(1L); + when(rs2.getString("FILE_NAME")).thenReturn("r.pxl"); + when(rs2.getInt("FILE_TYPE")).thenReturn(REGULAR_VALUE); + when(rs2.getInt("FILE_NUM_RG")).thenReturn(1); + when(rs2.getLong("FILE_MIN_ROW_ID")).thenReturn(0L); + when(rs2.getLong("FILE_MAX_ROW_ID")).thenReturn(0L); + when(rs2.getLong("PATHS_PATH_ID")).thenReturn(1L); + when(rs2.getLong("FILE_CLEANUP_AT")).thenReturn(0L); + when(rs2.wasNull()).thenReturn(true); // critical: NULL column + + MetadataProto.File proto2 = dao.getById(1L); + + assertNotNull(proto2); + assertFalse("NULL FILE_CLEANUP_AT column must surface as !hasCleanupAt()", + proto2.hasCleanupAt()); + } + + // ------------------------------------------------------------------------- + // helpers + // ------------------------------------------------------------------------- + + private PreparedStatement stubPreparedStatementForInsert() throws SQLException + { + PreparedStatement pst = mock(PreparedStatement.class); + when(mockConn.prepareStatement(anyString())).thenReturn(pst); + when(pst.executeUpdate()).thenReturn(1); + + // After a successful insert, the DAO calls executeQuery("SELECT LAST_INSERT_ID()") + // on the same PreparedStatement. Stub a single-row ResultSet so the call returns cleanly. + ResultSet idRs = mock(ResultSet.class); + when(pst.executeQuery(anyString())).thenReturn(idRs); + when(idRs.next()).thenReturn(true); + when(idRs.getLong(1)).thenReturn(1L); + return pst; + } + + private static MetadataProto.File.Builder baseFile(String name, int typeValue) + { + return MetadataProto.File.newBuilder() + .setName(name) + .setTypeValue(typeValue) + .setNumRowGroup(1) + .setMinRowId(0L) + .setMaxRowId(0L) + .setPathId(1L); + } + + /** + * Replace the private {@code connection} field in the {@link MetaDBUtil} singleton with + * the supplied connection, returning the previous value. Using reflection here keeps + * the production class untouched while still letting us inject a Mockito-managed + * {@link Connection} for the duration of a single test. + */ + private static Connection swapConnection(Connection replacement) throws Exception + { + Field f = MetaDBUtil.class.getDeclaredField("connection"); + f.setAccessible(true); + Connection previous = (Connection) f.get(MetaDBUtil.Instance()); + f.set(MetaDBUtil.Instance(), replacement); + return previous; + } +} diff --git a/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/retina/TestRetinaServer.java b/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/retina/TestRetinaServer.java index 1167cf6e86..cf4d0e526a 100644 --- a/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/retina/TestRetinaServer.java +++ b/pixels-daemon/src/test/java/io/pixelsdb/pixels/daemon/retina/TestRetinaServer.java @@ -19,12 +19,40 @@ */ package io.pixelsdb.pixels.daemon.retina; +import io.pixelsdb.pixels.common.exception.MetadataException; +import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.index.service.IndexService; +import io.pixelsdb.pixels.common.metadata.MetadataService; +import io.pixelsdb.pixels.common.metadata.domain.File; +import io.pixelsdb.pixels.common.metadata.domain.Layout; +import io.pixelsdb.pixels.common.metadata.domain.Path; +import io.pixelsdb.pixels.common.metadata.domain.Permission; +import io.pixelsdb.pixels.common.metadata.domain.Schema; +import io.pixelsdb.pixels.common.metadata.domain.Table; import io.pixelsdb.pixels.daemon.ServerContainer; import io.pixelsdb.pixels.daemon.metadata.MetadataServer; +import io.pixelsdb.pixels.retina.RetinaResourceManager; +import org.junit.Ignore; import org.junit.Test; +import org.mockito.InOrder; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class TestRetinaServer { + @Ignore("Integration test requires real metadata server, metadata DB, and fixed local ports.") @Test public void testRetinaServer() { @@ -34,4 +62,117 @@ public void testRetinaServer() RetinaServer retinaServer = new RetinaServer(18890); container.addServer("retina server", retinaServer); } + + @Test + public void testRetinaServerImplInitializationFailureIsFailClosed() throws Exception + { + MetadataService metadataService = mock(MetadataService.class); + IndexService indexService = mock(IndexService.class); + RetinaResourceManager resourceManager = mock(RetinaResourceManager.class); + + when(metadataService.getSchemas()).thenThrow(new MetadataException("metadata unavailable")); + + try + { + RetinaServerImpl server = new RetinaServerImpl(metadataService, indexService, resourceManager); + fail("RetinaServerImpl must fail closed when initialization fails: " + server); + } + catch (IllegalStateException e) + { + assertTrue(e.getMessage().contains("Failed to initialize RetinaServerImpl")); + } + + verify(resourceManager).recoverCheckpoints(); + verify(resourceManager, never()).startBackgroundGc(); + } + + @Test + public void testRetinaServerImplStartsBackgroundGcAfterSuccessfulInitialization() throws Exception + { + MetadataService metadataService = mock(MetadataService.class); + IndexService indexService = mock(IndexService.class); + RetinaResourceManager resourceManager = mock(RetinaResourceManager.class); + + Schema schema = new Schema(); + schema.setName("gc_schema"); + Table table = new Table(); + table.setName("gc_table"); + Path orderedPath = new Path(); + orderedPath.setId(11L); + orderedPath.setUri("file:///tmp/pixels/ordered"); + Path compactPath = new Path(); + compactPath.setId(12L); + compactPath.setUri("file:///tmp/pixels/compact"); + Layout layout = new Layout(); + layout.setPermission(Permission.READ_WRITE); + layout.setOrderedPaths(Collections.singletonList(orderedPath)); + layout.setCompactPaths(Collections.singletonList(compactPath)); + File orderedFile = new File(); + orderedFile.setName("ordered.pxl"); + File compactFile = new File(); + compactFile.setName("compact.pxl"); + List lifecycleEvents = Collections.synchronizedList(new ArrayList<>()); + + when(metadataService.getSchemas()).thenReturn(Collections.singletonList(schema)); + when(metadataService.getTables(schema.getName())).thenReturn(Collections.singletonList(table)); + when(metadataService.getLayouts(schema.getName(), table.getName())).thenReturn(Collections.singletonList(layout)); + when(metadataService.getFiles(orderedPath.getId())).thenReturn(Collections.singletonList(orderedFile)); + when(metadataService.getFiles(compactPath.getId())).thenReturn(Collections.singletonList(compactFile)); + doAnswer(invocation -> { + lifecycleEvents.add("recover"); + return null; + }).when(resourceManager).recoverCheckpoints(); + doAnswer(invocation -> { + lifecycleEvents.add("visibility:" + invocation.getArgument(0)); + return null; + }).when(resourceManager).addVisibility(org.mockito.ArgumentMatchers.anyString()); + doAnswer(invocation -> { + lifecycleEvents.add("writeBuffer"); + return null; + }).when(resourceManager).addWriteBuffer(schema.getName(), table.getName()); + doAnswer(invocation -> { + lifecycleEvents.add("startGc"); + return null; + }).when(resourceManager).startBackgroundGc(); + + new RetinaServerImpl(metadataService, indexService, resourceManager); + + assertTrue(lifecycleEvents.indexOf("recover") >= 0); + assertTrue(lifecycleEvents.contains("visibility:file:///tmp/pixels/ordered/ordered.pxl")); + assertTrue(lifecycleEvents.contains("visibility:file:///tmp/pixels/compact/compact.pxl")); + int writeBufferIndex = lifecycleEvents.indexOf("writeBuffer"); + assertTrue(writeBufferIndex > lifecycleEvents.indexOf("recover")); + assertTrue(writeBufferIndex > lifecycleEvents.indexOf("visibility:file:///tmp/pixels/ordered/ordered.pxl")); + assertTrue(writeBufferIndex > lifecycleEvents.indexOf("visibility:file:///tmp/pixels/compact/compact.pxl")); + assertTrue(lifecycleEvents.indexOf("startGc") > writeBufferIndex); + verify(resourceManager).addVisibility("file:///tmp/pixels/ordered/ordered.pxl"); + verify(resourceManager).addVisibility("file:///tmp/pixels/compact/compact.pxl"); + verify(resourceManager).startBackgroundGc(); + } + + @Test + public void testRetinaServerImplBackgroundGcStartFailureIsFailClosed() throws Exception + { + MetadataService metadataService = mock(MetadataService.class); + IndexService indexService = mock(IndexService.class); + RetinaResourceManager resourceManager = mock(RetinaResourceManager.class); + + when(metadataService.getSchemas()).thenReturn(Collections.emptyList()); + doThrow(new RetinaException("gc disabled by invalid lifecycle")) + .when(resourceManager).startBackgroundGc(); + + try + { + RetinaServerImpl server = new RetinaServerImpl(metadataService, indexService, resourceManager); + fail("RetinaServerImpl must fail closed when background GC cannot start: " + server); + } + catch (IllegalStateException e) + { + assertTrue(e.getMessage().contains("Failed to initialize RetinaServerImpl")); + } + + InOrder inOrder = inOrder(resourceManager); + inOrder.verify(resourceManager).recoverCheckpoints(); + inOrder.verify(resourceManager).startBackgroundGc(); + } } diff --git a/pixels-index/pixels-index-main-sqlite/README.md b/pixels-index/pixels-index-main-sqlite/README.md new file mode 100644 index 0000000000..74d53c74aa --- /dev/null +++ b/pixels-index/pixels-index-main-sqlite/README.md @@ -0,0 +1,156 @@ +# SQLite MainIndex + +This module implements the SQLite-backed `MainIndex`. It stores +`rowId -> RowLocation` mappings as row-id ranges in SQLite and uses a per-file +durable marker to make file-scoped persistence retryable. + +The primary table is `row_id_ranges`. A file-scoped persistence operation writes +the ranges for one file and one row in `row_id_range_flush_markers` in the same +SQLite transaction. The marker records the `file_id`, entry count, range count, +and a deterministic SHA-256 hash of the persisted ranges. + +If a later retry sees a matching marker, the file's ranges are already durable. +If it sees conflicting marker metadata, or ranges without a matching marker, the +backend fails closed instead of silently accepting ambiguous index state. + +## Test Setup + +Commands below assume they are run from the repository root: + +```bash +cd /path/to/pixels +``` + +If you are currently in this module directory, run: + +```bash +cd ../.. +``` + +The root `pom.xml` configures Surefire with `skipTests=true`, so +`mvn test -Dtest=...` still reports `Tests are skipped` for this module. To run +only a few SQLite tests without changing the POM, compile the module first and +then invoke Maven Failsafe directly. Failsafe is not bound by the inherited +Surefire `skipTests=true` setting. + +## Compile The Module + +```bash +mvn -pl pixels-index/pixels-index-main-sqlite -am \ + test-compile +``` + +This compiles the module and its reactor dependencies, including test classes, +but does not execute the JUnit tests. + +## Correctness Tests + +Run the main correctness suite: + +```bash +mvn -pl pixels-index/pixels-index-main-sqlite -am \ + test-compile \ + org.apache.maven.plugins:maven-failsafe-plugin:2.22.2:integration-test \ + org.apache.maven.plugins:maven-failsafe-plugin:2.22.2:verify \ + -Dit.test=TestSqliteMainIndex \ + -DfailIfNoTests=false +``` + +This covers normal put/get/delete behavior and the durable flush marker cases: + +- missing `fileId` flush is a no-op success; +- normal put -> flush -> lookup/delete; +- matching durable marker is accepted as an idempotent retry; +- marker metadata/hash conflicts fail closed and leave buffer retryable; +- dirty ranges without marker fail closed and leave buffer retryable; +- marker insert failure rolls back the range inserts; +- close/reopen flushes cached ranges and keeps rows readable. + +Run the JDBC range query correctness test: + +```bash +mvn -pl pixels-index/pixels-index-main-sqlite -am \ + test-compile \ + org.apache.maven.plugins:maven-failsafe-plugin:2.22.2:integration-test \ + org.apache.maven.plugins:maven-failsafe-plugin:2.22.2:verify \ + -Dit.test=TestSqliteMainIndexQuery \ + -DfailIfNoTests=false +``` + +This test writes a small file-scoped set of entries, flushes it, queries +`row_id_ranges` through JDBC, and asserts the persisted ranges are correct. + +## Performance Benchmark + +The benchmark is not a correctness gate. It is disabled by default and only runs +when explicitly enabled: + +```bash +mvn -pl pixels-index/pixels-index-main-sqlite -am \ + test-compile \ + org.apache.maven.plugins:maven-failsafe-plugin:2.22.2:integration-test \ + org.apache.maven.plugins:maven-failsafe-plugin:2.22.2:verify \ + -Dit.test=TestSqliteMainIndexBenchmark \ + -DfailIfNoTests=false \ + -Dpixels.sqlite.main.index.benchmark=true \ + -Dpixels.sqlite.main.index.benchmark.contiguousRows=1000000 \ + -Dpixels.sqlite.main.index.benchmark.fragmentedRows=10000 +``` + +Parameters: + +- `pixels.sqlite.main.index.benchmark`: must be `true` to run the benchmark. +- `pixels.sqlite.main.index.benchmark.contiguousRows`: row count for contiguous + rowId workloads. Default: `1000000`. +- `pixels.sqlite.main.index.benchmark.fragmentedRows`: row count for fragmented + rowId workloads. Default: `100000`. + +The benchmark prints a parameter block first, for example: + +```text +SQLite MainIndex benchmark parameters + -Dpixels.sqlite.main.index.benchmark=true + -Dpixels.sqlite.main.index.benchmark.contiguousRows=1000000 + -Dpixels.sqlite.main.index.benchmark.fragmentedRows=10000 + index.sqlite.path=/tmp/sqlite + java.version=23.0.2 + os.name=Linux + os.arch=amd64 +``` + +Then it prints a summary table: + +```text +SQLite MainIndex benchmark summary +rows = logical MainIndex entries; ranges = persisted row_id_ranges. +markerRetry = retry when a matching per-file durable marker already exists. +emptyRetry = immediate second flush after marker retry discarded the buffer. +workload shape rows ranges markers put(ms) put rows/s flush(ms) flush ranges/s markerRetry(ms) emptyRetry(ms) get(ms) get rows/s +hot put/get path contiguous, pre-flush get 1,000,000 1 1 ... +contiguous first flush contiguous rows -> 1 range 1,000,000 1 1 ... +fragmented first flush 1-row gaps -> many ranges 10,000 10,000 1 ... +marker-hit retry flush matching marker already durable 10,000 10,000 1 ... +``` + +How to read the table: + +- `rows`: logical entries inserted into `MainIndex`. +- `ranges`: persisted `row_id_ranges` count after flush. +- `markers`: persisted `row_id_range_flush_markers` count. +- `put(ms)` / `put rows/s`: in-memory `putEntry` hot path. +- `flush(ms)` / `flush ranges/s`: first durable flush path. +- `markerRetry(ms)`: retry path when SQLite already has a matching durable marker. +- `emptyRetry(ms)`: immediate second flush after marker retry discarded the buffer. +- `get(ms)` / `get rows/s`: lookup cost after the workload setup. + +For durable flush marker overhead, focus on: + +- `contiguous first flush` `flush(ms)`: best-case file flush, many rows become one + range plus one marker. +- `fragmented first flush` `flush(ms)`: many persisted ranges plus one marker. +- `marker-hit retry flush` `markerRetry(ms)`: crash/retry path after the previous + transaction committed but the in-memory buffer was not discarded. + +Large fragmented workloads can take much longer than contiguous workloads. That +is expected because `N` fragmented rows produce `N` SQLite ranges, while +contiguous rows often collapse into a single range. diff --git a/pixels-index/pixels-index-main-sqlite/src/main/java/io/pixelsdb/pixels/index/main/sqlite/SqliteMainIndex.java b/pixels-index/pixels-index-main-sqlite/src/main/java/io/pixelsdb/pixels/index/main/sqlite/SqliteMainIndex.java index be60cbf016..6958692a18 100644 --- a/pixels-index/pixels-index-main-sqlite/src/main/java/io/pixelsdb/pixels/index/main/sqlite/SqliteMainIndex.java +++ b/pixels-index/pixels-index-main-sqlite/src/main/java/io/pixelsdb/pixels/index/main/sqlite/SqliteMainIndex.java @@ -36,7 +36,10 @@ import java.io.File; import java.io.IOException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import java.sql.*; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -64,6 +67,13 @@ public class SqliteMainIndex implements MainIndex "(row_id_start BIGINT NOT NULL, row_id_end BIGINT NOT NULL, file_id BIGINT NOT NULL, rg_id INT NOT NULL," + "rg_row_offset_start INT NOT NULL, rg_row_offset_end INT NOT NULL, PRIMARY KEY (row_id_start, row_id_end))"; + /** + * The SQL statement to create the per-file flush marker table. + */ + private static final String createFlushMarkerTableSql = "CREATE TABLE IF NOT EXISTS row_id_range_flush_markers " + + "(file_id BIGINT NOT NULL PRIMARY KEY, entry_count BIGINT NOT NULL, range_count BIGINT NOT NULL, " + + "range_hash BLOB NOT NULL, committed_at_ms BIGINT NOT NULL)"; + /** * The SQL statement to query the row id range that covers the given row id (the two ? are of the same value). */ @@ -85,6 +95,42 @@ public class SqliteMainIndex implements MainIndex */ private static final String insertRangeSql = "INSERT INTO row_id_ranges VALUES(?, ?, ?, ?, ?, ?)"; + /** + * The SQL statement to query a per-file flush marker. + */ + private static final String queryFlushMarkerSql = + "SELECT entry_count, range_count, range_hash FROM row_id_range_flush_markers WHERE file_id = ?"; + + /** + * The SQL statement to insert a per-file flush marker. + */ + private static final String insertFlushMarkerSql = + "INSERT INTO row_id_range_flush_markers VALUES(?, ?, ?, ?, ?)"; + + private static final class FlushMarker + { + private final long fileId; + private final long entryCount; + private final long rangeCount; + private final byte[] rangeHash; + + private FlushMarker(long fileId, long entryCount, long rangeCount, byte[] rangeHash) + { + this.fileId = fileId; + this.entryCount = entryCount; + this.rangeCount = rangeCount; + this.rangeHash = rangeHash; + } + + private boolean matches(MainIndexBuffer.FlushSnapshot snapshot, byte[] snapshotHash) + { + return this.fileId == snapshot.getFileId() + && this.entryCount == snapshot.getEntryCount() + && this.rangeCount == snapshot.getRowIdRanges().size() + && Arrays.equals(this.rangeHash, snapshotHash); + } + } + private final long tableId; private final String sqlitePath; private final MainIndexBuffer indexBuffer; @@ -116,6 +162,7 @@ public SqliteMainIndex(long tableId, String sqlitePath) throws MainIndexExceptio try (Statement statement = connection.createStatement()) { statement.execute(createTableSql); + statement.execute(createFlushMarkerTableSql); } } catch (SQLException e) @@ -312,31 +359,68 @@ public List putEntries(List primaryEntrie @Override public boolean deleteRowIdRange(RowIdRange rowIdRange) throws MainIndexException { + long rowIdStart = rowIdRange.getRowIdStart(); + long rowIdEnd = rowIdRange.getRowIdEnd(); + if (rowIdEnd <= rowIdStart) + { + throw new MainIndexException("Invalid row id range to delete: [" + rowIdStart + ", " + rowIdEnd + ")"); + } + this.dbRwLock.writeLock().lock(); - try (PreparedStatement pst = connection.prepareStatement(deleteRangesSql)) - { - long rowIdStart = rowIdRange.getRowIdStart(); - long rowIdEnd = rowIdRange.getRowIdEnd(); - pst.setLong(1, rowIdStart); - pst.setLong(2, rowIdEnd); - RowIdRange leftBorderRange = getRowIdRangeFromSqlite(rowIdStart); - RowIdRange rightBorderRange = getRowIdRangeFromSqlite(rowIdEnd - 1); - boolean res = true; - if (leftBorderRange != null) + try + { + boolean originalAutoCommit = this.connection.getAutoCommit(); + try { - int width = (int) (rowIdStart - leftBorderRange.getRowIdStart()); - RowIdRange newLeftBorderRange = leftBorderRange.toBuilder() - .setRowIdEnd(rowIdStart).setRgRowOffsetEnd(leftBorderRange.getRgRowOffsetStart() + width).build(); - res &= updateRowIdRangeWidth(leftBorderRange, newLeftBorderRange); + this.connection.setAutoCommit(false); + RowIdRange leftBorderRange = getRowIdRangeFromSqlite(rowIdStart); + RowIdRange rightBorderRange = getRowIdRangeFromSqlite(rowIdEnd - 1); + boolean res = true; + try (PreparedStatement pst = connection.prepareStatement(deleteRangesSql)) + { + pst.setLong(1, rowIdStart); + pst.setLong(2, rowIdEnd); + pst.executeUpdate(); + } + if (leftBorderRange != null && rightBorderRange != null && + leftBorderRange.getRowIdStart() == rightBorderRange.getRowIdStart() && + leftBorderRange.getRowIdEnd() == rightBorderRange.getRowIdEnd()) + { + res &= trimSingleOverlappingRange(leftBorderRange, rowIdStart, rowIdEnd); + } + else + { + if (leftBorderRange != null && leftBorderRange.getRowIdStart() < rowIdStart && + rowIdStart < leftBorderRange.getRowIdEnd()) + { + int width = (int) (rowIdStart - leftBorderRange.getRowIdStart()); + RowIdRange newLeftBorderRange = leftBorderRange.toBuilder() + .setRowIdEnd(rowIdStart) + .setRgRowOffsetEnd(leftBorderRange.getRgRowOffsetStart() + width).build(); + res &= updateRowIdRangeWidth(leftBorderRange, newLeftBorderRange); + } + if (rightBorderRange != null && rightBorderRange.getRowIdStart() < rowIdEnd && + rowIdEnd < rightBorderRange.getRowIdEnd()) + { + int width = (int) (rightBorderRange.getRowIdEnd() - rowIdEnd); + RowIdRange newRightBorderRange = rightBorderRange.toBuilder() + .setRowIdStart(rowIdEnd) + .setRgRowOffsetStart(rightBorderRange.getRgRowOffsetEnd() - width).build(); + res &= updateRowIdRangeWidth(rightBorderRange, newRightBorderRange); + } + } + this.connection.commit(); + return res; } - if (rightBorderRange != null) + catch (SQLException | RowIdException e) { - int width = (int) (rightBorderRange.getRowIdEnd() - rowIdEnd); - RowIdRange newRightBorderRange = rightBorderRange.toBuilder() - .setRowIdStart(rowIdEnd).setRgRowOffsetStart(rightBorderRange.getRgRowOffsetEnd() - width).build(); - res &= updateRowIdRangeWidth(rightBorderRange, newRightBorderRange); + rollbackQuietly(e); + throw e; + } + finally + { + this.connection.setAutoCommit(originalAutoCommit); } - return res; } catch (SQLException | RowIdException e) { @@ -350,6 +434,46 @@ public boolean deleteRowIdRange(RowIdRange rowIdRange) throws MainIndexException } } + private boolean trimSingleOverlappingRange(RowIdRange range, long rowIdStart, long rowIdEnd) + throws RowIdException, SQLException + { + if (range.getRowIdStart() < rowIdStart && rowIdEnd < range.getRowIdEnd()) + { + int leftWidth = (int) (rowIdStart - range.getRowIdStart()); + RowIdRange newLeftRange = range.toBuilder() + .setRowIdEnd(rowIdStart) + .setRgRowOffsetEnd(range.getRgRowOffsetStart() + leftWidth).build(); + int rightWidth = (int) (range.getRowIdEnd() - rowIdEnd); + RowIdRange newRightRange = range.toBuilder() + .setRowIdStart(rowIdEnd) + .setRgRowOffsetStart(range.getRgRowOffsetEnd() - rightWidth).build(); + boolean res = updateRowIdRangeWidth(range, newLeftRange); + try (PreparedStatement pst = this.connection.prepareStatement(insertRangeSql)) + { + bindRangeInsertStatement(pst, newRightRange); + res &= pst.executeUpdate() > 0; + } + return res; + } + if (range.getRowIdStart() < rowIdStart && rowIdStart < range.getRowIdEnd()) + { + int width = (int) (rowIdStart - range.getRowIdStart()); + RowIdRange newLeftRange = range.toBuilder() + .setRowIdEnd(rowIdStart) + .setRgRowOffsetEnd(range.getRgRowOffsetStart() + width).build(); + return updateRowIdRangeWidth(range, newLeftRange); + } + if (range.getRowIdStart() < rowIdEnd && rowIdEnd < range.getRowIdEnd()) + { + int width = (int) (range.getRowIdEnd() - rowIdEnd); + RowIdRange newRightRange = range.toBuilder() + .setRowIdStart(rowIdEnd) + .setRgRowOffsetStart(range.getRgRowOffsetEnd() - width).build(); + return updateRowIdRangeWidth(range, newRightRange); + } + return true; + } + /** * Get the row id range that contains the given row id from sqlite. * @param rowId the given row id @@ -392,6 +516,16 @@ private RowIdRange getRowIdRangeFromSqlite (long rowId) throws RowIdException } } + private static void bindRangeInsertStatement(PreparedStatement pst, RowIdRange range) throws SQLException + { + pst.setLong(1, range.getRowIdStart()); + pst.setLong(2, range.getRowIdEnd()); + pst.setLong(3, range.getFileId()); + pst.setInt(4, range.getRgId()); + pst.setInt(5, range.getRgRowOffsetStart()); + pst.setInt(6, range.getRgRowOffsetEnd()); + } + /** * Update the width of an existing row id range. * @param oldRange the old row id range @@ -424,22 +558,52 @@ public boolean flushCache(long fileId) throws MainIndexException this.dbRwLock.writeLock().lock(); try { - List rowIdRanges = this.indexBuffer.flush(fileId); - try (PreparedStatement pst = this.connection.prepareStatement(insertRangeSql)) + MainIndexBuffer.FlushSnapshot snapshot = this.indexBuffer.snapshotForFlush(fileId); + if (snapshot.isEmpty()) { - for (RowIdRange range : rowIdRanges) + return true; + } + + byte[] snapshotHash = buildRangeHash(snapshot.getRowIdRanges()); + FlushMarker marker = readFlushMarker(snapshot.getFileId()); + if (marker != null) + { + if (!marker.matches(snapshot, snapshotHash)) { - pst.setLong(1, range.getRowIdStart()); - pst.setLong(2, range.getRowIdEnd()); - pst.setLong(3, range.getFileId()); - pst.setInt(4, range.getRgId()); - pst.setInt(5, range.getRgRowOffsetStart()); - pst.setInt(6, range.getRgRowOffsetEnd()); - pst.addBatch(); + throw new MainIndexException("Conflicting flush marker already exists for fileId=" + fileId); } - pst.executeBatch(); + this.indexBuffer.discardFlushed(snapshot); return true; } + + boolean originalAutoCommit = this.connection.getAutoCommit(); + try + { + this.connection.setAutoCommit(false); + try (PreparedStatement pst = this.connection.prepareStatement(insertRangeSql)) + { + for (RowIdRange range : snapshot.getRowIdRanges()) + { + bindRangeInsertStatement(pst, range); + pst.addBatch(); + } + pst.executeBatch(); + } + insertFlushMarker(snapshot, snapshotHash); + this.connection.commit(); + } + catch (SQLException e) + { + rollbackQuietly(e); + throw e; + } + finally + { + this.connection.setAutoCommit(originalAutoCommit); + } + + this.indexBuffer.discardFlushed(snapshot); + return true; } catch (MainIndexException | SQLException e) { @@ -452,6 +616,86 @@ public boolean flushCache(long fileId) throws MainIndexException } } + private FlushMarker readFlushMarker(long fileId) throws SQLException + { + try (PreparedStatement pst = this.connection.prepareStatement(queryFlushMarkerSql)) + { + pst.setLong(1, fileId); + try (ResultSet rs = pst.executeQuery()) + { + if (!rs.next()) + { + return null; + } + return new FlushMarker(fileId, rs.getLong("entry_count"), + rs.getLong("range_count"), rs.getBytes("range_hash")); + } + } + } + + private void insertFlushMarker(MainIndexBuffer.FlushSnapshot snapshot, byte[] rangeHash) throws SQLException + { + try (PreparedStatement pst = this.connection.prepareStatement(insertFlushMarkerSql)) + { + pst.setLong(1, snapshot.getFileId()); + pst.setLong(2, snapshot.getEntryCount()); + pst.setLong(3, snapshot.getRowIdRanges().size()); + pst.setBytes(4, rangeHash); + pst.setLong(5, System.currentTimeMillis()); + pst.executeUpdate(); + } + } + + private byte[] buildRangeHash(List rowIdRanges) throws MainIndexException + { + try + { + MessageDigest digest = MessageDigest.getInstance("SHA-256"); + for (RowIdRange range : rowIdRanges) + { + updateLong(digest, range.getRowIdStart()); + updateLong(digest, range.getRowIdEnd()); + updateLong(digest, range.getFileId()); + updateInt(digest, range.getRgId()); + updateInt(digest, range.getRgRowOffsetStart()); + updateInt(digest, range.getRgRowOffsetEnd()); + } + return digest.digest(); + } + catch (NoSuchAlgorithmException e) + { + throw new MainIndexException("Failed to build range hash for main index flush", e); + } + } + + private static void updateLong(MessageDigest digest, long value) + { + for (int shift = 56; shift >= 0; shift -= 8) + { + digest.update((byte) (value >>> shift)); + } + } + + private static void updateInt(MessageDigest digest, int value) + { + for (int shift = 24; shift >= 0; shift -= 8) + { + digest.update((byte) (value >>> shift)); + } + } + + private void rollbackQuietly(Exception failure) + { + try + { + this.connection.rollback(); + } + catch (SQLException rollbackException) + { + failure.addSuppressed(rollbackException); + } + } + @Override public void close() throws IOException { @@ -517,4 +761,4 @@ public boolean closeAndRemove() throws MainIndexException } return true; } -} \ No newline at end of file +} diff --git a/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndex.java b/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndex.java index ddf1a0aae3..9313977b12 100644 --- a/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndex.java +++ b/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndex.java @@ -29,11 +29,21 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; +import java.security.MessageDigest; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.Statement; +import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -42,16 +52,19 @@ public class TestSqliteMainIndex { - long tableId = 100L; + private static long nextTableId = 100L; + long tableId; + String sqlitePath; MainIndex mainIndex; @BeforeEach public void setUp() throws MainIndexException { + tableId = nextTableId++; // Create SQLite Directory try { - String sqlitePath = ConfigFactory.Instance().getProperty("index.sqlite.path"); + sqlitePath = ConfigFactory.Instance().getProperty("index.sqlite.path"); FileUtils.forceMkdir(new File(sqlitePath)); } catch (IOException e) @@ -65,12 +78,11 @@ public void setUp() throws MainIndexException @AfterEach public void tearDown() throws Exception { - mainIndex.close(); + MainIndexFactory.Instance().closeIndex(tableId, true); // Clear SQLite Directory try { - String sqlitePath = ConfigFactory.Instance().getProperty("index.sqlite.path"); FileUtils.deleteDirectory(new File(sqlitePath)); } catch (IOException e) @@ -79,6 +91,428 @@ public void tearDown() throws Exception } } + @Test + public void testFlushCacheMissingFileIsNoop() throws MainIndexException + { + Assertions.assertTrue(mainIndex.flushCache(987654321L)); + } + + @Test + public void testFlushCacheAcceptsMatchingCommittedMarker() throws Exception + { + long fileId = 42L; + RowIdRange firstRange = new RowIdRange(5000L, 5002L, fileId, 0, 0, 2); + RowIdRange secondRange = new RowIdRange(5010L, 5011L, fileId, 1, 0, 1); + List ranges = new ArrayList<>(); + ranges.add(firstRange); + ranges.add(secondRange); + putMainIndexEntry(5000L, fileId, 0, 0); + putMainIndexEntry(5001L, fileId, 0, 1); + putMainIndexEntry(5010L, fileId, 1, 0); + + insertRange(firstRange); + insertRange(secondRange); + insertFlushMarker(fileId, 3, ranges); + Assertions.assertEquals(2, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(2, countRangesForFile(fileId)); + assertLocation(5000L, fileId, 0, 0); + assertLocation(5001L, fileId, 0, 1); + assertLocation(5010L, fileId, 1, 0); + + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(2, countRangesForFile(fileId)); + } + + @Test + public void testFlushCacheConflictingMarkerKeepsBufferRetryable() throws Exception + { + long fileId = 43L; + putMainIndexEntry(6000L, fileId, 0, 0); + putMainIndexEntry(6001L, fileId, 0, 1); + putMainIndexEntry(6010L, fileId, 1, 0); + + insertFlushMarker(fileId, 3, new ArrayList<>()); + + Assertions.assertThrows(MainIndexException.class, () -> mainIndex.flushCache(fileId)); + Assertions.assertEquals(0, countExactRanges(6010L, 6011L)); + assertLocation(6000L, fileId, 0, 0); + assertLocation(6010L, fileId, 1, 0); + + deleteFlushMarker(fileId); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(2, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + assertLocation(6000L, fileId, 0, 0); + assertLocation(6010L, fileId, 1, 0); + } + + @Test + public void testFlushCacheRangeWithoutMarkerFailsAndKeepsBufferRetryable() throws Exception + { + long fileId = 44L; + putMainIndexEntry(7000L, fileId, 0, 0); + putMainIndexEntry(7001L, fileId, 0, 1); + putMainIndexEntry(7010L, fileId, 1, 0); + + insertRange(new RowIdRange(7000L, 7002L, fileId, 0, 0, 2)); + + Assertions.assertThrows(MainIndexException.class, () -> mainIndex.flushCache(fileId)); + Assertions.assertEquals(0, countExactRanges(7010L, 7011L)); + Assertions.assertEquals(0, countFlushMarkersForFile(fileId)); + assertLocation(7000L, fileId, 0, 0); + assertLocation(7010L, fileId, 1, 0); + + deleteExactRange(7000L, 7002L); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(2, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + } + + @Test + public void testFlushCacheRejectsFlushMarkerMetadataMismatches() throws Exception + { + long fileId = 45L; + putMainIndexEntry(8000L, fileId, 0, 0); + putMainIndexEntry(8001L, fileId, 0, 1); + + List ranges = Arrays.asList(new RowIdRange(8000L, 8002L, fileId, 0, 0, 2)); + byte[] rangeHash = buildRangeHash(ranges); + + insertFlushMarker(fileId, 1, ranges.size(), rangeHash); + assertFlushFailsAndBufferSurvives(fileId, 8000L, 8001L); + + deleteFlushMarker(fileId); + insertFlushMarker(fileId, 2, ranges.size() + 1, rangeHash); + assertFlushFailsAndBufferSurvives(fileId, 8000L, 8001L); + + deleteFlushMarker(fileId); + byte[] badHash = rangeHash.clone(); + badHash[0] = (byte) (badHash[0] ^ 0x7f); + insertFlushMarker(fileId, 2, ranges.size(), badHash); + assertFlushFailsAndBufferSurvives(fileId, 8000L, 8001L); + + deleteFlushMarker(fileId); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(1, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + } + + @Test + public void testFlushCacheRollsBackRangesWhenMarkerInsertFails() throws Exception + { + long fileId = 46L; + putMainIndexEntry(9000L, fileId, 0, 0); + putMainIndexEntry(9001L, fileId, 0, 1); + putMainIndexEntry(9010L, fileId, 1, 0); + + createFailingFlushMarkerTrigger(fileId); + Assertions.assertThrows(MainIndexException.class, () -> mainIndex.flushCache(fileId)); + Assertions.assertEquals(0, countRangesForFile(fileId)); + Assertions.assertEquals(0, countFlushMarkersForFile(fileId)); + assertLocation(9000L, fileId, 0, 0); + assertLocation(9010L, fileId, 1, 0); + + dropFailingFlushMarkerTrigger(); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(2, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + } + + @Test + public void testFlushCacheConvergesAfterUnknownCommittedStateWithOutOfOrderBuffer() throws Exception + { + long fileId = 48L; + List committedRanges = Arrays.asList( + new RowIdRange(11000L, 11003L, fileId, 0, 0, 3), + new RowIdRange(11010L, 11012L, fileId, 1, 7, 9)); + + putMainIndexEntry(11002L, fileId, 0, 2); + putMainIndexEntry(11000L, fileId, 0, 0); + putMainIndexEntry(11010L, fileId, 1, 7); + putMainIndexEntry(11001L, fileId, 0, 1); + putMainIndexEntry(11011L, fileId, 1, 8); + + for (RowIdRange range : committedRanges) + { + insertRange(range); + } + insertFlushMarker(fileId, 5, committedRanges); + + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertEquals(2, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + assertNoInvalidRanges(fileId); + assertLocation(11000L, fileId, 0, 0); + assertLocation(11002L, fileId, 0, 2); + assertLocation(11011L, fileId, 1, 8); + } + + @Test + public void testFlushCacheFailureForOneFileDoesNotDiscardOtherFileBuffers() throws Exception + { + long failingFileId = 49L; + long healthyFileId = 50L; + putMainIndexEntry(12000L, failingFileId, 0, 0); + putMainIndexEntry(12001L, failingFileId, 0, 1); + putMainIndexEntry(12100L, healthyFileId, 0, 0); + putMainIndexEntry(12101L, healthyFileId, 0, 1); + + createFailingFlushMarkerTrigger(failingFileId); + Assertions.assertThrows(MainIndexException.class, () -> mainIndex.flushCache(failingFileId)); + Assertions.assertEquals(0, countRangesForFile(failingFileId)); + Assertions.assertEquals(0, countFlushMarkersForFile(failingFileId)); + assertLocation(12000L, failingFileId, 0, 0); + + Assertions.assertTrue(mainIndex.flushCache(healthyFileId)); + Assertions.assertEquals(1, countRangesForFile(healthyFileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(healthyFileId)); + assertLocation(12101L, healthyFileId, 0, 1); + + dropFailingFlushMarkerTrigger(); + Assertions.assertTrue(mainIndex.flushCache(failingFileId)); + Assertions.assertEquals(1, countRangesForFile(failingFileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(failingFileId)); + } + + @Test + public void testPutEntriesFlushesDurableRangesAndLocations() throws Exception + { + long fileId = 51L; + List entries = Arrays.asList( + primaryEntry(13002L, fileId, 0, 2), + primaryEntry(13000L, fileId, 0, 0), + primaryEntry(13001L, fileId, 0, 1), + primaryEntry(13020L, fileId, 2, 4), + primaryEntry(13021L, fileId, 2, 5)); + + assertAllTrue(mainIndex.putEntries(entries)); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(2, ranges.size()); + assertRange(ranges.get(0), 13000L, 13003L, fileId, 0, 0, 3); + assertRange(ranges.get(1), 13020L, 13022L, fileId, 2, 4, 6); + assertNoInvalidRanges(fileId); + + List locations = mainIndex.getLocations(Arrays.asList(13000L, 13002L, 13021L)); + Assertions.assertEquals(3, locations.size()); + Assertions.assertEquals(0, locations.get(0).getRgRowOffset()); + Assertions.assertEquals(2, locations.get(1).getRgRowOffset()); + Assertions.assertEquals(5, locations.get(2).getRgRowOffset()); + } + + @Test + public void testCloseConvergesWhenPreviousFlushCommittedButBufferSurvived() throws Exception + { + long fileId = 52L; + RowIdRange committedRange = new RowIdRange(14000L, 14002L, fileId, 0, 0, 2); + putMainIndexEntry(14000L, fileId, 0, 0); + putMainIndexEntry(14001L, fileId, 0, 1); + + insertRange(committedRange); + insertFlushMarker(fileId, 2, Arrays.asList(committedRange)); + + MainIndexFactory.Instance().closeIndex(tableId, false); + mainIndex = MainIndexFactory.Instance().getMainIndex(tableId); + + Assertions.assertEquals(1, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + assertLocation(14000L, fileId, 0, 0); + assertLocation(14001L, fileId, 0, 1); + } + + @Test + public void testDeleteRowIdRangeRemovesExactRangeWithoutInvalidResidue() throws Exception + { + long fileId = 53L; + putContiguousEntries(fileId, 0, 15000L, 15004L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(15000L, 15004L, fileId, 0, 0, 4))); + + Assertions.assertEquals(0, countRangesForFile(fileId)); + assertNoInvalidRanges(fileId); + for (long rowId = 15000L; rowId < 15004L; rowId++) + { + assertLocationMissing(rowId); + } + } + + @Test + public void testDeleteRowIdRangeSplitsMiddleRangeForRecoveryCleanup() throws Exception + { + long fileId = 54L; + putContiguousEntries(fileId, 0, 16000L, 16010L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(16003L, 16007L, fileId, 0, 3, 7))); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(2, ranges.size()); + assertRange(ranges.get(0), 16000L, 16003L, fileId, 0, 0, 3); + assertRange(ranges.get(1), 16007L, 16010L, fileId, 0, 7, 10); + assertNoInvalidRanges(fileId); + assertLocation(16002L, fileId, 0, 2); + assertLocationMissing(16003L); + assertLocationMissing(16006L); + assertLocation(16007L, fileId, 0, 7); + } + + @Test + public void testDeleteRowIdRangeTrimsBordersAndDeletesCoveredRanges() throws Exception + { + long fileId = 55L; + putContiguousEntries(fileId, 0, 17000L, 17005L, 0); + putContiguousEntries(fileId, 1, 17010L, 17015L, 0); + putContiguousEntries(fileId, 2, 17020L, 17025L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(17003L, 17022L, fileId, 0, 3, 22))); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(2, ranges.size()); + assertRange(ranges.get(0), 17000L, 17003L, fileId, 0, 0, 3); + assertRange(ranges.get(1), 17022L, 17025L, fileId, 2, 2, 5); + assertNoInvalidRanges(fileId); + assertLocation(17002L, fileId, 0, 2); + assertLocationMissing(17010L); + assertLocationMissing(17021L); + assertLocation(17022L, fileId, 2, 2); + } + + @Test + public void testDeleteRowIdRangeLeftAlignedTrimsLeadingPortionOfSingleRange() throws Exception + { + long fileId = 60L; + putContiguousEntries(fileId, 0, 21000L, 21010L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + // Delete [21000, 21003) which shares its left edge with the existing range [21000, 21010). + // Expected to trim the leading portion and keep [21003, 21010). + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(21000L, 21003L, fileId, 0, 0, 3))); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(1, ranges.size()); + assertRange(ranges.get(0), 21003L, 21010L, fileId, 0, 3, 10); + assertNoInvalidRanges(fileId); + assertLocationMissing(21000L); + assertLocationMissing(21002L); + assertLocation(21003L, fileId, 0, 3); + assertLocation(21009L, fileId, 0, 9); + } + + @Test + public void testDeleteRowIdRangeRightAlignedTrimsTrailingPortionOfSingleRange() throws Exception + { + long fileId = 61L; + putContiguousEntries(fileId, 0, 22000L, 22010L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + // Delete [22007, 22010) which shares its right edge with the existing range [22000, 22010). + // Expected to trim the trailing portion and keep [22000, 22007). + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(22007L, 22010L, fileId, 0, 7, 10))); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(1, ranges.size()); + assertRange(ranges.get(0), 22000L, 22007L, fileId, 0, 0, 7); + assertNoInvalidRanges(fileId); + assertLocation(22000L, fileId, 0, 0); + assertLocation(22006L, fileId, 0, 6); + assertLocationMissing(22007L); + assertLocationMissing(22009L); + } + + @Test + public void testDeleteRowIdRangeFullyContainsSingleRangeRemovesItWithoutResidue() throws Exception + { + long fileId = 62L; + // Single committed range [23000, 23004) sitting in isolation. + putContiguousEntries(fileId, 0, 23000L, 23004L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + // Delete [22990, 23010) which strictly contains the entire range. + // No border range is partially overlapped, so the bulk DELETE clause should remove the range + // and leave no residue or split-out ranges. + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(22990L, 23010L, fileId, 0, 0, 20))); + + Assertions.assertEquals(0, countRangesForFile(fileId)); + assertNoInvalidRanges(fileId); + for (long rowId = 23000L; rowId < 23004L; rowId++) + { + assertLocationMissing(rowId); + } + } + + @Test + public void testDeleteRowIdRangeMissingAllRangesIsNoop() throws Exception + { + long fileId = 63L; + // Persist a single range [24000, 24004) so the table is non-empty. + putContiguousEntries(fileId, 0, 24000L, 24004L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + // Delete a row id window that does not overlap any committed range; should be a no-op. + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(30000L, 30010L, fileId, 0, 0, 10))); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(1, ranges.size()); + assertRange(ranges.get(0), 24000L, 24004L, fileId, 0, 0, 4); + assertNoInvalidRanges(fileId); + assertLocation(24000L, fileId, 0, 0); + assertLocation(24003L, fileId, 0, 3); + // Row ids inside the deleted (but never committed) window remain unknown. + assertLocationMissing(30000L); + assertLocationMissing(30009L); + } + + @Test + public void testDeleteRowIdRangeRollsBackSplitWhenRightRangeInsertFails() throws Exception + { + long fileId = 57L; + putContiguousEntries(fileId, 0, 19000L, 19010L, 0); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + + createFailingRangeInsertTrigger(19007L); + Assertions.assertThrows(MainIndexException.class, + () -> mainIndex.deleteRowIdRange(new RowIdRange(19003L, 19007L, fileId, 0, 3, 7))); + dropFailingRangeInsertTrigger(); + + List ranges = listRangesForFile(fileId); + Assertions.assertEquals(1, ranges.size()); + assertRange(ranges.get(0), 19000L, 19010L, fileId, 0, 0, 10); + assertNoInvalidRanges(fileId); + assertLocation(19003L, fileId, 0, 3); + assertLocation(19007L, fileId, 0, 7); + } + + @Test + public void testDeleteRowIdRangeRejectsEmptyOrReversedRange() throws Exception + { + Assertions.assertThrows(MainIndexException.class, + () -> mainIndex.deleteRowIdRange(new RowIdRange(20000L, 20000L, 58L, 0, 0, 0))); + Assertions.assertThrows(MainIndexException.class, + () -> mainIndex.deleteRowIdRange(new RowIdRange(20001L, 20000L, 58L, 0, 1, 0))); + } + + @Test + public void testCloseFlushesCacheWithMarkerAndReopenReadsRows() throws Exception + { + long fileId = 47L; + putMainIndexEntry(10000L, fileId, 0, 0); + putMainIndexEntry(10001L, fileId, 0, 1); + + MainIndexFactory.Instance().closeIndex(tableId, false); + mainIndex = MainIndexFactory.Instance().getMainIndex(tableId); + + Assertions.assertEquals(1, countRangesForFile(fileId)); + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + assertLocation(10000L, fileId, 0, 0); + assertLocation(10001L, fileId, 0, 1); + } + @Test public void testPutAndGetLocation() throws MainIndexException { @@ -95,7 +529,7 @@ public void testPutAndGetLocation() throws MainIndexException } @Test - public void testFlushCacheAndDeleteEntry() throws MainIndexException + public void testFlushCacheAndDeleteEntry() throws Exception { long rowId = 2000L; IndexProto.RowLocation location = IndexProto.RowLocation.newBuilder() @@ -107,52 +541,107 @@ public void testFlushCacheAndDeleteEntry() throws MainIndexException Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(rowId, rowId + 1, 2, 2, 0, 1))); - Assertions.assertNull(mainIndex.getLocation(rowId)); + assertLocationMissing(rowId); + Assertions.assertEquals(0, countRangesForFile(2)); + location = location.toBuilder().setFileId(3).build(); Assertions.assertTrue(mainIndex.putEntry(rowId, location)); Assertions.assertNotNull(mainIndex.getLocation(rowId)); - Assertions.assertTrue(mainIndex.flushCache(2)); + Assertions.assertTrue(mainIndex.flushCache(3)); Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(rowId - 1, rowId + 1, - 2, 2, 0, 2))); - Assertions.assertNull(mainIndex.getLocation(rowId)); + 3, 2, 0, 2))); + assertLocationMissing(rowId); + Assertions.assertEquals(0, countRangesForFile(3)); + location = location.toBuilder().setFileId(4).build(); Assertions.assertTrue(mainIndex.putEntry(rowId, location)); Assertions.assertNotNull(mainIndex.getLocation(rowId)); - Assertions.assertTrue(mainIndex.flushCache(2)); + Assertions.assertTrue(mainIndex.flushCache(4)); Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(rowId - 1, rowId, - 2, 2, 0, 1))); + 4, 2, 0, 1))); Assertions.assertNotNull(mainIndex.getLocation(rowId)); + } - Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(rowId, rowId + 1, - 2, 2, 0, 1))); + @Test + @Tag("performance") + public void testFlushCachePerformanceSmoke() throws Exception + { + int entryCount = Integer.getInteger("sqlite.main.index.perf.smoke.entries", 50_000); + long timeoutSeconds = Long.getLong("sqlite.main.index.perf.smoke.timeout.sec", 30L); + long fileId = 56L; + long rowIdBase = 18000L; + long[] elapsedMs = new long[4]; + + Assertions.assertTimeout(Duration.ofSeconds(timeoutSeconds), () -> { + IndexProto.RowLocation.Builder locationBuilder = IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(0); + long start = System.nanoTime(); + for (int i = 0; i < entryCount; i++) + { + Assertions.assertTrue(mainIndex.putEntry(rowIdBase + i, + locationBuilder.setRgRowOffset(i).build())); + } + elapsedMs[0] = nanosToMillis(System.nanoTime() - start); + + start = System.nanoTime(); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + elapsedMs[1] = nanosToMillis(System.nanoTime() - start); + + start = System.nanoTime(); + int sampleStep = Math.max(1, entryCount / 100); + for (int i = 0; i < entryCount; i += sampleStep) + { + IndexProto.RowLocation location = mainIndex.getLocation(rowIdBase + i); + Assertions.assertEquals(fileId, location.getFileId()); + Assertions.assertEquals(i, location.getRgRowOffset()); + } + elapsedMs[2] = nanosToMillis(System.nanoTime() - start); + + start = System.nanoTime(); + Assertions.assertTrue(mainIndex.flushCache(fileId)); + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange( + rowIdBase, rowIdBase + entryCount, fileId, 0, 0, entryCount))); + elapsedMs[3] = nanosToMillis(System.nanoTime() - start); + }); + + Assertions.assertEquals(1, countFlushMarkersForFile(fileId)); + Assertions.assertEquals(0, countRangesForFile(fileId)); + System.out.println("sqlite main index perf smoke entries=" + entryCount + + ", putMs=" + elapsedMs[0] + + ", flushMs=" + elapsedMs[1] + + ", sampledGetMs=" + elapsedMs[2] + + ", idempotentFlushAndDeleteMs=" + elapsedMs[3]); } @Test + @Disabled("Manual performance smoke test; not a correctness gate.") + @Tag("performance") public void testPutAndGetPerformance() throws MainIndexException { final long rowIdBase = 0L; + final int entryCount = Integer.getInteger("sqlite.main.index.perf.entries", 10_000_000); IndexProto.RowLocation.Builder locationBuilder = IndexProto.RowLocation.newBuilder() .setFileId(1L).setRgId(0); long start = System.currentTimeMillis(); - for (int i = 0; i < 10000000; i++) + for (int i = 0; i < entryCount; i++) { mainIndex.putEntry(rowIdBase + i, locationBuilder.setRgRowOffset(i).build()); } - System.out.println("put 10M entries in " + (System.currentTimeMillis() - start) + " ms"); + System.out.println("put " + entryCount + " entries in " + (System.currentTimeMillis() - start) + " ms"); start = System.currentTimeMillis(); - for (int i = 0; i < 10000000; i++) + for (int i = 0; i < entryCount; i++) { mainIndex.getLocation(rowIdBase + i); } - System.out.println("get 10M entries in " + (System.currentTimeMillis() - start) + " ms"); + System.out.println("get " + entryCount + " entries in " + (System.currentTimeMillis() - start) + " ms"); start = System.currentTimeMillis(); mainIndex.flushCache(1); System.out.println("flush cache in " + (System.currentTimeMillis() - start) + " ms"); start = System.currentTimeMillis(); mainIndex.deleteRowIdRange(new RowIdRange( - 0L, 10_000_000L, 1L, 0, 0, 10_000_000)); + 0L, entryCount, 1L, 0, 0, entryCount)); System.out.println("delete all entries in " + (System.currentTimeMillis() - start) + " ms"); } @@ -261,10 +750,10 @@ public void testConcurrentPutAndDeleteRowIds() throws Exception { mainIndex.flushCache(threadNum); RowIdRange range = ranges.get(threadNum); - Assertions.assertTrue(mainIndex.deleteRowIdRange(range)); - for (long id = range.getRowIdStart(); id <= range.getRowIdEnd(); id++) + mainIndex.deleteRowIdRange(range); + for (long id = range.getRowIdStart(); id < range.getRowIdEnd(); id++) { - Assertions.assertNull(mainIndex.getLocation(id)); + assertLocationMissing(id); } } finally @@ -282,4 +771,302 @@ public void testConcurrentPutAndDeleteRowIds() throws Exception } executor.shutdown(); } -} \ No newline at end of file + + private void putMainIndexEntry(long rowId, long fileId, int rgId, int rgRowOffset) + { + Assertions.assertTrue(mainIndex.putEntry(rowId, IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(rgId).setRgRowOffset(rgRowOffset).build())); + } + + private void putContiguousEntries(long fileId, int rgId, long rowIdStart, long rowIdEnd, int rgRowOffsetStart) + { + int offset = rgRowOffsetStart; + for (long rowId = rowIdStart; rowId < rowIdEnd; rowId++) + { + putMainIndexEntry(rowId, fileId, rgId, offset++); + } + } + + private IndexProto.PrimaryIndexEntry primaryEntry(long rowId, long fileId, int rgId, int rgRowOffset) + { + return IndexProto.PrimaryIndexEntry.newBuilder() + .setRowId(rowId) + .setRowLocation(IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(rgId).setRgRowOffset(rgRowOffset).build()) + .build(); + } + + private void assertAllTrue(List results) + { + for (Boolean result : results) + { + Assertions.assertTrue(result); + } + } + + private void assertLocation(long rowId, long fileId, int rgId, int rgRowOffset) throws MainIndexException + { + IndexProto.RowLocation location = mainIndex.getLocation(rowId); + Assertions.assertNotNull(location); + Assertions.assertEquals(fileId, location.getFileId()); + Assertions.assertEquals(rgId, location.getRgId()); + Assertions.assertEquals(rgRowOffset, location.getRgRowOffset()); + } + + private void assertLocationMissing(long rowId) + { + Assertions.assertThrows(MainIndexException.class, () -> mainIndex.getLocation(rowId)); + } + + private void assertFlushFailsAndBufferSurvives(long fileId, long firstRowId, long secondRowId) throws Exception + { + Assertions.assertThrows(MainIndexException.class, () -> mainIndex.flushCache(fileId)); + Assertions.assertEquals(0, countRangesForFile(fileId)); + assertLocation(firstRowId, fileId, 0, 0); + assertLocation(secondRowId, fileId, 0, 1); + } + + private void assertRange(RowIdRange range, long rowIdStart, long rowIdEnd, long fileId, + int rgId, int rgRowOffsetStart, int rgRowOffsetEnd) + { + Assertions.assertEquals(rowIdStart, range.getRowIdStart()); + Assertions.assertEquals(rowIdEnd, range.getRowIdEnd()); + Assertions.assertEquals(fileId, range.getFileId()); + Assertions.assertEquals(rgId, range.getRgId()); + Assertions.assertEquals(rgRowOffsetStart, range.getRgRowOffsetStart()); + Assertions.assertEquals(rgRowOffsetEnd, range.getRgRowOffsetEnd()); + } + + private void assertNoInvalidRanges(long fileId) throws Exception + { + Assertions.assertEquals(0, countInvalidRangesForFile(fileId)); + } + + private Connection openMainIndexConnection() throws Exception + { + String path = sqlitePath.endsWith("/") ? sqlitePath : sqlitePath + "/"; + return DriverManager.getConnection("jdbc:sqlite:" + path + tableId + ".main.index.db"); + } + + private void insertRange(RowIdRange range) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement("INSERT INTO row_id_ranges VALUES(?, ?, ?, ?, ?, ?)")) + { + pst.setLong(1, range.getRowIdStart()); + pst.setLong(2, range.getRowIdEnd()); + pst.setLong(3, range.getFileId()); + pst.setInt(4, range.getRgId()); + pst.setInt(5, range.getRgRowOffsetStart()); + pst.setInt(6, range.getRgRowOffsetEnd()); + pst.executeUpdate(); + } + } + + private void deleteExactRange(long rowIdStart, long rowIdEnd) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "DELETE FROM row_id_ranges WHERE row_id_start = ? AND row_id_end = ?")) + { + pst.setLong(1, rowIdStart); + pst.setLong(2, rowIdEnd); + pst.executeUpdate(); + } + } + + private void insertFlushMarker(long fileId, long entryCount, List ranges) throws Exception + { + insertFlushMarker(fileId, entryCount, ranges.size(), buildRangeHash(ranges)); + } + + private void insertFlushMarker(long fileId, long entryCount, long rangeCount, byte[] rangeHash) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "INSERT INTO row_id_range_flush_markers VALUES(?, ?, ?, ?, ?)")) + { + pst.setLong(1, fileId); + pst.setLong(2, entryCount); + pst.setLong(3, rangeCount); + pst.setBytes(4, rangeHash); + pst.setLong(5, System.currentTimeMillis()); + pst.executeUpdate(); + } + } + + private void deleteFlushMarker(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "DELETE FROM row_id_range_flush_markers WHERE file_id = ?")) + { + pst.setLong(1, fileId); + pst.executeUpdate(); + } + } + + private void createFailingFlushMarkerTrigger(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + Statement statement = connection.createStatement()) + { + statement.executeUpdate("DROP TRIGGER IF EXISTS fail_marker_insert"); + statement.executeUpdate("CREATE TRIGGER fail_marker_insert BEFORE INSERT ON row_id_range_flush_markers " + + "WHEN NEW.file_id = " + fileId + " BEGIN SELECT RAISE(ABORT, 'forced marker failure'); END"); + } + } + + private void dropFailingFlushMarkerTrigger() throws Exception + { + try (Connection connection = openMainIndexConnection(); + Statement statement = connection.createStatement()) + { + statement.executeUpdate("DROP TRIGGER IF EXISTS fail_marker_insert"); + } + } + + private void createFailingRangeInsertTrigger(long rowIdStart) throws Exception + { + try (Connection connection = openMainIndexConnection(); + Statement statement = connection.createStatement()) + { + statement.executeUpdate("DROP TRIGGER IF EXISTS fail_range_insert"); + statement.executeUpdate("CREATE TRIGGER fail_range_insert BEFORE INSERT ON row_id_ranges " + + "WHEN NEW.row_id_start = " + rowIdStart + " " + + "BEGIN SELECT RAISE(ABORT, 'forced range insert failure'); END"); + } + } + + private void dropFailingRangeInsertTrigger() throws Exception + { + try (Connection connection = openMainIndexConnection(); + Statement statement = connection.createStatement()) + { + statement.executeUpdate("DROP TRIGGER IF EXISTS fail_range_insert"); + } + } + + private List listRangesForFile(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "SELECT * FROM row_id_ranges WHERE file_id = ? ORDER BY row_id_start")) + { + pst.setLong(1, fileId); + List ranges = new ArrayList<>(); + try (ResultSet rs = pst.executeQuery()) + { + while (rs.next()) + { + ranges.add(new RowIdRange( + rs.getLong("row_id_start"), + rs.getLong("row_id_end"), + rs.getLong("file_id"), + rs.getInt("rg_id"), + rs.getInt("rg_row_offset_start"), + rs.getInt("rg_row_offset_end"))); + } + } + return ranges; + } + } + + private int countRangesForFile(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement("SELECT COUNT(*) FROM row_id_ranges WHERE file_id = ?")) + { + pst.setLong(1, fileId); + try (ResultSet rs = pst.executeQuery()) + { + Assertions.assertTrue(rs.next()); + return rs.getInt(1); + } + } + } + + private int countFlushMarkersForFile(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "SELECT COUNT(*) FROM row_id_range_flush_markers WHERE file_id = ?")) + { + pst.setLong(1, fileId); + try (ResultSet rs = pst.executeQuery()) + { + Assertions.assertTrue(rs.next()); + return rs.getInt(1); + } + } + } + + private int countExactRanges(long rowIdStart, long rowIdEnd) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "SELECT COUNT(*) FROM row_id_ranges WHERE row_id_start = ? AND row_id_end = ?")) + { + pst.setLong(1, rowIdStart); + pst.setLong(2, rowIdEnd); + try (ResultSet rs = pst.executeQuery()) + { + Assertions.assertTrue(rs.next()); + return rs.getInt(1); + } + } + } + + private int countInvalidRangesForFile(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "SELECT COUNT(*) FROM row_id_ranges WHERE file_id = ? AND " + + "(row_id_end <= row_id_start OR " + + "(row_id_end - row_id_start) != (rg_row_offset_end - rg_row_offset_start))")) + { + pst.setLong(1, fileId); + try (ResultSet rs = pst.executeQuery()) + { + Assertions.assertTrue(rs.next()); + return rs.getInt(1); + } + } + } + + private byte[] buildRangeHash(List ranges) throws Exception + { + MessageDigest digest = MessageDigest.getInstance("SHA-256"); + for (RowIdRange range : ranges) + { + updateLong(digest, range.getRowIdStart()); + updateLong(digest, range.getRowIdEnd()); + updateLong(digest, range.getFileId()); + updateInt(digest, range.getRgId()); + updateInt(digest, range.getRgRowOffsetStart()); + updateInt(digest, range.getRgRowOffsetEnd()); + } + return digest.digest(); + } + + private static void updateLong(MessageDigest digest, long value) + { + for (int shift = 56; shift >= 0; shift -= 8) + { + digest.update((byte) (value >>> shift)); + } + } + + private static void updateInt(MessageDigest digest, int value) + { + for (int shift = 24; shift >= 0; shift -= 8) + { + digest.update((byte) (value >>> shift)); + } + } + + private long nanosToMillis(long nanos) + { + return nanos / 1_000_000L; + } +} diff --git a/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexBenchmark.java b/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexBenchmark.java new file mode 100644 index 0000000000..d4b07de060 --- /dev/null +++ b/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexBenchmark.java @@ -0,0 +1,462 @@ +/* + * Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.index.main.sqlite; + +import io.pixelsdb.pixels.common.exception.MainIndexException; +import io.pixelsdb.pixels.common.index.MainIndex; +import io.pixelsdb.pixels.common.index.MainIndexFactory; +import io.pixelsdb.pixels.common.index.RowIdRange; +import io.pixelsdb.pixels.common.utils.ConfigFactory; +import io.pixelsdb.pixels.index.IndexProto; +import org.apache.commons.io.FileUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.security.MessageDigest; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +@Tag("benchmark") +public class TestSqliteMainIndexBenchmark +{ + private static final String ENABLE_PROPERTY = "pixels.sqlite.main.index.benchmark"; + private static final long NOT_APPLICABLE = -1L; + private static final int CONTIGUOUS_ROWS = Integer.getInteger( + "pixels.sqlite.main.index.benchmark.contiguousRows", 1_000_000); + private static final int FRAGMENTED_ROWS = Integer.getInteger( + "pixels.sqlite.main.index.benchmark.fragmentedRows", 100_000); + private static long nextTableId = 900_000L; + + private String sqlitePath; + private long tableId; + private MainIndex mainIndex; + + @BeforeEach + public void setUp() + { + Assumptions.assumeTrue(Boolean.getBoolean(ENABLE_PROPERTY), + "Set -D" + ENABLE_PROPERTY + "=true to run manual sqlite main-index benchmarks."); + } + + @AfterEach + public void tearDown() throws Exception + { + closeAndRemoveIndex(); + } + + @Test + public void benchmarkPutGetAndFlushPaths() throws Exception + { + System.out.println(); + printBenchmarkParameters(); + List results = new ArrayList<>(); + results.add(benchmarkHotPutGetPath()); + results.add(benchmarkContiguousFlush()); + results.add(benchmarkFragmentedFlush()); + results.add(benchmarkMarkerHitRetry()); + printBenchmarkSummary(results); + } + + private BenchmarkResult benchmarkHotPutGetPath() throws Exception + { + openFreshIndex(); + long fileId = 1L; + long rowIdBase = 1_000_000_000L; + + long putNs = elapsedNanos(() -> putContiguousEntries(CONTIGUOUS_ROWS, fileId, rowIdBase)); + long getNs = elapsedNanos(() -> getContiguousEntries(CONTIGUOUS_ROWS, rowIdBase)); + long cleanupFlushNs = elapsedNanos(() -> Assertions.assertTrue(mainIndex.flushCache(fileId))); + long ranges = countRangesForFile(fileId); + long markers = countFlushMarkersForFile(fileId); + + closeAndRemoveIndex(); + return new BenchmarkResult("hot put/get path", "contiguous, pre-flush get", + CONTIGUOUS_ROWS, ranges, markers, putNs, cleanupFlushNs, + NOT_APPLICABLE, NOT_APPLICABLE, getNs); + } + + private BenchmarkResult benchmarkContiguousFlush() throws Exception + { + openFreshIndex(); + long fileId = 2L; + long rowIdBase = 2_000_000_000L; + + long putNs = elapsedNanos(() -> putContiguousEntries(CONTIGUOUS_ROWS, fileId, rowIdBase)); + long flushNs = elapsedNanos(() -> Assertions.assertTrue(mainIndex.flushCache(fileId))); + long getNs = elapsedNanos(() -> getContiguousEntries(CONTIGUOUS_ROWS, rowIdBase)); + long ranges = countRangesForFile(fileId); + long markers = countFlushMarkersForFile(fileId); + + Assertions.assertEquals(1L, ranges); + Assertions.assertEquals(1L, markers); + closeAndRemoveIndex(); + return new BenchmarkResult("contiguous first flush", "contiguous rows -> 1 range", + CONTIGUOUS_ROWS, ranges, markers, putNs, flushNs, + NOT_APPLICABLE, NOT_APPLICABLE, getNs); + } + + private BenchmarkResult benchmarkFragmentedFlush() throws Exception + { + openFreshIndex(); + long fileId = 3L; + long rowIdBase = 3_000_000_000L; + + long putNs = elapsedNanos(() -> putFragmentedEntries(FRAGMENTED_ROWS, fileId, rowIdBase)); + long flushNs = elapsedNanos(() -> Assertions.assertTrue(mainIndex.flushCache(fileId))); + long getNs = elapsedNanos(() -> getFragmentedEntries(FRAGMENTED_ROWS, rowIdBase)); + long ranges = countRangesForFile(fileId); + long markers = countFlushMarkersForFile(fileId); + + Assertions.assertEquals(FRAGMENTED_ROWS, ranges); + Assertions.assertEquals(1L, markers); + closeAndRemoveIndex(); + return new BenchmarkResult("fragmented first flush", "1-row gaps -> many ranges", + FRAGMENTED_ROWS, ranges, markers, putNs, flushNs, + NOT_APPLICABLE, NOT_APPLICABLE, getNs); + } + + private BenchmarkResult benchmarkMarkerHitRetry() throws Exception + { + openFreshIndex(); + long fileId = 4L; + long rowIdBase = 4_000_000_000L; + List ranges = buildFragmentedRanges(FRAGMENTED_ROWS, fileId, rowIdBase); + + insertRangesAndMarker(fileId, FRAGMENTED_ROWS, ranges); + long putNs = elapsedNanos(() -> putFragmentedEntries(FRAGMENTED_ROWS, fileId, rowIdBase)); + long markerRetryNs = elapsedNanos(() -> Assertions.assertTrue(mainIndex.flushCache(fileId))); + long emptyRetryNs = elapsedNanos(() -> Assertions.assertTrue(mainIndex.flushCache(fileId))); + long getNs = elapsedNanos(() -> getFragmentedEntries(FRAGMENTED_ROWS, rowIdBase)); + long storedRanges = countRangesForFile(fileId); + long markers = countFlushMarkersForFile(fileId); + + Assertions.assertEquals(FRAGMENTED_ROWS, storedRanges); + Assertions.assertEquals(1L, markers); + closeAndRemoveIndex(); + return new BenchmarkResult("marker-hit retry flush", "matching marker already durable", + FRAGMENTED_ROWS, storedRanges, markers, putNs, NOT_APPLICABLE, + markerRetryNs, emptyRetryNs, getNs); + } + + private void openFreshIndex() throws Exception + { + closeAndRemoveIndex(); + this.tableId = nextTableId++; + this.sqlitePath = ConfigFactory.Instance().getProperty("index.sqlite.path"); + try + { + FileUtils.forceMkdir(new File(sqlitePath)); + } + catch (IOException e) + { + throw new MainIndexException("Failed to create SQLite benchmark directory", e); + } + this.mainIndex = MainIndexFactory.Instance().getMainIndex(tableId); + } + + private void closeAndRemoveIndex() throws Exception + { + if (this.mainIndex != null) + { + MainIndexFactory.Instance().closeIndex(this.tableId, true); + this.mainIndex = null; + } + if (this.sqlitePath != null) + { + FileUtils.deleteDirectory(new File(sqlitePath)); + } + } + + private void putContiguousEntries(int rowCount, long fileId, long rowIdBase) + { + IndexProto.RowLocation.Builder locationBuilder = IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(0); + for (int i = 0; i < rowCount; i++) + { + Assertions.assertTrue(mainIndex.putEntry(rowIdBase + i, locationBuilder.setRgRowOffset(i).build())); + } + } + + private void putFragmentedEntries(int rowCount, long fileId, long rowIdBase) + { + IndexProto.RowLocation.Builder locationBuilder = IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(0); + for (int i = 0; i < rowCount; i++) + { + Assertions.assertTrue(mainIndex.putEntry(rowIdBase + i * 2L, locationBuilder.setRgRowOffset(i).build())); + } + } + + private void getContiguousEntries(int rowCount, long rowIdBase) throws MainIndexException + { + for (int i = 0; i < rowCount; i++) + { + Assertions.assertNotNull(mainIndex.getLocation(rowIdBase + i)); + } + } + + private void getFragmentedEntries(int rowCount, long rowIdBase) throws MainIndexException + { + for (int i = 0; i < rowCount; i++) + { + Assertions.assertNotNull(mainIndex.getLocation(rowIdBase + i * 2L)); + } + } + + private List buildFragmentedRanges(int rowCount, long fileId, long rowIdBase) + { + List ranges = new ArrayList<>(rowCount); + for (int i = 0; i < rowCount; i++) + { + long rowId = rowIdBase + i * 2L; + ranges.add(new RowIdRange(rowId, rowId + 1, fileId, 0, i, i + 1)); + } + return ranges; + } + + private void insertRangesAndMarker(long fileId, long entryCount, List ranges) throws Exception + { + try (Connection connection = openMainIndexConnection()) + { + boolean originalAutoCommit = connection.getAutoCommit(); + connection.setAutoCommit(false); + try + { + try (PreparedStatement pst = connection.prepareStatement("INSERT INTO row_id_ranges VALUES(?, ?, ?, ?, ?, ?)")) + { + for (RowIdRange range : ranges) + { + pst.setLong(1, range.getRowIdStart()); + pst.setLong(2, range.getRowIdEnd()); + pst.setLong(3, range.getFileId()); + pst.setInt(4, range.getRgId()); + pst.setInt(5, range.getRgRowOffsetStart()); + pst.setInt(6, range.getRgRowOffsetEnd()); + pst.addBatch(); + } + pst.executeBatch(); + } + try (PreparedStatement pst = connection.prepareStatement( + "INSERT INTO row_id_range_flush_markers VALUES(?, ?, ?, ?, ?)")) + { + pst.setLong(1, fileId); + pst.setLong(2, entryCount); + pst.setLong(3, ranges.size()); + pst.setBytes(4, buildRangeHash(ranges)); + pst.setLong(5, System.currentTimeMillis()); + pst.executeUpdate(); + } + connection.commit(); + } + catch (Exception e) + { + connection.rollback(); + throw e; + } + finally + { + connection.setAutoCommit(originalAutoCommit); + } + } + } + + private Connection openMainIndexConnection() throws Exception + { + String path = sqlitePath.endsWith("/") ? sqlitePath : sqlitePath + "/"; + return DriverManager.getConnection("jdbc:sqlite:" + path + tableId + ".main.index.db"); + } + + private long countRangesForFile(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement("SELECT COUNT(*) FROM row_id_ranges WHERE file_id = ?")) + { + pst.setLong(1, fileId); + try (ResultSet rs = pst.executeQuery()) + { + Assertions.assertTrue(rs.next()); + return rs.getLong(1); + } + } + } + + private long countFlushMarkersForFile(long fileId) throws Exception + { + try (Connection connection = openMainIndexConnection(); + PreparedStatement pst = connection.prepareStatement( + "SELECT COUNT(*) FROM row_id_range_flush_markers WHERE file_id = ?")) + { + pst.setLong(1, fileId); + try (ResultSet rs = pst.executeQuery()) + { + Assertions.assertTrue(rs.next()); + return rs.getLong(1); + } + } + } + + private byte[] buildRangeHash(List ranges) throws Exception + { + MessageDigest digest = MessageDigest.getInstance("SHA-256"); + for (RowIdRange range : ranges) + { + updateLong(digest, range.getRowIdStart()); + updateLong(digest, range.getRowIdEnd()); + updateLong(digest, range.getFileId()); + updateInt(digest, range.getRgId()); + updateInt(digest, range.getRgRowOffsetStart()); + updateInt(digest, range.getRgRowOffsetEnd()); + } + return digest.digest(); + } + + private static void updateLong(MessageDigest digest, long value) + { + for (int shift = 56; shift >= 0; shift -= 8) + { + digest.update((byte) (value >>> shift)); + } + } + + private static void updateInt(MessageDigest digest, int value) + { + for (int shift = 24; shift >= 0; shift -= 8) + { + digest.update((byte) (value >>> shift)); + } + } + + private long elapsedNanos(ThrowingRunnable runnable) throws Exception + { + long start = System.nanoTime(); + runnable.run(); + return System.nanoTime() - start; + } + + private void printBenchmarkParameters() + { + System.out.println("SQLite MainIndex benchmark parameters"); + System.out.println(" -D" + ENABLE_PROPERTY + "=" + Boolean.getBoolean(ENABLE_PROPERTY)); + System.out.println(" -Dpixels.sqlite.main.index.benchmark.contiguousRows=" + CONTIGUOUS_ROWS); + System.out.println(" -Dpixels.sqlite.main.index.benchmark.fragmentedRows=" + FRAGMENTED_ROWS); + System.out.println(" index.sqlite.path=" + ConfigFactory.Instance().getProperty("index.sqlite.path")); + System.out.println(" java.version=" + System.getProperty("java.version")); + System.out.println(" os.name=" + System.getProperty("os.name")); + System.out.println(" os.arch=" + System.getProperty("os.arch")); + } + + private void printBenchmarkSummary(List results) + { + System.out.println(); + System.out.println("SQLite MainIndex benchmark summary"); + System.out.println("rows = logical MainIndex entries; ranges = persisted row_id_ranges."); + System.out.println("markerRetry = retry when a matching per-file durable marker already exists."); + System.out.println("emptyRetry = immediate second flush after marker retry discarded the buffer."); + System.out.println(String.format("%-27s %-31s %12s %10s %7s %10s %13s %10s %16s %15s %13s %10s %13s", + "workload", "shape", "rows", "ranges", "markers", "put(ms)", "put rows/s", + "flush(ms)", "flush ranges/s", "markerRetry(ms)", "emptyRetry(ms)", "get(ms)", "get rows/s")); + for (BenchmarkResult result : results) + { + System.out.println(String.format("%-27s %-31s %12s %10s %7s %10s %13s %10s %16s %15s %13s %10s %13s", + result.name, + result.shape, + formatLong(result.rows), + formatLong(result.ranges), + formatLong(result.markers), + formatMillis(result.putNs), + formatRate(result.rows, result.putNs), + formatMillis(result.flushNs), + formatRate(result.ranges, result.flushNs), + formatMillis(result.markerRetryNs), + formatMillis(result.emptyRetryNs), + formatMillis(result.getNs), + formatRate(result.rows, result.getNs))); + } + } + + private String formatLong(long value) + { + return String.format(Locale.US, "%,d", value); + } + + private String formatMillis(long nanos) + { + if (nanos < 0) + { + return "-"; + } + return String.format(Locale.US, "%,.3f", nanos / 1_000_000.0D); + } + + private String formatRate(long count, long nanos) + { + if (nanos <= 0) + { + return "-"; + } + double rate = count * 1_000_000_000.0D / nanos; + return String.format(Locale.US, "%,.0f", rate); + } + + private static final class BenchmarkResult + { + private final String name; + private final String shape; + private final long rows; + private final long ranges; + private final long markers; + private final long putNs; + private final long flushNs; + private final long markerRetryNs; + private final long emptyRetryNs; + private final long getNs; + + private BenchmarkResult(String name, String shape, long rows, long ranges, long markers, + long putNs, long flushNs, long markerRetryNs, long emptyRetryNs, long getNs) + { + this.name = name; + this.shape = shape; + this.rows = rows; + this.ranges = ranges; + this.markers = markers; + this.putNs = putNs; + this.flushNs = flushNs; + this.markerRetryNs = markerRetryNs; + this.emptyRetryNs = emptyRetryNs; + this.getNs = getNs; + } + } + + private interface ThrowingRunnable + { + void run() throws Exception; + } +} diff --git a/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexQuery.java b/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexQuery.java index 7847fcd34c..df5bbaaea0 100644 --- a/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexQuery.java +++ b/pixels-index/pixels-index-main-sqlite/src/test/java/io/pixelsdb/pixels/index/main/sqlite/TestSqliteMainIndexQuery.java @@ -19,8 +19,6 @@ */ package io.pixelsdb.pixels.index.main.sqlite; -import io.pixelsdb.pixels.common.exception.MainIndexException; -import io.pixelsdb.pixels.common.exception.RowIdException; import io.pixelsdb.pixels.common.index.MainIndex; import io.pixelsdb.pixels.common.index.MainIndexFactory; import io.pixelsdb.pixels.common.index.RowIdRange; @@ -39,63 +37,170 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; + public class TestSqliteMainIndexQuery { + private static long nextTableId = 3035L; + MainIndex mainIndex; - Long tableId =3035L; + long tableId; + String sqlitePath; Connection connection; + @BeforeEach public void setUp() throws Exception { - String sqlitePath = ConfigFactory.Instance().getProperty("index.sqlite.path"); - if (!sqlitePath.endsWith("/")) + tableId = nextTableId++; + sqlitePath = ConfigFactory.Instance().getProperty("index.sqlite.path"); + try { - sqlitePath += "/"; + FileUtils.forceMkdir(new File(sqlitePath)); } + catch (IOException e) + { + System.err.println("Failed to create SQLite test directory: " + e.getMessage()); + } + mainIndex = MainIndexFactory.Instance().getMainIndex(tableId); - connection = DriverManager.getConnection("jdbc:sqlite:" + sqlitePath + tableId + ".main.index.db"); + String path = sqlitePath.endsWith("/") ? sqlitePath : sqlitePath + "/"; + connection = DriverManager.getConnection("jdbc:sqlite:" + path + tableId + ".main.index.db"); + } + + @AfterEach + public void tearDown() throws Exception + { + if (connection != null) + { + connection.close(); + } + MainIndexFactory.Instance().closeIndex(tableId, true); + try + { + FileUtils.deleteDirectory(new File(sqlitePath)); + } + catch (IOException e) + { + System.err.println("Failed to clean up SQLite test directory: " + e.getMessage()); + } + } + + @Test + public void testQueryRowRangesFromCommittedFlush() throws Exception + { + putMainIndexEntry(11000L, 51L, 0, 0); + putMainIndexEntry(11001L, 51L, 0, 1); + putMainIndexEntry(11010L, 51L, 1, 0); + Assertions.assertTrue(mainIndex.flushCache(51L)); + + List rowIdRanges = queryRowRanges(); + Assertions.assertEquals(2, rowIdRanges.size()); + assertRange(rowIdRanges.get(0), 11000L, 11002L, 51L, 0, 0, 2); + assertRange(rowIdRanges.get(1), 11010L, 11011L, 51L, 1, 0, 1); + } + + @Test + public void testQueryRowRangesFromOutOfOrderBatchFlushesMultipleFiles() throws Exception + { + assertAllTrue(mainIndex.putEntries(Arrays.asList( + primaryEntry(11102L, 52L, 0, 2), + primaryEntry(11201L, 53L, 0, 1), + primaryEntry(11100L, 52L, 0, 0), + primaryEntry(11200L, 53L, 0, 0), + primaryEntry(11101L, 52L, 0, 1), + primaryEntry(11202L, 53L, 0, 2)))); + + Assertions.assertTrue(mainIndex.flushCache(53L)); + Assertions.assertTrue(mainIndex.flushCache(52L)); + + List rowIdRanges = queryRowRanges(); + Assertions.assertEquals(2, rowIdRanges.size()); + assertRange(rowIdRanges.get(0), 11100L, 11103L, 52L, 0, 0, 3); + assertRange(rowIdRanges.get(1), 11200L, 11203L, 53L, 0, 0, 3); } @Test - public void testQueryRowRanges() throws Exception + public void testQueryRowRangesReflectDeleteSplitForRecoveryCleanup() throws Exception { - String query = "SELECT * FROM row_id_ranges order by row_id_start"; - long fileid = 0; - try (PreparedStatement pst = this.connection.prepareStatement(query)) + putContiguousEntries(11300L, 11306L, 54L, 0, 0); + Assertions.assertTrue(mainIndex.flushCache(54L)); + + Assertions.assertTrue(mainIndex.deleteRowIdRange(new RowIdRange(11302L, 11305L, 54L, 0, 2, 5))); + + List rowIdRanges = queryRowRanges(); + Assertions.assertEquals(2, rowIdRanges.size()); + assertRange(rowIdRanges.get(0), 11300L, 11302L, 54L, 0, 0, 2); + assertRange(rowIdRanges.get(1), 11305L, 11306L, 54L, 0, 5, 6); + } + + private void putMainIndexEntry(long rowId, long fileId, int rgId, int rgRowOffset) + { + Assertions.assertTrue(mainIndex.putEntry(rowId, IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(rgId).setRgRowOffset(rgRowOffset).build())); + } + + private void putContiguousEntries(long rowIdStart, long rowIdEnd, long fileId, int rgId, int rgRowOffsetStart) + { + int offset = rgRowOffsetStart; + for (long rowId = rowIdStart; rowId < rowIdEnd; rowId++) { -// pst.setLong(1, fileid); - try (ResultSet rs = pst.executeQuery()) + putMainIndexEntry(rowId, fileId, rgId, offset++); + } + } + + private IndexProto.PrimaryIndexEntry primaryEntry(long rowId, long fileId, int rgId, int rgRowOffset) + { + return IndexProto.PrimaryIndexEntry.newBuilder() + .setRowId(rowId) + .setRowLocation(IndexProto.RowLocation.newBuilder() + .setFileId(fileId).setRgId(rgId).setRgRowOffset(rgRowOffset).build()) + .build(); + } + + private void assertAllTrue(List results) + { + for (Boolean result : results) + { + Assertions.assertTrue(result); + } + } + + private List queryRowRanges() throws Exception + { + String query = "SELECT * FROM row_id_ranges ORDER BY row_id_start"; + List ranges = new ArrayList<>(); + try (PreparedStatement pst = this.connection.prepareStatement(query); + ResultSet rs = pst.executeQuery()) + { + while (rs.next()) { - while (rs.next()) - { - long rowIdStart = rs.getLong("row_id_start"); - long rowIdEnd = rs.getLong("row_id_end"); - long fileId = rs.getLong("file_id"); - int rgId = rs.getInt("rg_id"); - int rgRowOffsetStart = rs.getInt("rg_row_offset_start"); - int rgRowOffsetEnd = rs.getInt("rg_row_offset_end"); - if (rowIdEnd - rowIdStart != rgRowOffsetEnd - rgRowOffsetStart) - { - throw new RowIdException("The width of row id range (" + rowIdStart + ", " + - rgRowOffsetEnd + ") does not match the width of row group row offset range (" + - rgRowOffsetStart + ", " + rgRowOffsetEnd + ")"); - } - System.out.println( - "rowIdStart=" + rowIdStart + - ", rowIdEnd=" + rowIdEnd + - ", fileId=" + fileId + - ", rgId=" + rgId + - ", rgRowOffsetStart=" + rgRowOffsetStart + - ", rgRowOffsetEnd=" + rgRowOffsetEnd - ); - } + long rowIdStart = rs.getLong("row_id_start"); + long rowIdEnd = rs.getLong("row_id_end"); + int rgRowOffsetStart = rs.getInt("rg_row_offset_start"); + int rgRowOffsetEnd = rs.getInt("rg_row_offset_end"); + Assertions.assertEquals(rowIdEnd - rowIdStart, rgRowOffsetEnd - rgRowOffsetStart); + + ranges.add(new RowIdRange( + rowIdStart, + rowIdEnd, + rs.getLong("file_id"), + rs.getInt("rg_id"), + rgRowOffsetStart, + rgRowOffsetEnd)); } } + return ranges; + } + private void assertRange(RowIdRange range, long rowIdStart, long rowIdEnd, long fileId, + int rgId, int rgRowOffsetStart, int rgRowOffsetEnd) + { + Assertions.assertEquals(rowIdStart, range.getRowIdStart()); + Assertions.assertEquals(rowIdEnd, range.getRowIdEnd()); + Assertions.assertEquals(fileId, range.getFileId()); + Assertions.assertEquals(rgId, range.getRgId()); + Assertions.assertEquals(rgRowOffsetStart, range.getRgRowOffsetStart()); + Assertions.assertEquals(rgRowOffsetEnd, range.getRgRowOffsetEnd()); } } diff --git a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/FileWriterManager.java b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/FileWriterManager.java index f470cb728e..7c463e0814 100644 --- a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/FileWriterManager.java +++ b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/FileWriterManager.java @@ -50,6 +50,9 @@ public class FileWriterManager private final long firstBlockId; private long lastBlockId = -1; private final int virtualNodeId; + // Initialized by PixelsWriteBuffer's single-threaded file publisher. + private CompletableFuture physicalCloseFuture; + /** * Creating pixelsWriter by passing in parameters avoids the need to read * the configuration file for each call. @@ -84,10 +87,13 @@ public FileWriterManager(long tableId, TypeDescription schema, MetadataService metadataService = MetadataService.Instance(); file = new File(); this.file.setName(targetFileName); - this.file.setType(File.Type.TEMPORARY); + this.file.setType(File.Type.TEMPORARY_INGEST); this.file.setNumRowGroup(1); this.file.setPathId(targetOrderedDirPath.getId()); - metadataService.addFiles(Collections.singletonList(file)); + if (!metadataService.addFiles(Collections.singletonList(file))) + { + throw new MetadataException("failed to add metadata for ingest file " + targetFilePath); + } this.file.setId(metadataService.getFileId(targetFilePath)); } catch (MetadataException e) { @@ -122,6 +128,17 @@ public FileWriterManager(long tableId, TypeDescription schema, } } + FileWriterManager(long tableId, PixelsWriter writer, File file, + long firstBlockId, long lastBlockId, int virtualNodeId) + { + this.tableId = tableId; + this.writer = writer; + this.file = file; + this.firstBlockId = firstBlockId; + this.lastBlockId = lastBlockId; + this.virtualNodeId = virtualNodeId; + } + public long getFileId() { return this.file.getId(); @@ -142,6 +159,19 @@ public long getLastBlockId() return this.lastBlockId; } + File getFileSnapshot() + { + File snapshot = new File(); + snapshot.setId(this.file.getId()); + snapshot.setName(this.file.getName()); + snapshot.setType(this.file.getType()); + snapshot.setNumRowGroup(this.file.getNumRowGroup()); + snapshot.setMinRowId(this.file.getMinRowId()); + snapshot.setMaxRowId(this.file.getMaxRowId()); + snapshot.setPathId(this.file.getPathId()); + return snapshot; + } + public void addRowBatch(VectorizedRowBatch rowBatch) throws RetinaException { try @@ -155,13 +185,22 @@ public void addRowBatch(VectorizedRowBatch rowBatch) throws RetinaException /** * Create a background thread to write the block of data stored in shared storage to a file. + * Metadata publication is handled by {@link PixelsWriteBuffer} after the + * physical close and index flush barrier both complete. */ - public CompletableFuture finish() + CompletableFuture finish() { + if (physicalCloseFuture != null) + { + return physicalCloseFuture; + } + CompletableFuture future = new CompletableFuture<>(); + physicalCloseFuture = future; new Thread(() -> { - try { + try + { for (long blockId = firstBlockId; blockId <= lastBlockId; ++blockId) { ObjectStorageManager objectStorageManager = ObjectStorageManager.Instance(); @@ -173,18 +212,12 @@ public CompletableFuture finish() this.writer.addRowBatch(VectorizedRowBatch.deserialize(data)); } this.writer.close(); - - // Update the file's type. - this.file.setType(File.Type.REGULAR); - MetadataService metadataService = MetadataService.Instance(); - metadataService.updateFile(this.file); - future.complete(null); } catch (Exception e) { future.completeExceptionally(e); } - }).start(); + }, "pixels-retina-file-finish-" + this.file.getId()).start(); return future; } diff --git a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/PixelsWriteBuffer.java b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/PixelsWriteBuffer.java index 0b9b47c80f..1880a2be63 100644 --- a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/PixelsWriteBuffer.java +++ b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/PixelsWriteBuffer.java @@ -26,6 +26,7 @@ import io.pixelsdb.pixels.common.index.service.IndexServiceProvider; import io.pixelsdb.pixels.common.index.RowIdAllocator; import io.pixelsdb.pixels.common.metadata.MetadataService; +import io.pixelsdb.pixels.common.metadata.domain.File; import io.pixelsdb.pixels.common.metadata.domain.Path; import io.pixelsdb.pixels.common.metadata.domain.SinglePointIndex; import io.pixelsdb.pixels.common.physical.Storage; @@ -43,7 +44,6 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; -import java.util.stream.LongStream; import static com.google.common.base.Preconditions.checkArgument; @@ -102,6 +102,7 @@ public class PixelsWriteBuffer // backend flush thread private final ExecutorService flushObjectExecutor; + // Single-threaded by design: it serializes file publishing and FileWriterManager physical close initialization. private final ScheduledExecutorService flushFileExecutor; private ScheduledFuture flushFileFuture; @@ -160,6 +161,7 @@ public PixelsWriteBuffer(long tableId, TypeDescription schema, Path targetOrdere this.objectEntries = new ArrayList<>(); this.flushObjectExecutor = Executors.newFixedThreadPool(Integer.parseInt(configFactory.getProperty("retina.buffer.object.flush.threads"))); + // Keep file publishing serialized: physical close, index flush, metadata publish, and cleanup are ordered per stream. this.flushFileExecutor = Executors.newSingleThreadScheduledExecutor(); this.fileWriterManagers = new ConcurrentLinkedQueue<>(); @@ -236,7 +238,7 @@ public long addRow(byte[][] values, long timestamp, IndexProto.RowLocation.Build { throw new RetinaException("Expect rgRowOffset >= 0, get " + rgRowOffset); } - builder.setFileId(activeMemTable.getFileId()) + builder.setFileId(currentMemTable.getFileId()) .setRgId(0) .setRgRowOffset(rgRowOffset); return rowId; @@ -368,6 +370,57 @@ public SuperVersion getCurrentVersion() } } + private void publishFinishedFile(FileWriterManager fileWriterManager) throws RetinaException + { + try + { + fileWriterManager.finish().get(); + + if (this.index == null) + { + this.index = MetadataService.Instance().getPrimaryIndex(tableId); + if (this.index == null) + { + throw new RetinaException("Primary index not found for table " + tableId); + } + } + + boolean flushed = IndexServiceProvider.getService(IndexServiceProvider.ServiceMode.local) + .flushIndexEntriesOfFile( + tableId, index.getId(), fileWriterManager.getFileId(), true, indexOption); + if (!flushed) + { + throw new RetinaException("Failed to flush main index for ingest file " + + fileWriterManager.getFileId()); + } + + File regularFile = fileWriterManager.getFileSnapshot(); + regularFile.setType(File.Type.REGULAR); + if (!MetadataService.Instance().updateFile(regularFile)) + { + throw new RetinaException("Failed to publish ingest file " + + fileWriterManager.getFileId() + " as REGULAR"); + } + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new RetinaException("Interrupted while publishing ingest file " + + fileWriterManager.getFileId(), e); + } catch (ExecutionException e) + { + throw new RetinaException("Failed to physically close ingest file " + + fileWriterManager.getFileId(), e.getCause()); + } catch (IndexException e) + { + throw new RetinaException("Failed to flush main index for ingest file " + + fileWriterManager.getFileId(), e); + } catch (MetadataException e) + { + throw new RetinaException("Failed to publish ingest file " + + fileWriterManager.getFileId() + " as REGULAR", e); + } + } + /** * Determine whether the last data block managed by fileWriterManager has * been written to Object. If it has been written, execute the file write @@ -378,47 +431,42 @@ private void startFlushObjectToFileScheduler(long intervalSeconds) this.flushFileFuture = this.flushFileExecutor.scheduleWithFixedDelay(() -> { try { - if(index == null) - { - try - { - index = MetadataService.Instance().getPrimaryIndex(tableId); - } catch (MetadataException ignored) - { - logger.warn("There isn't primary index on table {}", tableId); - } - } - Iterator iterator = this.fileWriterManagers.iterator(); while (iterator.hasNext()) { FileWriterManager fileWriterManager = iterator.next(); if (fileWriterManager.getLastBlockId() <= this.continuousFlushedId.get()) { - CompletableFuture finished = fileWriterManager.finish(); - iterator.remove(); - - // update super version + publishFinishedFile(fileWriterManager); + + /* + * Detach only the current write-buffer view while holding versionLock. + * Physical object deletion stays outside the lock so storage I/O does + * not run under the SuperVersion write lock. + */ + List toRemove; this.versionLock.writeLock().lock(); - Set idsToRemove = LongStream.rangeClosed(fileWriterManager.getFirstBlockId(), - fileWriterManager.getLastBlockId()).boxed().collect(Collectors.toSet()); - List toRemove = this.objectEntries.stream() - .filter(objectEntry -> idsToRemove.contains(objectEntry.getId())) - .collect(Collectors.toList()); - - this.objectEntries.removeAll(toRemove); - - SuperVersion oldVersion = this.currentVersion; - this.currentVersion = new SuperVersion(this.activeMemTable, this.immutableMemTables, this.objectEntries); - oldVersion.unref(); - this.versionLock.writeLock().unlock(); - - finished.get(); - if(index != null) + try + { + long firstBlockId = fileWriterManager.getFirstBlockId(); + long lastBlockId = fileWriterManager.getLastBlockId(); + toRemove = this.objectEntries.stream() + .filter(objectEntry -> + objectEntry.getId() >= firstBlockId && objectEntry.getId() <= lastBlockId) + .collect(Collectors.toList()); + + this.objectEntries.removeAll(toRemove); + + SuperVersion oldVersion = this.currentVersion; + this.currentVersion = new SuperVersion( + this.activeMemTable, this.immutableMemTables, this.objectEntries); + oldVersion.unref(); + } finally { - IndexServiceProvider.getService(IndexServiceProvider.ServiceMode.local) - .flushIndexEntriesOfFile(tableId, index.getId(), fileWriterManager.getFileId(), true, indexOption); + this.versionLock.writeLock().unlock(); } + + iterator.remove(); for (ObjectEntry objectEntry : toRemove) { if (objectEntry.unref()) @@ -473,7 +521,7 @@ public void close() throws RetinaException } SuperVersion sv = getCurrentVersion(); - List> futures = new ArrayList<>(); + boolean completed = false; try { long maxObjectKey = this.continuousFlushedId.get(); @@ -492,7 +540,7 @@ public void close() throws RetinaException iterator.remove(); } } - this.currentFileWriterManager.finish().get(); + publishFinishedFile(this.currentFileWriterManager); // process the remaining fileWriterManager for (FileWriterManager fileWriterManager : this.fileWriterManagers) @@ -503,7 +551,7 @@ public void close() throws RetinaException // all written to object if (lastBlockId <= maxObjectKey) { - futures.add(fileWriterManager.finish()); + publishFinishedFile(fileWriterManager); } else { // process elements in immutable memTable @@ -521,18 +569,10 @@ public void close() throws RetinaException // elements in object will be processed in finish() later fileWriterManager.setLastBlockId(maxObjectKey); - futures.add(fileWriterManager.finish()); + publishFinishedFile(fileWriterManager); } } - - CompletableFuture all = CompletableFuture.allOf( - futures.toArray(new CompletableFuture[0]) - ); - all.get(15, TimeUnit.SECONDS); - } catch (InterruptedException e) - { - Thread.currentThread().interrupt(); - throw new RetinaException("Data persistence was interrupted during close", e); + completed = true; } catch (Exception e) { throw new RetinaException("Failed to persist data during close operation. Data may be lost", e); @@ -548,8 +588,10 @@ public void close() throws RetinaException for (ObjectEntry objectEntry : sv.getObjectEntries()) { - objectEntry.unref(); - this.objectStorageManager.delete(this.tableId, virtualNodeId, objectEntry.getId()); + if (objectEntry.unref() && completed) + { + this.objectStorageManager.delete(this.tableId, virtualNodeId, objectEntry.getId()); + } } } } diff --git a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/RetinaResourceManager.java b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/RetinaResourceManager.java index 2eeb97f015..900d907b63 100644 --- a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/RetinaResourceManager.java +++ b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/RetinaResourceManager.java @@ -48,6 +48,7 @@ import java.nio.file.Paths; import java.util.*; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -66,7 +67,7 @@ public class RetinaResourceManager // GC related fields private final ScheduledExecutorService gcExecutor; - private final boolean storageGcEnabled; + private final AtomicBoolean gcScheduled; private final StorageGarbageCollector storageGarbageCollector; // Checkpoint related fields @@ -150,37 +151,20 @@ private RetinaResourceManager() return t; }); - ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(r -> { + this.gcExecutor = Executors.newSingleThreadScheduledExecutor(r -> { Thread t = new Thread(r, "retina-gc-thread"); t.setDaemon(true); return t; }); - try - { - long interval = Long.parseLong(config.getProperty("retina.gc.interval")); - if (interval > 0) - { - executor.scheduleAtFixedRate( - this::runGC, - interval, - interval, - TimeUnit.SECONDS - ); - } - } catch (Exception e) - { - logger.error("Failed to start retina background gc", e); - } - this.gcExecutor = executor; + this.gcScheduled = new AtomicBoolean(false); totalVirtualNodeNum = Integer.parseInt(ConfigFactory.Instance().getProperty("node.virtual.num")); this.retinaHostName = NetUtils.getLocalHostName(); - boolean gcEnabled = false; StorageGarbageCollector gc = null; try { - gcEnabled = Boolean.parseBoolean(config.getProperty("retina.storage.gc.enabled")); - if (gcEnabled) + boolean storageGcEnabled = Boolean.parseBoolean(config.getProperty("retina.storage.gc.enabled")); + if (storageGcEnabled) { double threshold = Double.parseDouble(config.getProperty("retina.storage.gc.threshold")); long targetFileSize = Long.parseLong(config.getProperty("retina.storage.gc.target.file.size")); @@ -200,10 +184,8 @@ private RetinaResourceManager() catch (Exception e) { logger.error("Failed to initialise StorageGarbageCollector, Storage GC will be disabled", e); - gcEnabled = false; gc = null; } - this.storageGcEnabled = gcEnabled; this.storageGarbageCollector = gc; } @@ -217,6 +199,62 @@ public static RetinaResourceManager Instance() return InstanceHolder.instance; } + /** + * Starts the periodic Retina GC scheduler after the service has reached the + * lifecycle point where background cleanup is safe to run. + * + *

The constructor intentionally does not schedule GC: recovery-capable + * startup must stay fail-closed until initialization succeeds. This method is + * idempotent so future lifecycle READY hooks can call it safely.

+ * + * @throws RetinaException if GC configuration is invalid or the scheduler cannot be started. + */ + public void startBackgroundGc() throws RetinaException + { + long interval; + try + { + interval = Long.parseLong(ConfigFactory.Instance().getProperty("retina.gc.interval")); + } + catch (Exception e) + { + throw new RetinaException("Invalid retina GC interval configuration", e); + } + + if (interval <= 0) + { + logger.info("Retina background GC is disabled"); + return; + } + + if (!this.gcScheduled.compareAndSet(false, true)) + { + logger.debug("Retina background GC scheduler has already been started"); + return; + } + + try + { + this.gcExecutor.scheduleAtFixedRate( + this::runGC, + interval, + interval, + TimeUnit.SECONDS + ); + logger.info("Retina background GC scheduler started with interval {} seconds", interval); + } + catch (RuntimeException e) + { + this.gcScheduled.set(false); + throw new RetinaException("Failed to start retina background GC", e); + } + } + + public boolean isBackgroundGcStarted() + { + return this.gcScheduled.get(); + } + public void addVisibility(long fileId, int rgId, int recordNum, long timestamp, long[] bitmap, boolean overwrite) { @@ -1000,7 +1038,7 @@ private void runGC() // Step 3: Storage GC — pass file-level stats so that candidate selection // uses O(1) lookups instead of per-RG aggregation loops. - if (storageGcEnabled && storageGarbageCollector != null) + if (storageGarbageCollector != null) { try { diff --git a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/StorageGarbageCollector.java b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/StorageGarbageCollector.java index fbc6da0e22..9aa31eaea1 100644 --- a/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/StorageGarbageCollector.java +++ b/pixels-retina/src/main/java/io/pixelsdb/pixels/retina/StorageGarbageCollector.java @@ -607,7 +607,7 @@ void processFileGroups(List fileGroups, long safeGcTs, * Rewrites all files in one {@link FileGroup} into a single new file, filtering out * rows marked as deleted in {@code gcSnapshotBitmaps}. * - *

The new file is registered as {@code TEMPORARY} in the catalog and its + *

The new file is registered as {@code TEMPORARY_GC} in the catalog and its * {@link RGVisibility} objects are initialised with {@code baseTimestamp = safeGcTs}. * *

After rewriting completes the {@code gcSnapshotBitmaps} entries for this group @@ -877,7 +877,7 @@ RewriteResult rewriteFileGroup(FileGroup group, long safeGcTs, backwardInfos.add(new BackwardInfo(fc.fileId, bwdMappings, oldFileRgRowStart)); } - // Register the new file as TEMPORARY in the catalog and initialise Visibility. + // Register the new file as TEMPORARY_GC in the catalog and initialise Visibility. // Track registration progress so that partial state can be cleaned up on failure. long newFileId = -1; int registeredRgCount = 0; @@ -891,12 +891,15 @@ RewriteResult rewriteFileGroup(FileGroup group, long safeGcTs, } File newFile = new File(); newFile.setName(newFileName); - newFile.setType(File.Type.TEMPORARY); + newFile.setType(File.Type.TEMPORARY_GC); newFile.setNumRowGroup(newFileRgCount); newFile.setMinRowId(minRowId); newFile.setMaxRowId(maxRowId); newFile.setPathId(group.files.get(0).file.getPathId()); - metadataService.addFiles(Collections.singletonList(newFile)); + if (!metadataService.addFiles(Collections.singletonList(newFile))) + { + throw new MetadataException("failed to add metadata for GC rewrite file " + newFilePath); + } newFileId = metadataService.getFileId(newFilePath); for (int rgId = 0; rgId < newFileRgCount; rgId++) @@ -917,7 +920,7 @@ RewriteResult rewriteFileGroup(FileGroup group, long safeGcTs, } /** - * Best-effort cleanup of a partially-created TEMPORARY file. Removes the + * Best-effort cleanup of a partially-created TEMPORARY_GC file. Removes the * catalog record, the physical file, and any RGVisibility keys that were * registered before the failure. */ @@ -939,7 +942,10 @@ private void cleanupTemporaryFile(Storage storage, String newFilePath, } try { - metadataService.deleteFiles(Collections.singletonList(newFileId)); + if (!metadataService.deleteFiles(Collections.singletonList(newFileId))) + { + throw new MetadataException("failed to delete temporary GC catalog entry for fileId=" + newFileId); + } } catch (Exception ex) { @@ -1167,7 +1173,7 @@ private List updateSinglePointIndex(RewriteResult result, long tableId, // ------------------------------------------------------------------------- /** - * Atomically promotes the new TEMPORARY file to REGULAR, deletes old files from + * Atomically promotes the new TEMPORARY_GC file to REGULAR, deletes old files from * the catalog, unregisters dual-write, and enqueues the old files for delayed cleanup. */ void commitFileGroup(RewriteResult result) throws Exception @@ -1263,7 +1269,10 @@ void rollback(RewriteResult result) try { - metadataService.deleteFiles(Collections.singletonList(result.newFileId)); + if (!metadataService.deleteFiles(Collections.singletonList(result.newFileId))) + { + throw new MetadataException("failed to rollback GC catalog entry for fileId=" + result.newFileId); + } } catch (Exception ex) { diff --git a/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestIngestFilePublisher.java b/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestIngestFilePublisher.java new file mode 100644 index 0000000000..817b27a1c2 --- /dev/null +++ b/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestIngestFilePublisher.java @@ -0,0 +1,664 @@ +/* + * Copyright 2026 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.retina; + +import io.pixelsdb.pixels.common.metadata.domain.File; +import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.core.PixelsWriter; +import io.pixelsdb.pixels.core.TypeDescription; +import io.pixelsdb.pixels.core.vector.VectorizedRowBatch; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class TestIngestFilePublisher +{ + @Test + public void finishClosesPhysicalFileOnlyOnceAndLeavesMetadataTemporary() throws Exception + { + CountingPixelsWriter writer = new CountingPixelsWriter(); + File file = temporaryFile(101L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + CompletableFuture firstFinish = fileWriterManager.finish(); + CompletableFuture secondFinish = fileWriterManager.finish(); + firstFinish.get(5, TimeUnit.SECONDS); + secondFinish.get(5, TimeUnit.SECONDS); + + assertSame(firstFinish, secondFinish); + assertEquals(1, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + assertEquals(File.Type.TEMPORARY_INGEST, fileWriterManager.getFileSnapshot().getType()); + assertTrue(firstFinish.isDone()); + assertFalse(firstFinish.isCompletedExceptionally()); + } + + @Test + public void finishFailureIsPropagatedAndDoesNotPublishMetadata() throws Exception + { + IOException closeFailure = new IOException("close failed"); + CountingPixelsWriter writer = new CountingPixelsWriter(null, null, closeFailure, null); + File file = temporaryFile(103L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + CompletableFuture firstFinish = fileWriterManager.finish(); + try + { + firstFinish.get(5, TimeUnit.SECONDS); + fail("Expected physical close failure"); + } catch (ExecutionException e) + { + assertSame(closeFailure, e.getCause()); + } + + CompletableFuture secondFinish = fileWriterManager.finish(); + assertSame(firstFinish, secondFinish); + assertTrue(secondFinish.isCompletedExceptionally()); + assertEquals(1, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + assertEquals(File.Type.TEMPORARY_INGEST, fileWriterManager.getFileSnapshot().getType()); + } + + @Test + public void fileSnapshotCopiesCurrentFileMetadata() + { + File file = temporaryFile(202L); + CountingPixelsWriter writer = new CountingPixelsWriter(); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + File snapshot = fileWriterManager.getFileSnapshot(); + + assertEquals(file.getId(), snapshot.getId()); + assertEquals(file.getName(), snapshot.getName()); + assertEquals(file.getType(), snapshot.getType()); + assertEquals(file.getNumRowGroup(), snapshot.getNumRowGroup()); + assertEquals(file.getMinRowId(), snapshot.getMinRowId()); + assertEquals(file.getMaxRowId(), snapshot.getMaxRowId()); + assertEquals(file.getPathId(), snapshot.getPathId()); + } + + @Test + public void fileSnapshotDoesNotExposeInternalFileState() + { + File file = temporaryFile(203L); + CountingPixelsWriter writer = new CountingPixelsWriter(); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + File snapshot = fileWriterManager.getFileSnapshot(); + + snapshot.setName("published.pxl"); + snapshot.setType(File.Type.REGULAR); + snapshot.setNumRowGroup(99); + snapshot.setMinRowId(1000); + snapshot.setMaxRowId(2000); + snapshot.setPathId(88L); + + File freshSnapshot = fileWriterManager.getFileSnapshot(); + assertEquals("ingest_203.pxl", file.getName()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + assertEquals(1, file.getNumRowGroup()); + assertEquals(0, file.getMinRowId()); + assertEquals(63, file.getMaxRowId()); + assertEquals(9L, file.getPathId()); + assertEquals(file.getName(), freshSnapshot.getName()); + assertEquals(file.getType(), freshSnapshot.getType()); + assertEquals(file.getNumRowGroup(), freshSnapshot.getNumRowGroup()); + assertEquals(file.getMinRowId(), freshSnapshot.getMinRowId()); + assertEquals(file.getMaxRowId(), freshSnapshot.getMaxRowId()); + assertEquals(file.getPathId(), freshSnapshot.getPathId()); + } + + @Test + public void fileSnapshotReflectsMutationsOnUnderlyingFile() + { + File file = temporaryFile(205L); + CountingPixelsWriter writer = new CountingPixelsWriter(); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + File before = fileWriterManager.getFileSnapshot(); + assertEquals(File.Type.TEMPORARY_INGEST, before.getType()); + assertEquals(63L, before.getMaxRowId()); + + // Mutations on the underlying file (e.g. visibility/row id updates) must be observed + // by snapshots taken afterwards. Snapshots taken earlier must remain unchanged. + file.setMaxRowId(127L); + file.setNumRowGroup(2); + + File after = fileWriterManager.getFileSnapshot(); + assertEquals(127L, after.getMaxRowId()); + assertEquals(2, after.getNumRowGroup()); + // The previously taken snapshot must keep its original values. + assertEquals(63L, before.getMaxRowId()); + assertEquals(1, before.getNumRowGroup()); + } + + @Test + public void gettersExposeConstructorArguments() + { + File file = temporaryFile(301L); + CountingPixelsWriter writer = new CountingPixelsWriter(); + FileWriterManager fileWriterManager = new FileWriterManager(7L, writer, file, 5L, 10L, 0); + + assertEquals(file.getId(), fileWriterManager.getFileId()); + assertEquals(5L, fileWriterManager.getFirstBlockId()); + assertEquals(10L, fileWriterManager.getLastBlockId()); + } + + @Test + public void setLastBlockIdUpdatesGetter() + { + File file = temporaryFile(302L); + CountingPixelsWriter writer = new CountingPixelsWriter(); + FileWriterManager fileWriterManager = new FileWriterManager(1L, writer, file, 0L, 0L, 0); + + fileWriterManager.setLastBlockId(42L); + assertEquals(42L, fileWriterManager.getLastBlockId()); + + // Allow lowering as well, e.g. when shrinking the range during close(). + fileWriterManager.setLastBlockId(-1L); + assertEquals(-1L, fileWriterManager.getLastBlockId()); + } + + @Test + public void addRowBatchSucceedsAndForwardsToWriter() throws Exception + { + CountingPixelsWriter writer = new CountingPixelsWriter(); + File file = temporaryFile(401L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + fileWriterManager.addRowBatch(null); + fileWriterManager.addRowBatch(null); + fileWriterManager.addRowBatch(null); + + assertEquals(3, writer.addRowBatchCount.get()); + assertEquals(0, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + } + + @Test + public void addRowBatchFailureLeavesManagerUsableForFinish() throws Exception + { + IOException writeFailure = new IOException("write failed"); + CountingPixelsWriter writer = new CountingPixelsWriter(null, null, null, writeFailure); + File file = temporaryFile(402L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + try + { + fileWriterManager.addRowBatch(null); + fail("Expected row batch write failure"); + } catch (RetinaException e) + { + assertSame(writeFailure, e.getCause()); + } + + // After a failed addRowBatch, finish() must still close the underlying writer exactly once + // and keep the file in TEMPORARY_INGEST state (publication is the buffer's responsibility). + fileWriterManager.finish().get(5, TimeUnit.SECONDS); + assertEquals(1, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + } + + @SuppressWarnings("unchecked") + @Test + public void finishIsIdempotentUnderConcurrentCallers() throws Exception + { + CountDownLatch closeStarted = new CountDownLatch(1); + CountDownLatch allowClose = new CountDownLatch(1); + CountingPixelsWriter writer = new CountingPixelsWriter(closeStarted, allowClose, null, null); + File file = temporaryFile(501L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + // Start the first finish() so the close thread is parked inside writer.close(). + CompletableFuture firstFinish = fileWriterManager.finish(); + assertTrue(closeStarted.await(5, TimeUnit.SECONDS)); + + int callerCount = 8; + ExecutorService callers = Executors.newFixedThreadPool(callerCount); + try + { + CountDownLatch readyLatch = new CountDownLatch(callerCount); + CountDownLatch startLatch = new CountDownLatch(1); + Future>[] results = new Future[callerCount]; + for (int i = 0; i < callerCount; ++i) + { + results[i] = callers.submit(() -> { + readyLatch.countDown(); + startLatch.await(); + return fileWriterManager.finish(); + }); + } + assertTrue(readyLatch.await(5, TimeUnit.SECONDS)); + startLatch.countDown(); + + for (Future> result : results) + { + CompletableFuture observed = result.get(5, TimeUnit.SECONDS); + assertSame(firstFinish, observed); + assertFalse(observed.isDone()); + } + } finally + { + allowClose.countDown(); + callers.shutdownNow(); + } + + firstFinish.get(5, TimeUnit.SECONDS); + assertEquals("writer.close() must run at most once even under concurrent finish() calls", + 1, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + } + + @Test + public void finishRunsCloseOnDedicatedNamedThread() throws Exception + { + CountDownLatch closeStarted = new CountDownLatch(1); + CountDownLatch allowClose = new CountDownLatch(1); + ThreadCapturingPixelsWriter writer = new ThreadCapturingPixelsWriter(closeStarted, allowClose); + File file = temporaryFile(601L); + FileWriterManager fileWriterManager = new FileWriterManager(1L, writer, file, 1L, 0L, 0); + + Thread caller = Thread.currentThread(); + CompletableFuture finishFuture = fileWriterManager.finish(); + assertTrue(closeStarted.await(5, TimeUnit.SECONDS)); + + Thread closeThread = writer.closeThread; + assertNotSame("close() must run off the caller thread", caller, closeThread); + assertEquals("pixels-retina-file-finish-" + file.getId(), closeThread.getName()); + + allowClose.countDown(); + finishFuture.get(5, TimeUnit.SECONDS); + } + + @Test + public void finishPropagatesRuntimeExceptionFromClose() throws Exception + { + RuntimeException closeFailure = new RuntimeException("boom"); + CountingPixelsWriter writer = new CountingPixelsWriter(null, null, null, null, + closeFailure); + File file = temporaryFile(701L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + CompletableFuture firstFinish = fileWriterManager.finish(); + try + { + firstFinish.get(5, TimeUnit.SECONDS); + fail("Expected runtime close failure"); + } catch (ExecutionException e) + { + assertSame(closeFailure, e.getCause()); + } + + // Subsequent calls must keep returning the same failed future and must not retry close(). + CompletableFuture secondFinish = fileWriterManager.finish(); + assertSame(firstFinish, secondFinish); + assertTrue(secondFinish.isCompletedExceptionally()); + assertEquals(1, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + } + + @Test(timeout = 10_000L) + public void finishDoesNotBlockCallerThread() throws Exception + { + CountDownLatch closeStarted = new CountDownLatch(1); + CountDownLatch allowClose = new CountDownLatch(1); + CountingPixelsWriter writer = new CountingPixelsWriter(closeStarted, allowClose, null, null); + File file = temporaryFile(801L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + long start = System.nanoTime(); + CompletableFuture finishFuture = fileWriterManager.finish(); + long elapsedMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + + // The caller thread must return promptly; the actual close() runs on the named thread. + assertTrue("finish() must not block on writer.close(); elapsedMillis=" + elapsedMillis, + elapsedMillis < 2_000L); + assertTrue(closeStarted.await(5, TimeUnit.SECONDS)); + assertFalse(finishFuture.isDone()); + try + { + finishFuture.get(200, TimeUnit.MILLISECONDS); + fail("finish() future must not complete before writer.close() returns"); + } catch (TimeoutException expected) + { + // expected: still in progress + } + allowClose.countDown(); + finishFuture.get(5, TimeUnit.SECONDS); + } + + @Test + public void concurrentAddRowBatchesAreAllForwardedToWriter() throws Exception + { + // FileWriterManager does not perform internal locking around addRowBatch; verify it does + // not lose calls or throw NPEs when several threads forward row batches concurrently. + CountingPixelsWriter writer = new CountingPixelsWriter(); + File file = temporaryFile(1601L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + int callerCount = 16; + int callsPerCaller = 25; + ExecutorService callers = Executors.newFixedThreadPool(callerCount); + try + { + CountDownLatch startLatch = new CountDownLatch(1); + List> results = new ArrayList<>(callerCount); + for (int i = 0; i < callerCount; ++i) + { + results.add(callers.submit(() -> { + startLatch.await(); + for (int j = 0; j < callsPerCaller; ++j) + { + fileWriterManager.addRowBatch(null); + } + return null; + })); + } + startLatch.countDown(); + for (Future result : results) + { + result.get(10, TimeUnit.SECONDS); + } + } finally + { + callers.shutdownNow(); + } + + assertEquals(callerCount * callsPerCaller, writer.addRowBatchCount.get()); + assertEquals(0, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + } + + @Test + public void finishIsRobustAgainstFileMetadataMutationsBeforeReturn() throws Exception + { + // Mutations on the underlying file (e.g. visibility/row id updates by other components) + // performed while finish() is in progress must not affect the success of physical close, + // and the post-close snapshot must reflect the mutated state because publication has + // not yet rewritten file.type. + CountDownLatch closeStarted = new CountDownLatch(1); + CountDownLatch allowClose = new CountDownLatch(1); + CountingPixelsWriter writer = new CountingPixelsWriter(closeStarted, allowClose, null, null); + File file = temporaryFile(2001L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + CompletableFuture finishFuture = fileWriterManager.finish(); + assertTrue(closeStarted.await(5, TimeUnit.SECONDS)); + + // Concurrently update row id bookkeeping; this is what the visibility layer does. + file.setMaxRowId(255L); + file.setNumRowGroup(3); + + allowClose.countDown(); + finishFuture.get(5, TimeUnit.SECONDS); + + File snapshot = fileWriterManager.getFileSnapshot(); + assertEquals(255L, snapshot.getMaxRowId()); + assertEquals(3, snapshot.getNumRowGroup()); + assertEquals(File.Type.TEMPORARY_INGEST, snapshot.getType()); + assertEquals(1, writer.closeCount.get()); + } + + @Test + public void addRowBatchPropagatesWriterRuntimeExceptionWithoutWrapping() throws Exception + { + // FileWriterManager only wraps IOException into RetinaException; unchecked exceptions + // (e.g. format-corruption indicators thrown by the underlying writer as RuntimeException) + // must propagate to the caller as-is so they are not silently masked. After such a failure + // the manager must remain usable and finish() must still close the writer exactly once. + RuntimeException formatFailure = new IllegalStateException("corrupted column vector"); + CountingPixelsWriter writer = new CountingPixelsWriter() + { + @Override + public boolean addRowBatch(VectorizedRowBatch rowBatch) throws IOException + { + addRowBatchCount.incrementAndGet(); + throw formatFailure; + } + }; + File file = temporaryFile(2101L); + FileWriterManager fileWriterManager = testFileWriterManager(writer, file); + + try + { + fileWriterManager.addRowBatch(null); + fail("Runtime exception from writer must propagate without being wrapped"); + } catch (RetinaException e) + { + fail("Runtime exception must not be wrapped as RetinaException, got: " + e); + } catch (IllegalStateException expected) + { + assertSame(formatFailure, expected); + } + assertEquals(1, writer.addRowBatchCount.get()); + + // After a runtime failure inside the writer, finish() must still be able to close it. + fileWriterManager.finish().get(5, TimeUnit.SECONDS); + assertEquals(1, writer.closeCount.get()); + assertEquals(File.Type.TEMPORARY_INGEST, file.getType()); + } + + private static File temporaryFile(long id) + { + File file = new File(); + file.setId(id); + file.setName("ingest_" + id + ".pxl"); + file.setType(File.Type.TEMPORARY_INGEST); + file.setNumRowGroup(1); + file.setMinRowId(0); + file.setMaxRowId(63); + file.setPathId(9L); + return file; + } + + private static FileWriterManager testFileWriterManager(CountingPixelsWriter writer, File file) + { + return new FileWriterManager(1L, writer, file, 1L, 0L, 0); + } + + private static class CountingPixelsWriter implements PixelsWriter + { + // Package-private so anonymous subclasses defined inside this test can observe call counts. + final AtomicInteger closeCount = new AtomicInteger(0); + final AtomicInteger addRowBatchCount = new AtomicInteger(0); + private final CountDownLatch closeStarted; + private final CountDownLatch allowClose; + private final IOException closeFailure; + private final IOException addRowBatchFailure; + private final RuntimeException closeRuntimeFailure; + + private CountingPixelsWriter() + { + this(null, null, null, null, null); + } + + private CountingPixelsWriter(CountDownLatch closeStarted, CountDownLatch allowClose, + IOException closeFailure, IOException addRowBatchFailure) + { + this(closeStarted, allowClose, closeFailure, addRowBatchFailure, null); + } + + private CountingPixelsWriter(CountDownLatch closeStarted, CountDownLatch allowClose, + IOException closeFailure, IOException addRowBatchFailure, + RuntimeException closeRuntimeFailure) + { + this.closeStarted = closeStarted; + this.allowClose = allowClose; + this.closeFailure = closeFailure; + this.addRowBatchFailure = addRowBatchFailure; + this.closeRuntimeFailure = closeRuntimeFailure; + } + + @Override + public boolean addRowBatch(VectorizedRowBatch rowBatch) throws IOException + { + addRowBatchCount.incrementAndGet(); + if (addRowBatchFailure != null) + { + throw addRowBatchFailure; + } + return true; + } + + @Override + public void addRowBatch(VectorizedRowBatch rowBatch, int hashValue) throws IOException + { + } + + @Override + public TypeDescription getSchema() + { + return null; + } + + @Override + public int getNumRowGroup() + { + return 0; + } + + @Override + public int getNumWriteRequests() + { + return 0; + } + + @Override + public long getCompletedBytes() + { + return 0; + } + + @Override + public void close() throws IOException + { + closeCount.incrementAndGet(); + if (closeStarted != null) + { + closeStarted.countDown(); + } + if (allowClose != null) + { + try + { + assertTrue(allowClose.await(5, TimeUnit.SECONDS)); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while waiting to close", e); + } + } + if (closeFailure != null) + { + throw closeFailure; + } + if (closeRuntimeFailure != null) + { + throw closeRuntimeFailure; + } + } + } + + private static class ThreadCapturingPixelsWriter implements PixelsWriter + { + private final CountDownLatch closeStarted; + private final CountDownLatch allowClose; + private volatile Thread closeThread; + + private ThreadCapturingPixelsWriter(CountDownLatch closeStarted, CountDownLatch allowClose) + { + this.closeStarted = closeStarted; + this.allowClose = allowClose; + } + + @Override + public boolean addRowBatch(VectorizedRowBatch rowBatch) + { + return true; + } + + @Override + public void addRowBatch(VectorizedRowBatch rowBatch, int hashValue) + { + } + + @Override + public TypeDescription getSchema() + { + return null; + } + + @Override + public int getNumRowGroup() + { + return 0; + } + + @Override + public int getNumWriteRequests() + { + return 0; + } + + @Override + public long getCompletedBytes() + { + return 0; + } + + @Override + public void close() throws IOException + { + this.closeThread = Thread.currentThread(); + if (closeStarted != null) + { + closeStarted.countDown(); + } + if (allowClose != null) + { + try + { + assertTrue(allowClose.await(5, TimeUnit.SECONDS)); + } catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while waiting to close", e); + } + } + } + } +} diff --git a/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestRetinaResourceManager.java b/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestRetinaResourceManager.java index 6edb341693..48986a7468 100644 --- a/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestRetinaResourceManager.java +++ b/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestRetinaResourceManager.java @@ -20,10 +20,27 @@ package io.pixelsdb.pixels.retina; import io.pixelsdb.pixels.common.exception.RetinaException; +import io.pixelsdb.pixels.common.utils.ConfigFactory; import io.pixelsdb.pixels.core.vector.VectorizedRowBatch; +import org.junit.Ignore; import org.junit.Test; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.nio.ByteBuffer; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; public class TestRetinaResourceManager { @@ -45,6 +62,150 @@ private boolean checkVisibility(long[] visibility, int rowId) return (targetLong & (1L << (rowId % 64))) != 0; } + private RetinaResourceManager newIsolatedManager() throws Exception + { + Constructor constructor = RetinaResourceManager.class.getDeclaredConstructor(); + constructor.setAccessible(true); + return constructor.newInstance(); + } + + private void setGcExecutor(RetinaResourceManager manager, + ScheduledExecutorService executor) throws Exception + { + Field field = RetinaResourceManager.class.getDeclaredField("gcExecutor"); + field.setAccessible(true); + field.set(manager, executor); + } + + @Test + public void testBackgroundGcIsNotStartedByConstructor() throws Exception + { + Constructor constructor = RetinaResourceManager.class.getDeclaredConstructor(); + constructor.setAccessible(true); + RetinaResourceManager manager = constructor.newInstance(); + + assertFalse("background GC must be started by lifecycle only", + manager.isBackgroundGcStarted()); + } + + @Test + public void testStartBackgroundGcIsExplicitAndIdempotent() throws Exception + { + String originalInterval = ConfigFactory.Instance().getProperty("retina.gc.interval"); + RetinaResourceManager manager = newIsolatedManager(); + ScheduledExecutorService executor = mock(ScheduledExecutorService.class); + setGcExecutor(manager, executor); + try + { + ConfigFactory.Instance().addProperty("retina.gc.interval", "300"); + + manager.startBackgroundGc(); + manager.startBackgroundGc(); + + assertTrue("explicit lifecycle start must mark background GC as started", + manager.isBackgroundGcStarted()); + verify(executor).scheduleAtFixedRate(any(Runnable.class), eq(300L), eq(300L), eq(TimeUnit.SECONDS)); + verifyNoMoreInteractions(executor); + } + finally + { + ConfigFactory.Instance().addProperty("retina.gc.interval", originalInterval); + } + } + + @Test + public void testStartBackgroundGcDisabledByNonPositiveInterval() throws Exception + { + String originalInterval = ConfigFactory.Instance().getProperty("retina.gc.interval"); + RetinaResourceManager manager = newIsolatedManager(); + try + { + ConfigFactory.Instance().addProperty("retina.gc.interval", "0"); + + manager.startBackgroundGc(); + + assertFalse("disabled interval must not mark background GC as started", + manager.isBackgroundGcStarted()); + } + finally + { + ConfigFactory.Instance().addProperty("retina.gc.interval", originalInterval); + } + } + + @Test + public void testStartBackgroundGcInvalidIntervalFailsWithoutStarting() throws Exception + { + String originalInterval = ConfigFactory.Instance().getProperty("retina.gc.interval"); + RetinaResourceManager manager = newIsolatedManager(); + try + { + ConfigFactory.Instance().addProperty("retina.gc.interval", "not-a-number"); + + try + { + manager.startBackgroundGc(); + fail("invalid GC interval must fail closed"); + } + catch (RetinaException e) + { + assertTrue(e.getMessage().contains("Invalid retina GC interval configuration")); + } + + assertFalse("failed lifecycle start must not mark GC as started", + manager.isBackgroundGcStarted()); + } + finally + { + ConfigFactory.Instance().addProperty("retina.gc.interval", originalInterval); + } + } + + @Test + public void testStartBackgroundGcSchedulerFailureRollsBackStartedFlag() throws Exception + { + String originalInterval = ConfigFactory.Instance().getProperty("retina.gc.interval"); + RetinaResourceManager manager = newIsolatedManager(); + ScheduledExecutorService executor = mock(ScheduledExecutorService.class); + setGcExecutor(manager, executor); + try + { + ConfigFactory.Instance().addProperty("retina.gc.interval", "300"); + when(executor.scheduleAtFixedRate(any(Runnable.class), eq(300L), eq(300L), eq(TimeUnit.SECONDS))) + .thenThrow(new RuntimeException("scheduler rejected")); + + try + { + manager.startBackgroundGc(); + fail("scheduler failure must fail closed"); + } + catch (RetinaException e) + { + assertTrue(e.getMessage().contains("Failed to start retina background GC")); + } + + assertFalse("scheduler failure must roll back started flag", + manager.isBackgroundGcStarted()); + } + finally + { + ConfigFactory.Instance().addProperty("retina.gc.interval", originalInterval); + } + } + + @Test + public void testRunGcBeforeLifecycleStartIsRejected() throws Exception + { + RetinaResourceManager manager = newIsolatedManager(); + Method runGc = RetinaResourceManager.class.getDeclaredMethod("runGC"); + runGc.setAccessible(true); + + runGc.invoke(manager); + + assertFalse("manual GC invocation before lifecycle start must be ignored", + manager.isBackgroundGcStarted()); + } + @Test public void TestVisibility() { @@ -80,6 +241,7 @@ private byte[][] createTpchNationRow(long nationKey, String name, long regionKey return row; } + @Ignore("Integration test requires real tpch.nation metadata and storage state.") @Test public void testWriteBuffer() { diff --git a/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestStorageGarbageCollector.java b/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestStorageGarbageCollector.java index 6281626267..81be9522cd 100644 --- a/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestStorageGarbageCollector.java +++ b/pixels-retina/src/test/java/io/pixelsdb/pixels/retina/TestStorageGarbageCollector.java @@ -21,6 +21,7 @@ import io.pixelsdb.pixels.common.metadata.MetadataService; import io.pixelsdb.pixels.common.utils.CheckpointFileIO; +import io.pixelsdb.pixels.common.utils.MetaDBUtil; import io.pixelsdb.pixels.common.utils.PixelsFileNameUtils; import io.pixelsdb.pixels.common.utils.RetinaUtils; import io.pixelsdb.pixels.common.metadata.domain.Column; @@ -52,6 +53,7 @@ import java.lang.reflect.Method; import java.nio.file.Files; import java.nio.file.Path; +import java.sql.PreparedStatement; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -70,6 +72,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for {@link StorageGarbageCollector}, covering scan/grouping, data rewrite, @@ -93,6 +96,7 @@ * * Legacy test names (pre-convention) are preserved for CI stability. */ +@Ignore("Integration suite requires a running metadata server and external metadata DB state.") public class TestStorageGarbageCollector { // ----------------------------------------------------------------------- @@ -702,96 +706,166 @@ public void testScanAndGroupFiles_skipsFilesWithNoVisibility() // ======================================================================= /** - * After {@code runStorageGC}, the {@code gcSnapshotBitmaps} map must have had - * non-candidate entries removed. Candidate bitmaps must be retained for the rewrite phase. + * When no file crosses the strict deletion-ratio threshold, + * {@code runStorageGC} must return before metadata scan and keep the bitmap + * snapshot intact for the already-written GC checkpoint. */ @Test - public void testRunStorageGC_trimsBitmapMapToCandidate() + public void testRunStorageGC_noCandidateDoesNotScanOrTrim() { - long candidateFileId = 66001L; - long otherFileId = 66002L; + long belowThresholdFileId = 66101L; + long exactlyThresholdFileId = 66102L; - Map bitmaps = new HashMap<>(); - bitmaps.put(candidateFileId + "_0", makeBitmap(100, 60)); - bitmaps.put(otherFileId + "_0", makeBitmap(100, 20)); - - // File-level stats: candidateFileId has 60% deletion, otherFileId has 20% Map fileStats = new HashMap<>(); - fileStats.put(candidateFileId, makeRgStats(100, 60)); - fileStats.put(otherFileId, makeRgStats(100, 20)); + fileStats.put(belowThresholdFileId, makeRgStats(100, 40)); + fileStats.put(exactlyThresholdFileId, makeRgStats(100, 50)); - List fakeFiles = Arrays.asList( - new FakeFileEntry(candidateFileId, 1, 1L, 0), - new FakeFileEntry(otherFileId, 1, 1L, 0)); + Map bitmaps = new HashMap<>(); + bitmaps.put(RetinaUtils.buildRgKey(belowThresholdFileId, 0), makeBitmap(100, 40)); + bitmaps.put(RetinaUtils.buildRgKey(exactlyThresholdFileId, 0), makeBitmap(100, 50)); - DirectScanStorageGC gc = new DirectScanStorageGC( - retinaManager, 0.5, 10, fakeFiles); + TrackingRunStorageGC trackingGc = new TrackingRunStorageGC(Collections.emptyList()); - gc.runStorageGC(300L, fileStats, bitmaps); + trackingGc.runStorageGC(301L, fileStats, bitmaps); - assertTrue("candidate RG key must be retained", - bitmaps.containsKey(candidateFileId + "_0")); - assertFalse("non-candidate RG key must be removed", - bitmaps.containsKey(otherFileId + "_0")); + assertFalse("no candidate means metadata scan must not run", trackingGc.scanCalled); + assertFalse("no candidate means process phase must not run", trackingGc.processCalled); + assertTrue("below-threshold bitmap must remain for checkpoint recovery", + bitmaps.containsKey(RetinaUtils.buildRgKey(belowThresholdFileId, 0))); + assertTrue("exact-threshold bitmap must remain because threshold is strict >", + bitmaps.containsKey(RetinaUtils.buildRgKey(exactlyThresholdFileId, 0))); + assertEquals("bitmap snapshot must remain unchanged", 2, bitmaps.size()); } - // ======================================================================= - // Section 4: runStorageGC end-to-end scan → process - // ======================================================================= - /** - * A file whose invalidRatio is exactly equal to the threshold (0.5) must NOT - * be selected as a candidate. The design uses strict {@code >}, not {@code >=}. + * Candidate selection must be driven by file-level stats only. Files at the + * threshold, with zero rows, or below threshold must not be passed to scan; + * their bitmap entries are released before rewrite processing starts. */ @Test - public void testRunStorageGC_thresholdExactlyEqual() + public void testRunStorageGC_passesOnlyStrictFileLevelCandidatesToScan() { - long fileId = 57001L; + long candidateA = 66201L; + long candidateB = 66202L; + long exactlyThreshold = 66203L; + long zeroRows = 66204L; + long belowThreshold = 66205L; Map fileStats = new HashMap<>(); - fileStats.put(fileId, makeRgStats(100, 50)); // exactly 50% = threshold + fileStats.put(candidateA, makeRgStats(100, 51)); + fileStats.put(candidateB, makeRgStats(200, 120)); + fileStats.put(exactlyThreshold, makeRgStats(100, 50)); + fileStats.put(zeroRows, new long[]{0, 10}); + fileStats.put(belowThreshold, makeRgStats(100, 49)); Map bitmaps = new HashMap<>(); - bitmaps.put(fileId + "_0", makeBitmap(100, 50)); + for (long fileId : Arrays.asList(candidateA, candidateB, exactlyThreshold, zeroRows, belowThreshold)) + { + bitmaps.put(RetinaUtils.buildRgKey(fileId, 0), makeBitmap(100, 1)); + } + bitmaps.put(RetinaUtils.buildRgKey(candidateB, 1), makeBitmap(100, 1)); + + TrackingRunStorageGC trackingGc = new TrackingRunStorageGC(Collections.emptyList()); + + trackingGc.runStorageGC(302L, fileStats, bitmaps); + + assertTrue("candidate scan must run when at least one file qualifies", trackingGc.scanCalled); + assertEquals(new HashSet<>(Arrays.asList(candidateA, candidateB)), trackingGc.capturedCandidateFileIds); + assertEquals("only candidate RG bitmaps should remain", 3, bitmaps.size()); + assertTrue(bitmaps.containsKey(RetinaUtils.buildRgKey(candidateA, 0))); + assertTrue(bitmaps.containsKey(RetinaUtils.buildRgKey(candidateB, 0))); + assertTrue(bitmaps.containsKey(RetinaUtils.buildRgKey(candidateB, 1))); + assertFalse(bitmaps.containsKey(RetinaUtils.buildRgKey(exactlyThreshold, 0))); + assertFalse(bitmaps.containsKey(RetinaUtils.buildRgKey(zeroRows, 0))); + assertFalse(bitmaps.containsKey(RetinaUtils.buildRgKey(belowThreshold, 0))); + assertFalse("empty scan result must skip process phase", trackingGc.processCalled); + } - DirectScanStorageGC gc = new DirectScanStorageGC( - retinaManager, 0.5, 10, - Collections.singletonList(new FakeFileEntry(fileId, 1, 1L, 0))); + /** + * The process phase must see the safe GC timestamp, the groups returned from + * scan, and a bitmap map already trimmed to candidate files. This protects + * the Storage GC rewrite path from accidentally consuming non-candidate RGs. + */ + @Test + public void testRunStorageGC_processSeesTrimmedCandidateBitmapsAndSafeTs() + { + long candidateFileId = 66301L; + long otherFileId = 66302L; + long safeGcTs = 303L; - gc.runStorageGC(400L, fileStats, bitmaps); + StorageGarbageCollector.FileGroup group = new StorageGarbageCollector.FileGroup( + 7L, 4, Collections.singletonList( + new StorageGarbageCollector.FileCandidate( + makeFile(candidateFileId, 2), "fake_candidate", candidateFileId, 2, 7L, 4, 0.75, 0L))); + TrackingRunStorageGC trackingGc = new TrackingRunStorageGC(Collections.singletonList(group)); - assertTrue("file at exactly threshold must NOT be trimmed (no candidates)", - bitmaps.containsKey(fileId + "_0")); - assertEquals(1, bitmaps.size()); + Map fileStats = new HashMap<>(); + fileStats.put(candidateFileId, makeRgStats(100, 75)); + fileStats.put(otherFileId, makeRgStats(100, 10)); + + Map bitmaps = new HashMap<>(); + bitmaps.put(RetinaUtils.buildRgKey(candidateFileId, 0), makeBitmap(100, 75)); + bitmaps.put(RetinaUtils.buildRgKey(candidateFileId, 1), makeBitmap(100, 60)); + bitmaps.put(RetinaUtils.buildRgKey(otherFileId, 0), makeBitmap(100, 10)); + + trackingGc.runStorageGC(safeGcTs, fileStats, bitmaps); + + assertTrue("process phase must run for non-empty groups", trackingGc.processCalled); + assertEquals("safeGcTs must be forwarded to process phase", safeGcTs, trackingGc.capturedSafeGcTs); + assertEquals("scan groups must be forwarded unchanged", 1, trackingGc.capturedFileGroups.size()); + assertEquals(candidateFileId, trackingGc.capturedFileGroups.get(0).files.get(0).fileId); + assertEquals(new HashSet<>(Arrays.asList( + RetinaUtils.buildRgKey(candidateFileId, 0), + RetinaUtils.buildRgKey(candidateFileId, 1))), trackingGc.bitmapKeysSeenByProcess); + assertFalse("non-candidate bitmap must be trimmed before process", + bitmaps.containsKey(RetinaUtils.buildRgKey(otherFileId, 0))); } /** - * A file whose {@code fileStats} entry has {@code totalRows=0} must not - * produce a candidate even if invalidCount is also 0 (division by zero guard). + * If the downstream process phase fails, {@code runStorageGC} must already + * have released non-candidate bitmaps. This mirrors the real GC ordering: + * checkpoint is complete, then candidate-only rewrite state is retained. */ @Test - public void testRunStorageGC_skipsTotalRowsZero() + public void testRunStorageGC_processFailureKeepsOnlyCandidateBitmaps() { - long fileId = 58001L; + long candidateFileId = 66401L; + long otherFileId = 66402L; + + StorageGarbageCollector.FileGroup group = new StorageGarbageCollector.FileGroup( + 8L, 0, Collections.singletonList( + new StorageGarbageCollector.FileCandidate( + makeFile(candidateFileId, 1), "fake_candidate", candidateFileId, 1, 8L, 0, 0.80, 0L))); + TrackingRunStorageGC trackingGc = new TrackingRunStorageGC(Collections.singletonList(group)); + trackingGc.processFailure = new RuntimeException("simulated process failure"); Map fileStats = new HashMap<>(); - fileStats.put(fileId, new long[]{0, 0}); // totalRows=0 + fileStats.put(candidateFileId, makeRgStats(100, 80)); + fileStats.put(otherFileId, makeRgStats(100, 20)); Map bitmaps = new HashMap<>(); - bitmaps.put(fileId + "_0", new long[]{0L}); + bitmaps.put(RetinaUtils.buildRgKey(candidateFileId, 0), makeBitmap(100, 80)); + bitmaps.put(RetinaUtils.buildRgKey(otherFileId, 0), makeBitmap(100, 20)); - DirectScanStorageGC gc = new DirectScanStorageGC( - retinaManager, 0.5, 10, - Collections.singletonList(new FakeFileEntry(fileId, 1, 1L, 0))); - - gc.runStorageGC(500L, fileStats, bitmaps); + try + { + trackingGc.runStorageGC(304L, fileStats, bitmaps); + fail("process failure should propagate to the caller"); + } + catch (RuntimeException e) + { + assertEquals("simulated process failure", e.getMessage()); + } - assertTrue("totalRows=0 file must remain untouched (no candidates)", - bitmaps.containsKey(fileId + "_0")); + assertTrue("process phase should have been entered", trackingGc.processCalled); + assertTrue("candidate bitmap remains available for failure handling", + bitmaps.containsKey(RetinaUtils.buildRgKey(candidateFileId, 0))); + assertFalse("non-candidate bitmap must remain released after failure", + bitmaps.containsKey(RetinaUtils.buildRgKey(otherFileId, 0))); } // ======================================================================= - // Section 4b: processFileGroups error handling + // Section 4: processFileGroups error handling // ======================================================================= /** @@ -1808,7 +1882,7 @@ public void testDualWrite_concurrentPressure() throws Exception // ======================================================================= /** - * Atomicity with multiple old files: one TEMPORARY new file and three REGULAR + * Atomicity with multiple old files: one TEMPORARY_GC new file and three REGULAR * old files are swapped in a single call. Verifies that after the call the new * file is promoted to REGULAR and all old files are removed from the * catalog—i.e., the UPDATE and DELETE execute as one indivisible transaction. @@ -1827,12 +1901,12 @@ public void testAtomicSwap_multipleOldFilesAtomicity() throws Exception new String[]{"atom_old1.pxl", "atom_old2.pxl", "atom_old3.pxl"}, new File.Type[]{File.Type.REGULAR, File.Type.REGULAR, File.Type.REGULAR}, new int[]{1, 1, 1}, new long[]{0, 0, 0}, new long[]{1, 1, 1}); - long newFileId = registerTestFile("atom_new.pxl", File.Type.TEMPORARY, 1, 0, 1); + long newFileId = registerTestFile("atom_new.pxl", File.Type.TEMPORARY_GC, 1, 0, 1); File preSwapNew = metadataService.getFileById(newFileId); assertNotNull("New file must exist before swap", preSwapNew); - assertEquals("New file should be TEMPORARY before swap", - File.Type.TEMPORARY, preSwapNew.getType()); + assertEquals("New file should be TEMPORARY_GC before swap", + File.Type.TEMPORARY_GC, preSwapNew.getType()); metadataService.atomicSwapFiles(newFileId, Arrays.asList(oldIds[0], oldIds[1], oldIds[2])); @@ -1853,7 +1927,7 @@ public void testAtomicSwap_idempotent() throws Exception { writeTestFile("idem_old.pxl", LONG_ID_SCHEMA, new long[]{0, 1, 2}, true, new long[]{100, 100, 100}); long oldFileId = registerTestFile("idem_old.pxl", File.Type.REGULAR, 1, 0, 2); - long newFileId = registerTestFile("idem_new.pxl", File.Type.TEMPORARY, 1, 0, 2); + long newFileId = registerTestFile("idem_new.pxl", File.Type.TEMPORARY_GC, 1, 0, 2); metadataService.atomicSwapFiles(newFileId, Collections.singletonList(oldFileId)); assertFileRegular(newFileId, "File should be REGULAR after first swap"); @@ -1865,8 +1939,8 @@ public void testAtomicSwap_idempotent() throws Exception } /** - * TEMPORARY visibility semantics: before the swap, {@code getFiles(pathId)} must - * not return the TEMPORARY new file (the DAO filters {@code FILE_TYPE <> 0}). + * TEMPORARY_GC visibility semantics: before the swap, {@code getFiles(pathId)} must + * not return the TEMPORARY_GC new file (the DAO filters {@code FILE_TYPE = REGULAR}). * After the swap the promoted file is visible and the old file disappears. */ @Test @@ -1875,7 +1949,7 @@ public void testAtomicSwap_temporaryInvisibleViaGetFiles() throws Exception writeTestFile("vis_old.pxl", LONG_ID_SCHEMA, new long[]{0, 1}, true, new long[]{100, 100}); long[] fileIds = registerTestFiles( new String[]{"vis_old.pxl", "vis_new_temp.pxl"}, - new File.Type[]{File.Type.REGULAR, File.Type.TEMPORARY}, + new File.Type[]{File.Type.REGULAR, File.Type.TEMPORARY_GC}, new int[]{1, 1}, new long[]{0, 0}, new long[]{1, 1}); long oldFileId = fileIds[0]; long tempFileId = fileIds[1]; @@ -1888,7 +1962,7 @@ public void testAtomicSwap_temporaryInvisibleViaGetFiles() throws Exception } assertTrue("REGULAR old file should be visible via getFiles before swap", beforeIds.contains(oldFileId)); - assertFalse("TEMPORARY new file must NOT be visible via getFiles before swap", + assertFalse("TEMPORARY_GC new file must NOT be visible via getFiles before swap", beforeIds.contains(tempFileId)); metadataService.atomicSwapFiles(tempFileId, Collections.singletonList(oldFileId)); @@ -1905,6 +1979,352 @@ public void testAtomicSwap_temporaryInvisibleViaGetFiles() throws Exception afterIds.contains(oldFileId)); } + // ----------------------------------------------------------------------- + // Coverage for getFiles(pathId) REGULAR-only enumeration. + // ----------------------------------------------------------------------- + + /** + * A path containing REGULAR and non-REGULAR FILE_TYPE values returns only REGULAR entries. + */ + @Test + public void testGetFiles_mixedAllFileTypes_onlyRegular() throws Exception + { + long regularId = -1L; + long tempId = -1L; + long nonRegularPositiveId = -1L; + long negativeId = -1L; + long extremeId = -1L; + try + { + String suffix = Long.toString(System.nanoTime()); + regularId = registerTestFile("mix_regular_" + suffix + ".pxl", + File.Type.REGULAR, 1, 0L, 1L); + tempId = registerTestFile("mix_temp_" + suffix + ".pxl", + File.Type.TEMPORARY_INGEST, 1, 0L, 1L); + nonRegularPositiveId = insertRawFileWithType("mix_non_regular_" + suffix + ".pxl", + File.Type.TEMPORARY_GC.getNumber(), 1, 0L, 1L); + negativeId = insertRawFileWithType("mix_negative_" + suffix + ".pxl", + -2, 1, 0L, 1L); + extremeId = insertRawFileWithType("mix_extreme_max_" + suffix + ".pxl", + Integer.MAX_VALUE, 1, 0L, 1L); + + List files = metadataService.getFiles(testPathId); + Set visible = new HashSet<>(); + for (File f : files) + { + assertEquals("getFiles must only emit REGULAR", + File.Type.REGULAR, f.getType()); + visible.add(f.getId()); + } + assertTrue("REGULAR member of the mix must be visible", + visible.contains(regularId)); + assertFalse("TEMPORARY_INGEST (FILE_TYPE=0) must be hidden", + visible.contains(tempId)); + assertFalse("non-REGULAR positive FILE_TYPE must be hidden", + visible.contains(nonRegularPositiveId)); + assertFalse("negative FILE_TYPE must be hidden", + visible.contains(negativeId)); + assertFalse("Integer.MAX_VALUE FILE_TYPE must be hidden", + visible.contains(extremeId)); + } + finally + { + List cleanup = new ArrayList<>(); + if (regularId > 0) cleanup.add(regularId); + if (tempId > 0) cleanup.add(tempId); + if (nonRegularPositiveId > 0) cleanup.add(nonRegularPositiveId); + if (negativeId > 0) cleanup.add(negativeId); + if (extremeId > 0) cleanup.add(extremeId); + if (!cleanup.isEmpty()) metadataService.deleteFiles(cleanup); + } + } + + // ------------------------------------------------------------------------- + // c01.1 regression — RETIRED is a new File.Type and must be invisible to + // query-time enumeration just like the two TEMPORARY_* states. These tests + // pin down the contract that the DAO filters FILE_TYPE = REGULAR and nothing + // else, so future refactors cannot accidentally widen the visible set. + // ------------------------------------------------------------------------- + + + + /** + * Exhaustive coverage: for every defined non-REGULAR {@link File.Type}, getFiles must + * exclude that file. Using {@link File.Type#values()} guards against future enum + * additions silently leaking into query results. + */ + @Test + public void testGetFiles_allNonRegularTypes_allHidden() throws Exception + { + List registeredIds = new ArrayList<>(); + long regularId = -1L; + try + { + String suffix = Long.toString(System.nanoTime()); + regularId = registerTestFile("all_types_regular_" + suffix + ".pxl", + File.Type.REGULAR, 1, 0L, 1L); + + // Register one file per non-REGULAR type, including RETIRED. + Set nonRegularIds = new HashSet<>(); + for (File.Type t : File.Type.values()) + { + if (t == File.Type.REGULAR) continue; + long id = insertRawFileWithType( + "all_types_" + t + "_" + suffix + ".pxl", + t.getNumber(), 1, 0L, 1L); + registeredIds.add(id); + nonRegularIds.add(id); + } + registeredIds.add(regularId); + + List visible = metadataService.getFiles(testPathId); + Set visibleIds = new HashSet<>(); + for (File f : visible) + { + assertEquals("every visible file must carry FILE_TYPE = REGULAR", + File.Type.REGULAR, f.getType()); + visibleIds.add(f.getId()); + } + assertTrue("the seed REGULAR file must be visible", + visibleIds.contains(regularId)); + for (long id : nonRegularIds) + { + assertFalse("non-REGULAR file (id=" + id + ") leaked into getFiles", + visibleIds.contains(id)); + } + } + finally + { + if (!registeredIds.isEmpty()) metadataService.deleteFiles(registeredIds); + } + } + + /** + * After the swap of a TEMPORARY_GC -> REGULAR, a RETIRED tombstone for the *old* file + * (i.e. the same file ids that were just deleted) cannot pollute the new visible set + * even if the catalog still carries unrelated RETIRED entries on the same path. + */ + @Test + public void testGetFiles_retiredCoexistsWithFreshlyPromoted() throws Exception + { + long oldRegularId = -1L; + long tempGcId = -1L; + long retiredCoexistingId = -1L; + try + { + String suffix = Long.toString(System.nanoTime()); + + // Pre-existing RETIRED file on the same path. This must remain hidden + // throughout the entire scenario. + retiredCoexistingId = insertRawFileWithType( + "coexist_retired_" + suffix + ".pxl", + File.Type.RETIRED.getNumber(), 1, 0L, 1L); + + // The classic swap pair. + oldRegularId = registerTestFile("coexist_old_regular_" + suffix + ".pxl", + File.Type.REGULAR, 1, 0L, 1L); + tempGcId = registerTestFile("coexist_new_temp_gc_" + suffix + ".pxl", + File.Type.TEMPORARY_GC, 1, 0L, 1L); + + // Before swap: only oldRegular visible; RETIRED + TEMPORARY_GC hidden. + Set beforeIds = new HashSet<>(); + for (File f : metadataService.getFiles(testPathId)) beforeIds.add(f.getId()); + assertTrue("old REGULAR must be visible before swap", + beforeIds.contains(oldRegularId)); + assertFalse("RETIRED tombstone must be hidden before swap", + beforeIds.contains(retiredCoexistingId)); + assertFalse("TEMPORARY_GC must be hidden before swap", + beforeIds.contains(tempGcId)); + + metadataService.atomicSwapFiles(tempGcId, Collections.singletonList(oldRegularId)); + + // After swap: tempGcId is now REGULAR (visible); old REGULAR is gone; the + // coexisting RETIRED file must STILL be hidden (the swap did not promote it). + Set afterIds = new HashSet<>(); + for (File f : metadataService.getFiles(testPathId)) + { + assertEquals("getFiles must only emit REGULAR after swap", + File.Type.REGULAR, f.getType()); + afterIds.add(f.getId()); + } + assertTrue("freshly-promoted file must be visible after swap", + afterIds.contains(tempGcId)); + assertFalse("the deleted old REGULAR must be gone after swap", + afterIds.contains(oldRegularId)); + assertFalse("the unrelated RETIRED tombstone must remain hidden after swap", + afterIds.contains(retiredCoexistingId)); + + // After the promote, the old file ids are deleted — clear the local handle so + // the cleanup block below does not double-delete a non-existent row. + oldRegularId = -1L; + } + finally + { + List cleanup = new ArrayList<>(); + if (oldRegularId > 0) cleanup.add(oldRegularId); + if (tempGcId > 0) cleanup.add(tempGcId); + if (retiredCoexistingId > 0) cleanup.add(retiredCoexistingId); + if (!cleanup.isEmpty()) metadataService.deleteFiles(cleanup); + } + } + + /** + * A minimum-size REGULAR file is returned with its catalog fields intact. + */ + @Test + public void testGetFiles_singleRegularMinimumData() throws Exception + { + long fileId = -1L; + try + { + fileId = registerTestFile("min_single_regular_" + System.nanoTime() + ".pxl", + File.Type.REGULAR, 1, 0L, 0L); + List files = metadataService.getFiles(testPathId); + File found = null; + for (File f : files) + { + if (f.getId() == fileId) + { + found = f; + } + assertEquals("every returned entry must be REGULAR", + File.Type.REGULAR, f.getType()); + } + assertNotNull("the single REGULAR minimum-data file must be visible", found); + assertEquals("type must be REGULAR", File.Type.REGULAR, found.getType()); + assertEquals("numRowGroup of minimum file must be 1", 1, found.getNumRowGroup()); + assertEquals("minRowId of minimum file must be 0", 0L, found.getMinRowId()); + assertEquals("maxRowId of minimum file must be 0", 0L, found.getMaxRowId()); + } + finally + { + if (fileId > 0) + { + metadataService.deleteFiles(Collections.singletonList(fileId)); + } + } + } + + /** + * A deleted REGULAR file is no longer returned by {@code getFiles}. + */ + @Test + public void testGetFiles_deletedRegular_notVisible() throws Exception + { + long regularId = registerTestFile("delete_visibility_" + System.nanoTime() + ".pxl", + File.Type.REGULAR, 1, 0L, 1L); + + List beforeDelete = metadataService.getFiles(testPathId); + Set beforeIds = new HashSet<>(); + for (File f : beforeDelete) beforeIds.add(f.getId()); + assertTrue("REGULAR file must be visible before delete", + beforeIds.contains(regularId)); + + metadataService.deleteFiles(Collections.singletonList(regularId)); + + List afterDelete = metadataService.getFiles(testPathId); + for (File f : afterDelete) + { + assertFalse("deleted REGULAR file must no longer be visible", + f.getId() == regularId); + } + } + + /** + * Concurrent readers observe a consistent REGULAR-only result. + */ + @Test + public void testGetFiles_concurrentReaders_consistentRegularOnly() throws Exception + { + long regularId = -1L; + long tempId = -1L; + long nonRegularPositiveId = -1L; + ExecutorService pool = null; + try + { + String suffix = Long.toString(System.nanoTime()); + regularId = registerTestFile("conc_regular_" + suffix + ".pxl", + File.Type.REGULAR, 1, 0L, 1L); + tempId = registerTestFile("conc_temp_" + suffix + ".pxl", + File.Type.TEMPORARY_INGEST, 1, 0L, 1L); + nonRegularPositiveId = insertRawFileWithType("conc_non_regular_" + suffix + ".pxl", + File.Type.TEMPORARY_GC.getNumber(), 1, 0L, 1L); + + final int threads = 8; + final int iterations = 16; + pool = Executors.newFixedThreadPool(threads); + CyclicBarrier startGate = new CyclicBarrier(threads); + AtomicInteger leakedTemporary = new AtomicInteger(); + AtomicInteger leakedNonRegular = new AtomicInteger(); + AtomicInteger missingRegular = new AtomicInteger(); + + List> futures = new ArrayList<>(); + final long pinnedRegular = regularId; + final long pinnedTemp = tempId; + final long pinnedNonRegular = nonRegularPositiveId; + for (int t = 0; t < threads; t++) + { + futures.add(CompletableFuture.runAsync(() -> + { + try + { + startGate.await(); + for (int i = 0; i < iterations; i++) + { + List snapshot = metadataService.getFiles(testPathId); + boolean sawRegular = false; + for (File f : snapshot) + { + if (f.getType() != File.Type.REGULAR) + { + leakedNonRegular.incrementAndGet(); + } + if (f.getId() == pinnedRegular) sawRegular = true; + if (f.getId() == pinnedTemp) leakedTemporary.incrementAndGet(); + if (f.getId() == pinnedNonRegular) leakedNonRegular.incrementAndGet(); + } + if (!sawRegular) missingRegular.incrementAndGet(); + } + } + catch (Exception e) + { + throw new RuntimeException(e); + } + }, pool)); + } + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .get(30, java.util.concurrent.TimeUnit.SECONDS); + + assertEquals("no concurrent reader may observe a TEMPORARY_INGEST file", + 0, leakedTemporary.get()); + assertEquals("no concurrent reader may observe a non-REGULAR file", + 0, leakedNonRegular.get()); + assertEquals("every concurrent reader must observe the REGULAR file", + 0, missingRegular.get()); + + // A follow-up call should remain REGULAR-only after the concurrent burst. + List followUp = metadataService.getFiles(testPathId); + assertNotNull("follow-up getFiles must not return null", followUp); + for (File f : followUp) + { + assertEquals("follow-up entries must all be REGULAR", + File.Type.REGULAR, f.getType()); + } + } + finally + { + if (pool != null) + { + pool.shutdownNow(); + } + List cleanup = new ArrayList<>(); + if (regularId > 0) cleanup.add(regularId); + if (tempId > 0) cleanup.add(tempId); + if (nonRegularPositiveId > 0) cleanup.add(nonRegularPositiveId); + if (!cleanup.isEmpty()) metadataService.deleteFiles(cleanup); + } + } + /** * Multiple serial swaps: Storage GC processes FileGroups serially on a single * thread, so {@code atomicSwapFiles} is never called concurrently in production. @@ -1931,7 +2351,7 @@ public void testAtomicSwap_multipleSerialSwaps() throws Exception long[] pair = registerTestFiles( new String[]{oldName, newName}, - new File.Type[]{File.Type.REGULAR, File.Type.TEMPORARY}, + new File.Type[]{File.Type.REGULAR, File.Type.TEMPORARY_GC}, new int[]{1, 1}, new long[]{0, 0}, new long[]{0, 0}); oldFileIds[i] = pair[0]; newFileIds[i] = pair[1]; @@ -1970,7 +2390,7 @@ public void testAtomicSwap_partialOldFilesAlreadyGone() throws Exception metadataService.deleteFiles(Collections.singletonList(oldIds[0])); assertFileGone(oldIds[0], "old1 should be gone before swap"); - long newFileId = registerTestFile("partial_new.pxl", File.Type.TEMPORARY, 1, 0, 1); + long newFileId = registerTestFile("partial_new.pxl", File.Type.TEMPORARY_GC, 1, 0, 1); metadataService.atomicSwapFiles(newFileId, Arrays.asList(oldIds[0], oldIds[1])); assertFileRegular(newFileId, "New file must be REGULAR"); @@ -1979,7 +2399,7 @@ public void testAtomicSwap_partialOldFilesAlreadyGone() throws Exception /** * Rollback after rewrite + dual-write: verifies that Visibility entries for the new - * file are removed, dual-write is unregistered, the TEMPORARY catalog entry is deleted, + * file are removed, dual-write is unregistered, the TEMPORARY_GC catalog entry is deleted, * and the physical file is cleaned up. */ @Test @@ -2063,7 +2483,7 @@ public void testAtomicSwap_delayedCleanup() throws Exception * Phase 3 (ts=200, dual-write active): delete row 3 → propagated to both files * Sync visibility → export + coord-transform + import * Phase 4 (ts=300, post-sync, dual-write still active): delete row 5 - * Commit → atomic swap (TEMPORARY→REGULAR), old file removed from catalog + * Commit -> atomic swap (TEMPORARY_GC -> REGULAR), old file removed from catalog * Verify: multi-snap_ts consistency on new file at ts=100..500 * Verify: old file gone from catalog, new file REGULAR * @@ -2952,6 +3372,27 @@ private long registerTestFile(String name, File.Type type, return id; } + private long insertRawFileWithType(String name, int fileType, + int numRg, long minRow, long maxRow) + throws Exception + { + String sql = "INSERT INTO FILES(FILE_NAME, FILE_TYPE, FILE_NUM_RG, FILE_MIN_ROW_ID, FILE_MAX_ROW_ID, PATHS_PATH_ID) " + + "VALUES (?, ?, ?, ?, ?, ?)"; + try (PreparedStatement pst = MetaDBUtil.Instance().getConnection().prepareStatement(sql)) + { + pst.setString(1, name); + pst.setInt(2, fileType); + pst.setInt(3, numRg); + pst.setLong(4, minRow); + pst.setLong(5, maxRow); + pst.setLong(6, testPathId); + assertEquals("raw test file insert should affect one row", 1, pst.executeUpdate()); + } + long id = metadataService.getFileId(testOrderedPathUri + "/" + name); + assertTrue(name + " must have valid id", id > 0); + return id; + } + private long[] registerTestFiles(String[] names, File.Type[] types, int[] numRgs, long[] minRows, long[] maxRows) throws Exception @@ -3719,6 +4160,53 @@ void processFileGroups(List fileGroups, long safeGcTs, } } + /** + * StorageGarbageCollector subclass that records the boundaries between + * {@code runStorageGC}'s candidate calculation, scan, bitmap trimming, and + * process phases without touching real metadata or Pixels files. + */ + static class TrackingRunStorageGC extends StorageGarbageCollector + { + private final List groupsToReturn; + boolean scanCalled; + boolean processCalled; + RuntimeException processFailure; + Set capturedCandidateFileIds = Collections.emptySet(); + List capturedFileGroups = Collections.emptyList(); + long capturedSafeGcTs = Long.MIN_VALUE; + Set bitmapKeysSeenByProcess = Collections.emptySet(); + + TrackingRunStorageGC(List groupsToReturn) + { + super(null, null, 0.5, 0L, Integer.MAX_VALUE, 10, + 1048576, EncodingLevel.EL2, 86_400_000L); + this.groupsToReturn = groupsToReturn; + } + + @Override + List scanAndGroupFiles(Set candidateFileIds, + Map fileStats) + { + this.scanCalled = true; + this.capturedCandidateFileIds = new HashSet<>(candidateFileIds); + return groupsToReturn; + } + + @Override + void processFileGroups(List fileGroups, long safeGcTs, + Map gcSnapshotBitmaps) + { + this.processCalled = true; + this.capturedFileGroups = new ArrayList<>(fileGroups); + this.capturedSafeGcTs = safeGcTs; + this.bitmapKeysSeenByProcess = new HashSet<>(gcSnapshotBitmaps.keySet()); + if (processFailure != null) + { + throw processFailure; + } + } + } + /** * StorageGarbageCollector subclass where {@code rewriteFileGroup} throws on * the first call and succeeds (cleaning up bitmaps) on subsequent calls. diff --git a/proto/metadata.proto b/proto/metadata.proto index 575b868918..b21556a102 100644 --- a/proto/metadata.proto +++ b/proto/metadata.proto @@ -174,8 +174,10 @@ message Path { message File { enum Type { - TEMPORARY = 0; + TEMPORARY_INGEST = 0; REGULAR = 1; + TEMPORARY_GC = 2; + RETIRED = 3; } uint64 id = 1; string name = 2; @@ -184,6 +186,7 @@ message File { uint64 minRowId = 5; uint64 maxRowId = 6; uint64 pathId = 7; + optional uint64 cleanupAt = 8; } message SchemaVersion { @@ -680,6 +683,7 @@ message AddFilesResponse { } message GetFilesRequest { + // Query-visible REGULAR file enumeration. RequestHeader header = 1; uint64 pathId = 2; } @@ -706,7 +710,7 @@ message GetFileTypeRequest { message GetFileTypeResponse { ResponseHeader header = 1; - File.Type fileType = 2; // the type of the file, e.g., REGULAR or EMPTY + File.Type fileType = 2; // the type of the file, e.g., REGULAR or RETIRED } message UpdateFileRequest { @@ -844,4 +848,4 @@ message DropViewRequest { message DropViewResponse { ResponseHeader header = 1; } -// end request/response definition for rpc services \ No newline at end of file +// end request/response definition for rpc services diff --git a/scripts/sql/metadata_schema.sql b/scripts/sql/metadata_schema.sql index 3f077e4417..c3b0c5a7e9 100644 --- a/scripts/sql/metadata_schema.sql +++ b/scripts/sql/metadata_schema.sql @@ -318,15 +318,17 @@ CREATE TABLE IF NOT EXISTS `pixels_metadata`.`PEER_PATHS` ( CREATE TABLE IF NOT EXISTS `pixels_metadata`.`FILES` ( `FILE_ID` BIGINT NOT NULL AUTO_INCREMENT, `FILE_NAME` VARCHAR(128) NOT NULL, - `FILE_TYPE` TINYINT NOT NULL COMMENT "Valid value can be 0 (temporary), or 1 (regular).", + `FILE_TYPE` TINYINT NOT NULL COMMENT "Valid value can be 0 (temporary ingest), 1 (regular), 2 (temporary gc), or 3 (retired).", `FILE_NUM_RG` INT NOT NULL, `FILE_MIN_ROW_ID` BIGINT NOT NULL, `FILE_MAX_ROW_ID` BIGINT NOT NULL, + `FILE_CLEANUP_AT` BIGINT NULL COMMENT "Earliest cleanup deadline in epoch milliseconds; meaningful only when FILE_TYPE = 3 (retired).", `PATHS_PATH_ID` BIGINT NOT NULL, PRIMARY KEY (`FILE_ID`), INDEX `fk_FILES_PATHS_idx` (`PATHS_PATH_ID` ASC), UNIQUE INDEX `PATH_ID_FILE_NAME_UNIQUE` (`PATHS_PATH_ID` ASC, `FILE_NAME` ASC), INDEX `FILE_ROW_ID_INDEX` USING BTREE (`FILE_MIN_ROW_ID`, `FILE_MAX_ROW_ID`), + INDEX `FILE_CLEANUP_AT_INDEX` USING BTREE (`FILE_TYPE`, `FILE_CLEANUP_AT`), CONSTRAINT `fk_FILES_PATHS` FOREIGN KEY (`PATHS_PATH_ID`) REFERENCES `pixels_metadata`.`PATHS` (`PATH_ID`)