You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/06/23 00:16:46 UTC

[GitHub] [hbase] saintstack opened a new pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

saintstack opened a new pull request #1955:
URL: https://github.com/apache/hbase/pull/1955


   …ableDescriptor
   
   Logging cleanup.
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
    Undo fetching Table Descriptor. Not reliably available at recovery time.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r445765597



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery

Review comment:
       bq. Actually the timeout can be set per rpc request on the same connection.
   
   You are right. Could try this later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] infraio commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
infraio commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r443942436



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery
+   *   given hfile has metadata on how it was written.
+   */
   private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
       long seqId, String familyName, boolean isMetaTable) throws IOException {
     Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
       tableName, regionName, familyName);
     StoreFileWriter.Builder writerBuilder =
         new StoreFileWriter.Builder(walSplitter.conf, CacheConfig.DISABLED, walSplitter.rootFS)
             .withOutputDir(outputDir);
-
-    TableDescriptor tableDesc =
-        tableDescCache.computeIfAbsent(tableName, t -> getTableDescriptor(t));
-    if (tableDesc == null) {
-      throw new IOException("Failed to get table descriptor for table " + tableName);
-    }
-    ColumnFamilyDescriptor cfd = tableDesc.getColumnFamily(Bytes.toBytesBinary(familyName));
-    HFileContext hFileContext = createFileContext(cfd, isMetaTable);
-    return writerBuilder.withFileContext(hFileContext).withBloomType(cfd.getBloomFilterType())
-        .build();
-  }
-
-  private HFileContext createFileContext(ColumnFamilyDescriptor cfd, boolean isMetaTable)
-      throws IOException {
-    return new HFileContextBuilder().withCompression(cfd.getCompressionType())
-        .withChecksumType(HStore.getChecksumType(walSplitter.conf))
-        .withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf))
-        .withBlockSize(cfd.getBlocksize()).withCompressTags(cfd.isCompressTags())
-        .withDataBlockEncoding(cfd.getDataBlockEncoding()).withCellComparator(
-          isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR)
-        .build();
-  }
-
-  private TableDescriptor getTableDescriptor(TableName tableName) {
-    if (walSplitter.rsServices != null) {
-      try {
-        return walSplitter.rsServices.getConnection().getAdmin().getDescriptor(tableName);
-      } catch (IOException e) {
-        LOG.warn("Failed to get table descriptor for {}", tableName, e);
-      }
-    }
-    LOG.info("Failed getting {} table descriptor from master; trying local", tableName);
-    try {
-      return walSplitter.tableDescriptors.get(tableName);

Review comment:
       The tableDescriptors may be removed from WalSplitter, too?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649112666


   Failed in here:
   [INFO] 
   [ERROR] Errors: 
   [ERROR] org.apache.hadoop.hbase.util.TestHBaseFsckCleanReplicationBarriers.null
   [ERROR]   Run 1: TestHBaseFsckCleanReplicationBarriers.testCleanReplicationBarrierWithNonExistTable:91->Object.wait:328->Object.wait:-2 » TestTimedOut
   [ERROR]   Run 2: TestHBaseFsckCleanReplicationBarriers »  Appears to be stuck in thread Time-li...
   [ERROR]   Run 3: TestHBaseFsckCleanReplicationBarriers.setUp:72 » IllegalState A mini-cluster i...
   [INFO] 
   [ERROR]   TestHBaseFsckCleanReplicationBarriers.testCleanReplicationBarrierWithNonExistTable:91 » InterruptedIO
   
   ... which is flakey.
   
   Let me rerun just in case.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-647845364


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 34s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 55s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   3m 23s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 45s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 16s |  hbase-server: The patch generated 1 new + 63 unchanged - 0 fixed = 64 total (was 63)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  13m  6s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   4m 30s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  44m 18s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 4eb894d8e3d6 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 4506f8d8ab |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-648259492


   Do we think this a good way forward?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r444294454



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -2037,8 +2037,7 @@ private void startServices() throws IOException {
       this.splitLogWorker = new SplitLogWorker(sinkConf, this,
           this, walFactory);
       splitLogWorker.start();
-    } else {
-      LOG.warn("SplitLogWorker Service NOT started; CoordinatedStateManager is null");
+      LOG.debug("SplitLogWorker started");

Review comment:
       Previous, we used to log when we did NOT start the splitlogworker service. This goes against our usual pattern of only logging started services (logging all the things we did not start would be an endless list -- smile)
   
   I changed the log here so that it conforms with our usual pattern.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649059556


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 48s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  6s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 24s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m  9s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 37s |  hbase-client in branch-2 failed.  |
   | -0 :warning: |  javadoc  |   0m 51s |  hbase-server in branch-2 failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 16s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 34s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 37s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 52s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  2s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 207m 56s |  hbase-server in the patch failed.  |
   |  |   | 249m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 01855400bda9 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/testReport/ |
   | Max. process+thread count | 2576 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649182541


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 14s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  5s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 58s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 34s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 22s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  7s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 50s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 50s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 50s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 206m 39s |  hbase-server in the patch passed.  |
   |  |   | 242m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk8-hadoop2-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 000c138c3342 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/testReport/ |
   | Max. process+thread count | 2684 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649021693


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  5s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  5s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 59s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 49s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   7m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 55s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 29s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 141m 49s |  hbase-server in the patch passed.  |
   |  |   | 177m 56s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-jdk8-hadoop2-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b2a902d411fe 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/testReport/ |
   | Max. process+thread count | 4257 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache9 commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r443946747



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery

Review comment:
       Should we try to get the TableDescriptor first? If it is not possible, then we fallback to write generic HFiles.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r444297274



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery
+   *   given hfile has metadata on how it was written.
+   */
   private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
       long seqId, String familyName, boolean isMetaTable) throws IOException {
     Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
       tableName, regionName, familyName);
     StoreFileWriter.Builder writerBuilder =
         new StoreFileWriter.Builder(walSplitter.conf, CacheConfig.DISABLED, walSplitter.rootFS)
             .withOutputDir(outputDir);
-
-    TableDescriptor tableDesc =
-        tableDescCache.computeIfAbsent(tableName, t -> getTableDescriptor(t));
-    if (tableDesc == null) {
-      throw new IOException("Failed to get table descriptor for table " + tableName);
-    }
-    ColumnFamilyDescriptor cfd = tableDesc.getColumnFamily(Bytes.toBytesBinary(familyName));
-    HFileContext hFileContext = createFileContext(cfd, isMetaTable);
-    return writerBuilder.withFileContext(hFileContext).withBloomType(cfd.getBloomFilterType())
-        .build();
-  }
-
-  private HFileContext createFileContext(ColumnFamilyDescriptor cfd, boolean isMetaTable)
-      throws IOException {
-    return new HFileContextBuilder().withCompression(cfd.getCompressionType())
-        .withChecksumType(HStore.getChecksumType(walSplitter.conf))
-        .withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf))
-        .withBlockSize(cfd.getBlocksize()).withCompressTags(cfd.isCompressTags())
-        .withDataBlockEncoding(cfd.getDataBlockEncoding()).withCellComparator(
-          isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR)
-        .build();
-  }
-
-  private TableDescriptor getTableDescriptor(TableName tableName) {
-    if (walSplitter.rsServices != null) {
-      try {
-        return walSplitter.rsServices.getConnection().getAdmin().getDescriptor(tableName);
-      } catch (IOException e) {
-        LOG.warn("Failed to get table descriptor for {}", tableName, e);
-      }
-    }
-    LOG.info("Failed getting {} table descriptor from master; trying local", tableName);
-    try {
-      return walSplitter.tableDescriptors.get(tableName);

Review comment:
       Good question.
   
   Looking at it, it usually gets them from the hosting Server, not by RPC to Master so there it should not suffer the issue we see here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649763073


   Merged. The failures vary and are flakies.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649158844


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 35s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  5s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 33s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   5m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in branch-2 failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in branch-2 failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 32s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 27s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 14s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 126m 48s |  hbase-server in the patch failed.  |
   |  |   | 157m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e7fd2218007c 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/testReport/ |
   | Max. process+thread count | 4181 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-649126958


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 15s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   2m  0s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   3m 59s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 18s |  hbase-server: The patch generated 1 new + 64 unchanged - 0 fixed = 65 total (was 64)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 38s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 44s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  42m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 6294f41ad7bf 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r444302316



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery

Review comment:
       Asking the Master for the table descriptor first and if it fails, read the fs is what we had before this patch. Problem was that default configuration had it that the RPC query wasn't timing out before the below happened:
   
    2020-06-18 19:53:54,175 ERROR [main] master.HMasterCommandLine: Master exiting
    java.lang.RuntimeException: Master not initialized after 200000ms
   
   I could dick around w/ timings inside here so RPC timed out sooner. It uses the server's Connection. I suppose I could make a new Connection everytime we want to query a Table schema and configure it to try once only but that is a lot of work to do at recovery time.
   
   On the compactions, don't we usually pick up the small files first?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-648447795


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 38s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  7s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  4s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 51s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 36s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 34s |  hbase-client in branch-2 failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in branch-2 failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 35s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 19s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 215m 41s |  hbase-server in the patch failed.  |
   |  |   | 253m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8baafa9bc8d3 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/testReport/ |
   | Max. process+thread count | 2610 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] infraio commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
infraio commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r443941820



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -2037,8 +2037,7 @@ private void startServices() throws IOException {
       this.splitLogWorker = new SplitLogWorker(sinkConf, this,
           this, walFactory);
       splitLogWorker.start();
-    } else {
-      LOG.warn("SplitLogWorker Service NOT started; CoordinatedStateManager is null");
+      LOG.debug("SplitLogWorker started");

Review comment:
       Move the log out of the if {} code block?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-648334000


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 49s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   3m 11s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 29s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 14s |  hbase-server: The patch generated 1 new + 64 unchanged - 0 fixed = 65 total (was 64)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 17s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 29s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  38m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux f8b9d1082f8d 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r443967906



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery

Review comment:
       In Jira, I added a comment about making sure we will compact all these tiny HFiles created as part of WAL split. If we can make sure that part, I would say it ok to create these tiny files with out any table specific things like compression/DBE etc. Anyways we know all these files are going to get compacted and rewritten once we open the region.  
   As of now we are not sure whether or when these tiny files will get compacted. In such case I would +1 ur ask. Do this HFile create with defaults as a fall back only.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r444337252



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery
+   *   given hfile has metadata on how it was written.
+   */
   private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
       long seqId, String familyName, boolean isMetaTable) throws IOException {
     Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
       tableName, regionName, familyName);
     StoreFileWriter.Builder writerBuilder =
         new StoreFileWriter.Builder(walSplitter.conf, CacheConfig.DISABLED, walSplitter.rootFS)
             .withOutputDir(outputDir);
-
-    TableDescriptor tableDesc =
-        tableDescCache.computeIfAbsent(tableName, t -> getTableDescriptor(t));
-    if (tableDesc == null) {
-      throw new IOException("Failed to get table descriptor for table " + tableName);
-    }
-    ColumnFamilyDescriptor cfd = tableDesc.getColumnFamily(Bytes.toBytesBinary(familyName));
-    HFileContext hFileContext = createFileContext(cfd, isMetaTable);
-    return writerBuilder.withFileContext(hFileContext).withBloomType(cfd.getBloomFilterType())
-        .build();
-  }
-
-  private HFileContext createFileContext(ColumnFamilyDescriptor cfd, boolean isMetaTable)
-      throws IOException {
-    return new HFileContextBuilder().withCompression(cfd.getCompressionType())
-        .withChecksumType(HStore.getChecksumType(walSplitter.conf))
-        .withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf))
-        .withBlockSize(cfd.getBlocksize()).withCompressTags(cfd.isCompressTags())
-        .withDataBlockEncoding(cfd.getDataBlockEncoding()).withCellComparator(
-          isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR)
-        .build();
-  }
-
-  private TableDescriptor getTableDescriptor(TableName tableName) {
-    if (walSplitter.rsServices != null) {
-      try {
-        return walSplitter.rsServices.getConnection().getAdmin().getDescriptor(tableName);
-      } catch (IOException e) {
-        LOG.warn("Failed to get table descriptor for {}", tableName, e);
-      }
-    }
-    LOG.info("Failed getting {} table descriptor from master; trying local", tableName);
-    try {
-      return walSplitter.tableDescriptors.get(tableName);

Review comment:
       Sorry, I get what you are saying now. You added this. Let me undo it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-648948126


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 41s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 39s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   3m  6s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 15s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  4s |  hbase-server: The patch generated 1 new + 64 unchanged - 0 fixed = 65 total (was 64)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 19s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 34s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 24s |  The patch does not generate ASF License warnings.  |
   |  |   |  37m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux c2ceb7cdd540 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] infraio commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
infraio commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r443942069



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
##########
@@ -48,7 +48,7 @@ public void process() {
     try {
       callable.call();
     } catch (Throwable t) {
-      LOG.error("Error when call RSProcedureCallable: ", t);
+      LOG.error("pid=" + this.procId, t);

Review comment:
       This log is too little?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache9 commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r444314587



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
##########
@@ -191,50 +186,22 @@ public boolean keepRegionEvent(Entry entry) {
     return false;
   }
 
+  /**
+   * @return Returns a base HFile without compressions or encodings; good enough for recovery

Review comment:
       Actually the timeout can be set per rpc request on the same connection.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#discussion_r444295419



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
##########
@@ -48,7 +48,7 @@ public void process() {
     try {
       callable.call();
     } catch (Throwable t) {
-      LOG.error("Error when call RSProcedureCallable: ", t);
+      LOG.error("pid=" + this.procId, t);

Review comment:
       I changed it so that it logs the pid same way we do it every where else. The thread/class/level already provides what we add here. I just removed redundancy.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack merged pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
saintstack merged pull request #1955:
URL: https://github.com/apache/hbase/pull/1955


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1955: HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1955:
URL: https://github.com/apache/hbase/pull/1955#issuecomment-648450873


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 37s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  6s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 37s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m 30s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   5m 42s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  2s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 29s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 29s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 33s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  1s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 30s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 229m 38s |  hbase-server in the patch passed.  |
   |  |   | 261m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/artifact/yetus-jdk8-hadoop2-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1955 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c76a0aae5a0d 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 017c374519 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/testReport/ |
   | Max. process+thread count | 2745 (vs. ulimit of 12500) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1955/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org