コード例 #1
0
  private static void assertFiles(
      String dirPath, Set<String> expectedFiles, boolean excludeNonExistingFiles)
      throws IOException {
    LogTransaction.waitForDeletions();

    File dir = new File(dirPath).getCanonicalFile();
    File[] files = dir.listFiles();
    if (files != null) {
      for (File file : files) {
        if (file.isDirectory()) continue;

        String filePath = file.getPath();
        assertTrue(
            String.format("%s not in [%s]", filePath, expectedFiles),
            expectedFiles.contains(filePath));
        expectedFiles.remove(filePath);
      }
    }

    if (excludeNonExistingFiles) {
      for (String filePath : expectedFiles) {
        File file = new File(filePath);
        if (!file.exists()) expectedFiles.remove(filePath);
      }
    }

    assertTrue(expectedFiles.toString(), expectedFiles.isEmpty());
  }
コード例 #2
0
  @Test
  public void testGetTemporaryFilesThrowsIfCompletingAfterObsoletion() throws Throwable {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
    File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);

    LogTransaction logs = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(logs);

    LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable);

    sstable.markObsolete(tidier);
    sstable.selfRef().release();

    LogTransaction.waitForDeletions();

    try {
      // This should race with the asynchronous deletion of txn log files
      // it should throw because we are violating the requirement that a transaction must
      // finish before deleting files (i.e. releasing sstables)
      getTemporaryFiles(dataFolder);
      fail("Expected runtime exception");
    } catch (RuntimeException e) {
      // pass as long as the cause is not an assertion
      assertFalse(e.getCause() instanceof AssertionError);
    }

    logs.finish();
  }
コード例 #3
0
      protected Throwable doAbort(Throwable accumulate) {
        tidier.abort();
        LogTransaction.waitForDeletions();

        Throwable ret = txnLogs.abort(accumulate);

        sstableNew.selfRef().release();
        sstableOld.selfRef().release();
        return ret;
      }
コード例 #4
0
      protected Throwable doCommit(Throwable accumulate) {
        sstableOld.markObsolete(tidier);
        sstableOld.selfRef().release();
        LogTransaction.waitForDeletions();

        Throwable ret = txnLogs.commit(accumulate);

        sstableNew.selfRef().release();
        return ret;
      }
コード例 #5
0
  @Test
  public void testUntrack() throws Throwable {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
    File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);

    // complete a transaction without keep the new files since they were untracked
    LogTransaction log = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(log);

    log.trackNew(sstableNew);
    log.untrackNew(sstableNew);

    log.finish();

    sstableNew.selfRef().release();
    Thread.sleep(1);
    LogTransaction.waitForDeletions();

    assertFiles(dataFolder.getPath(), Collections.<String>emptySet());
  }
コード例 #6
0
  @Test
  public void testCommitMultipleFolders() throws Throwable {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);

    File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    File dataFolder1 = new File(origiFolder, "1");
    File dataFolder2 = new File(origiFolder, "2");
    Files.createDirectories(dataFolder1.toPath());
    Files.createDirectories(dataFolder2.toPath());

    SSTableReader[] sstables = {
      sstable(dataFolder1, cfs, 0, 128),
      sstable(dataFolder1, cfs, 1, 128),
      sstable(dataFolder2, cfs, 2, 128),
      sstable(dataFolder2, cfs, 3, 128)
    };

    LogTransaction log = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(log);

    LogTransaction.SSTableTidier[] tidiers = {
      log.obsoleted(sstables[0]), log.obsoleted(sstables[2])
    };

    log.trackNew(sstables[1]);
    log.trackNew(sstables[3]);

    log.finish();

    sstables[0].markObsolete(tidiers[0]);
    sstables[2].markObsolete(tidiers[1]);

    Arrays.stream(sstables).forEach(s -> s.selfRef().release());
    LogTransaction.waitForDeletions();

    assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[1].getAllFilePaths()));
    assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[3].getAllFilePaths()));
  }
コード例 #7
0
  @Test
  public void testGetTemporaryFilesMultipleFolders() throws IOException {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);

    File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    File dataFolder1 = new File(origiFolder, "1");
    File dataFolder2 = new File(origiFolder, "2");
    Files.createDirectories(dataFolder1.toPath());
    Files.createDirectories(dataFolder2.toPath());

    SSTableReader[] sstables = {
      sstable(dataFolder1, cfs, 0, 128),
      sstable(dataFolder1, cfs, 1, 128),
      sstable(dataFolder2, cfs, 2, 128),
      sstable(dataFolder2, cfs, 3, 128)
    };

    // they should all have the same number of files since they are created in the same way
    int numSStableFiles = sstables[0].getAllFilePaths().size();

    LogTransaction log = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(log);

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(0, tmpFiles.size());
    }

    LogTransaction.SSTableTidier[] tidiers = {
      log.obsoleted(sstables[0]), log.obsoleted(sstables[2])
    };

    log.trackNew(sstables[1]);
    log.trackNew(sstables[3]);

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(numSStableFiles, tmpFiles.size());
    }

    log.finish();

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(numSStableFiles, tmpFiles.size());
    }

    sstables[0].markObsolete(tidiers[0]);
    sstables[2].markObsolete(tidiers[1]);

    Arrays.stream(sstables).forEach(s -> s.selfRef().release());
    LogTransaction.waitForDeletions();

    for (File dataFolder : new File[] {dataFolder1, dataFolder2}) {
      Set<File> tmpFiles = getTemporaryFiles(dataFolder);
      assertNotNull(tmpFiles);
      assertEquals(0, tmpFiles.size());
    }
  }
コード例 #8
0
  private static void testRemoveUnfinishedLeftovers_multipleFolders_errorConditions(
      Consumer<LogTransaction> modifier, boolean shouldCommit) throws Throwable {
    ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);

    File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
    File dataFolder1 = new File(origiFolder, "1");
    File dataFolder2 = new File(origiFolder, "2");
    Files.createDirectories(dataFolder1.toPath());
    Files.createDirectories(dataFolder2.toPath());

    SSTableReader[] sstables = {
      sstable(dataFolder1, cfs, 0, 128),
      sstable(dataFolder1, cfs, 1, 128),
      sstable(dataFolder2, cfs, 2, 128),
      sstable(dataFolder2, cfs, 3, 128)
    };

    LogTransaction log = new LogTransaction(OperationType.COMPACTION);
    assertNotNull(log);

    LogTransaction.SSTableTidier[] tidiers = {
      log.obsoleted(sstables[0]), log.obsoleted(sstables[2])
    };

    log.trackNew(sstables[1]);
    log.trackNew(sstables[3]);

    // fake some error condition on the txn logs
    modifier.accept(log);

    Arrays.stream(sstables).forEach(s -> s.selfRef().release());

    LogTransaction.removeUnfinishedLeftovers(Arrays.asList(dataFolder1, dataFolder2));
    LogTransaction.waitForDeletions();

    if (shouldCommit) {
      // only new sstables should still be there
      assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[1].getAllFilePaths()));
      assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[3].getAllFilePaths()));
    } else {
      // all files should still be there
      assertFiles(
          dataFolder1.getPath(),
          Sets.newHashSet(
              Iterables.concat(
                  sstables[0].getAllFilePaths(),
                  sstables[1].getAllFilePaths(),
                  Collections.singleton(log.logFilePaths().get(0)))));
      assertFiles(
          dataFolder2.getPath(),
          Sets.newHashSet(
              Iterables.concat(
                  sstables[2].getAllFilePaths(),
                  sstables[3].getAllFilePaths(),
                  Collections.singleton(log.logFilePaths().get(1)))));
    }

    // complete the transaction to avoid LEAK errors
    Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::run);
    log.txnFile().commit(); // just anything to make sure transaction tidier will finish
    assertNull(log.complete(null));
  }
コード例 #9
0
 protected AbstractTransactionalTest.TestableTransaction newTest() throws Exception {
   LogTransaction.waitForDeletions();
   SSTableReader.resetTidying();
   return new TxnTest();
 }