1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.wal;
19
20 import static org.junit.Assert.assertFalse;
21
22 import java.io.IOException;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.HBaseTestingUtility;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.HRegionInfo;
31 import org.apache.hadoop.hbase.HTableDescriptor;
32 import org.apache.hadoop.hbase.KeyValue;
33 import org.apache.hadoop.hbase.TableName;
34 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
35 import org.apache.hadoop.hbase.testclassification.SmallTests;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.util.FSTableDescriptors;
38 import org.apache.hadoop.hbase.util.FSUtils;
39 import org.apache.hadoop.hbase.wal.WAL;
40 import org.apache.hadoop.hbase.wal.WALFactory;
41 import org.apache.hadoop.hbase.wal.WALKey;
42 import org.junit.Test;
43 import org.junit.experimental.categories.Category;
44
45
46
47
48 @Category(SmallTests.class)
49 public class TestLogRollingNoCluster {
50 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
51 private final static byte [] EMPTY_1K_ARRAY = new byte[1024];
52 private static final int THREAD_COUNT = 100;
53
54
55
56
57
58
59
60 @Test
61 public void testContendedLogRolling() throws IOException, InterruptedException {
62 Path dir = TEST_UTIL.getDataTestDir();
63
64 TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, THREAD_COUNT);
65 final Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
66 FSUtils.setRootDir(conf, dir);
67 final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName());
68 final WAL wal = wals.getWAL(new byte[]{});
69
70 Appender [] appenders = null;
71
72 final int count = THREAD_COUNT;
73 appenders = new Appender[count];
74 try {
75 for (int i = 0; i < count; i++) {
76
77 appenders[i] = new Appender(wal, i, count);
78 }
79 for (int i = 0; i < count; i++) {
80 appenders[i].start();
81 }
82 for (int i = 0; i < count; i++) {
83
84 appenders[i].join();
85 }
86 } finally {
87 wals.close();
88 }
89 for (int i = 0; i < count; i++) {
90 assertFalse(appenders[i].isException());
91 }
92 }
93
94
95
96
97 static class Appender extends Thread {
98 private final Log log;
99 private final WAL wal;
100 private final int count;
101 private Exception e = null;
102
103 Appender(final WAL wal, final int index, final int count) {
104 super("" + index);
105 this.wal = wal;
106 this.count = count;
107 this.log = LogFactory.getLog("Appender:" + getName());
108 }
109
110
111
112
113 boolean isException() {
114 return !isAlive() && this.e != null;
115 }
116
117 Exception getException() {
118 return this.e;
119 }
120
121 @Override
122 public void run() {
123 this.log.info(getName() +" started");
124 final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
125 try {
126 for (int i = 0; i < this.count; i++) {
127 long now = System.currentTimeMillis();
128
129 if (i % 10 == 0) {
130 this.wal.rollWriter();
131 }
132 WALEdit edit = new WALEdit();
133 byte[] bytes = Bytes.toBytes(i);
134 edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
135 final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
136 final FSTableDescriptors fts = new FSTableDescriptors(TEST_UTIL.getConfiguration());
137 final HTableDescriptor htd = fts.get(TableName.META_TABLE_NAME);
138 final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(),
139 TableName.META_TABLE_NAME, now, mvcc), edit, true);
140 wal.sync(txid);
141 }
142 String msg = getName() + " finished";
143 if (isException())
144 this.log.info(msg, getException());
145 else
146 this.log.info(msg);
147 } catch (Exception e) {
148 this.e = e;
149 log.info("Caught exception from Appender:" + getName(), e);
150 } finally {
151
152 try {
153 this.wal.sync();
154 } catch (IOException e) {
155 throw new RuntimeException(e);
156 }
157 }
158 }
159 }
160
161
162
163
164 }