001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.procedure2.store.region;
019
020import static org.apache.hadoop.hbase.master.region.MasterRegionFactory.PROC_FAMILY;
021
022import java.io.PrintStream;
023import java.time.Instant;
024import java.time.ZoneId;
025import java.time.format.DateTimeFormatter;
026import java.util.Map;
027import org.apache.hadoop.fs.FileSystem;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.hbase.Cell;
030import org.apache.hadoop.hbase.HBaseInterfaceAudience;
031import org.apache.hadoop.hbase.procedure2.Procedure;
032import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
033import org.apache.hadoop.hbase.util.AbstractHBaseTool;
034import org.apache.hadoop.hbase.util.Bytes;
035import org.apache.hadoop.hbase.wal.WAL;
036import org.apache.hadoop.hbase.wal.WALEdit;
037import org.apache.hadoop.hbase.wal.WALFactory;
038import org.apache.hadoop.hbase.wal.WALKey;
039import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
040import org.apache.hadoop.hbase.wal.WALStreamReader;
041import org.apache.yetus.audience.InterfaceAudience;
042import org.apache.yetus.audience.InterfaceStability;
043
044import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
045
046import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
047
048/**
049 * A tool to dump the procedures in the WAL files.
050 * <p/>
051 * The different between this and {@link WALPrettyPrinter} is that, this class will decode the
052 * procedure in the WALEdit for better debugging. You are free to use {@link WALPrettyPrinter} to
053 * dump the same file as well.
054 */
055@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
056@InterfaceStability.Evolving
057public class WALProcedurePrettyPrinter extends AbstractHBaseTool {
058
059  private static final String KEY_TMPL = "Sequence=%s, at write timestamp=%s";
060
061  private static final DateTimeFormatter FORMATTER =
062    DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.systemDefault());
063
064  private String file;
065
066  private PrintStream out;
067
068  public WALProcedurePrettyPrinter() {
069    this(System.out);
070  }
071
072  public WALProcedurePrettyPrinter(PrintStream out) {
073    this.out = out;
074  }
075
076  @Override
077  protected void addOptions() {
078  }
079
080  @Override
081  protected void processOptions(CommandLine cmd) {
082    if (cmd.getArgList().size() != 1) {
083      throw new IllegalArgumentException("Please specify the file to dump");
084    }
085    file = cmd.getArgList().get(0);
086  }
087
088  @Override
089  protected int doWork() throws Exception {
090    Path path = new Path(file);
091    FileSystem fs = path.getFileSystem(conf);
092    try (WALStreamReader reader = WALFactory.createStreamReader(fs, path, conf)) {
093      for (;;) {
094        WAL.Entry entry = reader.next();
095        if (entry == null) {
096          return 0;
097        }
098        WALKey key = entry.getKey();
099        WALEdit edit = entry.getEdit();
100        long sequenceId = key.getSequenceId();
101        long writeTime = key.getWriteTime();
102        out.println(
103          String.format(KEY_TMPL, sequenceId, FORMATTER.format(Instant.ofEpochMilli(writeTime))));
104        for (Cell cell : edit.getCells()) {
105          Map<String, Object> op = WALPrettyPrinter.toStringMap(cell);
106          if (
107            !Bytes.equals(PROC_FAMILY, 0, PROC_FAMILY.length, cell.getFamilyArray(),
108              cell.getFamilyOffset(), cell.getFamilyLength())
109          ) {
110            // We could have cells other than procedure edits, for example, a flush marker
111            WALPrettyPrinter.printCell(out, op, false, false);
112            continue;
113          }
114          long procId = Bytes.toLong(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
115          out.println("pid=" + procId + ", type=" + op.get("type") + ", column=" + op.get("family")
116            + ":" + op.get("qualifier"));
117          if (cell.getType() == Cell.Type.Put) {
118            if (cell.getValueLength() > 0) {
119              // should be a normal put
120              Procedure<?> proc =
121                ProcedureUtil.convertToProcedure(ProcedureProtos.Procedure.parser()
122                  .parseFrom(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
123              out.println("\t" + proc.toStringDetails());
124            } else {
125              // should be a 'delete' put
126              out.println("\tmark deleted");
127            }
128          }
129          out.println("cell total size sum: " + cell.heapSize());
130        }
131        out.println("edit heap size: " + edit.heapSize());
132        out.println("position: " + reader.getPosition());
133      }
134    }
135  }
136
137  public static void main(String[] args) {
138    new WALProcedurePrettyPrinter().doStaticMain(args);
139  }
140}