Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 08FFAD259 for ; Sat, 13 Oct 2012 12:51:27 +0000 (UTC) Received: (qmail 83659 invoked by uid 500); 13 Oct 2012 12:51:26 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 83249 invoked by uid 500); 13 Oct 2012 12:51:20 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 83210 invoked by uid 99); 13 Oct 2012 12:51:18 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 13 Oct 2012 12:51:18 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 13 Oct 2012 12:51:14 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id DE272238899C for ; Sat, 13 Oct 2012 12:50:29 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1397821 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/io/hfile/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/regionserver/metrics/ test/java/org/apache/hadoop/hbase/regionserver/ Date: Sat, 13 Oct 2012 12:50:29 -0000 To: commits@hbase.apache.org From: mbautin@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20121013125029.DE272238899C@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: mbautin Date: Sat Oct 13 12:50:29 2012 New Revision: 1397821 URL: http://svn.apache.org/viewvc?rev=1397821&view=rev Log: [jira] [HBASE-6955] [89-fb] Block read time should be in ms, not in ns Author: mbautin Summary: We update block read time in nanoseconds, which is inconsistent with all other latency metrics in the system. Test Plan: Unit tests Reviewers: kannan Differential Revision: https://reviews.facebook.net/D5901 Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1397821&r1=1397820&r2=1397821&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Sat Oct 13 12:50:29 2012 @@ -170,33 +170,34 @@ public class HFile { static final AtomicInteger preadOps = new AtomicInteger(); static final AtomicLong preadTimeNano = new AtomicLong(); - // number of sequential reads - public static final int getReadOps() { + /** + * Get the number of sequential read (seek-and-read) operations and reset it to zero. + */ + public static final int getReadOpsAndReset() { return readOps.getAndSet(0); } - public static final long getReadTimeMs() { + /** + * Get the total time of sequential reads in milliseconds and reset it to zero. + */ + public static final long getReadTimeMsAndReset() { return readTimeNano.getAndSet(0) / 1000000; } - // number of positional reads - public static final int getPreadOps() { + /** + * Get the number of positional read operations and reset it to zero. + */ + public static final int getPreadOpsAndReset() { return preadOps.getAndSet(0); } - public static final long getPreadTimeMs() { + /** + * Get the total time of positional reads in milliseconds and reset it to zero. + */ + public static final long getPreadTimeMsAndReset() { return preadTimeNano.getAndSet(0) / 1000000; } - public static final int getWriteOps() { - return writeOps.getAndSet(0); - } - - public static final long getWriteTimeMs() { - return writeTimeNano.getAndSet(0) / 1000000; - } - - /** * Get the configured bytes per checksum for HFile * if not configured, return the default value @@ -361,9 +362,9 @@ public class HFile { } /** - * Returns the factory to be used to create {@link HFile} writers. Should - * always be {@link HFileWriterV2#WRITER_FACTORY_V2} in production, but - * can also be {@link HFileWriterV1#WRITER_FACTORY_V1} in testing. + * Returns the factory to be used to create {@link HFile} writers. Should always be an instance of + * {@link HFileWriterV2.WriterFactoryV2} in production, but can also be + * {@link HFileWriterV1.WriterFactoryV1} in testing. */ public static final WriterFactory getWriterFactory(Configuration conf, CacheConfig cacheConf) { Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java?rev=1397821&r1=1397820&r2=1397821&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java Sat Oct 13 12:50:29 2012 @@ -25,6 +25,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -221,14 +222,16 @@ public class HFileReaderV2 extends Abstr // Cache Miss, please load. } + final boolean pread = true; // we always do positional reads for meta blocks HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, - blockSize, -1, true); + blockSize, -1, pread); passSchemaMetricsTo(metaBlock); - long delta = System.nanoTime() - startTimeNs; - HFile.preadTimeNano.addAndGet(delta); + long deltaNs = System.nanoTime() - startTimeNs; + HFile.preadTimeNano.addAndGet(deltaNs); HFile.preadOps.incrementAndGet(); - getSchemaMetrics().updateOnCacheMiss(BlockCategory.META, false, delta); + getSchemaMetrics().updateOnCacheMiss(BlockCategory.META, false, + TimeUnit.NANOSECONDS.toMillis(deltaNs)); // Cache the block if (cacheBlock) { @@ -289,7 +292,7 @@ public class HFileReaderV2 extends Abstr IdLock.Entry lockEntry = offsetLock.getLockEntry(dataBlockOffset); try { // Double checking the block cache again within the IdLock - cachedBlock = this.getCachedBlock(cacheKey, cacheBlock, isCompaction, + cachedBlock = this.getCachedBlock(cacheKey, cacheBlock, isCompaction, expectedBlockType, false); if (cachedBlock != null) { return cachedBlock; @@ -305,15 +308,17 @@ public class HFileReaderV2 extends Abstr passSchemaMetricsTo(hfileBlock); BlockCategory blockCategory = hfileBlock.getBlockType().getCategory(); - long delta = System.nanoTime() - startTimeNs; + long deltaNs = System.nanoTime() - startTimeNs; if (pread) { - HFile.preadTimeNano.addAndGet(delta); + HFile.preadTimeNano.addAndGet(deltaNs); HFile.preadOps.incrementAndGet(); } else { - HFile.readTimeNano.addAndGet(delta); + HFile.readTimeNano.addAndGet(deltaNs); HFile.readOps.incrementAndGet(); } - getSchemaMetrics().updateOnCacheMiss(blockCategory, isCompaction, delta); + + getSchemaMetrics().updateOnCacheMiss(blockCategory, isCompaction, + TimeUnit.NANOSECONDS.toMillis(deltaNs)); // Cache the block if necessary if (cacheBlock && cacheConf.shouldCacheBlockOnRead( @@ -332,7 +337,7 @@ public class HFileReaderV2 extends Abstr hfileBlock.getBlockType().getCategory(), hfileBlock.getColumnFamilyName()), onDiskBlockSize); - pData.incLong(ProfilingData.TOTAL_BLOCK_READ_TIME_NS, delta); + pData.incLong(ProfilingData.TOTAL_BLOCK_READ_TIME_NS, deltaNs); } return hfileBlock; } finally { Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1397821&r1=1397820&r2=1397821&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Sat Oct 13 12:50:29 2012 @@ -1,4 +1,4 @@ -/* + /* * Copyright 2010 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java?rev=1397821&r1=1397820&r2=1397821&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java (original) +++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java Sat Oct 13 12:50:29 2012 @@ -38,14 +38,11 @@ import org.apache.hadoop.metrics.jvm.Jvm import org.apache.hadoop.metrics.util.MetricsIntValue; import org.apache.hadoop.metrics.util.MetricsLongValue; import org.apache.hadoop.metrics.util.MetricsRegistry; -import org.apache.hadoop.metrics.util.MetricsTimeVaryingInt; -import org.apache.hadoop.metrics.util.MetricsTimeVaryingLong; import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate; import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.MemoryUsage; -import java.util.ArrayList; import java.util.List; /** @@ -161,12 +158,18 @@ public class RegionServerMetrics impleme new MetricsIntValue("compactionQueueSize", registry); /** - * filesystem read latency + * filesystem read latency for seek-and-read operations */ public final MetricsTimeVaryingRate fsReadLatency = new MetricsTimeVaryingRate("fsReadLatency", registry); /** + * filesystem read latency for positional read operations + */ + public final MetricsTimeVaryingRate fsPreadLatency = + new MetricsTimeVaryingRate("fsPreadLatency", registry); + + /** * filesystem write latency */ public final MetricsTimeVaryingRate fsWriteLatency = @@ -310,9 +313,13 @@ public class RegionServerMetrics impleme addHLogMetric(HLog.getWriteSize(), this.fsWriteSize); addHLogMetric(HLog.getSyncTime(), this.fsSyncLatency); addHLogMetric(HLog.getGSyncTime(), this.fsGroupSyncLatency); + // HFile metrics - int ops = HFile.getReadOps(); - if (ops != 0) this.fsReadLatency.inc(ops, HFile.getReadTimeMs()); + collectHFileMetric(fsReadLatency, + HFile.getReadOpsAndReset(), HFile.getReadTimeMsAndReset()); + collectHFileMetric(fsPreadLatency, + HFile.getPreadOpsAndReset(), HFile.getPreadTimeMsAndReset()); + /* NOTE: removed HFile write latency. 2 reasons: * 1) Mixing HLog latencies are far higher priority since they're * on-demand and HFile is used in background (compact/flush) @@ -344,6 +351,20 @@ public class RegionServerMetrics impleme this.metricsRecord.update(); } + /** + * Increment the given latency metric using the number of operations and total read time + * obtained from HFile. + * @param latencyMetric latency metric to increment + * @param readOps the number of this type of read operations during the collection period + * @param readTimeMs the amount of total read time of this type in milliseconds during the period + */ + private static void collectHFileMetric(MetricsTimeVaryingRate latencyMetric, int readOps, + long readTimeMs) { + if (readOps != 0) { + latencyMetric.inc(readOps, readTimeMs); + } + } + private void addHLogMetric(HLog.Metric logMetric, MetricsTimeVaryingRate hadoopMetric) { if (logMetric.count > 0) Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java?rev=1397821&r1=1397820&r2=1397821&view=diff ============================================================================== --- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java (original) +++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java Sat Oct 13 12:50:29 2012 @@ -49,7 +49,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -61,7 +60,6 @@ import org.apache.hadoop.hbase.io.hfile. import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter; import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder; -import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.MD5Hash; import org.apache.hadoop.util.StringUtils; @@ -696,8 +694,8 @@ public class HFileReadWriteTest { // accumulate them here. HRegion metrics publishing thread should not // be running in this tool, so no one else should be resetting these // metrics. - totalSeekAndReads += HFile.getReadOps(); - totalPositionalReads += HFile.getPreadOps(); + totalSeekAndReads += HFile.getReadOpsAndReset(); + totalPositionalReads += HFile.getPreadOpsAndReset(); long totalBlocksRead = totalSeekAndReads + totalPositionalReads; double blkReadPerSec = totalBlocksRead / timeSec;