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