1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver.wal;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.concurrent.atomic.AtomicLong;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.Cell;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.HRegionInfo;
31 import org.apache.hadoop.hbase.HTableDescriptor;
32 import org.apache.hadoop.hbase.TableName;
33 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
34 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
35 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
36 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
37 import org.apache.hadoop.hbase.wal.WAL;
38 import org.apache.hadoop.hbase.wal.WALKey;
39
40 import com.google.protobuf.TextFormat;
41
42
43
44
45
46
47 @InterfaceAudience.Private
48 public class WALUtil {
49 static final Log LOG = LogFactory.getLog(WALUtil.class);
50
51
52
53
54
55
56
57
58 public static void writeCompactionMarker(WAL log, HTableDescriptor htd, HRegionInfo info,
59 final CompactionDescriptor c, AtomicLong sequenceId) throws IOException {
60 TableName tn = TableName.valueOf(c.getTableName().toByteArray());
61
62 WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
63 log.append(htd, info, key, WALEdit.createCompaction(info, c), sequenceId, false, null);
64 log.sync();
65 if (LOG.isTraceEnabled()) {
66 LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
67 }
68 }
69
70
71
72
73 public static long writeFlushMarker(WAL log, HTableDescriptor htd, HRegionInfo info,
74 final FlushDescriptor f, AtomicLong sequenceId, boolean sync) throws IOException {
75 TableName tn = TableName.valueOf(f.getTableName().toByteArray());
76
77 WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
78 long trx = log.append(htd, info, key, WALEdit.createFlushWALEdit(info, f), sequenceId, false,
79 null);
80 if (sync) log.sync(trx);
81 if (LOG.isTraceEnabled()) {
82 LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
83 }
84 return trx;
85 }
86
87
88
89
90 public static long writeRegionEventMarker(WAL log, HTableDescriptor htd, HRegionInfo info,
91 final RegionEventDescriptor r, AtomicLong sequenceId) throws IOException {
92 TableName tn = TableName.valueOf(r.getTableName().toByteArray());
93
94 WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
95 long trx = log.append(htd, info, key, WALEdit.createRegionEventWALEdit(info, r),
96 sequenceId, false, null);
97 log.sync(trx);
98 if (LOG.isTraceEnabled()) {
99 LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
100 }
101 return trx;
102 }
103
104
105
106
107
108
109
110
111
112
113
114
115
116 public static long writeBulkLoadMarkerAndSync(final WAL wal,
117 final HTableDescriptor htd,
118 final HRegionInfo info,
119 final WALProtos.BulkLoadDescriptor descriptor,
120 final AtomicLong sequenceId) throws IOException {
121 TableName tn = info.getTable();
122 WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
123
124
125 long trx = wal.append(htd,
126 info,
127 key,
128 WALEdit.createBulkLoadEvent(info, descriptor),
129 sequenceId,
130 false,
131 new ArrayList<Cell>());
132 wal.sync(trx);
133
134 if (LOG.isTraceEnabled()) {
135 LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(descriptor));
136 }
137 return trx;
138 }
139
140 }