Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

New bulk import api failed during test with agitation #1260

Closed
keith-turner opened this issue Jul 9, 2019 · 4 comments
Closed

New bulk import api failed during test with agitation #1260

keith-turner opened this issue Jul 9, 2019 · 4 comments
Labels
blocker This issue blocks any release version labeled on it. bug This issue has been verified to be a bug.
Milestone

Comments

@keith-turner
Copy link
Contributor

keith-turner commented Jul 9, 2019

While running bulk import with agitation on a small EC2 cluster I noticed a blip marker was not going away.

root@muchos> scan -b ~b -e ~c -t accumulo.metadata 
~blip/2/b-0000ldy : []    FATE[284d4940309b8f80]

I looked the master logs for the FATE transaction ID and found that it failed with the following error message.

2019-07-09 19:29:54,633 [zookeeper.DistributedReadWriteLock] INFO : Added lock entry 0 userData 284d4940309b8f80 lockTpye READ

2019-07-09 19:29:56,742 [master.EventCoordinator] INFO : worker4:9997 reported split 2;2a0009;29557, 2;2aaaaaaaaaaaaab;2a0009

2019-07-09 19:29:58,874 [fate.Fate] WARN : Failed to execute Repo, FATE[284d4940309b8f80]
java.util.NoSuchElementException
        at org.apache.accumulo.core.client.RowIterator.next(RowIterator.java:145)
        at org.apache.accumulo.core.client.RowIterator.next(RowIterator.java:35)
        at com.google.common.collect.TransformedIterator.next(TransformedIterator.java:47)
        at org.apache.accumulo.core.metadata.schema.LinkingIterator.next(LinkingIterator.java:131)
        at org.apache.accumulo.core.metadata.schema.LinkingIterator.next(LinkingIterator.java:48)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.findOverlappingTablets(LoadFiles.java:353)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.loadFiles(LoadFiles.java:331)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.isReady(LoadFiles.java:94)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.isReady(LoadFiles.java:71)
        at org.apache.accumulo.master.tableOps.TraceRepo.isReady(TraceRepo.java:42)
        at org.apache.accumulo.fate.Fate$TransactionRunner.run(Fate.java:72)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
        at java.lang.Thread.run(Thread.java:748)
2019-07-09 19:29:58,884 [fate.Fate] INFO : Updated status for Repo with FATE[284d4940309b8f80] to FAILED_IN_PROGRESS
2019-07-09 19:29:58,887 [master.EventCoordinator] INFO : [Normal Tablets]: 2 tablets are UNASSIGNED
2019-07-09 19:29:58,887 [master.EventCoordinator] INFO : [Normal Tablets]: 7 tablets are ASSIGNED
2019-07-09 19:29:58,887 [master.EventCoordinator] INFO : [Normal Tablets]: 5 tablets are HOSTED
2019-07-09 19:29:58,887 [master.EventCoordinator] INFO : [Normal Tablets]: 3 tablets unloaded
2019-07-09 19:29:58,902 [thrift.ProcessFunction] ERROR: Internal error processing waitForFateOperation
java.util.NoSuchElementException
        at org.apache.accumulo.core.client.RowIterator.next(RowIterator.java:145)
        at org.apache.accumulo.core.client.RowIterator.next(RowIterator.java:35)
        at com.google.common.collect.TransformedIterator.next(TransformedIterator.java:47)
        at org.apache.accumulo.core.metadata.schema.LinkingIterator.next(LinkingIterator.java:131)
        at org.apache.accumulo.core.metadata.schema.LinkingIterator.next(LinkingIterator.java:48)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.findOverlappingTablets(LoadFiles.java:353)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.loadFiles(LoadFiles.java:331)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.isReady(LoadFiles.java:94)
        at org.apache.accumulo.master.tableOps.bulkVer2.LoadFiles.isReady(LoadFiles.java:71)
        at org.apache.accumulo.master.tableOps.TraceRepo.isReady(TraceRepo.java:42)
        at org.apache.accumulo.fate.Fate$TransactionRunner.run(Fate.java:72)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
        at java.lang.Thread.run(Thread.java:748)
@keith-turner keith-turner added blocker This issue blocks any release version labeled on it. bug This issue has been verified to be a bug. labels Jul 10, 2019
@keith-turner
Copy link
Contributor Author

I wrote a test that continually creates a table with splits a and scans its metadata while another thread add splits. I found one case where the LinkingIterator misses the first tablet when a split is added at the beginning of the table when it starts scanning. Not sure if this is the cause of the problem seen here though.

@keith-turner
Copy link
Contributor Author

Wrote the following test to stress test the LinkingIterator while researching this issue. The test runs continually scanning the metadata table while a table is splitting. This is to look for issues where the linking iterator does not handle split properly. It did find one issue. Not sure what to do with the test thought as it just runs for a long time.

public class TmpIT extends AccumuloClusterHarness {

  @Override
  public int defaultTimeoutSeconds() {
    return 30;
  }

  @Test
  public void test() throws Exception {

    for (int i = 0; i < 1000; i++) {
      testScanWhileSplitting();
      System.out.println("I " + i);
    }
  }

  private void testScanWhileSplitting() throws AccumuloSecurityException, AccumuloException,
      TableExistsException, TableNotFoundException {
    Random rand = new Random();

    List<Text> initialSplits = new ArrayList<>();

    int initSize = rand.nextInt(1990) + 10;
    for (int i = 0; i < initSize; i++) {
      initialSplits.add(new Text(String.format("%016x", rand.nextLong())));
    }

    Collections.sort(initialSplits);

    // call the batchwriter with buffer of size zero
    String table = getUniqueNames(1)[0];
    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {

      ExecutorService es = Executors.newFixedThreadPool(1);

      NewTableConfiguration ntc = new NewTableConfiguration();
      ntc.withSplits(new TreeSet<>(initialSplits));
      c.tableOperations().create(table, ntc);

      String id = c.tableOperations().tableIdMap().get(table);

      Future<Void> splitFuture = es.submit(() -> {
        List<Text> nextSplits = new ArrayList<>();
        int nextsize = rand.nextInt(1990) + 10;
        for (int i = 0; i < nextsize; i++) {
          nextSplits.add(new Text(String.format("%016x", rand.nextLong())));
        }

        c.tableOperations().addSplits(table, new TreeSet<Text>(nextSplits));
        return null;
      });

      do {
        // TODO in LoadFiles following called overlapping() while building
        Iterator<TabletMetadata> tabletIter = TabletsMetadata.builder().forTable(TableId.of(id))
            .checkConsistency().fetchPrev().fetchLocation().fetchLoaded().build(c).iterator();

        ArrayDeque<Text> splits = new ArrayDeque<Text>(initialSplits);

        Text prev = null;

        while (tabletIter.hasNext() && !splits.isEmpty()) {
          Text split = splits.removeFirst();
          KeyExtent extent = tabletIter.next().getExtent();

          Assert.assertEquals(prev, extent.getPrevEndRow());

          while (extent.getEndRow().compareTo(split) < 0) {
            prev = extent.getEndRow();
            extent = tabletIter.next().getExtent();
            Assert.assertEquals(prev, extent.getPrevEndRow());
          }

          prev = extent.getEndRow();

          Assert.assertEquals(split, extent.getEndRow());
        }

        while (tabletIter.hasNext()) {
          KeyExtent extent = tabletIter.next().getExtent();
          Assert.assertEquals(prev, extent.getPrevEndRow());
          prev = extent.getEndRow();

          if (extent.getEndRow() == null) {
            Assert.assertFalse(tabletIter.hasNext());
          }
        }

        Assert.assertTrue(splits.isEmpty());

        // System.out.println("Finished a pass");

      } while (!splitFuture.isDone());

      es.shutdown();

      c.tableOperations().delete(table);
    }
  }

}

keith-turner added a commit to keith-turner/accumulo that referenced this issue Jul 11, 2019
The LinkingIterator checks the structure of the AccumuloMetadata table
as it is scans.   It was not properly handling a split of the first
tablet.  It was also not ensuring the last tablet for a table was seen.
This patch fixes those two issues.  These issues were discovered while
researching apache#1260, however I am not sure if these issues could have
caused apache#1260.
keith-turner added a commit that referenced this issue Jul 11, 2019
The LinkingIterator checks the structure of the AccumuloMetadata table
as it is scans.   It was not properly handling a split of the first
tablet.  It was also not ensuring the default tablet for a table was seen.
This patch fixes those two issues.  These issues were discovered while
researching #1260, however I am not sure if these issues could have
caused #1260.
keith-turner added a commit to keith-turner/accumulo that referenced this issue Jul 11, 2019
@keith-turner
Copy link
Contributor Author

I suspect that splitting could have been a cause for this issue. Yesterday I ran a test importing 100 directories into a table with a small split threshold (32M). This caused a lot of splits to happen while the imports were running. I did not see any problems. I am currently running a test to import 1000 directories with agitation incorporating some of the recent changes. I opened apache/accumulo-testing#94 because generating the 1000 dirs for import takes quite a while.

@keith-turner
Copy link
Contributor Author

I have done a lot of testing since seeing this and have not been able to reproduce this so far. I am going to close this for now.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
blocker This issue blocks any release version labeled on it. bug This issue has been verified to be a bug.
Projects
None yet
Development

No branches or pull requests

2 participants