You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/05/24 02:12:51 UTC

svn commit: r1485903 - in /hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase: io/hfile/ master/ regionserver/ regionserver/wal/ util/

Author: stack
Date: Fri May 24 00:12:50 2013
New Revision: 1485903

URL: http://svn.apache.org/r1485903
Log:
HBASE-8608 Do an edit of logs.. we log too much.

Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java Fri May 24 00:12:50 2013
@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
@@ -143,7 +142,7 @@ public class HFileWriterV2 extends Abstr
 
     // Meta data block index writer
     metaBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter();
-    LOG.debug("Initialized with " + cacheConf);
+    if (LOG.isTraceEnabled()) LOG.trace("Initialized with " + cacheConf);
   }
 
   /**

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java Fri May 24 00:12:50 2013
@@ -239,7 +239,7 @@ public abstract class TableLockManager {
         if (data == null) {
           return;
         }
-        LOG.debug("Table is locked by: " +
+        LOG.debug("Table is locked by " +
             String.format("[tableName=%s, lockOwner=%s, threadId=%s, " +
                 "purpose=%s, isShared=%s, createTime=%s]", Bytes.toString(data.getTableName().toByteArray()),
                 ProtobufUtil.toServerName(data.getLockOwner()), data.getThreadId(),
@@ -270,9 +270,9 @@ public abstract class TableLockManager {
 
       @Override
       public void acquire() throws IOException {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Attempt to acquire table " + (isShared ? "read" : "write")
-              + " lock on :" + tableNameStr + " for:" + purpose);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Attempt to acquire table " + (isShared ? "read" : "write") +
+            " lock on: " + tableNameStr + " for:" + purpose);
         }
 
         lock = createTableLock();
@@ -292,15 +292,15 @@ public abstract class TableLockManager {
           Thread.currentThread().interrupt();
           throw new InterruptedIOException("Interrupted acquiring a lock");
         }
-        LOG.debug("Acquired table " + (isShared ? "read" : "write")
-            + " lock on :" + tableNameStr + " for:" + purpose);
+        if (LOG.isTraceEnabled()) LOG.trace("Acquired table " + (isShared ? "read" : "write")
+            + " lock on " + tableNameStr + " for " + purpose);
       }
 
       @Override
       public void release() throws IOException {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Attempt to release table " + (isShared ? "read" : "write")
-              + " lock on :" + tableNameStr);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Attempt to release table " + (isShared ? "read" : "write")
+              + " lock on " + tableNameStr);
         }
         if (lock == null) {
           throw new IllegalStateException("Table " + tableNameStr +
@@ -314,8 +314,8 @@ public abstract class TableLockManager {
           Thread.currentThread().interrupt();
           throw new InterruptedIOException();
         }
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Released table lock on :" + tableNameStr);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Released table lock on " + tableNameStr);
         }
       }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java Fri May 24 00:12:50 2013
@@ -86,7 +86,9 @@ public class DefaultStoreFlusher extends
       scanner.close();
     }
     LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
-        + StringUtils.humanReadableInt(flushed) +", into tmp file " + writer.getPath());
+        + StringUtils.humanReadableInt(flushed) +
+        ", hasBloomFilter=" + writer.hasGeneralBloom() +
+        ", into tmp file " + writer.getPath());
     result.add(writer.getPath());
     return result;
   }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri May 24 00:12:50 2013
@@ -1554,10 +1554,10 @@ public class HRegion implements HeapSize
     } finally {
       this.updatesLock.writeLock().unlock();
     }
-    String s = "Finished snapshotting " + this +
-      ", commencing wait for mvcc, flushsize=" + flushsize;
+    String s = "Finished memstore snapshotting " + this +
+      ", syncing WAL and waiting on mvcc, flushsize=" + flushsize;
     status.setStatus(s);
-    LOG.debug(s);
+    if (LOG.isTraceEnabled()) LOG.trace(s);
 
     // sync unflushed WAL changes when deferred log sync is enabled
     // see HBASE-8208 for details
@@ -1572,8 +1572,9 @@ public class HRegion implements HeapSize
     // were removed via a rollbackMemstore could be written to Hfiles.
     mvcc.waitForRead(w);
 
-    status.setStatus("Flushing stores");
-    LOG.debug("Finished snapshotting, commencing flushing stores");
+    s = "Flushing stores of " + this;
+    status.setStatus(s);
+    if (LOG.isTraceEnabled()) LOG.trace(s);
 
     // Any failure from here on out will be catastrophic requiring server
     // restart so hlog content can be replayed and put back into the memstore.

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Fri May 24 00:12:50 2013
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.DataInput;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -36,7 +35,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
@@ -748,7 +746,7 @@ public class StoreFile {
 
     /** Bytes per Checksum */
     protected int bytesPerChecksum;
-    
+
     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
      * When flushing a memstore, we set TimeRange and use this variable to
@@ -783,7 +781,7 @@ public class StoreFile {
         CacheConfig cacheConf,
         final KVComparator comparator, BloomType bloomType, long maxKeys,
         final ChecksumType checksumType, final int bytesPerChecksum,
-        final boolean includeMVCCReadpoint, InetSocketAddress[] favoredNodes) 
+        final boolean includeMVCCReadpoint, InetSocketAddress[] favoredNodes)
             throws IOException {
       this.dataBlockEncoder = dataBlockEncoder != null ?
           dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
@@ -807,8 +805,8 @@ public class StoreFile {
 
       if (generalBloomFilterWriter != null) {
         this.bloomType = bloomType;
-        LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
-            + generalBloomFilterWriter.getClass().getSimpleName());
+        if (LOG.isTraceEnabled()) LOG.trace("Bloom filter type for " + path + ": " +
+          this.bloomType + ", " + generalBloomFilterWriter.getClass().getSimpleName());
       } else {
         // Not using Bloom filters.
         this.bloomType = BloomType.NONE;
@@ -824,7 +822,7 @@ public class StoreFile {
         deleteFamilyBloomFilterWriter = null;
       }
       if (deleteFamilyBloomFilterWriter != null) {
-        LOG.info("Delete Family Bloom filter type for " + path + ": "
+        if (LOG.isTraceEnabled()) LOG.trace("Delete Family Bloom filter type for " + path + ": "
             + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
       }
       this.checksumType = checksumType;
@@ -1044,9 +1042,11 @@ public class StoreFile {
 
       // Log final Bloom filter statistics. This needs to be done after close()
       // because compound Bloom filters might be finalized as part of closing.
-      StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
-          + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
-          + " was added to HFile (" + getPath() + ") ");
+      if (StoreFile.LOG.isTraceEnabled()) {
+        StoreFile.LOG.trace((hasGeneralBloom ? "" : "NO ") + "General Bloom and " +
+          (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily" + " was added to HFile " +
+          getPath());
+      }
 
     }
 
@@ -1401,9 +1401,11 @@ public class StoreFile {
             } else {
               generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
                   reader);
-              LOG.info("Loaded " + bloomFilterType.toString() + " ("
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("Loaded " + bloomFilterType.toString() + " "
                   + generalBloomFilter.getClass().getSimpleName()
-                  + ") metadata for " + reader.getName());
+                  + " metadata for " + reader.getName());
+              }
             }
           }
         } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java Fri May 24 00:12:50 2013
@@ -347,7 +347,7 @@ class FSHLog implements HLog, Syncable {
 
     this.logSyncer = new LogSyncer(this.optionalFlushInterval);
 
-    LOG.info("HLog configuration: blocksize=" +
+    LOG.info("WAL/HLog configuration: blocksize=" +
       StringUtils.byteDesc(this.blocksize) +
       ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
       ", enabled=" + this.enabled +
@@ -519,9 +519,10 @@ class FSHLog implements HLog, Syncable {
           this.hdfs_out = nextHdfsOut;
           this.numEntries.set(0);
         }
-        LOG.info("Rolled log" + (oldFile != null ? " for file=" + FSUtils.getPath(oldFile)
-          + ", entries=" + oldNumEntries + ", filesize=" + this.fs.getFileStatus(oldFile).getLen()
-          : "" ) + "; new path=" + FSUtils.getPath(newPath));
+        LOG.info("Rolled WAL " + (oldFile != null ?
+          FSUtils.getPath(oldFile) + ", entries=" + oldNumEntries + ", filesize=" +
+            StringUtils.humanReadableInt(this.fs.getFileStatus(oldFile).getLen()):
+          "" ) + "; new WAL=" + FSUtils.getPath(newPath));
 
         // Tell our listeners that a new log was created
         if (!this.listeners.isEmpty()) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java Fri May 24 00:12:50 2013
@@ -87,7 +87,9 @@ public class ProtobufLogWriter implement
     }
     // instantiate trailer to default value.
     trailer = WALTrailer.newBuilder().build();
-    LOG.debug("Writing protobuf WAL; path=" + path + ", compression=" + doCompress);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
+    }
   }
 
   @Override

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java Fri May 24 00:12:50 2013
@@ -135,8 +135,8 @@ public class CompoundBloomFilterWriter e
 
     chunk.compactBloom();
 
-    if (LOG.isDebugEnabled() && prevByteSize != chunk.getByteSize()) {
-      LOG.debug("Compacted Bloom chunk #" + readyChunk.chunkId + " from ["
+    if (LOG.isTraceEnabled() && prevByteSize != chunk.getByteSize()) {
+      LOG.trace("Compacted Bloom chunk #" + readyChunk.chunkId + " from ["
           + prevMaxKeys + " max keys, " + prevByteSize + " bytes] to ["
           + chunk.getMaxKeys() + " max keys, " + chunk.getByteSize()
           + " bytes]");

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=1485903&r1=1485902&r2=1485903&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java Fri May 24 00:12:50 2013
@@ -320,11 +320,11 @@ public abstract class FSUtils {
    * @return output stream to the created file
    * @throws IOException if the file cannot be created
    */
-  @SuppressWarnings("deprecation")
   public static FSDataOutputStream create(FileSystem fs, Path path,
       FsPermission perm, boolean overwrite) throws IOException {
-    LOG.debug("Creating file=" + path + " with permission=" + perm);
-
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Creating file=" + path + " with permission=" + perm + ", overwrite=" + overwrite);
+    }
     return fs.create(path, perm, overwrite, getDefaultBufferSize(fs),
         getDefaultReplication(fs, path), getDefaultBlockSize(fs, path), null);
   }