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 2019/12/28 14:30:47 UTC

[GitHub] [hbase] Apache9 opened a new pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Apache9 opened a new pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361810983
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
+
+  private static final String KEY_TMPL = "Sequence=%s, at write timestamp=%s";
 
 Review comment:
   What is a  'Sequence'.  Looks like its WALEdit sequenceid...

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361811400
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
+
+  private static final String KEY_TMPL = "Sequence=%s, at write timestamp=%s";
+
+  private static final DateTimeFormatter FORMATTER =
+    DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.systemDefault());
+
+  private String file;
+
+  private PrintStream out;
+
+  public WALProcedurePrettyPrinter() {
+    this(System.out);
+  }
+
+  public WALProcedurePrettyPrinter(PrintStream out) {
+    this.out = out;
+  }
+
+  @Override
+  protected void addOptions() {
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    if (cmd.getArgList().size() != 1) {
+      throw new IllegalArgumentException("Please specify the file to dump");
+    }
+    file = cmd.getArgList().get(0);
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    Path path = new Path(file);
+    FileSystem fs = path.getFileSystem(conf);
+    try (WAL.Reader reader = WALFactory.createReader(fs, path, conf)) {
+      for (;;) {
+        WAL.Entry entry = reader.next();
+        if (entry == null) {
+          return 0;
+        }
+        WALKey key = entry.getKey();
+        WALEdit edit = entry.getEdit();
+        long sequenceId = key.getSequenceId();
+        long writeTime = key.getWriteTime();
+        out.println(
+          String.format(KEY_TMPL, sequenceId, FORMATTER.format(Instant.ofEpochMilli(writeTime))));
+        for (Cell cell : edit.getCells()) {
+          Map<String, Object> op = WALPrettyPrinter.toStringMap(cell);
+          if (!Bytes.equals(FAMILY, 0, FAMILY.length, cell.getFamilyArray(), cell.getFamilyOffset(),
+            cell.getFamilyLength())) {
+            // We could have cells other than procedure edits, for example, a flush marker
+            WALPrettyPrinter.printCell(out, op, false);
+            continue;
+          }
+          long procId = Bytes.toLong(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
+          out.println("pid=" + procId + ", type=" + op.get("type") + ", column=" +
+            op.get("family") + ":" + op.get("qualifier"));
+          if (cell.getType() == Cell.Type.Put) {
+            if (cell.getValueLength() > 0) {
+              // should be a normal put
+              Procedure<?> proc =
+                ProcedureUtil.convertToProcedure(ProcedureProtos.Procedure.parser()
+                  .parseFrom(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+              out.println("\t" + proc.toStringDetails());
+            } else {
+              // should be a 'delete' put
+              out.println("\tmark deleted");
+            }
+          }
+          out.println("cell total size sum: " + cell.heapSize());
+        }
+        out.println("edit heap size: " + edit.heapSize());
+        out.println("position: " + reader.getPosition());
+      }
+    }
+  }
+}
 
 Review comment:
   Looks good.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#issuecomment-569440956
 
 
   :broken_heart: **-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.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m  1s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 30s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 53s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 51s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 29s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  3s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 35s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 57s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 165m 57s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   | 230m  8s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.security.access.TestSnapshotScannerHDFSAclController |
   |   | hadoop.hbase.master.assignment.TestRegionMoveAndAbandon |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/969 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 3eda1b619207 4.15.0-70-generic #79-Ubuntu SMP Tue Nov 12 10:36:11 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-969/out/precommit/personality/provided.sh |
   | git revision | master / 0ba84d8e95 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/1/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/1/testReport/ |
   | Max. process+thread count | 4857 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/1/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | 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


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361830832
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
 
 Review comment:
   Of course you can use WALPrettyPrinter on this file 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


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361810975
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
 
 Review comment:
   The WALPrettyPrinter doesn't work for these? The below adds extra info? Class comment to situate this tool? For debugging?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361830814
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
 
 Review comment:
   WALPrettyPrinter will not decode the procedure while this one will.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#issuecomment-569498006
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 29s |  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.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   8m 29s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 29s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 44s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 42s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 27s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 34s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 148m 35s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 27s |  The patch does not generate ASF License warnings.  |
   |  |   | 214m 54s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/969 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 39c0b58051e3 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-969/out/precommit/personality/provided.sh |
   | git revision | master / 2ca1e46d74 |
   | Default Java | 1.8.0_181 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/2/testReport/ |
   | Max. process+thread count | 4464 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-969/2/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | 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


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361850953
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
+
+  private static final String KEY_TMPL = "Sequence=%s, at write timestamp=%s";
 
 Review comment:
   I think we can file a separated issue to change them both if you think it is necessary.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361831049
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
+
+  private static final String KEY_TMPL = "Sequence=%s, at write timestamp=%s";
 
 Review comment:
   Just align with WALPrettyPrinter, it uses the word 'Sequence' to indicate the sequence id...

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 merged pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
Apache9 merged pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #969: HBASE-23618 Add a tool to dump procedure info in the WAL file
URL: https://github.com/apache/hbase/pull/969#discussion_r361811221
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/WALProcedurePrettyPrinter.java
 ##########
 @@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2.store.region;
+
+import static org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore.FAMILY;
+
+import java.io.PrintStream;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
+
+  private static final String KEY_TMPL = "Sequence=%s, at write timestamp=%s";
 
 Review comment:
   Usually we write sequence as seqid or sequenceid elsewhere logging WALKey sequenceids.

----------------------------------------------------------------
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


With regards,
Apache Git Services