You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bi...@apache.org on 2020/06/09 06:13:44 UTC

[hbase] branch branch-2.3 updated: HBASE-24468 Add region info when log meessages in HStore. (#1803)

This is an automated email from the ASF dual-hosted git repository.

binlijin pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 7c16828  HBASE-24468 Add region info when log meessages in HStore. (#1803)
7c16828 is described below

commit 7c16828eb8fbc6e808b94e95b7004c535f0b6865
Author: xincunSong <36...@qq.com>
AuthorDate: Tue Jun 9 14:12:44 2020 +0800

    HBASE-24468 Add region info when log meessages in HStore. (#1803)
    
    Signed-off-by: Anoop Sam John <an...@apache.org>
    Signed-off-by: binlijin <bi...@gmail.com>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../apache/hadoop/hbase/regionserver/HStore.java   | 54 ++++++++++------------
 1 file changed, 24 insertions(+), 30 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 556589f..a3e3150 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -278,7 +278,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     // used by ScanQueryMatcher
     long timeToPurgeDeletes =
         Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
-    LOG.trace("Time to purge deletes set to {}ms in store {}", timeToPurgeDeletes, this);
+    LOG.trace("Time to purge deletes set to {}ms in {}", timeToPurgeDeletes, this);
     // Get TTL
     long ttl = determineTTLFromFamily(family);
     // Why not just pass a HColumnDescriptor in here altogether?  Even if have
@@ -338,9 +338,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       confPrintThreshold = 10;
     }
     this.parallelPutCountPrintThreshold = confPrintThreshold;
-    LOG.info("Store={},  memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
+    LOG.info("{} created,  memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
             + "parallelPutCountPrintThreshold={}, encoding={}, compression={}",
-        getColumnFamilyName(), memstore.getClass().getSimpleName(), policyName, verifyBulkLoads,
+        this, memstore.getClass().getSimpleName(), policyName, verifyBulkLoads,
         parallelPutCountPrintThreshold, family.getDataBlockEncoding(),
         family.getCompressionType());
     cacheOnWriteLogged = false;
@@ -634,7 +634,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       List<HStoreFile> filesToRemove = new ArrayList<>(compactedStoreFiles.size());
       for (HStoreFile storeFile : results) {
         if (compactedStoreFiles.contains(storeFile.getPath().getName())) {
-          LOG.warn("Clearing the compacted storefile {} from this store", storeFile);
+          LOG.warn("Clearing the compacted storefile {} from {}", storeFile, this);
           storeFile.getReader().close(true);
           filesToRemove.add(storeFile);
         }
@@ -706,8 +706,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       return;
     }
 
-    LOG.info("Refreshing store files for region " + this.getRegionInfo().getRegionNameAsString()
-      + " files to add: " + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
+    LOG.info("Refreshing store files for " + this + " files to add: "
+      + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
 
     Set<HStoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
     for (StoreFileInfo sfi : toBeRemovedFiles) {
@@ -820,8 +820,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
     HFile.Reader reader  = null;
     try {
-      LOG.info("Validating hfile at " + srcPath + " for inclusion in "
-          + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
+      LOG.info("Validating hfile at " + srcPath + " for inclusion in " + this);
       FileSystem srcFs = srcPath.getFileSystem(conf);
       srcFs.access(srcPath, FsAction.READ_WRITE);
       reader = HFile.createReader(srcFs, srcPath, cacheConf, isPrimaryReplicaStore(), conf);
@@ -911,13 +910,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       }
     }
 
-    LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() + "' as "
+    LOG.info("Loaded HFile " + srcPath + " into " + this + " as "
         + dstPath + " - updating store file list.");
 
     HStoreFile sf = createStoreFileAndReader(dstPath);
     bulkLoadHFile(sf);
 
-    LOG.info("Successfully loaded store file {} into store {} (new location: {})",
+    LOG.info("Successfully loaded {} into {} (new location: {})",
         srcPath, this, dstPath);
 
     return dstPath;
@@ -945,7 +944,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       // the lock.
       this.lock.writeLock().unlock();
     }
-    LOG.info("Loaded HFile " + sf.getFileInfo() + " into store '" + getColumnFamilyName());
+    LOG.info("Loaded HFile " + sf.getFileInfo() + " into " + this);
     if (LOG.isTraceEnabled()) {
       String traceMessage = "BULK LOAD time,size,store size,store files ["
           + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize
@@ -1076,7 +1075,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
           }
         }
       } catch (IOException e) {
-        LOG.warn("Failed flushing store file, retrying num={}", i, e);
+        LOG.warn("Failed flushing store file for {}, retrying num={}", this, i, e);
         lastException = e;
       }
       if (lastException != null && i < (flushRetriesNumber - 1)) {
@@ -1093,8 +1092,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   }
 
   public HStoreFile tryCommitRecoveredHFile(Path path) throws IOException {
-    LOG.info("Validating recovered hfile at {} for inclusion in store {} region {}", path, this,
-      getRegionInfo().getRegionNameAsString());
+    LOG.info("Validating recovered hfile at {} for inclusion in store {}", path, this);
     FileSystem srcFs = path.getFileSystem(conf);
     srcFs.access(path, FsAction.READ_WRITE);
     try (HFile.Reader reader =
@@ -1185,9 +1183,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         .getCacheCompactedBlocksOnWriteThreshold()) {
         writerCacheConf.enableCacheOnWrite();
         if (!cacheOnWriteLogged) {
-          LOG.info("For Store {} , cacheCompactedBlocksOnWrite is true, hence enabled " +
-              "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks",
-            getColumnFamilyName());
+          LOG.info("For {} , cacheCompactedBlocksOnWrite is true, hence enabled " +
+              "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks", this);
           cacheOnWriteLogged = true;
         }
       } else {
@@ -1195,9 +1192,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         if (totalCompactedFilesSize > cacheConf.getCacheCompactedBlocksOnWriteThreshold()) {
           // checking condition once again for logging
           LOG.debug(
-            "For Store {}, setting cacheCompactedBlocksOnWrite as false as total size of compacted "
+            "For {}, setting cacheCompactedBlocksOnWrite as false as total size of compacted "
               + "files - {}, is greater than cacheCompactedBlocksOnWriteThreshold - {}",
-            getColumnFamilyName(), totalCompactedFilesSize,
+            this, totalCompactedFilesSize,
             cacheConf.getCacheCompactedBlocksOnWriteThreshold());
         }
       }
@@ -1206,8 +1203,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       if (shouldCacheDataOnWrite) {
         writerCacheConf.enableCacheOnWrite();
         if (!cacheOnWriteLogged) {
-          LOG.info("For Store {} , cacheDataOnWrite is true, hence enabled cacheOnWrite for " +
-            "Index blocks and Bloom filter blocks", getColumnFamilyName());
+          LOG.info("For {} , cacheDataOnWrite is true, hence enabled cacheOnWrite for " +
+            "Index blocks and Bloom filter blocks", this);
           cacheOnWriteLogged = true;
         }
       }
@@ -1967,8 +1964,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug(getRegionInfo().getEncodedName() + " - " + getColumnFamilyName()
-          + ": Initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
+      LOG.debug(this + " is initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
           + (request.isAllFiles() ? " (all files)" : ""));
     }
     this.region.reportCompactionRequestStart(request.isMajor());
@@ -1993,8 +1989,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       return;
     }
     if (getColumnFamilyDescriptor().getMinVersions() > 0) {
-      LOG.debug("Skipping expired store file removal due to min version being {}",
-          getColumnFamilyDescriptor().getMinVersions());
+      LOG.debug("Skipping expired store file removal due to min version of {} being {}",
+          this, getColumnFamilyDescriptor().getMinVersions());
       return;
     }
     this.lock.readLock().lock();
@@ -2021,8 +2017,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     replaceStoreFiles(delSfs, newFiles);
     completeCompaction(delSfs);
     LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
-        + this + " of " + this.getRegionInfo().getRegionNameAsString()
-        + "; total size for store is "
+        + this + "; total size is "
         + TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1));
   }
 
@@ -2225,7 +2220,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   @Override
   public String toString() {
-    return this.getColumnFamilyName();
+    return this.getRegionInfo().getShortNameToLog()+ "/" + this.getColumnFamilyName();
   }
 
   @Override
@@ -2514,8 +2509,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         HStore.this.totalUncompressedBytes
             .addAndGet(storeFile.getReader().getTotalUncompressedBytes());
         if (LOG.isInfoEnabled()) {
-          LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
-            " added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
+          LOG.info(this + " added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
               ", sequenceid=" + storeFile.getReader().getSequenceID() + ", filesize="
               + TraditionalBinaryPrefix.long2String(storeFile.getReader().length(), "", 1));
         }