/**
   * @return the total key count in the files being merged
   * @throws IOException
   */
  private long prepareForMerge() throws IOException {
    LOG.info("Merging " + inputFileNames);
    LOG.info("Using block size: " + blockSize);
    inputStoreFiles = new ArrayList<StoreFile>();

    long maxKeyCount = 0;
    for (String fileName : inputFileNames) {
      Path filePath = new Path(fileName);

      // Open without caching.
      StoreFile sf = openStoreFile(filePath, false);
      sf.createReader();
      inputStoreFiles.add(sf);

      StoreFile.Reader r = sf.getReader();
      if (r != null) {
        long keyCount = r.getFilterEntries();
        maxKeyCount += keyCount;
        LOG.info(
            "Compacting: "
                + sf
                + "; keyCount = "
                + keyCount
                + "; Bloom Type = "
                + r.getBloomFilterType().toString()
                + "; Size = "
                + StringUtils.humanReadableInt(r.length()));
      }
    }
    return maxKeyCount;
  }
Exemplo n.º 2
0
 private static CompactionRequest createDummyRequest() throws Exception {
   // "Files" are totally unused, it's Scanner class below that gives compactor fake KVs.
   // But compaction depends on everything under the sun, so stub everything with dummies.
   StoreFile sf = mock(StoreFile.class);
   StoreFile.Reader r = mock(StoreFile.Reader.class);
   when(r.length()).thenReturn(1L);
   when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
   when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
   when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong()))
       .thenReturn(mock(StoreFileScanner.class));
   when(sf.getReader()).thenReturn(r);
   when(sf.createReader()).thenReturn(r);
   return new CompactionRequest(Arrays.asList(sf));
 }
Exemplo n.º 3
0
 /**
  * Return an array of scanners corresponding to the given set of store files, And set the
  * ScanQueryMatcher for each store file scanner for further optimization
  */
 public static List<StoreFileScanner> getScannersForStoreFiles(
     Collection<StoreFile> files,
     boolean cacheBlocks,
     boolean isCompaction,
     ScanQueryMatcher matcher)
     throws IOException {
   List<StoreFileScanner> scanners = new ArrayList<StoreFileScanner>(files.size());
   for (StoreFile file : files) {
     StoreFile.Reader r = file.createReader();
     StoreFileScanner scanner = r.getStoreFileScanner(cacheBlocks, isCompaction);
     scanner.setScanQueryMatcher(matcher);
     scanners.add(scanner);
   }
   return scanners;
 }
    @Override
    public Boolean call() throws Exception {
      Thread.currentThread().setName("reader " + readerId);
      Random rand = new Random();
      StoreFileScanner scanner = reader.getStoreFileScanner(true, pread);

      while (System.currentTimeMillis() < endTime) {
        byte[] row = createRandomRow(rand, firstRow, lastRow);
        KeyValue kvToSeek = new KeyValue(row, family, createRandomQualifier(rand));
        if (rand.nextDouble() < 0.0001) {
          LOG.info("kvToSeek=" + kvToSeek);
        }
        boolean seekResult;
        try {
          seekResult = scanner.seek(kvToSeek);
        } catch (IOException ex) {
          throw new IOException("Seek failed for key " + kvToSeek + ", pread=" + pread, ex);
        }
        numSeeks.incrementAndGet();
        if (!seekResult) {
          error("Seek returned false for row " + Bytes.toStringBinary(row));
          return false;
        }
        for (int i = 0; i < rand.nextInt(10) + 1; ++i) {
          KeyValue kv = scanner.next();
          numKV.incrementAndGet();
          if (i == 0 && kv == null) {
            error(
                "scanner.next() returned null at the first iteration for "
                    + "row "
                    + Bytes.toStringBinary(row));
            return false;
          }
          if (kv == null) break;

          String keyHashStr = MD5Hash.getMD5AsHex(kv.getKey());
          keysRead.add(keyHashStr);
          totalBytes.addAndGet(kv.getLength());
        }
      }

      return true;
    }
Exemplo n.º 5
0
 @Override
 public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
   return reader.passesTimerangeFilter(scan, oldestUnexpiredTS)
       && reader.passesBloomFilter(scan, columns);
 }
Exemplo n.º 6
0
  /**
   * Pretend we have done a seek but don't do it yet, if possible. The hope is that we find
   * requested columns in more recent files and won't have to seek in older files. Creates a fake
   * key/value with the given row/column and the highest (most recent) possible timestamp we might
   * get from this file. When users of such "lazy scanner" need to know the next KV precisely (e.g.
   * when this scanner is at the top of the heap), they run {@link #enforceSeek()}.
   *
   * <p>Note that this function does guarantee that the current KV of this scanner will be advanced
   * to at least the given KV. Because of this, it does have to do a real seek in cases when the
   * seek timestamp is older than the highest timestamp of the file, e.g. when we are trying to seek
   * to the next row/column and use OLDEST_TIMESTAMP in the seek key.
   */
  @Override
  public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom) throws IOException {
    if (kv.getFamilyLength() == 0) {
      useBloom = false;
    }

    boolean haveToSeek = true;
    if (useBloom) {
      // check ROWCOL Bloom filter first.
      if (reader.getBloomFilterType() == StoreFile.BloomType.ROWCOL) {
        haveToSeek =
            reader.passesGeneralBloomFilter(
                kv.getBuffer(),
                kv.getRowOffset(),
                kv.getRowLength(),
                kv.getBuffer(),
                kv.getQualifierOffset(),
                kv.getQualifierLength());
      } else if (this.matcher != null && !matcher.hasNullColumnInQuery() && kv.isDeleteFamily()) {
        // if there is no such delete family kv in the store file,
        // then no need to seek.
        haveToSeek =
            reader.passesDeleteFamilyBloomFilter(
                kv.getBuffer(), kv.getRowOffset(), kv.getRowLength());
      }
    }

    delayedReseek = forward;
    delayedSeekKV = kv;

    if (haveToSeek) {
      // This row/column might be in this store file (or we did not use the
      // Bloom filter), so we still need to seek.
      realSeekDone = false;
      long maxTimestampInFile = reader.getMaxTimestamp();
      long seekTimestamp = kv.getTimestamp();
      if (seekTimestamp > maxTimestampInFile) {
        // Create a fake key that is not greater than the real next key.
        // (Lower timestamps correspond to higher KVs.)
        // To understand this better, consider that we are asked to seek to
        // a higher timestamp than the max timestamp in this file. We know that
        // the next point when we have to consider this file again is when we
        // pass the max timestamp of this file (with the same row/column).
        cur = kv.createFirstOnRowColTS(maxTimestampInFile);
      } else {
        // This will be the case e.g. when we need to seek to the next
        // row/column, and we don't know exactly what they are, so we set the
        // seek key's timestamp to OLDEST_TIMESTAMP to skip the rest of this
        // row/column.
        enforceSeek();
      }
      return cur != null;
    }

    // Multi-column Bloom filter optimization.
    // Create a fake key/value, so that this scanner only bubbles up to the top
    // of the KeyValueHeap in StoreScanner after we scanned this row/column in
    // all other store files. The query matcher will then just skip this fake
    // key/value and the store scanner will progress to the next column. This
    // is obviously not a "real real" seek, but unlike the fake KV earlier in
    // this method, we want this to be propagated to ScanQueryMatcher.
    cur = kv.createLastOnRowCol();

    realSeekDone = true;
    return true;
  }
Exemplo n.º 7
0
 @Override
 public long getSequenceID() {
   return reader.getSequenceID();
 }
  public boolean runRandomReadWorkload() throws IOException {
    if (inputFileNames.size() != 1) {
      throw new IOException("Need exactly one input file for random reads: " + inputFileNames);
    }

    Path inputPath = new Path(inputFileNames.get(0));

    // Make sure we are using caching.
    StoreFile storeFile = openStoreFile(inputPath, true);

    StoreFile.Reader reader = storeFile.createReader();

    LOG.info("First key: " + Bytes.toStringBinary(reader.getFirstKey()));
    LOG.info("Last key: " + Bytes.toStringBinary(reader.getLastKey()));

    KeyValue firstKV = KeyValue.createKeyValueFromKey(reader.getFirstKey());
    firstRow = firstKV.getRow();

    KeyValue lastKV = KeyValue.createKeyValueFromKey(reader.getLastKey());
    lastRow = lastKV.getRow();

    byte[] family = firstKV.getFamily();
    if (!Bytes.equals(family, lastKV.getFamily())) {
      LOG.error(
          "First and last key have different families: "
              + Bytes.toStringBinary(family)
              + " and "
              + Bytes.toStringBinary(lastKV.getFamily()));
      return false;
    }

    if (Bytes.equals(firstRow, lastRow)) {
      LOG.error(
          "First and last row are the same, cannot run read workload: "
              + "firstRow="
              + Bytes.toStringBinary(firstRow)
              + ", "
              + "lastRow="
              + Bytes.toStringBinary(lastRow));
      return false;
    }

    ExecutorService exec = Executors.newFixedThreadPool(numReadThreads + 1);
    int numCompleted = 0;
    int numFailed = 0;
    try {
      ExecutorCompletionService<Boolean> ecs = new ExecutorCompletionService<Boolean>(exec);
      endTime = System.currentTimeMillis() + 1000 * durationSec;
      boolean pread = true;
      for (int i = 0; i < numReadThreads; ++i) ecs.submit(new RandomReader(i, reader, pread));
      ecs.submit(new StatisticsPrinter());
      Future<Boolean> result;
      while (true) {
        try {
          result = ecs.poll(endTime + 1000 - System.currentTimeMillis(), TimeUnit.MILLISECONDS);
          if (result == null) break;
          try {
            if (result.get()) {
              ++numCompleted;
            } else {
              ++numFailed;
            }
          } catch (ExecutionException e) {
            LOG.error("Worker thread failure", e.getCause());
            ++numFailed;
          }
        } catch (InterruptedException ex) {
          LOG.error("Interrupted after " + numCompleted + " workers completed");
          Thread.currentThread().interrupt();
          continue;
        }
      }
    } finally {
      storeFile.closeReader(true);
      exec.shutdown();

      BlockCache c = cacheConf.getBlockCache();
      if (c != null) {
        c.shutdown();
      }
    }
    LOG.info("Worker threads completed: " + numCompleted);
    LOG.info("Worker threads failed: " + numFailed);
    return true;
  }
  /**
   * Do a minor/major compaction on an explicit set of storefiles from a Store.
   *
   * @param request the requested compaction that contains all necessary information to complete the
   *     compaction (i.e. the store, the files, etc.)
   * @return Product of compaction or null if all cells expired or deleted and nothing made it
   *     through the compaction.
   * @throws IOException
   */
  StoreFile.Writer compact(CompactionRequest request, long maxId) throws IOException {
    // Calculate maximum key count after compaction (for blooms)
    // Also calculate earliest put timestamp if major compaction
    int maxKeyCount = 0;
    long earliestPutTs = HConstants.LATEST_TIMESTAMP;
    long maxMVCCReadpoint = 0;

    // pull out the interesting things from the CR for ease later
    final Store store = request.getStore();
    final boolean majorCompaction = request.isMajor();
    final List<StoreFile> filesToCompact = request.getFiles();

    for (StoreFile file : filesToCompact) {
      StoreFile.Reader r = file.getReader();
      if (r == null) {
        LOG.warn("Null reader for " + file.getPath());
        continue;
      }
      // NOTE: getFilterEntries could cause under-sized blooms if the user
      //       switches bloom type (e.g. from ROW to ROWCOL)
      long keyCount =
          (r.getBloomFilterType() == store.getFamily().getBloomFilterType())
              ? r.getFilterEntries()
              : r.getEntries();
      maxKeyCount += keyCount;
      // Calculate the maximum MVCC readpoint used in any of the involved files
      Map<byte[], byte[]> fileInfo = r.loadFileInfo();
      byte[] tmp = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
      if (tmp != null) {
        maxMVCCReadpoint = Math.max(maxMVCCReadpoint, Bytes.toLong(tmp));
      }
      // For major compactions calculate the earliest put timestamp
      // of all involved storefiles. This is used to remove
      // family delete marker during the compaction.
      if (majorCompaction) {
        tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
        if (tmp == null) {
          // There's a file with no information, must be an old one
          // assume we have very old puts
          earliestPutTs = HConstants.OLDEST_TIMESTAMP;
        } else {
          earliestPutTs = Math.min(earliestPutTs, Bytes.toLong(tmp));
        }
      }
      if (LOG.isDebugEnabled()) {
        LOG.debug(
            "Compacting "
                + file
                + ", keycount="
                + keyCount
                + ", bloomtype="
                + r.getBloomFilterType().toString()
                + ", size="
                + StringUtils.humanReadableInt(r.length())
                + ", encoding="
                + r.getHFileReader().getEncodingOnDisk()
                + (majorCompaction ? ", earliestPutTs=" + earliestPutTs : ""));
      }
    }

    // keep track of compaction progress
    this.progress = new CompactionProgress(maxKeyCount);
    // Get some configs
    int compactionKVMax = getConf().getInt("hbase.hstore.compaction.kv.max", 10);
    Compression.Algorithm compression = store.getFamily().getCompression();
    // Avoid overriding compression setting for major compactions if the user
    // has not specified it separately
    Compression.Algorithm compactionCompression =
        (store.getFamily().getCompactionCompression() != Compression.Algorithm.NONE)
            ? store.getFamily().getCompactionCompression()
            : compression;

    // For each file, obtain a scanner:
    List<StoreFileScanner> scanners =
        StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, false, true);

    // Make the instantiation lazy in case compaction produces no product; i.e.
    // where all source cells are expired or deleted.
    StoreFile.Writer writer = null;
    // Find the smallest read point across all the Scanners.
    long smallestReadPoint = store.getHRegion().getSmallestReadPoint();
    MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
    try {
      InternalScanner scanner = null;
      try {
        if (store.getHRegion().getCoprocessorHost() != null) {
          scanner =
              store
                  .getHRegion()
                  .getCoprocessorHost()
                  .preCompactScannerOpen(
                      store,
                      scanners,
                      majorCompaction ? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT,
                      earliestPutTs,
                      request);
        }
        if (scanner == null) {
          Scan scan = new Scan();
          scan.setMaxVersions(store.getFamily().getMaxVersions());
          /* Include deletes, unless we are doing a major compaction */
          scanner =
              new StoreScanner(
                  store,
                  store.getScanInfo(),
                  scan,
                  scanners,
                  majorCompaction ? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT,
                  smallestReadPoint,
                  earliestPutTs);
        }
        if (store.getHRegion().getCoprocessorHost() != null) {
          InternalScanner cpScanner =
              store.getHRegion().getCoprocessorHost().preCompact(store, scanner, request);
          // NULL scanner returned from coprocessor hooks means skip normal processing
          if (cpScanner == null) {
            return null;
          }
          scanner = cpScanner;
        }

        int bytesWritten = 0;
        // since scanner.next() can return 'false' but still be delivering data,
        // we have to use a do/while loop.
        List<KeyValue> kvs = new ArrayList<KeyValue>();
        // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
        boolean hasMore;
        do {
          hasMore = scanner.next(kvs, compactionKVMax);
          if (writer == null && !kvs.isEmpty()) {
            writer =
                store.createWriterInTmp(
                    maxKeyCount,
                    compactionCompression,
                    true,
                    maxMVCCReadpoint >= smallestReadPoint);
          }
          if (writer != null) {
            // output to writer:
            for (KeyValue kv : kvs) {
              if (kv.getMemstoreTS() <= smallestReadPoint) {
                kv.setMemstoreTS(0);
              }
              writer.append(kv);
              // update progress per key
              ++progress.currentCompactedKVs;

              // check periodically to see if a system stop is requested
              if (Store.closeCheckInterval > 0) {
                bytesWritten += kv.getLength();
                if (bytesWritten > Store.closeCheckInterval) {
                  bytesWritten = 0;
                  isInterrupted(store, writer);
                }
              }
            }
          }
          kvs.clear();
        } while (hasMore);
      } finally {
        if (scanner != null) {
          scanner.close();
        }
      }
    } finally {
      if (writer != null) {
        writer.appendMetadata(maxId, majorCompaction);
        writer.close();
      }
    }
    return writer;
  }